You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by su...@apache.org on 2017/05/09 00:00:07 UTC

[01/42] hadoop git commit: YARN-6281. Cleanup when AMRMProxy fails to initialize a new interceptor chain. (Botong Huang via Subru) [Forced Update!]

Repository: hadoop
Updated Branches:
  refs/heads/YARN-2915 2f4191c50 -> 0a93a32e6 (forced update)


YARN-6281. Cleanup when AMRMProxy fails to initialize a new interceptor chain. (Botong Huang via Subru)

(cherry picked from commit 57a9afbd45b7ef8e6021cc58f96bd0074bf1389d)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/749e5c09
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/749e5c09
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/749e5c09

Branch: refs/heads/YARN-2915
Commit: 749e5c09b9990590c282ea944e24735b795351fc
Parents: cd9ff27
Author: Subru Krishnan <su...@apache.org>
Authored: Fri Mar 10 18:13:29 2017 -0800
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon May 8 16:55:47 2017 -0700

----------------------------------------------------------------------
 .../nodemanager/amrmproxy/AMRMProxyService.java | 25 +++++++++++------
 .../amrmproxy/BaseAMRMProxyTest.java            | 21 +++++++++-----
 .../amrmproxy/TestAMRMProxyService.java         | 29 ++++++++++++++++++++
 3 files changed, 59 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/749e5c09/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/AMRMProxyService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/AMRMProxyService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/AMRMProxyService.java
index 2696bca..aeb3be8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/AMRMProxyService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/AMRMProxyService.java
@@ -319,11 +319,16 @@ public class AMRMProxyService extends AbstractService implements
         + " ApplicationId:" + applicationAttemptId + " for the user: "
         + user);
 
-    RequestInterceptor interceptorChain =
-        this.createRequestInterceptorChain();
-    interceptorChain.init(createApplicationMasterContext(
-        applicationAttemptId, user, amrmToken, localToken));
-    chainWrapper.init(interceptorChain, applicationAttemptId);
+    try {
+      RequestInterceptor interceptorChain =
+          this.createRequestInterceptorChain();
+      interceptorChain.init(createApplicationMasterContext(this.nmContext,
+          applicationAttemptId, user, amrmToken, localToken));
+      chainWrapper.init(interceptorChain, applicationAttemptId);
+    } catch (Exception e) {
+      this.applPipelineMap.remove(applicationAttemptId.getApplicationId());
+      throw e;
+    }
   }
 
   /**
@@ -339,8 +344,10 @@ public class AMRMProxyService extends AbstractService implements
         this.applPipelineMap.remove(applicationId);
 
     if (pipeline == null) {
-      LOG.info("Request to stop an application that does not exist. Id:"
-          + applicationId);
+      LOG.info(
+          "No interceptor pipeline for application {},"
+              + " likely because its AM is not run in this node.",
+          applicationId);
     } else {
       // Remove the appAttempt in AMRMTokenSecretManager
       this.secretManager
@@ -413,11 +420,11 @@ public class AMRMProxyService extends AbstractService implements
   }
 
   private AMRMProxyApplicationContext createApplicationMasterContext(
-      ApplicationAttemptId applicationAttemptId, String user,
+      Context context, ApplicationAttemptId applicationAttemptId, String user,
       Token<AMRMTokenIdentifier> amrmToken,
       Token<AMRMTokenIdentifier> localToken) {
     AMRMProxyApplicationContextImpl appContext =
-        new AMRMProxyApplicationContextImpl(this.nmContext, getConfig(),
+        new AMRMProxyApplicationContextImpl(context, getConfig(),
             applicationAttemptId, user, amrmToken, localToken);
     return appContext;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/749e5c09/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/BaseAMRMProxyTest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/BaseAMRMProxyTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/BaseAMRMProxyTest.java
index 7f96947..6f5009e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/BaseAMRMProxyTest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/BaseAMRMProxyTest.java
@@ -121,9 +121,9 @@ public abstract class BaseAMRMProxyTest {
             + MockRequestInterceptor.class.getName());
 
     this.dispatcher = new AsyncDispatcher();
-    this.dispatcher.init(conf);
+    this.dispatcher.init(this.conf);
     this.dispatcher.start();
-    this.amrmProxyService = createAndStartAMRMProxyService();
+    createAndStartAMRMProxyService(this.conf);
   }
 
   @After
@@ -137,12 +137,19 @@ public abstract class BaseAMRMProxyTest {
     return threadpool;
   }
 
-  protected MockAMRMProxyService createAndStartAMRMProxyService() {
-    MockAMRMProxyService svc =
+  protected Configuration getConf() {
+    return this.conf;
+  }
+
+  protected void createAndStartAMRMProxyService(Configuration config) {
+    // Stop the existing instance first if not null
+    if (this.amrmProxyService != null) {
+      this.amrmProxyService.stop();
+    }
+    this.amrmProxyService =
         new MockAMRMProxyService(new NullContext(), dispatcher);
-    svc.init(conf);
-    svc.start();
-    return svc;
+    this.amrmProxyService.init(config);
+    this.amrmProxyService.start();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/749e5c09/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestAMRMProxyService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestAMRMProxyService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestAMRMProxyService.java
index 837278c..fa17f26 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestAMRMProxyService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestAMRMProxyService.java
@@ -21,9 +21,11 @@ package org.apache.hadoop.yarn.server.nodemanager.amrmproxy;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterResponse;
@@ -34,6 +36,7 @@ import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.nodemanager.amrmproxy.AMRMProxyService.RequestInterceptorChainWrapper;
 import org.apache.hadoop.yarn.util.Records;
@@ -95,6 +98,32 @@ public class TestAMRMProxyService extends BaseAMRMProxyTest {
   }
 
   /**
+   * Tests the case when interceptor pipeline initialization fails.
+   */
+  @Test
+  public void testInterceptorInitFailure() {
+    Configuration conf = this.getConf();
+    // Override with a bad interceptor configuration
+    conf.set(YarnConfiguration.AMRM_PROXY_INTERCEPTOR_CLASS_PIPELINE,
+        "class.that.does.not.exist");
+
+    // Reinitialize instance with the new config
+    createAndStartAMRMProxyService(conf);
+    int testAppId = 1;
+    try {
+      registerApplicationMaster(testAppId);
+      Assert.fail("Should not reach here. Expecting an exception thrown");
+    } catch (Exception e) {
+      Map<ApplicationId, RequestInterceptorChainWrapper> pipelines =
+          getAMRMProxyService().getPipelines();
+      ApplicationId id = getApplicationId(testAppId);
+      Assert.assertTrue(
+          "The interceptor pipeline should be removed if initializtion fails",
+          pipelines.get(id) == null);
+    }
+  }
+
+  /**
    * Tests the registration of multiple application master serially one at a
    * time.
    * 


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[20/42] hadoop git commit: YARN-3671. Integrate Federation services with ResourceManager. Contributed by Subru Krishnan

Posted by su...@apache.org.
YARN-3671. Integrate Federation services with ResourceManager. Contributed by Subru Krishnan


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/e3d508f0
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/e3d508f0
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/e3d508f0

Branch: refs/heads/YARN-2915
Commit: e3d508f07207026ad02a0fc73021dbe3284cd4b3
Parents: b96be3d
Author: Jian He <ji...@apache.org>
Authored: Tue Aug 30 12:20:52 2016 +0800
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon May 8 16:59:12 2017 -0700

----------------------------------------------------------------------
 .../hadoop/yarn/conf/YarnConfiguration.java     |  11 +-
 .../yarn/conf/TestYarnConfigurationFields.java  |   4 +-
 .../failover/FederationProxyProviderUtil.java   |   2 +-
 .../FederationRMFailoverProxyProvider.java      |   4 +-
 ...ationMembershipStateStoreInputValidator.java |   7 +-
 .../TestFederationStateStoreInputValidator.java |  10 +-
 .../server/resourcemanager/ResourceManager.java |  26 ++
 .../FederationStateStoreHeartbeat.java          | 108 +++++++
 .../federation/FederationStateStoreService.java | 304 +++++++++++++++++++
 .../federation/package-info.java                |  17 ++
 .../webapp/dao/ClusterMetricsInfo.java          |   5 +-
 .../TestFederationRMStateStoreService.java      | 170 +++++++++++
 12 files changed, 648 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3d508f0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
index 05321e1..3508fad 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
@@ -2546,9 +2546,6 @@ public class YarnConfiguration extends Configuration {
       FEDERATION_PREFIX + "failover.enabled";
   public static final boolean DEFAULT_FEDERATION_FAILOVER_ENABLED = true;
 
-  public static final String FEDERATION_SUBCLUSTER_ID =
-      FEDERATION_PREFIX + "sub-cluster.id";
-
   public static final String FEDERATION_STATESTORE_CLIENT_CLASS =
       FEDERATION_PREFIX + "state-store.class";
 
@@ -2561,6 +2558,14 @@ public class YarnConfiguration extends Configuration {
   // 5 minutes
   public static final int DEFAULT_FEDERATION_CACHE_TIME_TO_LIVE_SECS = 5 * 60;
 
+  public static final String FEDERATION_STATESTORE_HEARTBEAT_INTERVAL_SECS =
+      FEDERATION_PREFIX + "state-store.heartbeat-interval-secs";
+
+  // 5 minutes
+  public static final int
+      DEFAULT_FEDERATION_STATESTORE_HEARTBEAT_INTERVAL_SECS =
+      5 * 60;
+
   public static final String FEDERATION_MACHINE_LIST =
       FEDERATION_PREFIX + "machine-list";
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3d508f0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
index c4d8f38..5e0876f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
@@ -72,9 +72,9 @@ public class TestYarnConfigurationFields extends TestConfigurationFieldsBase {
     configurationPropsToSkipCompare
         .add(YarnConfiguration.DEFAULT_FEDERATION_STATESTORE_CLIENT_CLASS);
     configurationPropsToSkipCompare
-        .add(YarnConfiguration.FEDERATION_SUBCLUSTER_ID);
-    configurationPropsToSkipCompare
         .add(YarnConfiguration.FEDERATION_FAILOVER_ENABLED);
+    configurationPropsToSkipCompare
+        .add(YarnConfiguration.FEDERATION_STATESTORE_HEARTBEAT_INTERVAL_SECS);
 
     // Ignore blacklisting nodes for AM failures feature since it is still a
     // "work in progress"

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3d508f0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/failover/FederationProxyProviderUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/failover/FederationProxyProviderUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/failover/FederationProxyProviderUtil.java
index a986008..18f1338 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/failover/FederationProxyProviderUtil.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/failover/FederationProxyProviderUtil.java
@@ -134,7 +134,7 @@ public final class FederationProxyProviderUtil {
   // are based out of conf
   private static void updateConf(Configuration conf,
       SubClusterId subClusterId) {
-    conf.set(YarnConfiguration.FEDERATION_SUBCLUSTER_ID, subClusterId.getId());
+    conf.set(YarnConfiguration.RM_CLUSTER_ID, subClusterId.getId());
     // In a Federation setting, we will connect to not just the local cluster RM
     // but also multiple external RMs. The membership information of all the RMs
     // that are currently

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3d508f0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/failover/FederationRMFailoverProxyProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/failover/FederationRMFailoverProxyProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/failover/FederationRMFailoverProxyProvider.java
index 90a9239..0ffab0b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/failover/FederationRMFailoverProxyProvider.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/failover/FederationRMFailoverProxyProvider.java
@@ -74,8 +74,8 @@ public class FederationRMFailoverProxyProvider<T>
     this.protocol = proto;
     this.rmProxy.checkAllowedProtocols(this.protocol);
     String clusterId =
-        configuration.get(YarnConfiguration.FEDERATION_SUBCLUSTER_ID);
-    Preconditions.checkNotNull(clusterId, "Missing Federation SubClusterId");
+        configuration.get(YarnConfiguration.RM_CLUSTER_ID);
+    Preconditions.checkNotNull(clusterId, "Missing RM ClusterId");
     this.subClusterId = SubClusterId.newInstance(clusterId);
     this.facade = facade.getInstance();
     if (configuration instanceof YarnConfiguration) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3d508f0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationMembershipStateStoreInputValidator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationMembershipStateStoreInputValidator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationMembershipStateStoreInputValidator.java
index b587ee5..ff9d8e9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationMembershipStateStoreInputValidator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationMembershipStateStoreInputValidator.java
@@ -159,7 +159,10 @@ public final class FederationMembershipStateStoreInputValidator {
   }
 
   /**
-   * Validate if the SubCluster Info are present or not.
+   * Validate if all the required fields on {@link SubClusterInfo} are present
+   * or not. {@code Capability} will be empty as the corresponding
+   * {@code ResourceManager} is in the process of initialization during
+   * registration.
    *
    * @param subClusterInfo the information of the subcluster to be verified
    * @throws FederationStateStoreInvalidInputException if the SubCluster Info
@@ -194,8 +197,6 @@ public final class FederationMembershipStateStoreInputValidator {
     // validate subcluster state
     checkSubClusterState(subClusterInfo.getState());
 
-    // validate subcluster capability
-    checkCapability(subClusterInfo.getCapability());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3d508f0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/utils/TestFederationStateStoreInputValidator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/utils/TestFederationStateStoreInputValidator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/utils/TestFederationStateStoreInputValidator.java
index 13175ae..b95f17a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/utils/TestFederationStateStoreInputValidator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/utils/TestFederationStateStoreInputValidator.java
@@ -242,11 +242,8 @@ public class TestFederationStateStoreInputValidator {
           SubClusterRegisterRequest.newInstance(subClusterInfo);
       FederationMembershipStateStoreInputValidator
           .validateSubClusterRegisterRequest(request);
-      Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
-      LOG.info(e.getMessage());
-      Assert.assertTrue(
-          e.getMessage().startsWith("Invalid capability information."));
+      Assert.fail(e.getMessage());
     }
 
     // Execution with Empty Capability
@@ -260,11 +257,8 @@ public class TestFederationStateStoreInputValidator {
           SubClusterRegisterRequest.newInstance(subClusterInfo);
       FederationMembershipStateStoreInputValidator
           .validateSubClusterRegisterRequest(request);
-      Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
-      LOG.info(e.getMessage());
-      Assert.assertTrue(
-          e.getMessage().startsWith("Invalid capability information."));
+      Assert.fail(e.getMessage());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3d508f0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
index 81c3f1b..2a5f03e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
@@ -67,6 +67,7 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWriter;
 import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.ApplicationMasterLauncher;
+import org.apache.hadoop.yarn.server.resourcemanager.federation.FederationStateStoreService;
 import org.apache.hadoop.yarn.server.resourcemanager.metrics.NoOpSystemMetricPublisher;
 import org.apache.hadoop.yarn.server.resourcemanager.metrics.SystemMetricsPublisher;
 import org.apache.hadoop.yarn.server.resourcemanager.metrics.TimelineServiceV1Publisher;
@@ -188,6 +189,7 @@ public class ResourceManager extends CompositeService implements Recoverable {
   protected RMAppManager rmAppManager;
   protected ApplicationACLsManager applicationACLsManager;
   protected QueueACLsManager queueACLsManager;
+  private FederationStateStoreService federationStateStoreService;
   private WebApp webApp;
   private AppReportFetcher fetcher = null;
   protected ResourceTrackerService resourceTracker;
@@ -504,6 +506,10 @@ public class ResourceManager extends CompositeService implements Recoverable {
     return new RMTimelineCollectorManager(rmContext);
   }
 
+  private FederationStateStoreService createFederationStateStoreService() {
+    return new FederationStateStoreService(rmContext);
+  }
+
   protected SystemMetricsPublisher createSystemMetricsPublisher() {
     SystemMetricsPublisher publisher;
     if (YarnConfiguration.timelineServiceEnabled(conf) &&
@@ -732,6 +738,20 @@ public class ResourceManager extends CompositeService implements Recoverable {
         delegationTokenRenewer.setRMContext(rmContext);
       }
 
+      if(HAUtil.isFederationEnabled(conf)) {
+        String cId = YarnConfiguration.getClusterId(conf);
+        if (cId.isEmpty()) {
+          String errMsg =
+              "Cannot initialize RM as Federation is enabled"
+                  + " but cluster id is not configured.";
+          LOG.error(errMsg);
+          throw new YarnRuntimeException(errMsg);
+        }
+        federationStateStoreService = createFederationStateStoreService();
+        addIfService(federationStateStoreService);
+        LOG.info("Initialized Federation membership.");
+      }
+
       new RMNMInfo(rmContext, scheduler);
 
       super.serviceInit(conf);
@@ -1396,6 +1416,12 @@ public class ResourceManager extends CompositeService implements Recoverable {
   }
 
   @Private
+  @VisibleForTesting
+  public FederationStateStoreService getFederationStateStoreService() {
+    return this.federationStateStoreService;
+  }
+
+  @Private
   WebApp getWebapp() {
     return this.webApp;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3d508f0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/federation/FederationStateStoreHeartbeat.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/federation/FederationStateStoreHeartbeat.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/federation/FederationStateStoreHeartbeat.java
new file mode 100644
index 0000000..a4618a2
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/federation/FederationStateStoreHeartbeat.java
@@ -0,0 +1,108 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.federation;
+
+import java.io.StringWriter;
+
+import javax.xml.bind.JAXBException;
+
+import org.apache.hadoop.yarn.server.federation.store.FederationStateStore;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterHeartbeatRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterState;
+import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ClusterMetricsInfo;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.sun.jersey.api.json.JSONConfiguration;
+import com.sun.jersey.api.json.JSONJAXBContext;
+import com.sun.jersey.api.json.JSONMarshaller;
+
+/**
+ * Periodic heart beat from a <code>ResourceManager</code> participating in
+ * federation to indicate liveliness. The heart beat publishes the current
+ * capabilities as represented by {@link ClusterMetricsInfo} of the sub cluster.
+ *
+ */
+public class FederationStateStoreHeartbeat implements Runnable {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(FederationStateStoreHeartbeat.class);
+
+  private SubClusterId subClusterId;
+  private FederationStateStore stateStoreService;
+
+  private final ResourceScheduler rs;
+
+  private StringWriter currentClusterState;
+  private JSONJAXBContext jc;
+  private JSONMarshaller marshaller;
+  private String capability;
+
+  public FederationStateStoreHeartbeat(SubClusterId subClusterId,
+      FederationStateStore stateStoreClient, ResourceScheduler scheduler) {
+    this.stateStoreService = stateStoreClient;
+    this.subClusterId = subClusterId;
+    this.rs = scheduler;
+    // Initialize the JAXB Marshaller
+    this.currentClusterState = new StringWriter();
+    try {
+      this.jc = new JSONJAXBContext(
+          JSONConfiguration.mapped().rootUnwrapping(false).build(),
+          ClusterMetricsInfo.class);
+      marshaller = jc.createJSONMarshaller();
+    } catch (JAXBException e) {
+      LOG.warn("Exception while trying to initialize JAXB context.", e);
+    }
+    LOG.info("Initialized Federation membership for cluster with timestamp:  "
+        + ResourceManager.getClusterTimeStamp());
+  }
+
+  /**
+   * Get the current cluster state as a JSON string representation of the
+   * {@link ClusterMetricsInfo}.
+   */
+  private void updateClusterState() {
+    try {
+      // get the current state
+      currentClusterState.getBuffer().setLength(0);
+      ClusterMetricsInfo clusterMetricsInfo = new ClusterMetricsInfo(rs);
+      marshaller.marshallToJSON(clusterMetricsInfo, currentClusterState);
+      capability = currentClusterState.toString();
+    } catch (Exception e) {
+      LOG.warn("Exception while trying to generate cluster state,"
+          + " so reverting to last know state.", e);
+    }
+  }
+
+  @Override
+  public synchronized void run() {
+    try {
+      updateClusterState();
+      SubClusterHeartbeatRequest request = SubClusterHeartbeatRequest
+          .newInstance(subClusterId, SubClusterState.SC_RUNNING, capability);
+      stateStoreService.subClusterHeartbeat(request);
+      LOG.debug("Sending the heartbeat with capability: {}", capability);
+    } catch (Exception e) {
+      LOG.warn("Exception when trying to heartbeat: ", e);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3d508f0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/federation/FederationStateStoreService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/federation/FederationStateStoreService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/federation/FederationStateStoreService.java
new file mode 100644
index 0000000..9a01d7e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/federation/FederationStateStoreService.java
@@ -0,0 +1,304 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.federation;
+
+import java.net.InetSocketAddress;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.retry.RetryPolicy;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.util.concurrent.HadoopExecutors;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.hadoop.yarn.server.federation.store.FederationStateStore;
+import org.apache.hadoop.yarn.server.federation.store.records.AddApplicationHomeSubClusterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.AddApplicationHomeSubClusterResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.DeleteApplicationHomeSubClusterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.DeleteApplicationHomeSubClusterResponse;
+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.server.federation.store.records.GetApplicationsHomeSubClusterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.GetApplicationsHomeSubClusterResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterInfoRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterInfoResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterPoliciesConfigurationsRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterPoliciesConfigurationsResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterPolicyConfigurationRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterPolicyConfigurationResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClustersInfoRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClustersInfoResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.SetSubClusterPolicyConfigurationRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.SetSubClusterPolicyConfigurationResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterDeregisterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterDeregisterResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterHeartbeatRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterHeartbeatResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterRegisterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterRegisterResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterState;
+import org.apache.hadoop.yarn.server.federation.store.records.UpdateApplicationHomeSubClusterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.UpdateApplicationHomeSubClusterResponse;
+import org.apache.hadoop.yarn.server.federation.utils.FederationStateStoreFacade;
+import org.apache.hadoop.yarn.server.records.Version;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
+import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Implements {@link FederationStateStore} and provides a service for
+ * participating in the federation membership.
+ */
+public class FederationStateStoreService extends AbstractService
+    implements FederationStateStore {
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(FederationStateStoreService.class);
+
+  private Configuration config;
+  private ScheduledExecutorService scheduledExecutorService;
+  private FederationStateStoreHeartbeat stateStoreHeartbeat;
+  private FederationStateStore stateStoreClient = null;
+  private SubClusterId subClusterId;
+  private long heartbeatInterval;
+  private RMContext rmContext;
+
+  public FederationStateStoreService(RMContext rmContext) {
+    super(FederationStateStoreService.class.getName());
+    LOG.info("FederationStateStoreService initialized");
+    this.rmContext = rmContext;
+  }
+
+  @Override
+  protected void serviceInit(Configuration conf) throws Exception {
+
+    this.config = conf;
+
+    RetryPolicy retryPolicy =
+        FederationStateStoreFacade.createRetryPolicy(conf);
+
+    this.stateStoreClient =
+        (FederationStateStore) FederationStateStoreFacade.createRetryInstance(
+            conf, YarnConfiguration.FEDERATION_STATESTORE_CLIENT_CLASS,
+            YarnConfiguration.DEFAULT_FEDERATION_STATESTORE_CLIENT_CLASS,
+            FederationStateStore.class, retryPolicy);
+    this.stateStoreClient.init(conf);
+    LOG.info("Initialized state store client class");
+
+    this.subClusterId =
+        SubClusterId.newInstance(YarnConfiguration.getClusterId(conf));
+
+    heartbeatInterval = conf.getLong(
+        YarnConfiguration.FEDERATION_STATESTORE_HEARTBEAT_INTERVAL_SECS,
+        YarnConfiguration.DEFAULT_FEDERATION_STATESTORE_HEARTBEAT_INTERVAL_SECS);
+    if (heartbeatInterval <= 0) {
+      heartbeatInterval =
+          YarnConfiguration.DEFAULT_FEDERATION_STATESTORE_HEARTBEAT_INTERVAL_SECS;
+    }
+    LOG.info("Initialized federation membership service.");
+
+    super.serviceInit(conf);
+  }
+
+  @Override
+  protected void serviceStart() throws Exception {
+
+    registerAndInitializeHeartbeat();
+
+    super.serviceStart();
+  }
+
+  @Override
+  protected void serviceStop() throws Exception {
+    Exception ex = null;
+    try {
+      if (this.scheduledExecutorService != null
+          && !this.scheduledExecutorService.isShutdown()) {
+        this.scheduledExecutorService.shutdown();
+        LOG.info("Stopped federation membership heartbeat");
+      }
+    } catch (Exception e) {
+      LOG.error("Failed to shutdown ScheduledExecutorService", e);
+      ex = e;
+    }
+
+    if (this.stateStoreClient != null) {
+      try {
+        deregisterSubCluster(SubClusterDeregisterRequest
+            .newInstance(subClusterId, SubClusterState.SC_UNREGISTERED));
+      } finally {
+        this.stateStoreClient.close();
+      }
+    }
+
+    if (ex != null) {
+      throw ex;
+    }
+  }
+
+  // Return a client accessible string representation of the service address.
+  private String getServiceAddress(InetSocketAddress address) {
+    InetSocketAddress socketAddress = NetUtils.getConnectAddress(address);
+    return socketAddress.getAddress().getHostAddress() + ":"
+        + socketAddress.getPort();
+  }
+
+  private void registerAndInitializeHeartbeat() {
+    String clientRMAddress =
+        getServiceAddress(rmContext.getClientRMService().getBindAddress());
+    String amRMAddress = getServiceAddress(
+        rmContext.getApplicationMasterService().getBindAddress());
+    String rmAdminAddress = getServiceAddress(
+        config.getSocketAddr(YarnConfiguration.RM_ADMIN_ADDRESS,
+            YarnConfiguration.DEFAULT_RM_ADMIN_ADDRESS,
+            YarnConfiguration.DEFAULT_RM_ADMIN_PORT));
+    String webAppAddress =
+        WebAppUtils.getResolvedRemoteRMWebAppURLWithoutScheme(config);
+
+    SubClusterInfo subClusterInfo = SubClusterInfo.newInstance(subClusterId,
+        amRMAddress, clientRMAddress, rmAdminAddress, webAppAddress,
+        SubClusterState.SC_NEW, ResourceManager.getClusterTimeStamp(), "");
+    try {
+      registerSubCluster(SubClusterRegisterRequest.newInstance(subClusterInfo));
+      LOG.info("Successfully registered for federation subcluster: {}",
+          subClusterInfo);
+    } catch (Exception e) {
+      throw new YarnRuntimeException(
+          "Failed to register Federation membership with the StateStore", e);
+    }
+    stateStoreHeartbeat = new FederationStateStoreHeartbeat(subClusterId,
+        stateStoreClient, rmContext.getScheduler());
+    scheduledExecutorService =
+        HadoopExecutors.newSingleThreadScheduledExecutor();
+    scheduledExecutorService.scheduleWithFixedDelay(stateStoreHeartbeat,
+        heartbeatInterval, heartbeatInterval, TimeUnit.SECONDS);
+    LOG.info("Started federation membership heartbeat with interval: {}",
+        heartbeatInterval);
+  }
+
+  @VisibleForTesting
+  public FederationStateStore getStateStoreClient() {
+    return stateStoreClient;
+  }
+
+  @VisibleForTesting
+  public FederationStateStoreHeartbeat getStateStoreHeartbeatThread() {
+    return stateStoreHeartbeat;
+  }
+
+  @Override
+  public Version getCurrentVersion() {
+    return stateStoreClient.getCurrentVersion();
+  }
+
+  @Override
+  public Version loadVersion() {
+    return stateStoreClient.getCurrentVersion();
+  }
+
+  @Override
+  public GetSubClusterPolicyConfigurationResponse getPolicyConfiguration(
+      GetSubClusterPolicyConfigurationRequest request) throws YarnException {
+    return stateStoreClient.getPolicyConfiguration(request);
+  }
+
+  @Override
+  public SetSubClusterPolicyConfigurationResponse setPolicyConfiguration(
+      SetSubClusterPolicyConfigurationRequest request) throws YarnException {
+    return stateStoreClient.setPolicyConfiguration(request);
+  }
+
+  @Override
+  public GetSubClusterPoliciesConfigurationsResponse getPoliciesConfigurations(
+      GetSubClusterPoliciesConfigurationsRequest request) throws YarnException {
+    return stateStoreClient.getPoliciesConfigurations(request);
+  }
+
+  @Override
+  public SubClusterRegisterResponse registerSubCluster(
+      SubClusterRegisterRequest registerSubClusterRequest)
+      throws YarnException {
+    return stateStoreClient.registerSubCluster(registerSubClusterRequest);
+  }
+
+  @Override
+  public SubClusterDeregisterResponse deregisterSubCluster(
+      SubClusterDeregisterRequest subClusterDeregisterRequest)
+      throws YarnException {
+    return stateStoreClient.deregisterSubCluster(subClusterDeregisterRequest);
+  }
+
+  @Override
+  public SubClusterHeartbeatResponse subClusterHeartbeat(
+      SubClusterHeartbeatRequest subClusterHeartbeatRequest)
+      throws YarnException {
+    return stateStoreClient.subClusterHeartbeat(subClusterHeartbeatRequest);
+  }
+
+  @Override
+  public GetSubClusterInfoResponse getSubCluster(
+      GetSubClusterInfoRequest subClusterRequest) throws YarnException {
+    return stateStoreClient.getSubCluster(subClusterRequest);
+  }
+
+  @Override
+  public GetSubClustersInfoResponse getSubClusters(
+      GetSubClustersInfoRequest subClustersRequest) throws YarnException {
+    return stateStoreClient.getSubClusters(subClustersRequest);
+  }
+
+  @Override
+  public AddApplicationHomeSubClusterResponse addApplicationHomeSubCluster(
+      AddApplicationHomeSubClusterRequest request) throws YarnException {
+    return stateStoreClient.addApplicationHomeSubCluster(request);
+  }
+
+  @Override
+  public UpdateApplicationHomeSubClusterResponse updateApplicationHomeSubCluster(
+      UpdateApplicationHomeSubClusterRequest request) throws YarnException {
+    return stateStoreClient.updateApplicationHomeSubCluster(request);
+  }
+
+  @Override
+  public GetApplicationHomeSubClusterResponse getApplicationHomeSubCluster(
+      GetApplicationHomeSubClusterRequest request) throws YarnException {
+    return stateStoreClient.getApplicationHomeSubCluster(request);
+  }
+
+  @Override
+  public GetApplicationsHomeSubClusterResponse getApplicationsHomeSubCluster(
+      GetApplicationsHomeSubClusterRequest request) throws YarnException {
+    return stateStoreClient.getApplicationsHomeSubCluster(request);
+  }
+
+  @Override
+  public DeleteApplicationHomeSubClusterResponse deleteApplicationHomeSubCluster(
+      DeleteApplicationHomeSubClusterRequest request) throws YarnException {
+    return stateStoreClient.deleteApplicationHomeSubCluster(request);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3d508f0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/federation/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/federation/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/federation/package-info.java
new file mode 100644
index 0000000..47c7c65
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/federation/package-info.java
@@ -0,0 +1,17 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.yarn.server.resourcemanager.federation;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3d508f0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ClusterMetricsInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ClusterMetricsInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ClusterMetricsInfo.java
index f083b05..dc42eb6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ClusterMetricsInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ClusterMetricsInfo.java
@@ -65,7 +65,10 @@ public class ClusterMetricsInfo {
   } // JAXB needs this
 
   public ClusterMetricsInfo(final ResourceManager rm) {
-    ResourceScheduler rs = rm.getResourceScheduler();
+    this(rm.getResourceScheduler());
+  }
+
+  public ClusterMetricsInfo(final ResourceScheduler rs) {
     QueueMetrics metrics = rs.getRootQueueMetrics();
     ClusterMetrics clusterMetrics = ClusterMetrics.getMetrics();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3d508f0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/federation/TestFederationRMStateStoreService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/federation/TestFederationRMStateStoreService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/federation/TestFederationRMStateStoreService.java
new file mode 100644
index 0000000..30f69b5
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/federation/TestFederationRMStateStoreService.java
@@ -0,0 +1,170 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.federation;
+
+import java.io.IOException;
+import java.io.StringReader;
+
+import javax.xml.bind.JAXBException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ha.HAServiceProtocol;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.federation.store.FederationStateStore;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterInfoRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterDeregisterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterState;
+import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ClusterMetricsInfo;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.sun.jersey.api.json.JSONConfiguration;
+import com.sun.jersey.api.json.JSONJAXBContext;
+import com.sun.jersey.api.json.JSONUnmarshaller;
+
+/**
+ * Unit tests for FederationStateStoreService.
+ */
+public class TestFederationRMStateStoreService {
+
+  private final HAServiceProtocol.StateChangeRequestInfo requestInfo =
+      new HAServiceProtocol.StateChangeRequestInfo(
+          HAServiceProtocol.RequestSource.REQUEST_BY_USER);
+  private final SubClusterId subClusterId = SubClusterId.newInstance("SC-1");
+  private final GetSubClusterInfoRequest request =
+      GetSubClusterInfoRequest.newInstance(subClusterId);
+
+  private Configuration conf;
+  private FederationStateStore stateStore;
+  private long lastHearbeatTS = 0;
+  private JSONJAXBContext jc;
+  private JSONUnmarshaller unmarshaller;
+
+  @Before
+  public void setUp() throws IOException, YarnException, JAXBException {
+    conf = new YarnConfiguration();
+    jc = new JSONJAXBContext(
+        JSONConfiguration.mapped().rootUnwrapping(false).build(),
+        ClusterMetricsInfo.class);
+    unmarshaller = jc.createJSONUnmarshaller();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    unmarshaller = null;
+    jc = null;
+  }
+
+  @Test
+  public void testFederationStateStoreService() throws Exception {
+    conf.setBoolean(YarnConfiguration.FEDERATION_ENABLED, true);
+    conf.set(YarnConfiguration.RM_CLUSTER_ID, subClusterId.getId());
+    final MockRM rm = new MockRM(conf);
+
+    // Initially there should be no entry for the sub-cluster
+    rm.init(conf);
+    stateStore = rm.getFederationStateStoreService().getStateStoreClient();
+    try {
+      stateStore.getSubCluster(request);
+      Assert.fail("There should be no entry for the sub-cluster.");
+    } catch (YarnException e) {
+      Assert.assertTrue(e.getMessage().endsWith("does not exist"));
+    }
+
+    // Validate if sub-cluster is registered
+    rm.start();
+    String capability = checkSubClusterInfo(SubClusterState.SC_NEW);
+    Assert.assertTrue(capability.isEmpty());
+
+    // Heartbeat to see if sub-cluster transitions to running
+    FederationStateStoreHeartbeat storeHeartbeat =
+        rm.getFederationStateStoreService().getStateStoreHeartbeatThread();
+    storeHeartbeat.run();
+    capability = checkSubClusterInfo(SubClusterState.SC_RUNNING);
+    checkClusterMetricsInfo(capability, 0);
+
+    // heartbeat again after adding a node.
+    rm.registerNode("127.0.0.1:1234", 4 * 1024);
+    storeHeartbeat.run();
+    capability = checkSubClusterInfo(SubClusterState.SC_RUNNING);
+    checkClusterMetricsInfo(capability, 1);
+
+    // Validate sub-cluster deregistration
+    rm.getFederationStateStoreService()
+        .deregisterSubCluster(SubClusterDeregisterRequest
+            .newInstance(subClusterId, SubClusterState.SC_UNREGISTERED));
+    checkSubClusterInfo(SubClusterState.SC_UNREGISTERED);
+
+    // check after failover
+    explicitFailover(rm);
+
+    capability = checkSubClusterInfo(SubClusterState.SC_NEW);
+    Assert.assertTrue(capability.isEmpty());
+
+    // Heartbeat to see if sub-cluster transitions to running
+    storeHeartbeat =
+        rm.getFederationStateStoreService().getStateStoreHeartbeatThread();
+    storeHeartbeat.run();
+    capability = checkSubClusterInfo(SubClusterState.SC_RUNNING);
+    checkClusterMetricsInfo(capability, 0);
+
+    // heartbeat again after adding a node.
+    rm.registerNode("127.0.0.1:1234", 4 * 1024);
+    storeHeartbeat.run();
+    capability = checkSubClusterInfo(SubClusterState.SC_RUNNING);
+    checkClusterMetricsInfo(capability, 1);
+
+    rm.stop();
+  }
+
+  private void explicitFailover(MockRM rm) throws IOException {
+    rm.getAdminService().transitionToStandby(requestInfo);
+    Assert.assertTrue(rm.getRMContext()
+        .getHAServiceState() == HAServiceProtocol.HAServiceState.STANDBY);
+    rm.getAdminService().transitionToActive(requestInfo);
+    Assert.assertTrue(rm.getRMContext()
+        .getHAServiceState() == HAServiceProtocol.HAServiceState.ACTIVE);
+    lastHearbeatTS = 0;
+    stateStore = rm.getFederationStateStoreService().getStateStoreClient();
+  }
+
+  private void checkClusterMetricsInfo(String capability, int numNodes)
+      throws JAXBException {
+    ClusterMetricsInfo clusterMetricsInfo = unmarshaller.unmarshalFromJSON(
+        new StringReader(capability), ClusterMetricsInfo.class);
+    Assert.assertEquals(numNodes, clusterMetricsInfo.getTotalNodes());
+  }
+
+  private String checkSubClusterInfo(SubClusterState state)
+      throws YarnException {
+    Assert.assertNotNull(stateStore.getSubCluster(request));
+    SubClusterInfo response =
+        stateStore.getSubCluster(request).getSubClusterInfo();
+    Assert.assertEquals(state, response.getState());
+    Assert.assertTrue(response.getLastHeartBeat() >= lastHearbeatTS);
+    lastHearbeatTS = response.getLastHeartBeat();
+    return response.getCapability();
+  }
+
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[17/42] hadoop git commit: YARN-5407. In-memory based implementation of the FederationApplicationStateStore/FederationPolicyStateStore. (Ellen Hui via Subru)

Posted by su...@apache.org.
YARN-5407. In-memory based implementation of the FederationApplicationStateStore/FederationPolicyStateStore. (Ellen Hui via Subru)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/a82a0dfc
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/a82a0dfc
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/a82a0dfc

Branch: refs/heads/YARN-2915
Commit: a82a0dfc488c03020be38b9eb672db278d7873b3
Parents: 27b3ce0
Author: Subru Krishnan <su...@apache.org>
Authored: Tue Aug 9 16:07:55 2016 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon May 8 16:59:12 2017 -0700

----------------------------------------------------------------------
 .../store/impl/MemoryFederationStateStore.java  | 158 +++++++-
 ...SubClusterPoliciesConfigurationsRequest.java |   2 +-
 ...ubClusterPoliciesConfigurationsResponse.java |   2 +-
 ...GetSubClusterPolicyConfigurationRequest.java |   3 +-
 ...etSubClusterPolicyConfigurationResponse.java |   2 +-
 ...SetSubClusterPolicyConfigurationRequest.java |  20 +-
 ...etSubClusterPolicyConfigurationResponse.java |   2 +-
 .../records/SubClusterPolicyConfiguration.java  |  27 +-
 ...tApplicationHomeSubClusterRequestPBImpl.java |   4 +
 ...ClusterPolicyConfigurationRequestPBImpl.java |  17 -
 .../pb/SubClusterPolicyConfigurationPBImpl.java |  17 +
 .../proto/yarn_server_federation_protos.proto   |   8 +-
 .../impl/FederationStateStoreBaseTest.java      | 367 ++++++++++++++++++-
 .../impl/TestMemoryFederationStateStore.java    |   4 +-
 14 files changed, 558 insertions(+), 75 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a82a0dfc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/MemoryFederationStateStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/MemoryFederationStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/MemoryFederationStateStore.java
index cea4ac2..a540dff 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/MemoryFederationStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/MemoryFederationStateStore.java
@@ -20,35 +20,72 @@ package org.apache.hadoop.yarn.server.federation.store.impl;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.concurrent.ConcurrentHashMap;
 
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
-import org.apache.hadoop.yarn.server.federation.store.FederationMembershipStateStore;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyConfiguration;
+import org.apache.hadoop.yarn.server.federation.store.FederationStateStore;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterDeregisterRequest;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterDeregisterResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.AddApplicationHomeSubClusterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.AddApplicationHomeSubClusterResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.ApplicationHomeSubCluster;
+import org.apache.hadoop.yarn.server.federation.store.records.DeleteApplicationHomeSubClusterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.DeleteApplicationHomeSubClusterResponse;
+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.server.federation.store.records.GetApplicationsHomeSubClusterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.GetApplicationsHomeSubClusterResponse;
 import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterInfoRequest;
 import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterInfoResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterPoliciesConfigurationsRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterPoliciesConfigurationsResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterPolicyConfigurationRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterPolicyConfigurationResponse;
 import org.apache.hadoop.yarn.server.federation.store.records.GetSubClustersInfoRequest;
 import org.apache.hadoop.yarn.server.federation.store.records.GetSubClustersInfoResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.SetSubClusterPolicyConfigurationRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.SetSubClusterPolicyConfigurationResponse;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterHeartbeatRequest;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterHeartbeatResponse;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterRegisterRequest;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterRegisterResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.UpdateApplicationHomeSubClusterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.UpdateApplicationHomeSubClusterResponse;
+import org.apache.hadoop.yarn.server.records.Version;
 import org.apache.hadoop.yarn.util.MonotonicClock;
 
 /**
- * In-memory implementation of FederationMembershipStateStore.
+ * In-memory implementation of {@link FederationStateStore}.
  */
-public class MemoryFederationStateStore
-    implements FederationMembershipStateStore {
+public class MemoryFederationStateStore implements FederationStateStore {
+
+  private Map<SubClusterId, SubClusterInfo> membership;
+  private Map<ApplicationId, SubClusterId> applications;
+  private Map<String, SubClusterPolicyConfiguration> policies;
 
-  private final Map<SubClusterId, SubClusterInfo> membership =
-      new ConcurrentHashMap<SubClusterId, SubClusterInfo>();
   private final MonotonicClock clock = new MonotonicClock();
 
   @Override
+  public void init(Configuration conf) {
+    membership = new ConcurrentHashMap<SubClusterId, SubClusterInfo>();
+    applications = new ConcurrentHashMap<ApplicationId, SubClusterId>();
+    policies = new ConcurrentHashMap<String, SubClusterPolicyConfiguration>();
+  }
+
+  @Override
+  public void close() {
+    membership = null;
+    applications = null;
+    policies = null;
+  }
+
+  @Override
   public SubClusterRegisterResponse registerSubCluster(
       SubClusterRegisterRequest request) throws YarnException {
     SubClusterInfo subClusterInfo = request.getSubClusterInfo();
@@ -116,4 +153,113 @@ public class MemoryFederationStateStore
     return GetSubClustersInfoResponse.newInstance(result);
   }
 
+  // FederationApplicationHomeSubClusterStore methods
+
+  @Override
+  public AddApplicationHomeSubClusterResponse addApplicationHomeSubClusterMap(
+      AddApplicationHomeSubClusterRequest request) throws YarnException {
+    ApplicationId appId =
+        request.getApplicationHomeSubCluster().getApplicationId();
+    if (applications.containsKey(appId)) {
+      throw new YarnException("Application " + appId + " already exists");
+    }
+
+    applications.put(appId,
+        request.getApplicationHomeSubCluster().getHomeSubCluster());
+    return AddApplicationHomeSubClusterResponse.newInstance();
+  }
+
+  @Override
+  public UpdateApplicationHomeSubClusterResponse updateApplicationHomeSubClusterMap(
+      UpdateApplicationHomeSubClusterRequest request) throws YarnException {
+    ApplicationId appId =
+        request.getApplicationHomeSubCluster().getApplicationId();
+    if (!applications.containsKey(appId)) {
+      throw new YarnException("Application " + appId + " does not exist");
+    }
+
+    applications.put(appId,
+        request.getApplicationHomeSubCluster().getHomeSubCluster());
+    return UpdateApplicationHomeSubClusterResponse.newInstance();
+  }
+
+  @Override
+  public GetApplicationHomeSubClusterResponse getApplicationHomeSubClusterMap(
+      GetApplicationHomeSubClusterRequest request) throws YarnException {
+    ApplicationId appId = request.getApplicationId();
+    if (!applications.containsKey(appId)) {
+      throw new YarnException("Application " + appId + " does not exist");
+    }
+
+    return GetApplicationHomeSubClusterResponse.newInstance(
+        ApplicationHomeSubCluster.newInstance(appId, applications.get(appId)));
+  }
+
+  @Override
+  public GetApplicationsHomeSubClusterResponse getApplicationsHomeSubClusterMap(
+      GetApplicationsHomeSubClusterRequest request) throws YarnException {
+    List<ApplicationHomeSubCluster> result =
+        new ArrayList<ApplicationHomeSubCluster>();
+    for (Entry<ApplicationId, SubClusterId> e : applications.entrySet()) {
+      result
+          .add(ApplicationHomeSubCluster.newInstance(e.getKey(), e.getValue()));
+    }
+
+    GetApplicationsHomeSubClusterResponse.newInstance(result);
+    return GetApplicationsHomeSubClusterResponse.newInstance(result);
+  }
+
+  @Override
+  public DeleteApplicationHomeSubClusterResponse deleteApplicationHomeSubClusterMap(
+      DeleteApplicationHomeSubClusterRequest request) throws YarnException {
+    ApplicationId appId = request.getApplicationId();
+    if (!applications.containsKey(appId)) {
+      throw new YarnException("Application " + appId + " does not exist");
+    }
+
+    applications.remove(appId);
+    return DeleteApplicationHomeSubClusterResponse.newInstance();
+  }
+
+  @Override
+  public GetSubClusterPolicyConfigurationResponse getPolicyConfiguration(
+      GetSubClusterPolicyConfigurationRequest request) throws YarnException {
+    String queue = request.getQueue();
+    if (!policies.containsKey(queue)) {
+      throw new YarnException("Policy for queue " + queue + " does not exist");
+    }
+
+    return GetSubClusterPolicyConfigurationResponse
+        .newInstance(policies.get(queue));
+  }
+
+  @Override
+  public SetSubClusterPolicyConfigurationResponse setPolicyConfiguration(
+      SetSubClusterPolicyConfigurationRequest request) throws YarnException {
+    policies.put(request.getPolicyConfiguration().getQueue(),
+        request.getPolicyConfiguration());
+    return SetSubClusterPolicyConfigurationResponse.newInstance();
+  }
+
+  @Override
+  public GetSubClusterPoliciesConfigurationsResponse getPoliciesConfigurations(
+      GetSubClusterPoliciesConfigurationsRequest request) throws YarnException {
+    ArrayList<SubClusterPolicyConfiguration> result =
+        new ArrayList<SubClusterPolicyConfiguration>();
+    for (SubClusterPolicyConfiguration policy : policies.values()) {
+      result.add(policy);
+    }
+    return GetSubClusterPoliciesConfigurationsResponse.newInstance(result);
+  }
+
+  @Override
+  public Version getCurrentVersion() {
+    return null;
+  }
+
+  @Override
+  public Version loadVersion() {
+    return null;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a82a0dfc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetSubClusterPoliciesConfigurationsRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetSubClusterPoliciesConfigurationsRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetSubClusterPoliciesConfigurationsRequest.java
index 404521b..8cb84f3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetSubClusterPoliciesConfigurationsRequest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetSubClusterPoliciesConfigurationsRequest.java
@@ -29,7 +29,7 @@ import org.apache.hadoop.yarn.util.Records;
 @Private
 @Unstable
 public abstract class GetSubClusterPoliciesConfigurationsRequest {
-  public GetSubClusterPoliciesConfigurationsRequest newInstance() {
+  public static GetSubClusterPoliciesConfigurationsRequest newInstance() {
     return Records.newRecord(GetSubClusterPoliciesConfigurationsRequest.class);
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a82a0dfc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetSubClusterPoliciesConfigurationsResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetSubClusterPoliciesConfigurationsResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetSubClusterPoliciesConfigurationsResponse.java
index 6554d68..2eaeb51 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetSubClusterPoliciesConfigurationsResponse.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetSubClusterPoliciesConfigurationsResponse.java
@@ -36,7 +36,7 @@ public abstract class GetSubClusterPoliciesConfigurationsResponse {
 
   @Private
   @Unstable
-  public GetSubClusterPoliciesConfigurationsResponse newInstance(
+  public static GetSubClusterPoliciesConfigurationsResponse newInstance(
       List<SubClusterPolicyConfiguration> policyConfigurations) {
     GetSubClusterPoliciesConfigurationsResponse response =
         Records.newRecord(GetSubClusterPoliciesConfigurationsResponse.class);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a82a0dfc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetSubClusterPolicyConfigurationRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetSubClusterPolicyConfigurationRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetSubClusterPolicyConfigurationRequest.java
index 7b7d8c4..c3f49e1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetSubClusterPolicyConfigurationRequest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetSubClusterPolicyConfigurationRequest.java
@@ -33,7 +33,8 @@ public abstract class GetSubClusterPolicyConfigurationRequest {
 
   @Private
   @Unstable
-  public GetSubClusterPolicyConfigurationRequest newInstance(String queueName) {
+  public static GetSubClusterPolicyConfigurationRequest newInstance(
+      String queueName) {
     GetSubClusterPolicyConfigurationRequest request =
         Records.newRecord(GetSubClusterPolicyConfigurationRequest.class);
     request.setQueue(queueName);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a82a0dfc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetSubClusterPolicyConfigurationResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetSubClusterPolicyConfigurationResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetSubClusterPolicyConfigurationResponse.java
index 11a46e0..350b239 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetSubClusterPolicyConfigurationResponse.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetSubClusterPolicyConfigurationResponse.java
@@ -34,7 +34,7 @@ public abstract class GetSubClusterPolicyConfigurationResponse {
 
   @Private
   @Unstable
-  public GetSubClusterPolicyConfigurationResponse newInstance(
+  public static GetSubClusterPolicyConfigurationResponse newInstance(
       SubClusterPolicyConfiguration policy) {
     GetSubClusterPolicyConfigurationResponse response =
         Records.newRecord(GetSubClusterPolicyConfigurationResponse.class);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a82a0dfc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SetSubClusterPolicyConfigurationRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SetSubClusterPolicyConfigurationRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SetSubClusterPolicyConfigurationRequest.java
index 06d5399..743ad0e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SetSubClusterPolicyConfigurationRequest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SetSubClusterPolicyConfigurationRequest.java
@@ -32,7 +32,7 @@ import org.apache.hadoop.yarn.util.Records;
 public abstract class SetSubClusterPolicyConfigurationRequest {
   @Private
   @Unstable
-  public SetSubClusterPolicyConfigurationRequest newInstance(
+  public static SetSubClusterPolicyConfigurationRequest newInstance(
       SubClusterPolicyConfiguration policy) {
     SetSubClusterPolicyConfigurationRequest request =
         Records.newRecord(SetSubClusterPolicyConfigurationRequest.class);
@@ -41,24 +41,6 @@ public abstract class SetSubClusterPolicyConfigurationRequest {
   }
 
   /**
-   * Get the name of the queue for which we are configuring a policy.
-   *
-   * @return the name of the queue
-   */
-  @Public
-  @Unstable
-  public abstract String getQueue();
-
-  /**
-   * Sets the name of the queue for which we are configuring a policy.
-   *
-   * @param queueName the name of the queue
-   */
-  @Private
-  @Unstable
-  public abstract void setQueue(String queueName);
-
-  /**
    * Get the policy configuration assigned to the queue.
    *
    * @return the policy for the specified queue

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a82a0dfc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SetSubClusterPolicyConfigurationResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SetSubClusterPolicyConfigurationResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SetSubClusterPolicyConfigurationResponse.java
index 33c4043..401e984 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SetSubClusterPolicyConfigurationResponse.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SetSubClusterPolicyConfigurationResponse.java
@@ -30,7 +30,7 @@ import org.apache.hadoop.yarn.util.Records;
 @Private
 @Unstable
 public abstract class SetSubClusterPolicyConfigurationResponse {
-  public SetSubClusterPolicyConfigurationResponse newInstance() {
+  public static SetSubClusterPolicyConfigurationResponse newInstance() {
     return Records.newRecord(SetSubClusterPolicyConfigurationResponse.class);
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a82a0dfc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterPolicyConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterPolicyConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterPolicyConfiguration.java
index bc12acb..2839139 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterPolicyConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterPolicyConfiguration.java
@@ -29,8 +29,8 @@ import java.nio.ByteBuffer;
 
 /**
  * {@link SubClusterPolicyConfiguration} is a class that represents a
- * configuration of a policy. It contains a policy type (resolve to a class
- * name) and its params as an opaque {@link ByteBuffer}.
+ * configuration of a policy. For a single queue, it contains a policy type
+ * (resolve to a class name) and its params as an opaque {@link ByteBuffer}.
  *
  * Note: by design the params are an opaque ByteBuffer, this allows for enough
  * flexibility to evolve the policies without impacting the protocols to/from
@@ -42,16 +42,35 @@ public abstract class SubClusterPolicyConfiguration {
 
   @Private
   @Unstable
-  public static SubClusterPolicyConfiguration newInstance(String policyType,
-      ByteBuffer policyParams) {
+  public static SubClusterPolicyConfiguration newInstance(String queue,
+      String policyType, ByteBuffer policyParams) {
     SubClusterPolicyConfiguration policy =
         Records.newRecord(SubClusterPolicyConfiguration.class);
+    policy.setQueue(queue);
     policy.setType(policyType);
     policy.setParams(policyParams);
     return policy;
   }
 
   /**
+   * Get the name of the queue for which we are configuring a policy.
+   *
+   * @return the name of the queue
+   */
+  @Public
+  @Unstable
+  public abstract String getQueue();
+
+  /**
+   * Sets the name of the queue for which we are configuring a policy.
+   *
+   * @param queueName the name of the queue
+   */
+  @Private
+  @Unstable
+  public abstract void setQueue(String queueName);
+
+  /**
    * Get the type of the policy. This could be random, round-robin, load-based,
    * etc.
    *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a82a0dfc/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/GetApplicationHomeSubClusterRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/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/GetApplicationHomeSubClusterRequestPBImpl.java b/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/GetApplicationHomeSubClusterRequestPBImpl.java
index 865d0c4..585ba81 100644
--- a/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/GetApplicationHomeSubClusterRequestPBImpl.java
+++ b/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/GetApplicationHomeSubClusterRequestPBImpl.java
@@ -108,6 +108,10 @@ public class GetApplicationHomeSubClusterRequestPBImpl
   public ApplicationId getApplicationId() {
     GetApplicationHomeSubClusterRequestProtoOrBuilder p =
         viaProto ? proto : builder;
+    if (applicationId != null) {
+      return applicationId;
+    }
+
     if (!p.hasApplicationId()) {
       return null;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a82a0dfc/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/SetSubClusterPolicyConfigurationRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/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/SetSubClusterPolicyConfigurationRequestPBImpl.java b/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/SetSubClusterPolicyConfigurationRequestPBImpl.java
index 5e29bd5..7b7f89d 100644
--- a/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/SetSubClusterPolicyConfigurationRequestPBImpl.java
+++ b/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/SetSubClusterPolicyConfigurationRequestPBImpl.java
@@ -107,23 +107,6 @@ public class SetSubClusterPolicyConfigurationRequestPBImpl
   }
 
   @Override
-  public String getQueue() {
-    SetSubClusterPolicyConfigurationRequestProtoOrBuilder p =
-        viaProto ? proto : builder;
-    return p.getQueue();
-  }
-
-  @Override
-  public void setQueue(String queueName) {
-    maybeInitBuilder();
-    if (queueName == null) {
-      builder.clearQueue();
-      return;
-    }
-    builder.setQueue(queueName);
-  }
-
-  @Override
   public SubClusterPolicyConfiguration getPolicyConfiguration() {
     SetSubClusterPolicyConfigurationRequestProtoOrBuilder p =
         viaProto ? proto : builder;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a82a0dfc/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/SubClusterPolicyConfigurationPBImpl.java
----------------------------------------------------------------------
diff --git a/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/SubClusterPolicyConfigurationPBImpl.java b/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/SubClusterPolicyConfigurationPBImpl.java
index fe9d9db..305a8d3 100644
--- a/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/SubClusterPolicyConfigurationPBImpl.java
+++ b/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/SubClusterPolicyConfigurationPBImpl.java
@@ -87,6 +87,23 @@ public class SubClusterPolicyConfigurationPBImpl
   }
 
   @Override
+  public String getQueue() {
+    SubClusterPolicyConfigurationProtoOrBuilder p = viaProto ? proto : builder;
+    return p.getQueue();
+  }
+
+  @Override
+  public void setQueue(String queueName) {
+    maybeInitBuilder();
+    if (queueName == null) {
+      builder.clearType();
+      return;
+    }
+    builder.setQueue(queueName);
+
+  }
+
+  @Override
   public String getType() {
     SubClusterPolicyConfigurationProtoOrBuilder p = viaProto ? proto : builder;
     return p.getType();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a82a0dfc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_federation_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_federation_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_federation_protos.proto
index 3f1cee9..11f786f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_federation_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_federation_protos.proto
@@ -136,8 +136,9 @@ message DeleteApplicationHomeSubClusterResponseProto {
 }
 
 message SubClusterPolicyConfigurationProto {
-  optional string type = 1;
-  optional bytes params = 2;
+  optional string queue = 1;
+  optional string type = 2;
+  optional bytes params = 3;
 }
 
 message GetSubClusterPolicyConfigurationRequestProto {
@@ -149,8 +150,7 @@ message GetSubClusterPolicyConfigurationResponseProto {
 }
 
 message SetSubClusterPolicyConfigurationRequestProto {
-  optional string queue = 1;
-  optional SubClusterPolicyConfigurationProto policy_configuration = 2;
+  optional SubClusterPolicyConfigurationProto policy_configuration = 1;
 }
 
 message SetSubClusterPolicyConfigurationResponseProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a82a0dfc/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
----------------------------------------------------------------------
diff --git a/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 b/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
index c76a485..165dd78 100644
--- a/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
+++ b/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
@@ -18,18 +18,39 @@
 package org.apache.hadoop.yarn.server.federation.store.impl;
 
 import java.io.IOException;
+import java.nio.ByteBuffer;
 
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.exceptions.YarnException;
-import org.apache.hadoop.yarn.server.federation.store.FederationMembershipStateStore;
+import org.apache.hadoop.yarn.server.federation.store.FederationStateStore;
+import org.apache.hadoop.yarn.server.federation.store.records.AddApplicationHomeSubClusterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.AddApplicationHomeSubClusterResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.ApplicationHomeSubCluster;
+import org.apache.hadoop.yarn.server.federation.store.records.DeleteApplicationHomeSubClusterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.DeleteApplicationHomeSubClusterResponse;
+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.server.federation.store.records.GetApplicationsHomeSubClusterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.GetApplicationsHomeSubClusterResponse;
 import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterInfoRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterPoliciesConfigurationsRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterPoliciesConfigurationsResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterPolicyConfigurationRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterPolicyConfigurationResponse;
 import org.apache.hadoop.yarn.server.federation.store.records.GetSubClustersInfoRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.SetSubClusterPolicyConfigurationRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.SetSubClusterPolicyConfigurationResponse;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterDeregisterRequest;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterHeartbeatRequest;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyConfiguration;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterRegisterRequest;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterRegisterResponse;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterState;
+import org.apache.hadoop.yarn.server.federation.store.records.UpdateApplicationHomeSubClusterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.UpdateApplicationHomeSubClusterResponse;
 import org.apache.hadoop.yarn.util.MonotonicClock;
 import org.junit.After;
 import org.junit.Assert;
@@ -42,20 +63,21 @@ import org.junit.Test;
 public abstract class FederationStateStoreBaseTest {
 
   private static final MonotonicClock CLOCK = new MonotonicClock();
+  private FederationStateStore stateStore = createStateStore();
 
-  private FederationMembershipStateStore stateStore;
+  protected abstract FederationStateStore createStateStore();
 
   @Before
-  public void before() throws IOException {
-    stateStore = getCleanStateStore();
+  public void before() throws IOException, YarnException {
+    stateStore.init(new Configuration());
   }
 
   @After
-  public void after() {
-    stateStore = null;
+  public void after() throws Exception {
+    stateStore.close();
   }
 
-  protected abstract FederationMembershipStateStore getCleanStateStore();
+  // Test FederationMembershipStateStore
 
   @Test
   public void testRegisterSubCluster() throws Exception {
@@ -72,10 +94,7 @@ public abstract class FederationStateStoreBaseTest {
   @Test
   public void testDeregisterSubCluster() throws Exception {
     SubClusterId subClusterId = SubClusterId.newInstance("SC");
-    SubClusterInfo subClusterInfo = createSubClusterInfo(subClusterId);
-
-    stateStore.registerSubCluster(
-        SubClusterRegisterRequest.newInstance(subClusterInfo));
+    registerSubCluster(subClusterId);
 
     SubClusterDeregisterRequest deregisterRequest = SubClusterDeregisterRequest
         .newInstance(subClusterId, SubClusterState.SC_UNREGISTERED);
@@ -105,9 +124,7 @@ public abstract class FederationStateStoreBaseTest {
 
     SubClusterId subClusterId = SubClusterId.newInstance("SC");
     SubClusterInfo subClusterInfo = createSubClusterInfo(subClusterId);
-
-    stateStore.registerSubCluster(
-        SubClusterRegisterRequest.newInstance(subClusterInfo));
+    registerSubCluster(subClusterId);
 
     GetSubClusterInfoRequest request =
         GetSubClusterInfoRequest.newInstance(subClusterId);
@@ -167,10 +184,7 @@ public abstract class FederationStateStoreBaseTest {
   @Test
   public void testSubClusterHeartbeat() throws Exception {
     SubClusterId subClusterId = SubClusterId.newInstance("SC");
-    SubClusterInfo subClusterInfo = createSubClusterInfo(subClusterId);
-
-    stateStore.registerSubCluster(
-        SubClusterRegisterRequest.newInstance(subClusterInfo));
+    registerSubCluster(subClusterId);
 
     SubClusterHeartbeatRequest heartbeatRequest = SubClusterHeartbeatRequest
         .newInstance(subClusterId, SubClusterState.SC_RUNNING, "cabability");
@@ -196,6 +210,271 @@ public abstract class FederationStateStoreBaseTest {
     }
   }
 
+  // Test FederationApplicationHomeSubClusterStore
+
+  @Test
+  public void testAddApplicationHomeSubClusterMap() throws Exception {
+    ApplicationId appId = ApplicationId.newInstance(1, 1);
+    SubClusterId subClusterId = SubClusterId.newInstance("SC");
+    ApplicationHomeSubCluster ahsc =
+        ApplicationHomeSubCluster.newInstance(appId, subClusterId);
+
+    AddApplicationHomeSubClusterRequest request =
+        AddApplicationHomeSubClusterRequest.newInstance(ahsc);
+    AddApplicationHomeSubClusterResponse response =
+        stateStore.addApplicationHomeSubClusterMap(request);
+
+    Assert.assertNotNull(response);
+    Assert.assertEquals(subClusterId, queryApplicationHomeSC(appId));
+
+  }
+
+  @Test
+  public void testAddApplicationHomeSubClusterMapAppAlreadyExists()
+      throws Exception {
+    ApplicationId appId = ApplicationId.newInstance(1, 1);
+    SubClusterId subClusterId1 = SubClusterId.newInstance("SC1");
+    addApplicationHomeSC(appId, subClusterId1);
+
+    SubClusterId subClusterId2 = SubClusterId.newInstance("SC2");
+    ApplicationHomeSubCluster ahsc2 =
+        ApplicationHomeSubCluster.newInstance(appId, subClusterId2);
+
+    try {
+      stateStore.addApplicationHomeSubClusterMap(
+          AddApplicationHomeSubClusterRequest.newInstance(ahsc2));
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertTrue(e.getMessage()
+          .startsWith("Application " + appId.toString() + " already exists"));
+    }
+
+    Assert.assertEquals(subClusterId1, queryApplicationHomeSC(appId));
+
+  }
+
+  @Test
+  public void testDeleteApplicationHomeSubClusterMap() throws Exception {
+    ApplicationId appId = ApplicationId.newInstance(1, 1);
+    SubClusterId subClusterId = SubClusterId.newInstance("SC");
+    addApplicationHomeSC(appId, subClusterId);
+
+    DeleteApplicationHomeSubClusterRequest delRequest =
+        DeleteApplicationHomeSubClusterRequest.newInstance(appId);
+
+    DeleteApplicationHomeSubClusterResponse response =
+        stateStore.deleteApplicationHomeSubClusterMap(delRequest);
+
+    Assert.assertNotNull(response);
+    try {
+      queryApplicationHomeSC(appId);
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertTrue(e.getMessage()
+          .startsWith("Application " + appId + " does not exist"));
+    }
+
+  }
+
+  @Test
+  public void testDeleteApplicationHomeSubClusterMapUnknownApp()
+      throws Exception {
+    ApplicationId appId = ApplicationId.newInstance(1, 1);
+    DeleteApplicationHomeSubClusterRequest delRequest =
+        DeleteApplicationHomeSubClusterRequest.newInstance(appId);
+
+    try {
+      stateStore.deleteApplicationHomeSubClusterMap(delRequest);
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertTrue(e.getMessage()
+          .startsWith("Application " + appId.toString() + " does not exist"));
+    }
+  }
+
+  @Test
+  public void testGetApplicationHomeSubClusterMap() throws Exception {
+    ApplicationId appId = ApplicationId.newInstance(1, 1);
+    SubClusterId subClusterId = SubClusterId.newInstance("SC");
+    addApplicationHomeSC(appId, subClusterId);
+
+    GetApplicationHomeSubClusterRequest getRequest =
+        GetApplicationHomeSubClusterRequest.newInstance(appId);
+
+    GetApplicationHomeSubClusterResponse result =
+        stateStore.getApplicationHomeSubClusterMap(getRequest);
+
+    Assert.assertEquals(appId,
+        result.getApplicationHomeSubCluster().getApplicationId());
+    Assert.assertEquals(subClusterId,
+        result.getApplicationHomeSubCluster().getHomeSubCluster());
+  }
+
+  @Test
+  public void testGetApplicationHomeSubClusterMapUnknownApp() throws Exception {
+    ApplicationId appId = ApplicationId.newInstance(1, 1);
+    GetApplicationHomeSubClusterRequest request =
+        GetApplicationHomeSubClusterRequest.newInstance(appId);
+
+    try {
+      stateStore.getApplicationHomeSubClusterMap(request);
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertTrue(e.getMessage()
+          .startsWith("Application " + appId.toString() + " does not exist"));
+    }
+  }
+
+  @Test
+  public void testGetApplicationsHomeSubClusterMap() throws Exception {
+    ApplicationId appId1 = ApplicationId.newInstance(1, 1);
+    SubClusterId subClusterId1 = SubClusterId.newInstance("SC1");
+    ApplicationHomeSubCluster ahsc1 =
+        ApplicationHomeSubCluster.newInstance(appId1, subClusterId1);
+
+    ApplicationId appId2 = ApplicationId.newInstance(1, 2);
+    SubClusterId subClusterId2 = SubClusterId.newInstance("SC2");
+    ApplicationHomeSubCluster ahsc2 =
+        ApplicationHomeSubCluster.newInstance(appId2, subClusterId2);
+
+    addApplicationHomeSC(appId1, subClusterId1);
+    addApplicationHomeSC(appId2, subClusterId2);
+
+    GetApplicationsHomeSubClusterRequest getRequest =
+        GetApplicationsHomeSubClusterRequest.newInstance();
+
+    GetApplicationsHomeSubClusterResponse result =
+        stateStore.getApplicationsHomeSubClusterMap(getRequest);
+
+    Assert.assertEquals(2, result.getAppsHomeSubClusters().size());
+    Assert.assertTrue(result.getAppsHomeSubClusters().contains(ahsc1));
+    Assert.assertTrue(result.getAppsHomeSubClusters().contains(ahsc2));
+  }
+
+  @Test
+  public void testUpdateApplicationHomeSubClusterMap() throws Exception {
+    ApplicationId appId = ApplicationId.newInstance(1, 1);
+    SubClusterId subClusterId1 = SubClusterId.newInstance("SC1");
+    addApplicationHomeSC(appId, subClusterId1);
+
+    SubClusterId subClusterId2 = SubClusterId.newInstance("SC2");
+    ApplicationHomeSubCluster ahscUpdate =
+        ApplicationHomeSubCluster.newInstance(appId, subClusterId2);
+
+    UpdateApplicationHomeSubClusterRequest updateRequest =
+        UpdateApplicationHomeSubClusterRequest.newInstance(ahscUpdate);
+
+    UpdateApplicationHomeSubClusterResponse response =
+        stateStore.updateApplicationHomeSubClusterMap(updateRequest);
+
+    Assert.assertNotNull(response);
+
+    Assert.assertEquals(subClusterId2, queryApplicationHomeSC(appId));
+  }
+
+  @Test
+  public void testUpdateApplicationHomeSubClusterMapUnknownApp()
+      throws Exception {
+    ApplicationId appId = ApplicationId.newInstance(1, 1);
+    SubClusterId subClusterId1 = SubClusterId.newInstance("SC1");
+    ApplicationHomeSubCluster ahsc =
+        ApplicationHomeSubCluster.newInstance(appId, subClusterId1);
+
+    UpdateApplicationHomeSubClusterRequest updateRequest =
+        UpdateApplicationHomeSubClusterRequest.newInstance(ahsc);
+
+    try {
+      stateStore.updateApplicationHomeSubClusterMap((updateRequest));
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertTrue(e.getMessage()
+          .startsWith("Application " + appId.toString() + " does not exist"));
+    }
+  }
+
+  // Test FederationPolicyStore
+
+  @Test
+  public void testSetPolicyConfiguration() throws Exception {
+    SetSubClusterPolicyConfigurationRequest request =
+        SetSubClusterPolicyConfigurationRequest
+            .newInstance(createSCPolicyConf("Queue", "PolicyType"));
+
+    SetSubClusterPolicyConfigurationResponse result =
+        stateStore.setPolicyConfiguration(request);
+
+    Assert.assertNotNull(result);
+    Assert.assertEquals(createSCPolicyConf("Queue", "PolicyType"),
+        queryPolicy("Queue"));
+
+  }
+
+  @Test
+  public void testSetPolicyConfigurationUpdateExisting() throws Exception {
+    setPolicyConf("Queue", "PolicyType1");
+
+    SetSubClusterPolicyConfigurationRequest request2 =
+        SetSubClusterPolicyConfigurationRequest
+            .newInstance(createSCPolicyConf("Queue", "PolicyType2"));
+    SetSubClusterPolicyConfigurationResponse result =
+        stateStore.setPolicyConfiguration(request2);
+
+    Assert.assertNotNull(result);
+    Assert.assertEquals(createSCPolicyConf("Queue", "PolicyType2"),
+        queryPolicy("Queue"));
+  }
+
+  @Test
+  public void testGetPolicyConfiguration() throws Exception {
+    setPolicyConf("Queue", "PolicyType");
+
+    GetSubClusterPolicyConfigurationRequest getRequest =
+        GetSubClusterPolicyConfigurationRequest.newInstance("Queue");
+    GetSubClusterPolicyConfigurationResponse result =
+        stateStore.getPolicyConfiguration(getRequest);
+
+    Assert.assertNotNull(result);
+    Assert.assertEquals(createSCPolicyConf("Queue", "PolicyType"),
+        result.getPolicyConfiguration());
+
+  }
+
+  @Test
+  public void testGetPolicyConfigurationUnknownQueue() throws Exception {
+
+    GetSubClusterPolicyConfigurationRequest request =
+        GetSubClusterPolicyConfigurationRequest.newInstance("Queue");
+    try {
+      stateStore.getPolicyConfiguration(request);
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertTrue(
+          e.getMessage().startsWith("Policy for queue Queue does not exist"));
+    }
+  }
+
+  @Test
+  public void testGetPoliciesConfigurations() throws Exception {
+    setPolicyConf("Queue1", "PolicyType1");
+    setPolicyConf("Queue2", "PolicyType2");
+
+    GetSubClusterPoliciesConfigurationsResponse response =
+        stateStore.getPoliciesConfigurations(
+            GetSubClusterPoliciesConfigurationsRequest.newInstance());
+
+    Assert.assertNotNull(response);
+    Assert.assertNotNull(response.getPoliciesConfigs());
+
+    Assert.assertEquals(2, response.getPoliciesConfigs().size());
+
+    Assert.assertTrue(response.getPoliciesConfigs()
+        .contains(createSCPolicyConf("Queue1", "PolicyType1")));
+    Assert.assertTrue(response.getPoliciesConfigs()
+        .contains(createSCPolicyConf("Queue2", "PolicyType2")));
+  }
+
+  // Convenience methods
+
   private SubClusterInfo createSubClusterInfo(SubClusterId subClusterId) {
 
     String amRMAddress = "1.2.3.4:1";
@@ -208,6 +487,37 @@ public abstract class FederationStateStoreBaseTest {
         CLOCK.getTime(), "cabability");
   }
 
+  private SubClusterPolicyConfiguration createSCPolicyConf(String queueName,
+      String policyType) {
+    return SubClusterPolicyConfiguration.newInstance(queueName, policyType,
+        ByteBuffer.allocate(1));
+  }
+
+  private void addApplicationHomeSC(ApplicationId appId,
+      SubClusterId subClusterId) throws YarnException {
+    ApplicationHomeSubCluster ahsc =
+        ApplicationHomeSubCluster.newInstance(appId, subClusterId);
+    AddApplicationHomeSubClusterRequest request =
+        AddApplicationHomeSubClusterRequest.newInstance(ahsc);
+    stateStore.addApplicationHomeSubClusterMap(request);
+  }
+
+  private void setPolicyConf(String queue, String policyType)
+      throws YarnException {
+    SetSubClusterPolicyConfigurationRequest request =
+        SetSubClusterPolicyConfigurationRequest
+            .newInstance(createSCPolicyConf(queue, policyType));
+    stateStore.setPolicyConfiguration(request);
+  }
+
+  private void registerSubCluster(SubClusterId subClusterId)
+      throws YarnException {
+
+    SubClusterInfo subClusterInfo = createSubClusterInfo(subClusterId);
+    stateStore.registerSubCluster(
+        SubClusterRegisterRequest.newInstance(subClusterInfo));
+  }
+
   private SubClusterInfo querySubClusterInfo(SubClusterId subClusterId)
       throws YarnException {
     GetSubClusterInfoRequest request =
@@ -215,4 +525,25 @@ public abstract class FederationStateStoreBaseTest {
     return stateStore.getSubCluster(request).getSubClusterInfo();
   }
 
+  private SubClusterId queryApplicationHomeSC(ApplicationId appId)
+      throws YarnException {
+    GetApplicationHomeSubClusterRequest request =
+        GetApplicationHomeSubClusterRequest.newInstance(appId);
+
+    GetApplicationHomeSubClusterResponse response =
+        stateStore.getApplicationHomeSubClusterMap(request);
+
+    return response.getApplicationHomeSubCluster().getHomeSubCluster();
+  }
+
+  private SubClusterPolicyConfiguration queryPolicy(String queue)
+      throws YarnException {
+    GetSubClusterPolicyConfigurationRequest request =
+        GetSubClusterPolicyConfigurationRequest.newInstance(queue);
+
+    GetSubClusterPolicyConfigurationResponse result =
+        stateStore.getPolicyConfiguration(request);
+    return result.getPolicyConfiguration();
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a82a0dfc/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
----------------------------------------------------------------------
diff --git a/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 b/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
index 9396eda..74404c7 100644
--- a/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
+++ b/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
@@ -17,7 +17,7 @@
 
 package org.apache.hadoop.yarn.server.federation.store.impl;
 
-import org.apache.hadoop.yarn.server.federation.store.FederationMembershipStateStore;
+import org.apache.hadoop.yarn.server.federation.store.FederationStateStore;
 
 /**
  * Unit tests for MemoryFederationStateStore.
@@ -26,7 +26,7 @@ public class TestMemoryFederationStateStore
     extends FederationStateStoreBaseTest {
 
   @Override
-  protected FederationMembershipStateStore getCleanStateStore() {
+  protected FederationStateStore createStateStore() {
     return new MemoryFederationStateStore();
   }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[27/42] hadoop git commit: YARN-5601. Make the RM epoch base value configurable. Contributed by Subru Krishnan

Posted by su...@apache.org.
YARN-5601. Make the RM epoch base value configurable. Contributed by Subru Krishnan


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/7d17701f
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/7d17701f
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/7d17701f

Branch: refs/heads/YARN-2915
Commit: 7d17701ff2b94a6f4ccffe7fd2b376f53c466fa3
Parents: e5067ec
Author: Jian He <ji...@apache.org>
Authored: Fri Sep 2 12:23:57 2016 +0800
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon May 8 16:59:12 2017 -0700

----------------------------------------------------------------------
 .../hadoop-yarn/dev-support/findbugs-exclude.xml             | 5 ++++-
 .../java/org/apache/hadoop/yarn/conf/YarnConfiguration.java  | 3 +++
 .../apache/hadoop/yarn/conf/TestYarnConfigurationFields.java | 2 ++
 .../hadoop/yarn/server/resourcemanager/ResourceManager.java  | 7 +++++++
 .../resourcemanager/recovery/FileSystemRMStateStore.java     | 2 +-
 .../server/resourcemanager/recovery/LeveldbRMStateStore.java | 2 +-
 .../server/resourcemanager/recovery/MemoryRMStateStore.java  | 1 +
 .../yarn/server/resourcemanager/recovery/RMStateStore.java   | 4 ++++
 .../yarn/server/resourcemanager/recovery/ZKRMStateStore.java | 2 +-
 .../resourcemanager/recovery/RMStateStoreTestBase.java       | 8 +++++---
 .../server/resourcemanager/recovery/TestFSRMStateStore.java  | 1 +
 .../resourcemanager/recovery/TestLeveldbRMStateStore.java    | 1 +
 .../server/resourcemanager/recovery/TestZKRMStateStore.java  | 1 +
 13 files changed, 32 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7d17701f/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml b/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
index 2f5451d..bbd03a9 100644
--- a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
+++ b/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
@@ -293,7 +293,10 @@
   </Match>
   <Match>
     <Class name="org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore" />
-    <Field name="resourceManager"/>
+    <Or>
+      <Field name="resourceManager"/>
+      <Field name="baseEpoch"/>
+    </Or>
     <Bug pattern="IS2_INCONSISTENT_SYNC" />
   </Match>
   <Match>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7d17701f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
index 3508fad..9930e3a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
@@ -143,6 +143,9 @@ public class YarnConfiguration extends Configuration {
 
   public static final String RM_HOSTNAME = RM_PREFIX + "hostname";
 
+  public static final String RM_EPOCH = RM_PREFIX + "epoch";
+  public static final long DEFAULT_RM_EPOCH = 0L;
+
   /** The address of the applications manager interface in the RM.*/
   public static final String RM_ADDRESS = 
     RM_PREFIX + "address";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7d17701f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
index 5e0876f..3f3a06c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
@@ -75,6 +75,8 @@ public class TestYarnConfigurationFields extends TestConfigurationFieldsBase {
         .add(YarnConfiguration.FEDERATION_FAILOVER_ENABLED);
     configurationPropsToSkipCompare
         .add(YarnConfiguration.FEDERATION_STATESTORE_HEARTBEAT_INTERVAL_SECS);
+    configurationPropsToSkipCompare
+        .add(YarnConfiguration.RM_EPOCH);
 
     // Ignore blacklisting nodes for AM failures feature since it is still a
     // "work in progress"

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7d17701f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
index 2a5f03e..64a4142 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
@@ -780,6 +780,13 @@ public class ResourceManager extends CompositeService implements Recoverable {
           LOG.error("Failed to load/recover state", e);
           throw e;
         }
+      } else {
+        if (HAUtil.isFederationEnabled(conf)) {
+          long epoch = conf.getLong(YarnConfiguration.RM_EPOCH,
+              YarnConfiguration.DEFAULT_RM_EPOCH);
+          rmContext.setEpoch(epoch);
+          LOG.info("Epoch set for Federation: " + epoch);
+        }
       }
 
       super.serviceStart();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7d17701f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/FileSystemRMStateStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/FileSystemRMStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/FileSystemRMStateStore.java
index 9591945..7cbeda3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/FileSystemRMStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/FileSystemRMStateStore.java
@@ -197,7 +197,7 @@ public class FileSystemRMStateStore extends RMStateStore {
   @Override
   public synchronized long getAndIncrementEpoch() throws Exception {
     Path epochNodePath = getNodePath(rootDirPath, EPOCH_NODE);
-    long currentEpoch = 0;
+    long currentEpoch = baseEpoch;
     FileStatus status = getFileStatusWithRetries(epochNodePath);
     if (status != null) {
       // load current epoch

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7d17701f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/LeveldbRMStateStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/LeveldbRMStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/LeveldbRMStateStore.java
index 2ca53db..16ae1d3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/LeveldbRMStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/LeveldbRMStateStore.java
@@ -255,7 +255,7 @@ public class LeveldbRMStateStore extends RMStateStore {
 
   @Override
   public synchronized long getAndIncrementEpoch() throws Exception {
-    long currentEpoch = 0;
+    long currentEpoch = baseEpoch;
     byte[] dbKeyBytes = bytes(EPOCH_NODE);
     try {
       byte[] data = db.get(dbKeyBytes);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7d17701f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/MemoryRMStateStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/MemoryRMStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/MemoryRMStateStore.java
index 5f3328b..5041000 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/MemoryRMStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/MemoryRMStateStore.java
@@ -83,6 +83,7 @@ public class MemoryRMStateStore extends RMStateStore {
   
   @Override
   public synchronized void initInternal(Configuration conf) {
+    epoch = baseEpoch;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7d17701f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStore.java
index 5e3cf22..2a1948a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStore.java
@@ -99,6 +99,7 @@ public abstract class RMStateStore extends AbstractService {
       "ReservationSystemRoot";
   protected static final String VERSION_NODE = "RMVersionNode";
   protected static final String EPOCH_NODE = "EpochNode";
+  protected long baseEpoch;
   protected ResourceManager resourceManager;
   private final ReadLock readLock;
   private final WriteLock writeLock;
@@ -684,6 +685,9 @@ public abstract class RMStateStore extends AbstractService {
     dispatcher.register(RMStateStoreEventType.class, 
                         rmStateStoreEventHandler);
     dispatcher.setDrainEventsOnStop();
+    // read the base epoch value from conf
+    baseEpoch = conf.getLong(YarnConfiguration.RM_EPOCH,
+        YarnConfiguration.DEFAULT_RM_EPOCH);
     initInternal(conf);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7d17701f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKRMStateStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKRMStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKRMStateStore.java
index 86f7a5b..b19e841 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKRMStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKRMStateStore.java
@@ -430,7 +430,7 @@ public class ZKRMStateStore extends RMStateStore {
   @Override
   public synchronized long getAndIncrementEpoch() throws Exception {
     String epochNodePath = getNodePath(zkRootNodePath, EPOCH_NODE);
-    long currentEpoch = 0;
+    long currentEpoch = baseEpoch;
 
     if (exists(epochNodePath)) {
       // load current epoch

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7d17701f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStoreTestBase.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStoreTestBase.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStoreTestBase.java
index ca97914..06a16ff 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStoreTestBase.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStoreTestBase.java
@@ -91,6 +91,8 @@ public class RMStateStoreTestBase {
 
   public static final Log LOG = LogFactory.getLog(RMStateStoreTestBase.class);
 
+  protected final long epoch = 10L;
+
   static class TestDispatcher implements Dispatcher, EventHandler<Event> {
 
     ApplicationAttemptId attemptId;
@@ -564,13 +566,13 @@ public class RMStateStoreTestBase {
     store.setRMDispatcher(new TestDispatcher());
     
     long firstTimeEpoch = store.getAndIncrementEpoch();
-    Assert.assertEquals(0, firstTimeEpoch);
+    Assert.assertEquals(epoch, firstTimeEpoch);
     
     long secondTimeEpoch = store.getAndIncrementEpoch();
-    Assert.assertEquals(1, secondTimeEpoch);
+    Assert.assertEquals(epoch + 1, secondTimeEpoch);
     
     long thirdTimeEpoch = store.getAndIncrementEpoch();
-    Assert.assertEquals(2, thirdTimeEpoch);
+    Assert.assertEquals(epoch + 2, thirdTimeEpoch);
   }
 
   public void testAppDeletion(RMStateStoreHelper stateStoreHelper)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7d17701f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestFSRMStateStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestFSRMStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestFSRMStateStore.java
index 5eeb528..0738730 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestFSRMStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestFSRMStateStore.java
@@ -117,6 +117,7 @@ public class TestFSRMStateStore extends RMStateStoreTestBase {
       conf.setInt(YarnConfiguration.FS_RM_STATE_STORE_NUM_RETRIES, 8);
       conf.setLong(YarnConfiguration.FS_RM_STATE_STORE_RETRY_INTERVAL_MS,
               900L);
+      conf.setLong(YarnConfiguration.RM_EPOCH, epoch);
       if (adminCheckEnable) {
         conf.setBoolean(
           YarnConfiguration.YARN_INTERMEDIATE_DATA_ENCRYPTION, true);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7d17701f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestLeveldbRMStateStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestLeveldbRMStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestLeveldbRMStateStore.java
index e3d0f9c..afd0c77 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestLeveldbRMStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestLeveldbRMStateStore.java
@@ -82,6 +82,7 @@ public class TestLeveldbRMStateStore extends RMStateStoreTestBase {
 
   @Test(timeout = 60000)
   public void testEpoch() throws Exception {
+    conf.setLong(YarnConfiguration.RM_EPOCH, epoch);
     LeveldbStateStoreTester tester = new LeveldbStateStoreTester();
     testEpoch(tester);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7d17701f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestZKRMStateStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestZKRMStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestZKRMStateStore.java
index 7c40ddf..58f7340 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestZKRMStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestZKRMStateStore.java
@@ -184,6 +184,7 @@ public class TestZKRMStateStore extends RMStateStoreTestBase {
       conf.set(YarnConfiguration.RM_ZK_ADDRESS,
           curatorTestingServer.getConnectString());
       conf.set(YarnConfiguration.ZK_RM_STATE_STORE_PARENT_PATH, workingZnode);
+      conf.setLong(YarnConfiguration.RM_EPOCH, epoch);
       this.store = new TestZKRMStateStoreInternal(conf, workingZnode);
       return this.store;
     }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[26/42] hadoop git commit: YARN-5324. Stateless Federation router policies implementation. (Carlo Curino via Subru).

Posted by su...@apache.org.
YARN-5324. Stateless Federation router policies implementation. (Carlo Curino via Subru).


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/5d38f164
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/5d38f164
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/5d38f164

Branch: refs/heads/YARN-2915
Commit: 5d38f1645e2c0cafb6a1b450710c5d8f72d8b56c
Parents: 77e3985
Author: Subru Krishnan <su...@apache.org>
Authored: Thu Sep 22 17:06:57 2016 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon May 8 16:59:12 2017 -0700

----------------------------------------------------------------------
 .../policies/FederationPolicyConfigurator.java  |  91 -------
 .../FederationPolicyInitializationContext.java  |  11 +-
 .../policies/FederationPolicyManager.java       | 126 +++++++++
 .../policies/FederationPolicyWriter.java        |  45 ----
 .../policies/dao/WeightedPolicyInfo.java        | 253 +++++++++++++++++++
 .../federation/policies/dao/package-info.java   |  20 ++
 .../router/BaseWeightedRouterPolicy.java        | 150 +++++++++++
 .../policies/router/LoadBasedRouterPolicy.java  | 109 ++++++++
 .../policies/router/PriorityRouterPolicy.java   |  66 +++++
 .../router/UniformRandomRouterPolicy.java       |  85 +++++++
 .../router/WeightedRandomRouterPolicy.java      |  79 ++++++
 .../store/records/SubClusterIdInfo.java         |  75 ++++++
 .../policies/BaseFederationPoliciesTest.java    | 155 ++++++++++++
 ...ionPolicyInitializationContextValidator.java |  17 +-
 .../router/TestLoadBasedRouterPolicy.java       | 109 ++++++++
 .../router/TestPriorityRouterPolicy.java        |  87 +++++++
 .../router/TestUniformRandomRouterPolicy.java   |  65 +++++
 .../router/TestWeightedRandomRouterPolicy.java  | 127 ++++++++++
 .../utils/FederationPoliciesTestUtil.java       |  82 +++++-
 19 files changed, 1604 insertions(+), 148 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d38f164/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyConfigurator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyConfigurator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyConfigurator.java
deleted file mode 100644
index fdc3857..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyConfigurator.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership.  The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations under
- * the License.
- */
-
-package org.apache.hadoop.yarn.server.federation.policies;
-
-import org.apache.hadoop.yarn.server.federation.policies.amrmproxy.FederationAMRMProxyPolicy;
-
-
-import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
-
-import org.apache.hadoop.yarn.server.federation.policies.router
-    .FederationRouterPolicy;
-
-/**
- * Implementors of this interface are capable to instantiate and (re)initalize
- * {@link FederationAMRMProxyPolicy} and {@link FederationRouterPolicy} based on
- * a {@link FederationPolicyInitializationContext}. The reason to bind these two
- * policies together is to make sure we remain consistent across the router and
- * amrmproxy policy decisions.
- */
-public interface FederationPolicyConfigurator {
-
-  /**
-   * If the current instance is compatible, this method returns the same
-   * instance of {@link FederationAMRMProxyPolicy} reinitialized with the
-   * current context, otherwise a new instance initialized with the current
-   * context is provided. If the instance is compatible with the current class
-   * the implementors should attempt to reinitalize (retaining state). To affect
-   * a complete policy reset oldInstance should be null.
-   *
-   * @param federationPolicyInitializationContext the current context
-   * @param oldInstance                           the existing (possibly null)
-   *                                              instance.
-   *
-   * @return an updated {@link FederationAMRMProxyPolicy
-  }.
-   *
-   * @throws FederationPolicyInitializationException if the initialization
-   *                                                 cannot be completed
-   *                                                 properly. The oldInstance
-   *                                                 should be still valid in
-   *                                                 case of failed
-   *                                                 initialization.
-   */
-  FederationAMRMProxyPolicy getAMRMPolicy(
-      FederationPolicyInitializationContext
-          federationPolicyInitializationContext,
-      FederationAMRMProxyPolicy oldInstance)
-      throws FederationPolicyInitializationException;
-
-  /**
-   * If the current instance is compatible, this method returns the same
-   * instance of {@link FederationRouterPolicy} reinitialized with the current
-   * context, otherwise a new instance initialized with the current context is
-   * provided. If the instance is compatible with the current class the
-   * implementors should attempt to reinitalize (retaining state). To affect a
-   * complete policy reset oldInstance shoulb be set to null.
-   *
-   * @param federationPolicyInitializationContext the current context
-   * @param oldInstance                           the existing (possibly null)
-   *                                              instance.
-   *
-   * @return an updated {@link FederationRouterPolicy}.
-   *
-   * @throws FederationPolicyInitializationException if the initalization cannot
-   *                                                 be completed properly. The
-   *                                                 oldInstance should be still
-   *                                                 valid in case of failed
-   *                                                 initialization.
-   */
-  FederationRouterPolicy getRouterPolicy(
-      FederationPolicyInitializationContext
-          federationPolicyInitializationContext,
-      FederationRouterPolicy oldInstance)
-      throws FederationPolicyInitializationException;
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d38f164/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyInitializationContext.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyInitializationContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyInitializationContext.java
index 879ccee..9347fd0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyInitializationContext.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyInitializationContext.java
@@ -59,13 +59,12 @@ public class FederationPolicyInitializationContext {
   /**
    * Setter for the {@link SubClusterPolicyConfiguration}.
    *
-   * @param federationPolicyConfiguration the
-   * {@link SubClusterPolicyConfiguration}
-   *                                      to be used for initialization.
+   * @param fedPolicyConfiguration the {@link SubClusterPolicyConfiguration}
+   *                               to be used for initialization.
    */
-  public void setFederationPolicyConfiguration(
-      SubClusterPolicyConfiguration federationPolicyConfiguration) {
-    this.federationPolicyConfiguration = federationPolicyConfiguration;
+  public void setSubClusterPolicyConfiguration(
+      SubClusterPolicyConfiguration fedPolicyConfiguration) {
+    this.federationPolicyConfiguration = fedPolicyConfiguration;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d38f164/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyManager.java
new file mode 100644
index 0000000..e5dba63
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyManager.java
@@ -0,0 +1,126 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies;
+
+import org.apache.hadoop.yarn.server.federation.policies.amrmproxy.FederationAMRMProxyPolicy;
+import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
+import org.apache.hadoop.yarn.server.federation.policies.router.FederationRouterPolicy;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyConfiguration;
+
+/**
+ *
+ * Implementors need to provide the ability to serliaze a policy and its
+ * configuration as a {@link SubClusterPolicyConfiguration}, as well as
+ * provide (re)initialization mechanics for the underlying
+ * {@link FederationAMRMProxyPolicy} and {@link FederationRouterPolicy}.
+ *
+ * The serialization aspects are used by admin APIs or a policy engine to
+ * store a serialized configuration in the {@code FederationStateStore},
+ * while the getters methods are used to obtain a propertly inizialized
+ * policy in the {@code Router} and {@code AMRMProxy} respectively.
+ *
+ * This interface by design binds together
+ * {@link FederationAMRMProxyPolicy} and {@link FederationRouterPolicy} and
+ * provide lifecycle support for serialization and deserialization, to reduce
+ * configuration mistakes (combining incompatible policies).
+ *
+ */
+public interface FederationPolicyManager {
+
+  /**
+   * If the current instance is compatible, this method returns the same
+   * instance of {@link FederationAMRMProxyPolicy} reinitialized with the
+   * current context, otherwise a new instance initialized with the current
+   * context is provided. If the instance is compatible with the current class
+   * the implementors should attempt to reinitalize (retaining state). To affect
+   * a complete policy reset oldInstance should be null.
+   *
+   * @param federationPolicyInitializationContext the current context
+   * @param oldInstance                           the existing (possibly null)
+   *                                              instance.
+   *
+   * @return an updated {@link FederationAMRMProxyPolicy
+  }.
+   *
+   * @throws FederationPolicyInitializationException if the initialization
+   *                                                 cannot be completed
+   *                                                 properly. The oldInstance
+   *                                                 should be still valid in
+   *                                                 case of failed
+   *                                                 initialization.
+   */
+  FederationAMRMProxyPolicy getAMRMPolicy(
+      FederationPolicyInitializationContext
+          federationPolicyInitializationContext,
+      FederationAMRMProxyPolicy oldInstance)
+      throws FederationPolicyInitializationException;
+
+  /**
+   * If the current instance is compatible, this method returns the same
+   * instance of {@link FederationRouterPolicy} reinitialized with the current
+   * context, otherwise a new instance initialized with the current context is
+   * provided. If the instance is compatible with the current class the
+   * implementors should attempt to reinitalize (retaining state). To affect a
+   * complete policy reset oldInstance shoulb be set to null.
+   *
+   * @param federationPolicyInitializationContext the current context
+   * @param oldInstance                           the existing (possibly null)
+   *                                              instance.
+   *
+   * @return an updated {@link FederationRouterPolicy}.
+   *
+   * @throws FederationPolicyInitializationException if the initalization cannot
+   *                                                 be completed properly. The
+   *                                                 oldInstance should be still
+   *                                                 valid in case of failed
+   *                                                 initialization.
+   */
+  FederationRouterPolicy getRouterPolicy(
+      FederationPolicyInitializationContext
+          federationPolicyInitializationContext,
+      FederationRouterPolicy oldInstance)
+      throws FederationPolicyInitializationException;
+
+  /**
+   * This method is invoked to derive a {@link SubClusterPolicyConfiguration}.
+   * This is to be used when writing a policy object in the federation policy
+   * store.
+   *
+   * @return a valid policy configuration representing this object
+   * parametrization.
+   *
+   * @throws FederationPolicyInitializationException if the current state cannot
+   *                                                 be serialized properly
+   */
+  SubClusterPolicyConfiguration serializeConf()
+      throws FederationPolicyInitializationException;
+
+
+  /**
+   * This method returns the queue this policy is configured for.
+   * @return the name of the queue.
+   */
+  String getQueue();
+
+  /**
+   * This methods provides a setter for the queue this policy is specified for.
+   * @param queue the name of the queue.
+   */
+  void setQueue(String queue);
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d38f164/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyWriter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyWriter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyWriter.java
deleted file mode 100644
index 5034b7e..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyWriter.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership.  The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations under
- * the License.
- */
-
-package org.apache.hadoop.yarn.server.federation.policies;
-
-import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
-import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyConfiguration;
-
-/**
- * Implementors of this class are able to serializeConf the configuraiton of a
- * policy as a {@link SubClusterPolicyConfiguration}. This is used during the
- * lifetime of a policy from the admin APIs or policy engine to serializeConf
- * the policy into the policy store.
- */
-public interface FederationPolicyWriter {
-
-  /**
-   /**
-   * This method is invoked to derive a {@link SubClusterPolicyConfiguration}.
-   * This is to be used when writing a policy object in the federation policy
-   * store.
-   *
-   * @return a valid policy configuration representing this object
-   * parametrization.
-   *
-   * @throws FederationPolicyInitializationException if the current state cannot
-   *                                                 be serialized properly
-   */
-  SubClusterPolicyConfiguration serializeConf()
-      throws FederationPolicyInitializationException;
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d38f164/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/dao/WeightedPolicyInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/dao/WeightedPolicyInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/dao/WeightedPolicyInfo.java
new file mode 100644
index 0000000..a0fa37f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/dao/WeightedPolicyInfo.java
@@ -0,0 +1,253 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies.dao;
+
+import com.sun.jersey.api.json.JSONConfiguration;
+import com.sun.jersey.api.json.JSONJAXBContext;
+import com.sun.jersey.api.json.JSONMarshaller;
+import com.sun.jersey.api.json.JSONUnmarshaller;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterIdInfo;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.xml.bind.JAXBException;
+import javax.xml.bind.Marshaller;
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlRootElement;
+import java.io.StringReader;
+import java.io.StringWriter;
+import java.nio.ByteBuffer;
+import java.nio.charset.Charset;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * This is a DAO class for the configuration of parameteres for federation
+ * policies. This generalizes several possible configurations as two lists of
+ * {@link SubClusterIdInfo} and corresponding weights as a
+ * {@link Float}. The interpretation of the weight is left to the logic in
+ * the policy.
+ */
+
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+@XmlRootElement(name = "federation-policy")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class WeightedPolicyInfo {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(WeightedPolicyInfo.class);
+
+  private Map<SubClusterIdInfo, Float> routerPolicyWeights = new HashMap<>();
+  private Map<SubClusterIdInfo, Float> amrmPolicyWeights = new HashMap<>();
+  private float headroomAlpha;
+
+  private static JSONJAXBContext jsonjaxbContext = initContext();
+
+  private static JSONJAXBContext initContext() {
+    try {
+      return new JSONJAXBContext(JSONConfiguration.DEFAULT,
+          WeightedPolicyInfo.class);
+    } catch (JAXBException e) {
+      LOG.error("Error parsing the policy.", e);
+    }
+    return null;
+  }
+
+  public WeightedPolicyInfo() {
+    //JAXB needs this
+  }
+
+  /**
+   * Setter method for Router weights.
+   *
+   * @param policyWeights the router weights.
+   */
+  public void setRouterPolicyWeights(
+      Map<SubClusterIdInfo, Float> policyWeights) {
+    this.routerPolicyWeights = policyWeights;
+  }
+
+  /**
+   * Setter method for ARMRMProxy weights.
+   *
+   * @param policyWeights the amrmproxy weights.
+   */
+  public void setAMRMPolicyWeights(
+      Map<SubClusterIdInfo, Float> policyWeights) {
+    this.amrmPolicyWeights = policyWeights;
+  }
+
+  /**
+   * Getter of the router weights.
+   * @return the router weights.
+   */
+  public Map<SubClusterIdInfo, Float> getRouterPolicyWeights() {
+    return routerPolicyWeights;
+  }
+
+  /**
+   * Getter for AMRMProxy weights.
+   * @return the AMRMProxy weights.
+   */
+  public Map<SubClusterIdInfo, Float> getAMRMPolicyWeights() {
+    return amrmPolicyWeights;
+  }
+
+  /**
+   * Deserializes a {@link WeightedPolicyInfo} from a byte UTF-8 JSON
+   * representation.
+   *
+   * @param bb the input byte representation.
+   *
+   * @return the {@link WeightedPolicyInfo} represented.
+   *
+   * @throws FederationPolicyInitializationException if a deserializaiton error
+   *                                                 occurs.
+   */
+  public static WeightedPolicyInfo fromByteBuffer(ByteBuffer bb)
+      throws FederationPolicyInitializationException {
+
+    if (jsonjaxbContext == null) {
+      throw new FederationPolicyInitializationException("JSONJAXBContext should"
+          + " not be null.");
+    }
+
+    try {
+      JSONUnmarshaller unmarshaller = jsonjaxbContext.createJSONUnmarshaller();
+      final byte[] bytes = new byte[bb.remaining()];
+      bb.get(bytes);
+      String params = new String(bytes, Charset.forName("UTF-8"));
+
+      WeightedPolicyInfo weightedPolicyInfo = unmarshaller
+          .unmarshalFromJSON(new StringReader(params),
+              WeightedPolicyInfo.class);
+      return weightedPolicyInfo;
+    } catch (JAXBException j) {
+      throw new FederationPolicyInitializationException(j);
+    }
+  }
+
+  /**
+   * Converts the policy into a byte array representation in the input {@link
+   * ByteBuffer}.
+   *
+   * @return byte array representation of this policy configuration.
+   *
+   * @throws FederationPolicyInitializationException if a serialization error
+   *                                                 occurs.
+   */
+  public ByteBuffer toByteBuffer()
+      throws FederationPolicyInitializationException {
+    if (jsonjaxbContext == null) {
+      throw new FederationPolicyInitializationException("JSONJAXBContext should"
+          + " not be null.");
+    }
+    try {
+      String s = toJSONString();
+      return ByteBuffer.wrap(s.getBytes(Charset.forName("UTF-8")));
+    } catch (JAXBException j) {
+      throw new FederationPolicyInitializationException(j);
+    }
+  }
+
+  private String toJSONString() throws JAXBException {
+    JSONMarshaller marshaller = jsonjaxbContext.createJSONMarshaller();
+    marshaller.setProperty(Marshaller.JAXB_FORMATTED_OUTPUT, true);
+    StringWriter sw = new StringWriter(256);
+    marshaller.marshallToJSON(this, sw);
+    return sw.toString();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+
+    if (other == null || !other.getClass().equals(this.getClass())) {
+      return false;
+    }
+
+    WeightedPolicyInfo otherPolicy =
+        (WeightedPolicyInfo) other;
+    Map<SubClusterIdInfo, Float> otherAMRMWeights =
+        otherPolicy.getAMRMPolicyWeights();
+    Map<SubClusterIdInfo, Float> otherRouterWeights =
+        otherPolicy.getRouterPolicyWeights();
+
+    boolean amrmWeightsMatch = otherAMRMWeights != null &&
+        getAMRMPolicyWeights() != null &&
+        CollectionUtils.isEqualCollection(otherAMRMWeights.entrySet(),
+            getAMRMPolicyWeights().entrySet());
+
+    boolean routerWeightsMatch = otherRouterWeights != null &&
+        getRouterPolicyWeights() != null &&
+        CollectionUtils.isEqualCollection(otherRouterWeights.entrySet(),
+            getRouterPolicyWeights().entrySet());
+
+    return amrmWeightsMatch && routerWeightsMatch;
+  }
+
+  @Override
+  public int hashCode() {
+    return 31 * amrmPolicyWeights.hashCode() + routerPolicyWeights.hashCode();
+  }
+
+  /**
+   * Return the parameter headroomAlpha, used by policies that balance
+   * weight-based and load-based considerations in their decisions.
+   *
+   * For policies that use this parameter, values close to 1 indicate that
+   * most of the decision should be based on currently observed headroom from
+   * various sub-clusters, values close to zero, indicate that the decision
+   * should be mostly based on weights and practically ignore current load.
+   *
+   * @return the value of headroomAlpha.
+   */
+  public float getHeadroomAlpha() {
+    return headroomAlpha;
+  }
+
+  /**
+   * Set the parameter headroomAlpha, used by policies that balance
+   * weight-based and load-based considerations in their decisions.
+   *
+   * For policies that use this parameter, values close to 1 indicate that
+   * most of the decision should be based on currently observed headroom from
+   * various sub-clusters, values close to zero, indicate that the decision
+   * should be mostly based on weights and practically ignore current load.
+   *
+   * @param headroomAlpha the value to use for balancing.
+   */
+  public void setHeadroomAlpha(float headroomAlpha) {
+    this.headroomAlpha = headroomAlpha;
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return toJSONString();
+    } catch (JAXBException e) {
+      e.printStackTrace();
+      return "Error serializing to string.";
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d38f164/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/dao/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/dao/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/dao/package-info.java
new file mode 100644
index 0000000..43f5b83
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/dao/package-info.java
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/** DAO objects for serializing/deserializing policy configurations. **/
+package org.apache.hadoop.yarn.server.federation.policies.dao;
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d38f164/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/BaseWeightedRouterPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/BaseWeightedRouterPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/BaseWeightedRouterPolicy.java
new file mode 100644
index 0000000..e888979
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/BaseWeightedRouterPolicy.java
@@ -0,0 +1,150 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies.router;
+
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.federation.policies.FederationPolicyInitializationContext;
+import org.apache.hadoop.yarn.server.federation.policies.FederationPolicyInitializationContextValidator;
+import org.apache.hadoop.yarn.server.federation.policies.dao.WeightedPolicyInfo;
+import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
+import org.apache.hadoop.yarn.server.federation.policies.exceptions.NoActiveSubclustersException;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterIdInfo;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
+
+import java.util.Map;
+
+/**
+ * Abstract class provides common validation of reinitialize(), for all
+ * policies that are "weight-based".
+ */
+public abstract class BaseWeightedRouterPolicy
+    implements FederationRouterPolicy {
+
+  private WeightedPolicyInfo policyInfo = null;
+  private FederationPolicyInitializationContext policyContext;
+
+  public BaseWeightedRouterPolicy() {
+  }
+
+  @Override
+  public void reinitialize(FederationPolicyInitializationContext
+      federationPolicyContext)
+      throws FederationPolicyInitializationException {
+    FederationPolicyInitializationContextValidator
+        .validate(federationPolicyContext, this.getClass().getCanonicalName());
+
+    // perform consistency checks
+    WeightedPolicyInfo newPolicyInfo = WeightedPolicyInfo
+        .fromByteBuffer(
+            federationPolicyContext.getSubClusterPolicyConfiguration()
+                .getParams());
+
+    // if nothing has changed skip the rest of initialization
+    if (policyInfo != null && policyInfo.equals(newPolicyInfo)) {
+      return;
+    }
+
+    validate(newPolicyInfo);
+    setPolicyInfo(newPolicyInfo);
+    this.policyContext = federationPolicyContext;
+  }
+
+  /**
+   * Overridable validation step for the policy configuration.
+   * @param newPolicyInfo the configuration to test.
+   * @throws FederationPolicyInitializationException if the configuration is
+   * not valid.
+   */
+  public void validate(WeightedPolicyInfo newPolicyInfo) throws
+      FederationPolicyInitializationException {
+    if (newPolicyInfo == null) {
+      throw new FederationPolicyInitializationException("The policy to "
+          + "validate should not be null.");
+    }
+    Map<SubClusterIdInfo, Float> newWeights =
+        newPolicyInfo.getRouterPolicyWeights();
+    if (newWeights == null || newWeights.size() < 1) {
+      throw new FederationPolicyInitializationException(
+          "Weight vector cannot be null/empty.");
+    }
+  }
+
+
+  /**
+   * Getter method for the configuration weights.
+   *
+   * @return the {@link WeightedPolicyInfo} representing the policy
+   * configuration.
+   */
+  public WeightedPolicyInfo getPolicyInfo() {
+    return policyInfo;
+  }
+
+  /**
+   * Setter method for the configuration weights.
+   *
+   * @param policyInfo the {@link WeightedPolicyInfo} representing the policy
+   *                   configuration.
+   */
+  public void setPolicyInfo(
+      WeightedPolicyInfo policyInfo) {
+    this.policyInfo = policyInfo;
+  }
+
+  /**
+   * Getter method for the {@link FederationPolicyInitializationContext}.
+   * @return the context for this policy.
+   */
+  public FederationPolicyInitializationContext getPolicyContext() {
+    return policyContext;
+  }
+
+  /**
+   * Setter method for the {@link FederationPolicyInitializationContext}.
+   * @param policyContext the context to assign to this policy.
+   */
+  public void setPolicyContext(
+      FederationPolicyInitializationContext policyContext) {
+    this.policyContext = policyContext;
+  }
+
+  /**
+   * This methods gets active subclusters map from the {@code
+   * FederationStateStoreFacade} and validate it not being null/empty.
+   *
+   * @return the map of ids to info for all active subclusters.
+   * @throws YarnException if we can't get the list.
+   */
+  protected Map<SubClusterId, SubClusterInfo> getActiveSubclusters()
+      throws YarnException {
+
+    Map<SubClusterId, SubClusterInfo> activeSubclusters = getPolicyContext()
+        .getFederationStateStoreFacade().getSubClusters(true);
+
+    if (activeSubclusters == null || activeSubclusters.size() < 1) {
+      throw new NoActiveSubclustersException(
+          "Zero active subclusters, cannot pick where to send job.");
+    }
+    return activeSubclusters;
+  }
+
+
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d38f164/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/LoadBasedRouterPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/LoadBasedRouterPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/LoadBasedRouterPolicy.java
new file mode 100644
index 0000000..e57709f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/LoadBasedRouterPolicy.java
@@ -0,0 +1,109 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies.router;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.federation.policies.FederationPolicyInitializationContext;
+import org.apache.hadoop.yarn.server.federation.policies.dao.WeightedPolicyInfo;
+import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterIdInfo;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
+import org.codehaus.jettison.json.JSONException;
+import org.codehaus.jettison.json.JSONObject;
+
+import java.util.Map;
+
+/**
+ * This implements a simple load-balancing policy. The policy "weights" are
+ * binary 0/1 values that enable/disable each sub-cluster, and the policy peaks
+ * the sub-cluster with the least load to forward this application.
+ */
+public class LoadBasedRouterPolicy
+    extends BaseWeightedRouterPolicy {
+
+  private static final Log LOG =
+      LogFactory.getLog(LoadBasedRouterPolicy.class);
+
+  @Override
+  public void reinitialize(FederationPolicyInitializationContext
+      federationPolicyContext)
+      throws FederationPolicyInitializationException {
+
+    // remember old policyInfo
+    WeightedPolicyInfo tempPolicy = getPolicyInfo();
+
+    //attempt new initialization
+    super.reinitialize(federationPolicyContext);
+
+    //check extra constraints
+    for (Float weight : getPolicyInfo().getRouterPolicyWeights().values()) {
+      if (weight != 0 && weight != 1) {
+        //reset to old policyInfo if check fails
+        setPolicyInfo(tempPolicy);
+        throw new FederationPolicyInitializationException(
+            this.getClass().getCanonicalName()
+                + " policy expects all weights to be either "
+                + "\"0\" or \"1\"");
+      }
+    }
+  }
+
+  @Override
+  public SubClusterId getHomeSubcluster(
+      ApplicationSubmissionContext appSubmissionContext)
+      throws YarnException {
+
+    Map<SubClusterId, SubClusterInfo> activeSubclusters =
+        getActiveSubclusters();
+
+    Map<SubClusterIdInfo, Float> weights = getPolicyInfo()
+        .getRouterPolicyWeights();
+    SubClusterIdInfo chosen = null;
+    long currBestMem = -1;
+    for (Map.Entry<SubClusterId, SubClusterInfo> entry :
+        activeSubclusters
+        .entrySet()) {
+      SubClusterIdInfo id = new SubClusterIdInfo(entry.getKey());
+      if (weights.containsKey(id) && weights.get(id) > 0) {
+        long availableMemory = getAvailableMemory(entry.getValue());
+        if (availableMemory > currBestMem) {
+          currBestMem = availableMemory;
+          chosen = id;
+        }
+      }
+    }
+
+    return chosen.toId();
+  }
+
+  private long getAvailableMemory(SubClusterInfo value)
+      throws YarnException {
+    try {
+      long mem = -1;
+      JSONObject obj = new JSONObject(value.getCapability());
+      mem = obj.getJSONObject("clusterMetrics").getLong("availableMB");
+      return mem;
+    } catch (JSONException j) {
+      throw new YarnException("FederationSubCluserInfo cannot be parsed", j);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d38f164/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/PriorityRouterPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/PriorityRouterPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/PriorityRouterPolicy.java
new file mode 100644
index 0000000..a8ac5f7
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/PriorityRouterPolicy.java
@@ -0,0 +1,66 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies.router;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterIdInfo;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
+
+import java.util.Map;
+
+/**
+ * This implements a policy that interprets "weights" as a ordered list of
+ * preferences among sub-clusters. Highest weight among active subclusters is
+ * chosen.
+ */
+public class PriorityRouterPolicy
+    extends BaseWeightedRouterPolicy {
+
+  private static final Log LOG =
+      LogFactory.getLog(PriorityRouterPolicy.class);
+
+  @Override
+  public SubClusterId getHomeSubcluster(
+      ApplicationSubmissionContext appSubmissionContext)
+      throws YarnException {
+
+    Map<SubClusterId, SubClusterInfo> activeSubclusters =
+        getActiveSubclusters();
+
+    // This finds the sub-cluster with the highest weight among the
+    // currently active ones.
+    Map<SubClusterIdInfo, Float> weights = getPolicyInfo()
+        .getRouterPolicyWeights();
+    SubClusterId chosen = null;
+    Float currentBest = Float.MIN_VALUE;
+    for (SubClusterId id : activeSubclusters.keySet()) {
+      SubClusterIdInfo idInfo = new SubClusterIdInfo(id);
+      if (weights.containsKey(idInfo) && weights.get(idInfo) > currentBest) {
+        currentBest = weights.get(idInfo);
+        chosen = id;
+      }
+    }
+
+    return chosen;
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d38f164/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/UniformRandomRouterPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/UniformRandomRouterPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/UniformRandomRouterPolicy.java
new file mode 100644
index 0000000..1774961
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/UniformRandomRouterPolicy.java
@@ -0,0 +1,85 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies.router;
+
+import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.federation.policies.FederationPolicyInitializationContext;
+import org.apache.hadoop.yarn.server.federation.policies.FederationPolicyInitializationContextValidator;
+import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+
+/**
+ * This simple policy picks at uniform random among any of the currently active
+ * subclusters. This policy is easy to use and good for testing.
+ *
+ * NOTE: this is "almost" subsumed by the {@code WeightedRandomRouterPolicy}.
+ * Behavior only diverges when there are active sub-clusters that are not part
+ * of the "weights", in which case the {@link UniformRandomRouterPolicy} send
+ * load to them, while {@code WeightedRandomRouterPolicy} does not.
+ */
+public class UniformRandomRouterPolicy extends BaseWeightedRouterPolicy {
+
+  private Random rand;
+
+  public UniformRandomRouterPolicy() {
+    rand = new Random(System.currentTimeMillis());
+  }
+
+  @Override
+  public void reinitialize(
+      FederationPolicyInitializationContext federationPolicyContext)
+      throws FederationPolicyInitializationException {
+    FederationPolicyInitializationContextValidator
+        .validate(federationPolicyContext, this.getClass().getCanonicalName());
+
+    //note: this overrides BaseWeighterRouterPolicy and ignores the weights
+
+    setPolicyContext(federationPolicyContext);
+  }
+
+  /**
+   * Simply picks a random active subcluster to start the AM (this does NOT
+   * depend on the weights in the policy).
+   *
+   * @param appSubmissionContext the context for the app being submitted
+   *                             (ignored).
+   *
+   * @return a randomly chosen subcluster.
+   *
+   * @throws YarnException if there are no active subclusters.
+   */
+  public SubClusterId getHomeSubcluster(
+      ApplicationSubmissionContext appSubmissionContext)
+      throws YarnException {
+
+    Map<SubClusterId, SubClusterInfo> activeSubclusters =
+        getActiveSubclusters();
+
+    List<SubClusterId> list =
+        new ArrayList<>(activeSubclusters.keySet());
+    return list.get(rand.nextInt(list.size()));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d38f164/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/WeightedRandomRouterPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/WeightedRandomRouterPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/WeightedRandomRouterPolicy.java
new file mode 100644
index 0000000..0777677
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/WeightedRandomRouterPolicy.java
@@ -0,0 +1,79 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies.router;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterIdInfo;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
+
+import java.util.Map;
+import java.util.Random;
+
+/**
+ * This policy implements a weighted random sample among currently active
+ * sub-clusters.
+ */
+public class WeightedRandomRouterPolicy
+    extends BaseWeightedRouterPolicy {
+
+  private static final Log LOG =
+      LogFactory.getLog(WeightedRandomRouterPolicy.class);
+  private Random rand = new Random(System.currentTimeMillis());
+
+  @Override
+  public SubClusterId getHomeSubcluster(
+      ApplicationSubmissionContext appSubmissionContext)
+      throws YarnException {
+
+    Map<SubClusterId, SubClusterInfo> activeSubclusters =
+        getActiveSubclusters();
+
+    // note: we cannot pre-compute the weights, as the set of activeSubcluster
+    // changes dynamically (and this would unfairly spread the load to
+    // sub-clusters adjacent to an inactive one), hence we need to count/scan
+    // the list and based on weight pick the next sub-cluster.
+    Map<SubClusterIdInfo, Float> weights = getPolicyInfo()
+        .getRouterPolicyWeights();
+
+    float totActiveWeight = 0;
+    for(Map.Entry<SubClusterIdInfo, Float> entry : weights.entrySet()){
+      if(entry.getKey()!=null && activeSubclusters.containsKey(entry.getKey()
+          .toId())){
+        totActiveWeight += entry.getValue();
+      }
+    }
+    float lookupValue = rand.nextFloat() * totActiveWeight;
+
+    for (SubClusterId id : activeSubclusters.keySet()) {
+      SubClusterIdInfo idInfo = new SubClusterIdInfo(id);
+      if (weights.containsKey(idInfo)) {
+        lookupValue -= weights.get(idInfo);
+      }
+      if (lookupValue <= 0) {
+        return id;
+      }
+    }
+    //should never happen
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d38f164/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterIdInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterIdInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterIdInfo.java
new file mode 100644
index 0000000..e2260a1
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterIdInfo.java
@@ -0,0 +1,75 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlRootElement;
+
+/**
+ * This class represent a sub-cluster identifier in the JSON representation
+ * of the policy configuration.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+@XmlRootElement(name = "federation-policy")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class SubClusterIdInfo {
+
+  private String id;
+
+  public SubClusterIdInfo() {
+    //JAXB needs this
+  }
+
+  public SubClusterIdInfo(String subClusterId) {
+    this.id = subClusterId;
+  }
+
+  public SubClusterIdInfo(SubClusterId subClusterId) {
+    this.id = subClusterId.getId();
+  }
+
+  /**
+   * Get the sub-cluster identifier as {@link SubClusterId}.
+   * @return the sub-cluster id.
+   */
+  public SubClusterId toId() {
+    return SubClusterId.newInstance(id);
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other instanceof SubClusterIdInfo) {
+      if (((SubClusterIdInfo) other).id.equals(this.id)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  @Override
+  public int hashCode() {
+    return id.hashCode();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d38f164/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/BaseFederationPoliciesTest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/BaseFederationPoliciesTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/BaseFederationPoliciesTest.java
new file mode 100644
index 0000000..8da92b9
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/BaseFederationPoliciesTest.java
@@ -0,0 +1,155 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies;
+
+import static org.mockito.Mockito.mock;
+
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Random;
+
+import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.federation.policies.dao.WeightedPolicyInfo;
+import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
+import org.apache.hadoop.yarn.server.federation.policies.exceptions.NoActiveSubclustersException;
+import org.apache.hadoop.yarn.server.federation.policies.router.FederationRouterPolicy;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyConfiguration;
+import org.apache.hadoop.yarn.server.federation.utils.FederationPoliciesTestUtil;
+import org.junit.Test;
+
+/**
+ * Base class for policies tests, tests for common reinitialization cases.
+ */
+public abstract class BaseFederationPoliciesTest {
+
+  private ConfigurableFederationPolicy policy;
+  private WeightedPolicyInfo policyInfo;
+  private Map<SubClusterId, SubClusterInfo> activeSubclusters = new HashMap<>();
+  private FederationPolicyInitializationContext federationPolicyContext;
+  private ApplicationSubmissionContext applicationSubmissionContext =
+      mock(ApplicationSubmissionContext.class);
+  private Random rand = new Random();
+
+  @Test
+  public void testReinitilialize() throws YarnException {
+    FederationPolicyInitializationContext fpc =
+        new FederationPolicyInitializationContext();
+    ByteBuffer buf = getPolicyInfo().toByteBuffer();
+    fpc.setSubClusterPolicyConfiguration(SubClusterPolicyConfiguration
+        .newInstance("queue1", getPolicy().getClass().getCanonicalName(), buf));
+    fpc.setFederationSubclusterResolver(
+        FederationPoliciesTestUtil.initResolver());
+    fpc.setFederationStateStoreFacade(FederationPoliciesTestUtil.initFacade());
+    getPolicy().reinitialize(fpc);
+  }
+
+  @Test(expected = FederationPolicyInitializationException.class)
+  public void testReinitilializeBad1() throws YarnException {
+    getPolicy().reinitialize(null);
+  }
+
+  @Test(expected = FederationPolicyInitializationException.class)
+  public void testReinitilializeBad2() throws YarnException {
+    FederationPolicyInitializationContext fpc =
+        new FederationPolicyInitializationContext();
+    getPolicy().reinitialize(fpc);
+  }
+
+  @Test(expected = FederationPolicyInitializationException.class)
+  public void testReinitilializeBad3() throws YarnException {
+    FederationPolicyInitializationContext fpc =
+        new FederationPolicyInitializationContext();
+    ByteBuffer buf = mock(ByteBuffer.class);
+    fpc.setSubClusterPolicyConfiguration(SubClusterPolicyConfiguration
+        .newInstance("queue1", "WrongPolicyName", buf));
+    fpc.setFederationSubclusterResolver(
+        FederationPoliciesTestUtil.initResolver());
+    fpc.setFederationStateStoreFacade(FederationPoliciesTestUtil.initFacade());
+    getPolicy().reinitialize(fpc);
+  }
+
+  @Test(expected = NoActiveSubclustersException.class)
+  public void testNoSubclusters() throws YarnException {
+    // empty the activeSubclusters map
+    FederationPoliciesTestUtil.initializePolicyContext(getPolicy(),
+        getPolicyInfo(), new HashMap<>());
+
+    ConfigurableFederationPolicy currentPolicy = getPolicy();
+    if (currentPolicy instanceof FederationRouterPolicy) {
+      ((FederationRouterPolicy) currentPolicy)
+          .getHomeSubcluster(getApplicationSubmissionContext());
+    }
+  }
+
+  public ConfigurableFederationPolicy getPolicy() {
+    return policy;
+  }
+
+  public void setPolicy(ConfigurableFederationPolicy policy) {
+    this.policy = policy;
+  }
+
+  public WeightedPolicyInfo getPolicyInfo() {
+    return policyInfo;
+  }
+
+  public void setPolicyInfo(WeightedPolicyInfo policyInfo) {
+    this.policyInfo = policyInfo;
+  }
+
+  public Map<SubClusterId, SubClusterInfo> getActiveSubclusters() {
+    return activeSubclusters;
+  }
+
+  public void setActiveSubclusters(
+      Map<SubClusterId, SubClusterInfo> activeSubclusters) {
+    this.activeSubclusters = activeSubclusters;
+  }
+
+  public FederationPolicyInitializationContext getFederationPolicyContext() {
+    return federationPolicyContext;
+  }
+
+  public void setFederationPolicyContext(
+      FederationPolicyInitializationContext federationPolicyContext) {
+    this.federationPolicyContext = federationPolicyContext;
+  }
+
+  public ApplicationSubmissionContext getApplicationSubmissionContext() {
+    return applicationSubmissionContext;
+  }
+
+  public void setApplicationSubmissionContext(
+      ApplicationSubmissionContext applicationSubmissionContext) {
+    this.applicationSubmissionContext = applicationSubmissionContext;
+  }
+
+  public Random getRand() {
+    return rand;
+  }
+
+  public void setRand(Random rand) {
+    this.rand = rand;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d38f164/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestFederationPolicyInitializationContextValidator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestFederationPolicyInitializationContextValidator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestFederationPolicyInitializationContextValidator.java
index 4ec04d5..e840b3f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestFederationPolicyInitializationContextValidator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestFederationPolicyInitializationContextValidator.java
@@ -77,7 +77,7 @@ public class TestFederationPolicyInitializationContextValidator {
 
   @Test(expected = FederationPolicyInitializationException.class)
   public void nullConf() throws Exception {
-    context.setFederationPolicyConfiguration(null);
+    context.setSubClusterPolicyConfiguration(null);
     FederationPolicyInitializationContextValidator.validate(context,
         MockPolicyManager.class.getCanonicalName());
   }
@@ -96,8 +96,8 @@ public class TestFederationPolicyInitializationContextValidator {
         MockPolicyManager.class.getCanonicalName());
   }
 
-  private class MockPolicyManager
-      implements FederationPolicyWriter, FederationPolicyConfigurator {
+  private class MockPolicyManager implements FederationPolicyManager {
+
     @Override
     public FederationAMRMProxyPolicy getAMRMPolicy(
         FederationPolicyInitializationContext
@@ -123,6 +123,17 @@ public class TestFederationPolicyInitializationContextValidator {
       return SubClusterPolicyConfiguration
           .newInstance("queue1", this.getClass().getCanonicalName(), buf);
     }
+
+    @Override
+    public String getQueue() {
+      return "default";
+    }
+
+    @Override
+    public void setQueue(String queue) {
+
+    }
+
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d38f164/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestLoadBasedRouterPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestLoadBasedRouterPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestLoadBasedRouterPolicy.java
new file mode 100644
index 0000000..9e94f72
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestLoadBasedRouterPolicy.java
@@ -0,0 +1,109 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies.router;
+
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.federation.policies.BaseFederationPoliciesTest;
+import org.apache.hadoop.yarn.server.federation.policies.dao.WeightedPolicyInfo;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterIdInfo;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterState;
+import org.apache.hadoop.yarn.server.federation.utils.FederationPoliciesTestUtil;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Simple test class for the {@link LoadBasedRouterPolicy}. Test that the
+ * load is properly considered for allocation.
+ */
+public class TestLoadBasedRouterPolicy extends BaseFederationPoliciesTest {
+
+  @Before
+  public void setUp() throws Exception {
+    setPolicy(new LoadBasedRouterPolicy());
+    setPolicyInfo(new WeightedPolicyInfo());
+    Map<SubClusterIdInfo, Float> routerWeights = new HashMap<>();
+    Map<SubClusterIdInfo, Float> amrmWeights = new HashMap<>();
+
+    // simulate 20 active subclusters
+    for (int i = 0; i < 20; i++) {
+      SubClusterIdInfo sc =
+          new SubClusterIdInfo(String.format("sc%02d", i));
+      SubClusterInfo federationSubClusterInfo =
+          SubClusterInfo.newInstance(sc.toId(), null, null, null, null, -1,
+              SubClusterState.SC_RUNNING, -1,
+              generateClusterMetricsInfo(i));
+      getActiveSubclusters().put(sc.toId(), federationSubClusterInfo);
+      float weight = getRand().nextInt(2);
+      if (i == 5) {
+        weight = 1.0f;
+      }
+
+      // 5% chance we omit one of the weights
+      if (i <= 5 || getRand().nextFloat() > 0.05f) {
+        routerWeights.put(sc, weight);
+        amrmWeights.put(sc, weight);
+      }
+    }
+    getPolicyInfo().setRouterPolicyWeights(routerWeights);
+    getPolicyInfo().setAMRMPolicyWeights(amrmWeights);
+
+    FederationPoliciesTestUtil.initializePolicyContext(getPolicy(),
+        getPolicyInfo(), getActiveSubclusters());
+
+  }
+
+  private String generateClusterMetricsInfo(int id) {
+
+    long mem = 1024 * getRand().nextInt(277 * 100 - 1);
+    //plant a best cluster
+    if (id == 5) {
+      mem = 1024 * 277 * 100;
+    }
+    String clusterMetrics =
+        "{\"clusterMetrics\":{\"appsSubmitted\":65," + "\"appsCompleted\":64,"
+            + "\"appsPending\":0,\"appsRunning\":0,\"appsFailed\":0,"
+            + "\"appsKilled\":1,\"reservedMB\":0,\"availableMB\":" + mem + ","
+            + "\"allocatedMB\":0,\"reservedVirtualCores\":0,"
+            + "\"availableVirtualCores\":2216,\"allocatedVirtualCores\":0,"
+            + "\"containersAllocated\":0,\"containersReserved\":0,"
+            + "\"containersPending\":0,\"totalMB\":28364800,"
+            + "\"totalVirtualCores\":2216,\"totalNodes\":278,\"lostNodes\":1,"
+            + "\"unhealthyNodes\":0,\"decommissionedNodes\":0,"
+            + "\"rebootedNodes\":0,\"activeNodes\":277}}\n";
+
+    return clusterMetrics;
+
+  }
+
+  @Test
+  public void testLoadIsRespected() throws YarnException {
+
+    SubClusterId chosen = ((FederationRouterPolicy) getPolicy())
+        .getHomeSubcluster(getApplicationSubmissionContext());
+
+    // check the "planted" best cluster is chosen
+    Assert.assertEquals("sc05", chosen.getId());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d38f164/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestPriorityRouterPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestPriorityRouterPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestPriorityRouterPolicy.java
new file mode 100644
index 0000000..ff5175d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestPriorityRouterPolicy.java
@@ -0,0 +1,87 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.yarn.server.federation.policies.router;
+
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.federation.policies.BaseFederationPoliciesTest;
+import org.apache.hadoop.yarn.server.federation.policies.dao.WeightedPolicyInfo;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterIdInfo;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterState;
+import org.apache.hadoop.yarn.server.federation.utils.FederationPoliciesTestUtil;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * Simple test class for the {@link PriorityRouterPolicy}. Tests that the
+ * weights are correctly used for ordering the choice of sub-clusters.
+ */
+public class TestPriorityRouterPolicy extends BaseFederationPoliciesTest {
+
+  @Before
+  public void setUp() throws Exception {
+    setPolicy(new PriorityRouterPolicy());
+    setPolicyInfo(new WeightedPolicyInfo());
+    Map<SubClusterIdInfo, Float> routerWeights = new HashMap<>();
+    Map<SubClusterIdInfo, Float> amrmWeights = new HashMap<>();
+
+    // simulate 20 subclusters with a 5% chance of being inactive
+    for (int i = 0; i < 20; i++) {
+      SubClusterIdInfo sc = new SubClusterIdInfo("sc" + i);
+
+      // with 5% omit a subcluster
+      if (getRand().nextFloat() < 0.95f || i == 5) {
+        SubClusterInfo sci = mock(SubClusterInfo.class);
+        when(sci.getState()).thenReturn(SubClusterState.SC_RUNNING);
+        when(sci.getSubClusterId()).thenReturn(sc.toId());
+        getActiveSubclusters().put(sc.toId(), sci);
+      }
+      float weight = getRand().nextFloat();
+      if (i == 5) {
+        weight = 1.1f; // guaranteed to be the largest.
+      }
+
+      // 5% chance we omit one of the weights
+      if (i <= 5 || getRand().nextFloat() > 0.05f) {
+        routerWeights.put(sc, weight);
+        amrmWeights.put(sc, weight);
+      }
+    }
+    getPolicyInfo().setRouterPolicyWeights(routerWeights);
+    getPolicyInfo().setAMRMPolicyWeights(amrmWeights);
+    FederationPoliciesTestUtil.initializePolicyContext(getPolicy(),
+        getPolicyInfo(),
+        getActiveSubclusters());
+
+  }
+
+  @Test
+  public void testPickLowestWeight() throws YarnException {
+    SubClusterId chosen = ((FederationRouterPolicy) getPolicy())
+        .getHomeSubcluster(getApplicationSubmissionContext());
+    Assert.assertEquals("sc5", chosen.getId());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d38f164/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestUniformRandomRouterPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestUniformRandomRouterPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestUniformRandomRouterPolicy.java
new file mode 100644
index 0000000..ac41ab5
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestUniformRandomRouterPolicy.java
@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies.router;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.federation.policies.BaseFederationPoliciesTest;
+import org.apache.hadoop.yarn.server.federation.policies.dao.WeightedPolicyInfo;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterIdInfo;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterState;
+import org.apache.hadoop.yarn.server.federation.utils.FederationPoliciesTestUtil;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Simple test class for the {@link UniformRandomRouterPolicy}. Tests that one
+ * of the active subcluster is chosen.
+ */
+public class TestUniformRandomRouterPolicy extends BaseFederationPoliciesTest {
+
+  @Before
+  public void setUp() throws Exception {
+    setPolicy(new UniformRandomRouterPolicy());
+    // needed for base test to work
+    setPolicyInfo(mock(WeightedPolicyInfo.class));
+    for (int i = 1; i <= 2; i++) {
+      SubClusterIdInfo sc = new SubClusterIdInfo("sc" + i);
+      SubClusterInfo sci = mock(SubClusterInfo.class);
+      when(sci.getState()).thenReturn(SubClusterState.SC_RUNNING);
+      when(sci.getSubClusterId()).thenReturn(sc.toId());
+      getActiveSubclusters().put(sc.toId(), sci);
+    }
+
+    FederationPoliciesTestUtil.initializePolicyContext(getPolicy(),
+        mock(WeightedPolicyInfo.class), getActiveSubclusters());
+  }
+
+  @Test
+  public void testOneSubclusterIsChosen() throws YarnException {
+    SubClusterId chosen = ((FederationRouterPolicy) getPolicy())
+        .getHomeSubcluster(getApplicationSubmissionContext());
+    Assert.assertTrue(getActiveSubclusters().keySet().contains(chosen));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d38f164/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestWeightedRandomRouterPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestWeightedRandomRouterPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestWeightedRandomRouterPolicy.java
new file mode 100644
index 0000000..a612685
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestWeightedRandomRouterPolicy.java
@@ -0,0 +1,127 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies.router;
+
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.federation.policies.BaseFederationPoliciesTest;
+import org.apache.hadoop.yarn.server.federation.policies.dao.WeightedPolicyInfo;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterIdInfo;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterState;
+import org.apache.hadoop.yarn.server.federation.utils.FederationPoliciesTestUtil;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * Simple test class for the {@link WeightedRandomRouterPolicy}. Generate large
+ * number of randomized tests to check we are weighiting correctly even if
+ * clusters go inactive.
+ */
+public class TestWeightedRandomRouterPolicy extends BaseFederationPoliciesTest {
+
+  @Before
+  public void setUp() throws Exception {
+    setPolicy(new WeightedRandomRouterPolicy());
+    setPolicyInfo(new WeightedPolicyInfo());
+    Map<SubClusterIdInfo, Float> routerWeights = new HashMap<>();
+    Map<SubClusterIdInfo, Float> amrmWeights = new HashMap<>();
+
+    // simulate 20 subclusters with a 5% chance of being inactive
+    for (int i = 0; i < 20; i++) {
+      SubClusterIdInfo sc = new SubClusterIdInfo("sc" + i);
+      // with 5% omit a subcluster
+      if (getRand().nextFloat() < 0.95f) {
+        SubClusterInfo sci = mock(SubClusterInfo.class);
+        when(sci.getState()).thenReturn(SubClusterState.SC_RUNNING);
+        when(sci.getSubClusterId()).thenReturn(sc.toId());
+        getActiveSubclusters().put(sc.toId(), sci);
+      }
+      // 5% chance we omit one of the weights
+      float weight = getRand().nextFloat();
+      if (i <= 5 || getRand().nextFloat() > 0.05f) {
+        routerWeights.put(sc, weight);
+        amrmWeights.put(sc, weight);
+      }
+    }
+    getPolicyInfo().setRouterPolicyWeights(routerWeights);
+    getPolicyInfo().setAMRMPolicyWeights(amrmWeights);
+
+    FederationPoliciesTestUtil.initializePolicyContext(getPolicy(),
+        getPolicyInfo(),
+        getActiveSubclusters());
+
+  }
+
+  @Test
+  public void testClusterChosenWithRightProbability() throws YarnException {
+
+    Map<SubClusterId, AtomicLong> counter = new HashMap<>();
+    for (SubClusterIdInfo id : getPolicyInfo().getRouterPolicyWeights()
+        .keySet()) {
+      counter.put(id.toId(), new AtomicLong(0));
+    }
+
+    float numberOfDraws = 1000000;
+
+    for (float i = 0; i < numberOfDraws; i++) {
+      SubClusterId chosenId = ((FederationRouterPolicy) getPolicy()).
+          getHomeSubcluster(getApplicationSubmissionContext());
+      counter.get(chosenId).incrementAndGet();
+    }
+
+    float totalActiveWeight = 0;
+    for (SubClusterId id : getActiveSubclusters().keySet()) {
+      SubClusterIdInfo idInfo = new SubClusterIdInfo(id);
+      if (getPolicyInfo().getRouterPolicyWeights().containsKey(idInfo)) {
+        totalActiveWeight +=
+            getPolicyInfo().getRouterPolicyWeights().get(idInfo);
+      }
+    }
+
+    for (Map.Entry<SubClusterId, AtomicLong> counterEntry : counter
+        .entrySet()) {
+      float expectedWeight = getPolicyInfo().getRouterPolicyWeights()
+          .get(new SubClusterIdInfo(counterEntry.getKey())) / totalActiveWeight;
+      float actualWeight = counterEntry.getValue().floatValue() / numberOfDraws;
+
+      // make sure that the weights is respected among active subclusters
+      // and no jobs are routed to inactive subclusters.
+      if (getActiveSubclusters().containsKey(counterEntry.getKey())) {
+        Assert.assertTrue(
+            "Id " + counterEntry.getKey() + " Actual weight: " + actualWeight
+                + " expected weight: " + expectedWeight, expectedWeight == 0 ||
+                (actualWeight / expectedWeight) < 1.1
+                    && (actualWeight / expectedWeight) > 0.9);
+      } else {
+        Assert.assertTrue(
+            "Id " + counterEntry.getKey() + " Actual weight: " + actualWeight
+                + " expected weight: " + expectedWeight, actualWeight == 0);
+
+      }
+    }
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[08/42] hadoop git commit: YARN-3662. Federation Membership State Store internal APIs.

Posted by su...@apache.org.
YARN-3662. Federation Membership State Store internal APIs.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/abe79cb6
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/abe79cb6
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/abe79cb6

Branch: refs/heads/YARN-2915
Commit: abe79cb611afe723a4c5834840b64b499f532cb3
Parents: 7fe7ebb
Author: Subru Krishnan <su...@apache.org>
Authored: Fri Jul 29 16:53:40 2016 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon May 8 16:59:12 2017 -0700

----------------------------------------------------------------------
 .../hadoop-yarn-server-common/pom.xml           |   8 +
 .../store/FederationMembershipStateStore.java   | 126 +++++++++
 .../server/federation/store/package-info.java   |  17 ++
 .../store/records/GetSubClusterInfoRequest.java |  62 +++++
 .../records/GetSubClusterInfoResponse.java      |  62 +++++
 .../records/GetSubClustersInfoRequest.java      |  66 +++++
 .../records/GetSubClustersInfoResponse.java     |  66 +++++
 .../records/SubClusterDeregisterRequest.java    |  89 +++++++
 .../records/SubClusterDeregisterResponse.java   |  42 +++
 .../records/SubClusterHeartbeatRequest.java     | 149 +++++++++++
 .../records/SubClusterHeartbeatResponse.java    |  45 ++++
 .../federation/store/records/SubClusterId.java  | 100 +++++++
 .../store/records/SubClusterInfo.java           | 263 ++++++++++++++++++
 .../records/SubClusterRegisterRequest.java      |  74 +++++
 .../records/SubClusterRegisterResponse.java     |  44 +++
 .../store/records/SubClusterState.java          |  60 +++++
 .../impl/pb/GetSubClusterInfoRequestPBImpl.java | 125 +++++++++
 .../pb/GetSubClusterInfoResponsePBImpl.java     | 134 ++++++++++
 .../pb/GetSubClustersInfoRequestPBImpl.java     | 108 ++++++++
 .../pb/GetSubClustersInfoResponsePBImpl.java    | 184 +++++++++++++
 .../pb/SubClusterDeregisterRequestPBImpl.java   | 156 +++++++++++
 .../pb/SubClusterDeregisterResponsePBImpl.java  |  77 ++++++
 .../pb/SubClusterHeartbeatRequestPBImpl.java    | 192 +++++++++++++
 .../pb/SubClusterHeartbeatResponsePBImpl.java   |  77 ++++++
 .../records/impl/pb/SubClusterIdPBImpl.java     |  75 ++++++
 .../records/impl/pb/SubClusterInfoPBImpl.java   | 267 +++++++++++++++++++
 .../pb/SubClusterRegisterRequestPBImpl.java     | 134 ++++++++++
 .../pb/SubClusterRegisterResponsePBImpl.java    |  77 ++++++
 .../store/records/impl/pb/package-info.java     |  17 ++
 .../federation/store/records/package-info.java  |  17 ++
 .../proto/yarn_server_federation_protos.proto   |  93 +++++++
 .../records/TestFederationProtocolRecords.java  | 133 +++++++++
 32 files changed, 3139 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/abe79cb6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml
index fc23af8..9cc3cae 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml
@@ -58,6 +58,13 @@
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-yarn-common</artifactId>
     </dependency>
+    <!-- 'mvn dependency:analyze' fails to detect use of this dependency -->
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-common</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
 
     <dependency>
       <groupId>com.google.guava</groupId>
@@ -146,6 +153,7 @@
                   <include>yarn_server_common_protos.proto</include>
                   <include>yarn_server_common_service_protos.proto</include>
                   <include>yarn_server_common_service_protos.proto</include>
+                  <include>yarn_server_federation_protos.proto</include>
                   <include>ResourceTracker.proto</include>
                   <include>SCMUploader.proto</include>
                   <include>collectornodemanager_protocol.proto</include>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/abe79cb6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/FederationMembershipStateStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/FederationMembershipStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/FederationMembershipStateStore.java
new file mode 100644
index 0000000..378eadc
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/FederationMembershipStateStore.java
@@ -0,0 +1,126 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterInfoRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterInfoResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClustersInfoRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClustersInfoResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterDeregisterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterDeregisterResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterHeartbeatRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterHeartbeatResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterRegisterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterRegisterResponse;
+import org.apache.hadoop.yarn.server.records.Version;
+
+/**
+ * FederationMembershipStateStore maintains the state of all
+ * <em>subcluster(s)</em> as encapsulated by {@code SubClusterInfo} for all the
+ * subcluster(s) that are participating in federation.
+ */
+@Private
+@Unstable
+public interface FederationMembershipStateStore {
+
+  /**
+   * Get the {@link Version} of the underlying federation membership state
+   * store.
+   *
+   * @return the {@link Version} of the underlying federation membership state
+   *         store
+   */
+  Version getMembershipStateStoreVersion();
+
+  /**
+   * Register a <em>subcluster</em> by publishing capabilities as represented by
+   * {@code SubClusterInfo} to indicate participation in federation. This is
+   * typically done during initialization or restart/failover of the
+   * subcluster's <code>ResourceManager</code>. Upon successful registration, an
+   * identifier for the <em>subcluster</em> which is unique across the federated
+   * cluster is returned. The identifier is static, i.e. preserved across
+   * restarts and failover.
+   *
+   * @param registerSubClusterRequest the capabilities of the subcluster that
+   *          wants to participate in federation. The subcluster id is also
+   *          specified in case registration is triggered by restart/failover
+   * @return response empty on successfully if registration was successful
+   * @throws YarnException if the request is invalid/fails
+   */
+  SubClusterRegisterResponse registerSubCluster(
+      SubClusterRegisterRequest registerSubClusterRequest) throws YarnException;
+
+  /**
+   * Deregister a <em>subcluster</em> identified by {@code SubClusterId} to
+   * change state in federation. This can be done to mark the sub cluster lost,
+   * deregistered, or decommissioned.
+   *
+   * @param subClusterDeregisterRequest - the request to deregister the
+   *          sub-cluster from federation.
+   * @return response empty on successfully deregistering the subcluster state
+   * @throws YarnException if the request is invalid/fails
+   */
+  SubClusterDeregisterResponse deregisterSubCluster(
+      SubClusterDeregisterRequest subClusterDeregisterRequest)
+      throws YarnException;
+
+  /**
+   * Periodic heartbeat from a <code>ResourceManager</code> participating in
+   * federation to indicate liveliness. The heartbeat publishes the current
+   * capabilities as represented by {@code SubClusterInfo} of the subcluster.
+   * Currently response is empty if the operation was successful, if not an
+   * exception reporting reason for a failure.
+   *
+   * @param subClusterHeartbeatRequest the capabilities of the subcluster that
+   *          wants to keep alive its participation in federation
+   * @return response currently empty on if heartbeat was successfully processed
+   * @throws YarnException if the request is invalid/fails
+   */
+  SubClusterHeartbeatResponse subClusterHeartbeat(
+      SubClusterHeartbeatRequest subClusterHeartbeatRequest)
+      throws YarnException;
+
+  /**
+   * Get the membership information of <em>subcluster</em> as identified by
+   * {@code SubClusterId}. The membership information includes the cluster
+   * endpoint and current capabilities as represented by {@code SubClusterInfo}.
+   *
+   * @param subClusterRequest the subcluster whose information is required
+   * @return the {@code SubClusterInfo}
+   * @throws YarnException if the request is invalid/fails
+   */
+  GetSubClusterInfoResponse getSubCluster(
+      GetSubClusterInfoRequest subClusterRequest) throws YarnException;
+
+  /**
+   * Get the membership information of all the <em>subclusters</em> that are
+   * currently participating in federation. The membership information includes
+   * the cluster endpoint and current capabilities as represented by
+   * {@code SubClusterInfo}.
+   *
+   * @param subClustersRequest request for sub-clusters information
+   * @return a map of {@code SubClusterInfo} keyed by the {@code SubClusterId}
+   * @throws YarnException if the request is invalid/fails
+   */
+  GetSubClustersInfoResponse getSubClusters(
+      GetSubClustersInfoRequest subClustersRequest) throws YarnException;
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/abe79cb6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/package-info.java
new file mode 100644
index 0000000..33179e9
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/package-info.java
@@ -0,0 +1,17 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.yarn.server.federation.store;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/abe79cb6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetSubClusterInfoRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetSubClusterInfoRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetSubClusterInfoRequest.java
new file mode 100644
index 0000000..656dea9
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetSubClusterInfoRequest.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * Request class to obtain information about a sub-cluster identified by its
+ * {@link SubClusterId}.
+ */
+@Private
+@Unstable
+public abstract class GetSubClusterInfoRequest {
+
+  @Private
+  @Unstable
+  public static GetSubClusterInfoRequest newInstance(
+      SubClusterId subClusterId) {
+    GetSubClusterInfoRequest subClusterRequest =
+        Records.newRecord(GetSubClusterInfoRequest.class);
+    subClusterRequest.setSubClusterId(subClusterId);
+    return subClusterRequest;
+  }
+
+  /**
+   * Get the {@link SubClusterId} representing the unique identifier of the
+   * subcluster.
+   *
+   * @return the subcluster identifier
+   */
+  @Public
+  @Unstable
+  public abstract SubClusterId getSubClusterId();
+
+  /**
+   * Set the {@link SubClusterId} representing the unique identifier of the
+   * subcluster.
+   *
+   * @param subClusterId the subcluster identifier
+   */
+  @Public
+  @Unstable
+  public abstract void setSubClusterId(SubClusterId subClusterId);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/abe79cb6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetSubClusterInfoResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetSubClusterInfoResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetSubClusterInfoResponse.java
new file mode 100644
index 0000000..f7bc74d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetSubClusterInfoResponse.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * Response to a query with {@link SubClusterInfo} about a sub-cluster.
+ */
+@Private
+@Unstable
+public abstract class GetSubClusterInfoResponse {
+
+  @Private
+  @Unstable
+  public static GetSubClusterInfoResponse newInstance(
+      SubClusterInfo subClusterInfo) {
+    GetSubClusterInfoResponse registerSubClusterRequest =
+        Records.newRecord(GetSubClusterInfoResponse.class);
+    registerSubClusterRequest.setSubClusterInfo(subClusterInfo);
+    return registerSubClusterRequest;
+  }
+
+  /**
+   * Get the {@link SubClusterInfo} encapsulating the information about the
+   * sub-cluster.
+   *
+   * @return the information pertaining to the sub-cluster
+   */
+  @Public
+  @Unstable
+  public abstract SubClusterInfo getSubClusterInfo();
+
+  /**
+   * Set the {@link SubClusterInfo} encapsulating the information about the
+   * sub-cluster.
+   *
+   * @param subClusterInfo the information pertaining to the sub-cluster
+   */
+  @Private
+  @Unstable
+  public abstract void setSubClusterInfo(SubClusterInfo subClusterInfo);
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/abe79cb6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetSubClustersInfoRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetSubClustersInfoRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetSubClustersInfoRequest.java
new file mode 100644
index 0000000..3264d81
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetSubClustersInfoRequest.java
@@ -0,0 +1,66 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * Request class to obtain information about all sub-clusters that are
+ * participating in federation.
+ */
+@Private
+@Unstable
+public abstract class GetSubClustersInfoRequest {
+
+  @Public
+  @Unstable
+  public static GetSubClustersInfoRequest newInstance(
+      boolean filterInactiveSubClusters) {
+    GetSubClustersInfoRequest request =
+        Records.newRecord(GetSubClustersInfoRequest.class);
+    request.setFilterInactiveSubClusters(filterInactiveSubClusters);
+    return request;
+  }
+
+  /**
+   * Get the flag that indicates whether only active sub-clusters should be
+   * returned.
+   *
+   * @return whether to filter out inactive sub-clusters
+   */
+  @Public
+  @Unstable
+  public abstract boolean getFilterInactiveSubClusters();
+
+  /**
+   * Set the flag that indicates whether only active sub-clusters should be
+   * returned.
+   *
+   * @param filterInactiveSubClusters whether to filter out inactive
+   *          sub-clusters
+   */
+  @Public
+  @Unstable
+  public abstract void setFilterInactiveSubClusters(
+      boolean filterInactiveSubClusters);
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/abe79cb6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetSubClustersInfoResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetSubClustersInfoResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetSubClustersInfoResponse.java
new file mode 100644
index 0000000..bcf75ab
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetSubClustersInfoResponse.java
@@ -0,0 +1,66 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records;
+
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * Response to a query with list of {@link SubClusterInfo} about all
+ * sub-clusters that are currently participating in Federation.
+ */
+@Private
+@Unstable
+public abstract class GetSubClustersInfoResponse {
+
+  @Public
+  @Unstable
+  public static GetSubClustersInfoResponse newInstance(
+      List<SubClusterInfo> subClusters) {
+    GetSubClustersInfoResponse subClusterInfos =
+        Records.newRecord(GetSubClustersInfoResponse.class);
+    subClusterInfos.setSubClusters(subClusters);
+    return subClusterInfos;
+  }
+
+  /**
+   * Get the list of {@link SubClusterInfo} representing the information about
+   * all sub-clusters that are currently participating in Federation.
+   *
+   * @return the list of {@link SubClusterInfo}
+   */
+  @Public
+  @Unstable
+  public abstract List<SubClusterInfo> getSubClusters();
+
+  /**
+   * Set the list of {@link SubClusterInfo} representing the information about
+   * all sub-clusters that are currently participating in Federation.
+   *
+   * @param subClusters the list of {@link SubClusterInfo}
+   */
+  @Private
+  @Unstable
+  public abstract void setSubClusters(List<SubClusterInfo> subClusters);
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/abe79cb6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterDeregisterRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterDeregisterRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterDeregisterRequest.java
new file mode 100644
index 0000000..50a50a1
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterDeregisterRequest.java
@@ -0,0 +1,89 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * <p>
+ * The request sent to set the state of a subcluster to either
+ * SC_DECOMMISSIONED, SC_LOST, or SC_DEREGISTERED.
+ *
+ * <p>
+ * The update includes details such as:
+ * <ul>
+ * <li>{@link SubClusterId}</li>
+ * <li>{@link SubClusterState}</li>
+ * </ul>
+ */
+@Private
+@Unstable
+public abstract class SubClusterDeregisterRequest {
+
+  @Private
+  @Unstable
+  public static SubClusterDeregisterRequest newInstance(
+      SubClusterId subClusterId, SubClusterState subClusterState) {
+    SubClusterDeregisterRequest registerRequest =
+        Records.newRecord(SubClusterDeregisterRequest.class);
+    registerRequest.setSubClusterId(subClusterId);
+    registerRequest.setState(subClusterState);
+    return registerRequest;
+  }
+
+  /**
+   * Get the {@link SubClusterId} representing the unique identifier of the
+   * subcluster.
+   *
+   * @return the subcluster identifier
+   */
+  @Public
+  @Unstable
+  public abstract SubClusterId getSubClusterId();
+
+  /**
+   * Set the {@link SubClusterId} representing the unique identifier of the
+   * subcluster.
+   *
+   * @param subClusterId the subcluster identifier
+   */
+  @Private
+  @Unstable
+  public abstract void setSubClusterId(SubClusterId subClusterId);
+
+  /**
+   * Get the {@link SubClusterState} of the subcluster.
+   *
+   * @return the state of the subcluster
+   */
+  @Public
+  @Unstable
+  public abstract SubClusterState getState();
+
+  /**
+   * Set the {@link SubClusterState} of the subcluster.
+   *
+   * @param state the state of the subCluster
+   */
+  @Private
+  @Unstable
+  public abstract void setState(SubClusterState state);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/abe79cb6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterDeregisterResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterDeregisterResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterDeregisterResponse.java
new file mode 100644
index 0000000..74fe994
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterDeregisterResponse.java
@@ -0,0 +1,42 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * SubClusterDeregisterResponse contains the answer from the {@code
+ * FederationMembershipStateStore} to a request to deregister the sub cluster.
+ * Currently response is empty if the operation was successful, if not an
+ * exception reporting reason for a failure.
+ */
+@Private
+@Unstable
+public abstract class SubClusterDeregisterResponse {
+
+  @Private
+  @Unstable
+  public static SubClusterDeregisterResponse newInstance() {
+    SubClusterDeregisterResponse response =
+        Records.newRecord(SubClusterDeregisterResponse.class);
+    return response;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/abe79cb6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterHeartbeatRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterHeartbeatRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterHeartbeatRequest.java
new file mode 100644
index 0000000..3a07c18
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterHeartbeatRequest.java
@@ -0,0 +1,149 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * <p>
+ * SubClusterHeartbeatRequest is a report of the runtime information of the
+ * subcluster that is participating in federation.
+ *
+ * <p>
+ * It includes information such as:
+ * <ul>
+ * <li>{@link SubClusterId}</li>
+ * <li>The URL of the subcluster</li>
+ * <li>The timestamp representing the last start time of the subCluster</li>
+ * <li>{@code FederationsubClusterState}</li>
+ * <li>The current capacity and utilization of the subCluster</li>
+ * </ul>
+ */
+@Private
+@Unstable
+public abstract class SubClusterHeartbeatRequest {
+
+  @Private
+  @Unstable
+  public static SubClusterHeartbeatRequest newInstance(
+      SubClusterId subClusterId, SubClusterState state, String capability) {
+    return newInstance(subClusterId, 0, state, capability);
+  }
+
+  @Private
+  @Unstable
+  public static SubClusterHeartbeatRequest newInstance(
+      SubClusterId subClusterId, long lastHeartBeat, SubClusterState state,
+      String capability) {
+    SubClusterHeartbeatRequest subClusterHeartbeatRequest =
+        Records.newRecord(SubClusterHeartbeatRequest.class);
+    subClusterHeartbeatRequest.setSubClusterId(subClusterId);
+    subClusterHeartbeatRequest.setLastHeartBeat(lastHeartBeat);
+    subClusterHeartbeatRequest.setState(state);
+    subClusterHeartbeatRequest.setCapability(capability);
+    return subClusterHeartbeatRequest;
+  }
+
+  /**
+   * Get the {@link SubClusterId} representing the unique identifier of the
+   * subcluster.
+   *
+   * @return the subcluster identifier
+   */
+  @Public
+  @Unstable
+  public abstract SubClusterId getSubClusterId();
+
+  /**
+   * Set the {@link SubClusterId} representing the unique identifier of the
+   * subCluster.
+   *
+   * @param subClusterId the subCluster identifier
+   */
+  @Private
+  @Unstable
+  public abstract void setSubClusterId(SubClusterId subClusterId);
+
+  /**
+   * Get the last heart beat time of the subcluster.
+   *
+   * @return the state of the subcluster
+   */
+  @Public
+  @Unstable
+  public abstract long getLastHeartBeat();
+
+  /**
+   * Set the last heartbeat time of the subcluster.
+   *
+   * @param time the last heartbeat time of the subcluster
+   */
+  @Private
+  @Unstable
+  public abstract void setLastHeartBeat(long time);
+
+  /**
+   * Get the {@link SubClusterState} of the subcluster.
+   *
+   * @return the state of the subcluster
+   */
+  @Public
+  @Unstable
+  public abstract SubClusterState getState();
+
+  /**
+   * Set the {@link SubClusterState} of the subcluster.
+   *
+   * @param state the state of the subCluster
+   */
+  @Private
+  @Unstable
+  public abstract void setState(SubClusterState state);
+
+  /**
+   * Get the current capacity and utilization of the subcluster. This is the
+   * JAXB marshalled string representation of the <code>ClusterMetrics</code>.
+   *
+   * @return the current capacity and utilization of the subcluster
+   */
+  @Public
+  @Unstable
+  public abstract String getCapability();
+
+  /**
+   * Set the current capacity and utilization of the subCluster. This is the
+   * JAXB marshalled string representation of the <code>ClusterMetrics</code>.
+   *
+   * @param capability the current capacity and utilization of the subcluster
+   */
+  @Private
+  @Unstable
+  public abstract void setCapability(String capability);
+
+  @Override
+  public String toString() {
+    return "SubClusterHeartbeatRequest [getSubClusterId() = "
+        + getSubClusterId() + ", getState() = " + getState()
+        + ", getLastHeartBeat = " + getLastHeartBeat() + ", getCapability() = "
+        + getCapability() + "]";
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/abe79cb6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterHeartbeatResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterHeartbeatResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterHeartbeatResponse.java
new file mode 100644
index 0000000..0b7fd8c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterHeartbeatResponse.java
@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * SubClusterHeartbeatResponse contains the response from the {@code
+ * FederationMembershipStateStore} to a periodic heartbeat to indicate
+ * liveliness from a <code>ResourceManager</code> participating in federation.
+ * Currently response is empty if the operation was successful, if not an
+ * exception reporting reason for a failure.
+ * <p>
+ * NOTE: This can be extended to push down policies in future
+ */
+@Private
+@Unstable
+public abstract class SubClusterHeartbeatResponse {
+
+  @Private
+  @Unstable
+  public static SubClusterHeartbeatResponse newInstance() {
+    SubClusterHeartbeatResponse response =
+        Records.newRecord(SubClusterHeartbeatResponse.class);
+    return response;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/abe79cb6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterId.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterId.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterId.java
new file mode 100644
index 0000000..fec967d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterId.java
@@ -0,0 +1,100 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * <p>
+ * SubClusterId represents the <em>globally unique</em> identifier for a
+ * subcluster that is participating in federation.
+ *
+ * <p>
+ * The globally unique nature of the identifier is obtained from the
+ * <code>FederationMembershipStateStore</code> on initialization.
+ */
+@Private
+@Unstable
+public abstract class SubClusterId implements Comparable<SubClusterId> {
+
+  @Private
+  @Unstable
+  public static SubClusterId newInstance(String subClusterId) {
+    SubClusterId id = Records.newRecord(SubClusterId.class);
+    id.setId(subClusterId);
+    return id;
+  }
+
+  /**
+   * Get the string identifier of the <em>subcluster</em> which is unique across
+   * the federated cluster. The identifier is static, i.e. preserved across
+   * restarts and failover.
+   *
+   * @return unique identifier of the subcluster
+   */
+  @Public
+  @Unstable
+  public abstract String getId();
+
+  /**
+   * Set the string identifier of the <em>subcluster</em> which is unique across
+   * the federated cluster. The identifier is static, i.e. preserved across
+   * restarts and failover.
+   *
+   * @param subClusterId unique identifier of the subcluster
+   */
+  @Private
+  @Unstable
+  protected abstract void setId(String subClusterId);
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (getClass() != obj.getClass()) {
+      return false;
+    }
+    SubClusterId other = (SubClusterId) obj;
+    return this.getId().equals(other.getId());
+  }
+
+  @Override
+  public int hashCode() {
+    return getId().hashCode();
+  }
+
+  @Override
+  public int compareTo(SubClusterId other) {
+    return getId().compareTo(other.getId());
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append(getId());
+    return sb.toString();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/abe79cb6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterInfo.java
new file mode 100644
index 0000000..f13c8f1
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterInfo.java
@@ -0,0 +1,263 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * <p>
+ * SubClusterInfo is a report of the runtime information of the subcluster that
+ * is participating in federation.
+ *
+ * <p>
+ * It includes information such as:
+ * <ul>
+ * <li>{@link SubClusterId}</li>
+ * <li>The URL of the subcluster</li>
+ * <li>The timestamp representing the last start time of the subCluster</li>
+ * <li>{@code FederationsubClusterState}</li>
+ * <li>The current capacity and utilization of the subCluster</li>
+ * </ul>
+ */
+@Private
+@Unstable
+public abstract class SubClusterInfo {
+
+  @Private
+  @Unstable
+  public static SubClusterInfo newInstance(SubClusterId subClusterId,
+      String amRMServiceAddress, String clientRMServiceAddress,
+      String rmAdminServiceAddress, String rmWebServiceAddress,
+      SubClusterState state, long lastStartTime, String capability) {
+    return newInstance(subClusterId, amRMServiceAddress, clientRMServiceAddress,
+        rmAdminServiceAddress, rmWebServiceAddress, 0, state, lastStartTime,
+        capability);
+  }
+
+  @Private
+  @Unstable
+  public static SubClusterInfo newInstance(SubClusterId subClusterId,
+      String amRMServiceAddress, String clientRMServiceAddress,
+      String rmAdminServiceAddress, String rmWebServiceAddress,
+      long lastHeartBeat, SubClusterState state, long lastStartTime,
+      String capability) {
+    SubClusterInfo subClusterInfo = Records.newRecord(SubClusterInfo.class);
+    subClusterInfo.setSubClusterId(subClusterId);
+    subClusterInfo.setAMRMServiceAddress(amRMServiceAddress);
+    subClusterInfo.setClientRMServiceAddress(clientRMServiceAddress);
+    subClusterInfo.setRMAdminServiceAddress(rmAdminServiceAddress);
+    subClusterInfo.setRMWebServiceAddress(rmWebServiceAddress);
+    subClusterInfo.setLastHeartBeat(lastHeartBeat);
+    subClusterInfo.setState(state);
+    subClusterInfo.setLastStartTime(lastStartTime);
+    subClusterInfo.setCapability(capability);
+    return subClusterInfo;
+  }
+
+  /**
+   * Get the {@link SubClusterId} representing the unique identifier of the
+   * subcluster.
+   *
+   * @return the subcluster identifier
+   */
+  @Public
+  @Unstable
+  public abstract SubClusterId getSubClusterId();
+
+  /**
+   * Set the {@link SubClusterId} representing the unique identifier of the
+   * subCluster.
+   *
+   * @param subClusterId the subCluster identifier
+   */
+  @Private
+  @Unstable
+  public abstract void setSubClusterId(SubClusterId subClusterId);
+
+  /**
+   * Get the URL of the AM-RM service endpoint of the subcluster
+   * <code>ResourceManager</code>.
+   *
+   * @return the URL of the AM-RM service endpoint of the subcluster
+   *         <code>ResourceManager</code>
+   */
+  @Public
+  @Unstable
+  public abstract String getAMRMServiceAddress();
+
+  /**
+   * Set the URL of the AM-RM service endpoint of the subcluster
+   * <code>ResourceManager</code>.
+   *
+   * @param amRMServiceAddress the URL of the AM-RM service endpoint of the
+   *          subcluster <code>ResourceManager</code>
+   */
+  @Private
+  @Unstable
+  public abstract void setAMRMServiceAddress(String amRMServiceAddress);
+
+  /**
+   * Get the URL of the client-RM service endpoint of the subcluster
+   * <code>ResourceManager</code>.
+   *
+   * @return the URL of the client-RM service endpoint of the subcluster
+   *         <code>ResourceManager</code>
+   */
+  @Public
+  @Unstable
+  public abstract String getClientRMServiceAddress();
+
+  /**
+   * Set the URL of the client-RM service endpoint of the subcluster
+   * <code>ResourceManager</code>.
+   *
+   * @param clientRMServiceAddress the URL of the client-RM service endpoint of
+   *          the subCluster <code>ResourceManager</code>
+   */
+  @Private
+  @Unstable
+  public abstract void setClientRMServiceAddress(String clientRMServiceAddress);
+
+  /**
+   * Get the URL of the <code>ResourceManager</code> administration service.
+   *
+   * @return the URL of the <code>ResourceManager</code> administration service
+   */
+  @Public
+  @Unstable
+  public abstract String getRMAdminServiceAddress();
+
+  /**
+   * Set the URL of the <code>ResourceManager</code> administration service.
+   *
+   * @param rmAdminServiceAddress the URL of the <code>ResourceManager</code>
+   *          administration service.
+   */
+  @Private
+  @Unstable
+  public abstract void setRMAdminServiceAddress(String rmAdminServiceAddress);
+
+  /**
+   * Get the URL of the <code>ResourceManager</code> web application interface.
+   *
+   * @return the URL of the <code>ResourceManager</code> web application
+   *         interface.
+   */
+  @Public
+  @Unstable
+  public abstract String getRMWebServiceAddress();
+
+  /**
+   * Set the URL of the <code>ResourceManager</code> web application interface.
+   *
+   * @param rmWebServiceAddress the URL of the <code>ResourceManager</code> web
+   *          application interface.
+   */
+  @Private
+  @Unstable
+  public abstract void setRMWebServiceAddress(String rmWebServiceAddress);
+
+  /**
+   * Get the last heart beat time of the subcluster.
+   *
+   * @return the state of the subcluster
+   */
+  @Public
+  @Unstable
+  public abstract long getLastHeartBeat();
+
+  /**
+   * Set the last heartbeat time of the subcluster.
+   *
+   * @param time the last heartbeat time of the subcluster
+   */
+  @Private
+  @Unstable
+  public abstract void setLastHeartBeat(long time);
+
+  /**
+   * Get the {@link SubClusterState} of the subcluster.
+   *
+   * @return the state of the subcluster
+   */
+  @Public
+  @Unstable
+  public abstract SubClusterState getState();
+
+  /**
+   * Set the {@link SubClusterState} of the subcluster.
+   *
+   * @param state the state of the subCluster
+   */
+  @Private
+  @Unstable
+  public abstract void setState(SubClusterState state);
+
+  /**
+   * Get the timestamp representing the last start time of the subcluster.
+   *
+   * @return the timestamp representing the last start time of the subcluster
+   */
+  @Public
+  @Unstable
+  public abstract long getLastStartTime();
+
+  /**
+   * Set the timestamp representing the last start time of the subcluster.
+   *
+   * @param lastStartTime the timestamp representing the last start time of the
+   *          subcluster
+   */
+  @Private
+  @Unstable
+  public abstract void setLastStartTime(long lastStartTime);
+
+  /**
+   * Get the current capacity and utilization of the subcluster. This is the
+   * JAXB marshalled string representation of the <code>ClusterMetrics</code>.
+   *
+   * @return the current capacity and utilization of the subcluster
+   */
+  @Public
+  @Unstable
+  public abstract String getCapability();
+
+  /**
+   * Set the current capacity and utilization of the subCluster. This is the
+   * JAXB marshalled string representation of the <code>ClusterMetrics</code>.
+   *
+   * @param capability the current capacity and utilization of the subcluster
+   */
+  @Private
+  @Unstable
+  public abstract void setCapability(String capability);
+
+  @Override
+  public String toString() {
+    return "SubClusterInfo [getSubClusterId() = " + getSubClusterId()
+        + ", getAMRMServiceAddress() = " + getAMRMServiceAddress()
+        + ", getClientRMServiceAddress() = " + getClientRMServiceAddress()
+        + ", getRMAdminServiceAddress() = " + getRMAdminServiceAddress()
+        + ", getRMWebServiceAddress() = " + getRMWebServiceAddress()
+        + ", getState() = " + getState() + ", getLastStartTime() = "
+        + getLastStartTime() + ", getCapability() = " + getCapability() + "]";
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/abe79cb6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterRegisterRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterRegisterRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterRegisterRequest.java
new file mode 100644
index 0000000..8864fe3
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterRegisterRequest.java
@@ -0,0 +1,74 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * <p>
+ * SubClusterRegisterRequest is a request by a sub-cluster
+ * {@code ResourceManager} to participate in federation.
+ *
+ * <p>
+ * It includes information such as:
+ * <ul>
+ * <li>{@link SubClusterId}</li>
+ * <li>The URL of the subcluster</li>
+ * <li>The timestamp representing the last start time of the subCluster</li>
+ * <li>{@code FederationsubClusterState}</li>
+ * <li>The current capacity and utilization of the subCluster</li>
+ * </ul>
+ */
+@Private
+@Unstable
+public abstract class SubClusterRegisterRequest {
+
+  @Private
+  @Unstable
+  public static SubClusterRegisterRequest newInstance(
+      SubClusterInfo subClusterInfo) {
+    SubClusterRegisterRequest registerSubClusterRequest =
+        Records.newRecord(SubClusterRegisterRequest.class);
+    registerSubClusterRequest.setSubClusterInfo(subClusterInfo);
+    return registerSubClusterRequest;
+  }
+
+  /**
+   * Get the {@link SubClusterInfo} encapsulating the information about the
+   * sub-cluster.
+   *
+   * @return the information pertaining to the sub-cluster
+   */
+  @Public
+  @Unstable
+  public abstract SubClusterInfo getSubClusterInfo();
+
+  /**
+   * Set the {@link SubClusterInfo} encapsulating the information about the
+   * sub-cluster.
+   *
+   * @param subClusterInfo the information pertaining to the sub-cluster
+   */
+  @Public
+  @Unstable
+  public abstract void setSubClusterInfo(SubClusterInfo subClusterInfo);
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/abe79cb6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterRegisterResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterRegisterResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterRegisterResponse.java
new file mode 100644
index 0000000..060a857
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterRegisterResponse.java
@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * SubClusterRegisterResponse contains the response from the {@code
+ * FederationMembershipStateStore} to a registration request from a
+ * <code>ResourceManager</code> to participate in federation.
+ *
+ * Currently response is empty if the operation was successful, if not an
+ * exception reporting reason for a failure.
+ */
+@Private
+@Unstable
+public abstract class SubClusterRegisterResponse {
+
+  @Private
+  @Unstable
+  public static SubClusterRegisterResponse newInstance() {
+    SubClusterRegisterResponse response =
+        Records.newRecord(SubClusterRegisterResponse.class);
+    return response;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/abe79cb6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterState.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterState.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterState.java
new file mode 100644
index 0000000..22cec99
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterState.java
@@ -0,0 +1,60 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+
+/**
+ * <p>
+ * State of a <code>SubCluster</code>.
+ * </p>
+ */
+@Private
+@Unstable
+public enum SubClusterState {
+  /** Newly registered subcluster, before the first heartbeat. */
+  SC_NEW,
+
+  /** Subcluster is registered and the RM sent a heartbeat recently. */
+  SC_RUNNING,
+
+  /** Subcluster is unhealthy. */
+  SC_UNHEALTHY,
+
+  /** Subcluster is in the process of being out of service. */
+  SC_DECOMMISSIONING,
+
+  /** Subcluster is out of service. */
+  SC_DECOMMISSIONED,
+
+  /** RM has not sent a heartbeat for some configured time threshold. */
+  SC_LOST,
+
+  /** Subcluster has unregistered. */
+  SC_UNREGISTERED;
+
+  public boolean isUnusable() {
+    return (this != SC_RUNNING && this != SC_NEW);
+  }
+
+  public boolean isFinal() {
+    return (this == SC_UNREGISTERED || this == SC_DECOMMISSIONED
+        || this == SC_LOST);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/abe79cb6/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/GetSubClusterInfoRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/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/GetSubClusterInfoRequestPBImpl.java b/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/GetSubClusterInfoRequestPBImpl.java
new file mode 100644
index 0000000..c61c419
--- /dev/null
+++ b/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/GetSubClusterInfoRequestPBImpl.java
@@ -0,0 +1,125 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records.impl.pb;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.GetSubClusterInfoRequestProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.GetSubClusterInfoRequestProtoOrBuilder;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SubClusterIdProto;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterInfoRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+
+import com.google.protobuf.TextFormat;
+
+/**
+ * Protocol buffer based implementation of {@link GetSubClusterInfoRequest}.
+ */
+@Private
+@Unstable
+public class GetSubClusterInfoRequestPBImpl extends GetSubClusterInfoRequest {
+
+  private GetSubClusterInfoRequestProto proto =
+      GetSubClusterInfoRequestProto.getDefaultInstance();
+  private GetSubClusterInfoRequestProto.Builder builder = null;
+  private boolean viaProto = false;
+
+  public GetSubClusterInfoRequestPBImpl() {
+    builder = GetSubClusterInfoRequestProto.newBuilder();
+  }
+
+  public GetSubClusterInfoRequestPBImpl(GetSubClusterInfoRequestProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public GetSubClusterInfoRequestProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = GetSubClusterInfoRequestProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  private void mergeLocalToBuilder() {
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null) {
+      return false;
+    }
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
+  }
+
+  @Override
+  public SubClusterId getSubClusterId() {
+    GetSubClusterInfoRequestProtoOrBuilder p = viaProto ? proto : builder;
+    if (!p.hasSubClusterId()) {
+      return null;
+    }
+    return convertFromProtoFormat(p.getSubClusterId());
+  }
+
+  @Override
+  public void setSubClusterId(SubClusterId subClusterId) {
+    maybeInitBuilder();
+    if (subClusterId == null) {
+      builder.clearSubClusterId();
+      return;
+    }
+    builder.setSubClusterId(convertToProtoFormat(subClusterId));
+  }
+
+  private SubClusterId convertFromProtoFormat(SubClusterIdProto sc) {
+    return new SubClusterIdPBImpl(sc);
+  }
+
+  private SubClusterIdProto convertToProtoFormat(SubClusterId sc) {
+    return ((SubClusterIdPBImpl) sc).getProto();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/abe79cb6/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/GetSubClusterInfoResponsePBImpl.java
----------------------------------------------------------------------
diff --git a/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/GetSubClusterInfoResponsePBImpl.java b/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/GetSubClusterInfoResponsePBImpl.java
new file mode 100644
index 0000000..d0bcc33
--- /dev/null
+++ b/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/GetSubClusterInfoResponsePBImpl.java
@@ -0,0 +1,134 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records.impl.pb;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.GetSubClusterInfoResponseProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.GetSubClusterInfoResponseProtoOrBuilder;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SubClusterInfoProto;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterInfoResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
+
+import com.google.protobuf.TextFormat;
+
+/**
+ * Protocol buffer based implementation of {@link GetSubClusterInfoResponse}.
+ */
+@Private
+@Unstable
+public class GetSubClusterInfoResponsePBImpl extends GetSubClusterInfoResponse {
+
+  private GetSubClusterInfoResponseProto proto =
+      GetSubClusterInfoResponseProto.getDefaultInstance();
+  private GetSubClusterInfoResponseProto.Builder builder = null;
+  private boolean viaProto = false;
+
+  private SubClusterInfo subClusterInfo = null;
+
+  public GetSubClusterInfoResponsePBImpl() {
+    builder = GetSubClusterInfoResponseProto.newBuilder();
+  }
+
+  public GetSubClusterInfoResponsePBImpl(GetSubClusterInfoResponseProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public GetSubClusterInfoResponseProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = GetSubClusterInfoResponseProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.subClusterInfo != null) {
+      builder.setSubClusterInfo(convertToProtoFormat(this.subClusterInfo));
+    }
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null) {
+      return false;
+    }
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
+  }
+
+  @Override
+  public SubClusterInfo getSubClusterInfo() {
+    GetSubClusterInfoResponseProtoOrBuilder p = viaProto ? proto : builder;
+    if (this.subClusterInfo != null) {
+      return this.subClusterInfo;
+    }
+    if (!p.hasSubClusterInfo()) {
+      return null;
+    }
+    this.subClusterInfo = convertFromProtoFormat(p.getSubClusterInfo());
+    return this.subClusterInfo;
+  }
+
+  @Override
+  public void setSubClusterInfo(SubClusterInfo subClusterInfo) {
+    maybeInitBuilder();
+    if (subClusterInfo == null) {
+      builder.clearSubClusterInfo();
+    }
+    this.subClusterInfo = subClusterInfo;
+  }
+
+  private SubClusterInfo convertFromProtoFormat(
+      SubClusterInfoProto clusterInfo) {
+    return new SubClusterInfoPBImpl(clusterInfo);
+  }
+
+  private SubClusterInfoProto convertToProtoFormat(SubClusterInfo clusterInfo) {
+    return ((SubClusterInfoPBImpl) clusterInfo).getProto();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/abe79cb6/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/GetSubClustersInfoRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/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/GetSubClustersInfoRequestPBImpl.java b/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/GetSubClustersInfoRequestPBImpl.java
new file mode 100644
index 0000000..2b848c0
--- /dev/null
+++ b/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/GetSubClustersInfoRequestPBImpl.java
@@ -0,0 +1,108 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records.impl.pb;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.GetSubClustersInfoRequestProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.GetSubClustersInfoRequestProtoOrBuilder;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClustersInfoRequest;
+
+import com.google.protobuf.TextFormat;
+
+/**
+ * Protocol buffer based implementation of {@link GetSubClustersInfoRequest}.
+ */
+@Private
+@Unstable
+public class GetSubClustersInfoRequestPBImpl extends GetSubClustersInfoRequest {
+
+  private GetSubClustersInfoRequestProto proto =
+      GetSubClustersInfoRequestProto.getDefaultInstance();
+  private GetSubClustersInfoRequestProto.Builder builder = null;
+  private boolean viaProto = false;
+
+  public GetSubClustersInfoRequestPBImpl() {
+    builder = GetSubClustersInfoRequestProto.newBuilder();
+  }
+
+  public GetSubClustersInfoRequestPBImpl(GetSubClustersInfoRequestProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public GetSubClustersInfoRequestProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = GetSubClustersInfoRequestProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  private void mergeLocalToBuilder() {
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null) {
+      return false;
+    }
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
+  }
+
+  @Override
+  public boolean getFilterInactiveSubClusters() {
+    GetSubClustersInfoRequestProtoOrBuilder p = viaProto ? proto : builder;
+    return p.getFilterInactiveSubclusters();
+  }
+
+  @Override
+  public void setFilterInactiveSubClusters(boolean filterInactiveSubClusters) {
+    maybeInitBuilder();
+    builder.setFilterInactiveSubclusters(filterInactiveSubClusters);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/abe79cb6/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/GetSubClustersInfoResponsePBImpl.java
----------------------------------------------------------------------
diff --git a/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/GetSubClustersInfoResponsePBImpl.java b/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/GetSubClustersInfoResponsePBImpl.java
new file mode 100644
index 0000000..d39ef7f
--- /dev/null
+++ b/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/GetSubClustersInfoResponsePBImpl.java
@@ -0,0 +1,184 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records.impl.pb;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.GetSubClustersInfoResponseProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.GetSubClustersInfoResponseProtoOrBuilder;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SubClusterInfoProto;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClustersInfoResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
+
+import com.google.protobuf.TextFormat;
+
+/**
+ * Protocol buffer based implementation of {@link GetSubClustersInfoResponse}.
+ */
+@Private
+@Unstable
+public class GetSubClustersInfoResponsePBImpl
+    extends GetSubClustersInfoResponse {
+
+  private GetSubClustersInfoResponseProto proto =
+      GetSubClustersInfoResponseProto.getDefaultInstance();
+  private GetSubClustersInfoResponseProto.Builder builder = null;
+  private boolean viaProto = false;
+
+  private List<SubClusterInfo> subClusterInfos;
+
+  public GetSubClustersInfoResponsePBImpl() {
+    builder = GetSubClustersInfoResponseProto.newBuilder();
+  }
+
+  public GetSubClustersInfoResponsePBImpl(
+      GetSubClustersInfoResponseProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public GetSubClustersInfoResponseProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.subClusterInfos != null) {
+      addReservationResourcesToProto();
+    }
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = GetSubClustersInfoResponseProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  @Override
+  public List<SubClusterInfo> getSubClusters() {
+    initSubClustersInfoList();
+    return subClusterInfos;
+  }
+
+  @Override
+  public void setSubClusters(List<SubClusterInfo> subClusters) {
+    if (subClusters == null) {
+      builder.clearSubClusterInfos();
+      return;
+    }
+    this.subClusterInfos = subClusters;
+  }
+
+  private void initSubClustersInfoList() {
+    if (this.subClusterInfos != null) {
+      return;
+    }
+    GetSubClustersInfoResponseProtoOrBuilder p = viaProto ? proto : builder;
+    List<SubClusterInfoProto> subClusterInfosList = p.getSubClusterInfosList();
+    subClusterInfos = new ArrayList<SubClusterInfo>();
+
+    for (SubClusterInfoProto r : subClusterInfosList) {
+      subClusterInfos.add(convertFromProtoFormat(r));
+    }
+  }
+
+  private void addReservationResourcesToProto() {
+    maybeInitBuilder();
+    builder.clearSubClusterInfos();
+    if (subClusterInfos == null) {
+      return;
+    }
+    Iterable<SubClusterInfoProto> iterable =
+        new Iterable<SubClusterInfoProto>() {
+          @Override
+          public Iterator<SubClusterInfoProto> iterator() {
+            return new Iterator<SubClusterInfoProto>() {
+
+              private Iterator<SubClusterInfo> iter =
+                  subClusterInfos.iterator();
+
+              @Override
+              public boolean hasNext() {
+                return iter.hasNext();
+              }
+
+              @Override
+              public SubClusterInfoProto next() {
+                return convertToProtoFormat(iter.next());
+              }
+
+              @Override
+              public void remove() {
+                throw new UnsupportedOperationException();
+              }
+
+            };
+
+          }
+
+        };
+    builder.addAllSubClusterInfos(iterable);
+  }
+
+  private SubClusterInfoProto convertToProtoFormat(SubClusterInfo r) {
+    return ((SubClusterInfoPBImpl) r).getProto();
+  }
+
+  private SubClusterInfoPBImpl convertFromProtoFormat(SubClusterInfoProto r) {
+    return new SubClusterInfoPBImpl(r);
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null) {
+      return false;
+    }
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
+  }
+
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[32/42] hadoop git commit: YARN-5411. Create a proxy chain for ApplicationClientProtocol in the Router. (Giovanni Matteo Fumarola via Subru).

Posted by su...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a93a32e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/DefaultClientRequestInterceptor.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/DefaultClientRequestInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/DefaultClientRequestInterceptor.java
new file mode 100644
index 0000000..12b933b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/DefaultClientRequestInterceptor.java
@@ -0,0 +1,334 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.router.clientrm;
+
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
+import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.FailApplicationAttemptRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.FailApplicationAttemptResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptsRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeLabelsRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeLabelsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerReportRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerReportResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainersRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainersResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetLabelsToNodesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetLabelsToNodesResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.MoveApplicationAcrossQueuesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.MoveApplicationAcrossQueuesResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationListRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationListResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.SignalContainerRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.SignalContainerResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.UpdateApplicationPriorityRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.UpdateApplicationPriorityResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.UpdateApplicationTimeoutsRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.UpdateApplicationTimeoutsResponse;
+import org.apache.hadoop.yarn.client.ClientRMProxy;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Extends the AbstractRequestInterceptorClient class and provides an
+ * implementation that simply forwards the client requests to the cluster
+ * resource manager.
+ *
+ */
+public class DefaultClientRequestInterceptor
+    extends AbstractClientRequestInterceptor {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(DefaultClientRequestInterceptor.class);
+  private ApplicationClientProtocol clientRMProxy;
+  private UserGroupInformation user = null;
+
+  @Override
+  public void init(String userName) {
+    super.init(userName);
+    try {
+      // Do not create a proxy user if user name matches the user name on
+      // current UGI
+      if (userName.equalsIgnoreCase(
+          UserGroupInformation.getCurrentUser().getUserName())) {
+        user = UserGroupInformation.getCurrentUser();
+      } else {
+        user = UserGroupInformation.createProxyUser(userName,
+            UserGroupInformation.getCurrentUser());
+      }
+
+      final Configuration conf = this.getConf();
+
+      clientRMProxy =
+          user.doAs(new PrivilegedExceptionAction<ApplicationClientProtocol>() {
+            @Override
+            public ApplicationClientProtocol run() throws Exception {
+              return ClientRMProxy.createRMProxy(conf,
+                  ApplicationClientProtocol.class);
+            }
+          });
+    } catch (IOException e) {
+      String message = "Error while creating Router ClientRM Service for user:";
+      if (user != null) {
+        message += ", user: " + user;
+      }
+
+      LOG.info(message);
+      throw new YarnRuntimeException(message, e);
+    } catch (Exception e) {
+      throw new YarnRuntimeException(e);
+    }
+  }
+
+  @Override
+  public void setNextInterceptor(ClientRequestInterceptor next) {
+    throw new YarnRuntimeException(
+        "setNextInterceptor is being called on DefaultRequestInterceptor,"
+            + "which should be the last one in the chain "
+            + "Check if the interceptor pipeline configuration is correct");
+  }
+
+  @Override
+  public GetNewApplicationResponse getNewApplication(
+      GetNewApplicationRequest request) throws YarnException, IOException {
+    return clientRMProxy.getNewApplication(request);
+  }
+
+  @Override
+  public SubmitApplicationResponse submitApplication(
+      SubmitApplicationRequest request) throws YarnException, IOException {
+    return clientRMProxy.submitApplication(request);
+  }
+
+  @Override
+  public KillApplicationResponse forceKillApplication(
+      KillApplicationRequest request) throws YarnException, IOException {
+    return clientRMProxy.forceKillApplication(request);
+  }
+
+  @Override
+  public GetClusterMetricsResponse getClusterMetrics(
+      GetClusterMetricsRequest request) throws YarnException, IOException {
+    return clientRMProxy.getClusterMetrics(request);
+  }
+
+  @Override
+  public GetClusterNodesResponse getClusterNodes(GetClusterNodesRequest request)
+      throws YarnException, IOException {
+    return clientRMProxy.getClusterNodes(request);
+  }
+
+  @Override
+  public GetQueueInfoResponse getQueueInfo(GetQueueInfoRequest request)
+      throws YarnException, IOException {
+    return clientRMProxy.getQueueInfo(request);
+  }
+
+  @Override
+  public GetQueueUserAclsInfoResponse getQueueUserAcls(
+      GetQueueUserAclsInfoRequest request) throws YarnException, IOException {
+    return clientRMProxy.getQueueUserAcls(request);
+  }
+
+  @Override
+  public MoveApplicationAcrossQueuesResponse moveApplicationAcrossQueues(
+      MoveApplicationAcrossQueuesRequest request)
+      throws YarnException, IOException {
+    return clientRMProxy.moveApplicationAcrossQueues(request);
+  }
+
+  @Override
+  public GetNewReservationResponse getNewReservation(
+      GetNewReservationRequest request) throws YarnException, IOException {
+    return clientRMProxy.getNewReservation(request);
+  }
+
+  @Override
+  public ReservationSubmissionResponse submitReservation(
+      ReservationSubmissionRequest request) throws YarnException, IOException {
+    return clientRMProxy.submitReservation(request);
+  }
+
+  @Override
+  public ReservationListResponse listReservations(
+      ReservationListRequest request) throws YarnException, IOException {
+    return clientRMProxy.listReservations(request);
+  }
+
+  @Override
+  public ReservationUpdateResponse updateReservation(
+      ReservationUpdateRequest request) throws YarnException, IOException {
+    return clientRMProxy.updateReservation(request);
+  }
+
+  @Override
+  public ReservationDeleteResponse deleteReservation(
+      ReservationDeleteRequest request) throws YarnException, IOException {
+    return clientRMProxy.deleteReservation(request);
+  }
+
+  @Override
+  public GetNodesToLabelsResponse getNodeToLabels(
+      GetNodesToLabelsRequest request) throws YarnException, IOException {
+    return clientRMProxy.getNodeToLabels(request);
+  }
+
+  @Override
+  public GetLabelsToNodesResponse getLabelsToNodes(
+      GetLabelsToNodesRequest request) throws YarnException, IOException {
+    return clientRMProxy.getLabelsToNodes(request);
+  }
+
+  @Override
+  public GetClusterNodeLabelsResponse getClusterNodeLabels(
+      GetClusterNodeLabelsRequest request) throws YarnException, IOException {
+    return clientRMProxy.getClusterNodeLabels(request);
+  }
+
+  @Override
+  public GetApplicationReportResponse getApplicationReport(
+      GetApplicationReportRequest request) throws YarnException, IOException {
+    return clientRMProxy.getApplicationReport(request);
+  }
+
+  @Override
+  public GetApplicationsResponse getApplications(GetApplicationsRequest request)
+      throws YarnException, IOException {
+    return clientRMProxy.getApplications(request);
+  }
+
+  @Override
+  public GetApplicationAttemptReportResponse getApplicationAttemptReport(
+      GetApplicationAttemptReportRequest request)
+      throws YarnException, IOException {
+    return clientRMProxy.getApplicationAttemptReport(request);
+  }
+
+  @Override
+  public GetApplicationAttemptsResponse getApplicationAttempts(
+      GetApplicationAttemptsRequest request) throws YarnException, IOException {
+    return clientRMProxy.getApplicationAttempts(request);
+  }
+
+  @Override
+  public GetContainerReportResponse getContainerReport(
+      GetContainerReportRequest request) throws YarnException, IOException {
+    return clientRMProxy.getContainerReport(request);
+  }
+
+  @Override
+  public GetContainersResponse getContainers(GetContainersRequest request)
+      throws YarnException, IOException {
+    return clientRMProxy.getContainers(request);
+  }
+
+  @Override
+  public GetDelegationTokenResponse getDelegationToken(
+      GetDelegationTokenRequest request) throws YarnException, IOException {
+    return clientRMProxy.getDelegationToken(request);
+  }
+
+  @Override
+  public RenewDelegationTokenResponse renewDelegationToken(
+      RenewDelegationTokenRequest request) throws YarnException, IOException {
+    return clientRMProxy.renewDelegationToken(request);
+  }
+
+  @Override
+  public CancelDelegationTokenResponse cancelDelegationToken(
+      CancelDelegationTokenRequest request) throws YarnException, IOException {
+    return clientRMProxy.cancelDelegationToken(request);
+  }
+
+  @Override
+  public FailApplicationAttemptResponse failApplicationAttempt(
+      FailApplicationAttemptRequest request) throws YarnException, IOException {
+    return clientRMProxy.failApplicationAttempt(request);
+  }
+
+  @Override
+  public UpdateApplicationPriorityResponse updateApplicationPriority(
+      UpdateApplicationPriorityRequest request)
+      throws YarnException, IOException {
+    return clientRMProxy.updateApplicationPriority(request);
+  }
+
+  @Override
+  public SignalContainerResponse signalToContainer(
+      SignalContainerRequest request) throws YarnException, IOException {
+    return clientRMProxy.signalToContainer(request);
+  }
+
+  @Override
+  public UpdateApplicationTimeoutsResponse updateApplicationTimeouts(
+      UpdateApplicationTimeoutsRequest request)
+      throws YarnException, IOException {
+    return clientRMProxy.updateApplicationTimeouts(request);
+  }
+
+  @VisibleForTesting
+  public void setRMClient(ApplicationClientProtocol clientRM) {
+    this.clientRMProxy = clientRM;
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a93a32e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/RouterClientRMService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/RouterClientRMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/RouterClientRMService.java
new file mode 100644
index 0000000..00016dd
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/RouterClientRMService.java
@@ -0,0 +1,544 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.router.clientrm;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ipc.Server;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
+import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.FailApplicationAttemptRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.FailApplicationAttemptResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptsRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeLabelsRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeLabelsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerReportRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerReportResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainersRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainersResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetLabelsToNodesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetLabelsToNodesResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.MoveApplicationAcrossQueuesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.MoveApplicationAcrossQueuesResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationListRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationListResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.SignalContainerRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.SignalContainerResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.UpdateApplicationPriorityRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.UpdateApplicationPriorityResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.UpdateApplicationTimeoutsRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.UpdateApplicationTimeoutsResponse;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.hadoop.yarn.ipc.YarnRPC;
+import org.apache.hadoop.yarn.util.LRUCacheHashMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * RouterClientRMService is a service that runs on each router that can be used
+ * to intercept and inspect ApplicationClientProtocol messages from client to
+ * the cluster resource manager. It listens ApplicationClientProtocol messages
+ * from the client and creates a request intercepting pipeline instance for each
+ * client. The pipeline is a chain of intercepter instances that can inspect and
+ * modify the request/response as needed. The main difference with
+ * AMRMProxyService is the protocol they implement.
+ */
+public class RouterClientRMService extends AbstractService
+    implements ApplicationClientProtocol {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RouterClientRMService.class);
+
+  private Server server;
+  private InetSocketAddress listenerEndpoint;
+
+  // For each user we store an interceptors' pipeline.
+  // For performance issue we use LRU cache to keep in memory the newest ones
+  // and remove the oldest used ones.
+  private Map<String, RequestInterceptorChainWrapper> userPipelineMap;
+
+  public RouterClientRMService() {
+    super(RouterClientRMService.class.getName());
+  }
+
+  @Override
+  protected void serviceStart() throws Exception {
+    LOG.info("Starting Router ClientRMService");
+    Configuration conf = getConfig();
+    YarnRPC rpc = YarnRPC.create(conf);
+    UserGroupInformation.setConfiguration(conf);
+
+    this.listenerEndpoint =
+        conf.getSocketAddr(YarnConfiguration.ROUTER_CLIENTRM_ADDRESS,
+            YarnConfiguration.DEFAULT_ROUTER_CLIENTRM_ADDRESS,
+            YarnConfiguration.DEFAULT_ROUTER_CLIENTRM_PORT);
+
+    int maxCacheSize =
+        conf.getInt(YarnConfiguration.ROUTER_CLIENTRM_PIPELINE_CACHE_MAX_SIZE,
+            YarnConfiguration.DEFAULT_ROUTER_CLIENTRM_PIPELINE_CACHE_MAX_SIZE);
+    this.userPipelineMap = Collections.synchronizedMap(
+        new LRUCacheHashMap<String, RequestInterceptorChainWrapper>(
+            maxCacheSize, true));
+
+    Configuration serverConf = new Configuration(conf);
+
+    int numWorkerThreads =
+        serverConf.getInt(YarnConfiguration.RM_CLIENT_THREAD_COUNT,
+            YarnConfiguration.DEFAULT_RM_CLIENT_THREAD_COUNT);
+
+    this.server = rpc.getServer(ApplicationClientProtocol.class, this,
+        listenerEndpoint, serverConf, null, numWorkerThreads);
+
+    this.server.start();
+    LOG.info("Router ClientRMService listening on address: "
+        + this.server.getListenerAddress());
+    super.serviceStart();
+  }
+
+  @Override
+  protected void serviceStop() throws Exception {
+    LOG.info("Stopping Router ClientRMService");
+    if (this.server != null) {
+      this.server.stop();
+    }
+    userPipelineMap.clear();
+    super.serviceStop();
+  }
+
+  /**
+   * Returns the comma separated intercepter class names from the configuration.
+   *
+   * @param conf
+   * @return the intercepter class names as an instance of ArrayList
+   */
+  private List<String> getInterceptorClassNames(Configuration conf) {
+    String configuredInterceptorClassNames =
+        conf.get(YarnConfiguration.ROUTER_CLIENTRM_INTERCEPTOR_CLASS_PIPELINE,
+            YarnConfiguration.DEFAULT_ROUTER_CLIENTRM_INTERCEPTOR_CLASS);
+
+    List<String> interceptorClassNames = new ArrayList<String>();
+    Collection<String> tempList =
+        StringUtils.getStringCollection(configuredInterceptorClassNames);
+    for (String item : tempList) {
+      interceptorClassNames.add(item.trim());
+    }
+
+    return interceptorClassNames;
+  }
+
+  @Override
+  public GetNewApplicationResponse getNewApplication(
+      GetNewApplicationRequest request) throws YarnException, IOException {
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getNewApplication(request);
+  }
+
+  @Override
+  public SubmitApplicationResponse submitApplication(
+      SubmitApplicationRequest request) throws YarnException, IOException {
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().submitApplication(request);
+  }
+
+  @Override
+  public KillApplicationResponse forceKillApplication(
+      KillApplicationRequest request) throws YarnException, IOException {
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().forceKillApplication(request);
+  }
+
+  @Override
+  public GetClusterMetricsResponse getClusterMetrics(
+      GetClusterMetricsRequest request) throws YarnException, IOException {
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getClusterMetrics(request);
+  }
+
+  @Override
+  public GetClusterNodesResponse getClusterNodes(GetClusterNodesRequest request)
+      throws YarnException, IOException {
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getClusterNodes(request);
+  }
+
+  @Override
+  public GetQueueInfoResponse getQueueInfo(GetQueueInfoRequest request)
+      throws YarnException, IOException {
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getQueueInfo(request);
+  }
+
+  @Override
+  public GetQueueUserAclsInfoResponse getQueueUserAcls(
+      GetQueueUserAclsInfoRequest request) throws YarnException, IOException {
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getQueueUserAcls(request);
+  }
+
+  @Override
+  public MoveApplicationAcrossQueuesResponse moveApplicationAcrossQueues(
+      MoveApplicationAcrossQueuesRequest request)
+      throws YarnException, IOException {
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().moveApplicationAcrossQueues(request);
+  }
+
+  @Override
+  public GetNewReservationResponse getNewReservation(
+      GetNewReservationRequest request) throws YarnException, IOException {
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getNewReservation(request);
+  }
+
+  @Override
+  public ReservationSubmissionResponse submitReservation(
+      ReservationSubmissionRequest request) throws YarnException, IOException {
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().submitReservation(request);
+  }
+
+  @Override
+  public ReservationListResponse listReservations(
+      ReservationListRequest request) throws YarnException, IOException {
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().listReservations(request);
+  }
+
+  @Override
+  public ReservationUpdateResponse updateReservation(
+      ReservationUpdateRequest request) throws YarnException, IOException {
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().updateReservation(request);
+  }
+
+  @Override
+  public ReservationDeleteResponse deleteReservation(
+      ReservationDeleteRequest request) throws YarnException, IOException {
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().deleteReservation(request);
+  }
+
+  @Override
+  public GetNodesToLabelsResponse getNodeToLabels(
+      GetNodesToLabelsRequest request) throws YarnException, IOException {
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getNodeToLabels(request);
+  }
+
+  @Override
+  public GetLabelsToNodesResponse getLabelsToNodes(
+      GetLabelsToNodesRequest request) throws YarnException, IOException {
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getLabelsToNodes(request);
+  }
+
+  @Override
+  public GetClusterNodeLabelsResponse getClusterNodeLabels(
+      GetClusterNodeLabelsRequest request) throws YarnException, IOException {
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getClusterNodeLabels(request);
+  }
+
+  @Override
+  public GetApplicationReportResponse getApplicationReport(
+      GetApplicationReportRequest request) throws YarnException, IOException {
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getApplicationReport(request);
+  }
+
+  @Override
+  public GetApplicationsResponse getApplications(GetApplicationsRequest request)
+      throws YarnException, IOException {
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getApplications(request);
+  }
+
+  @Override
+  public GetApplicationAttemptReportResponse getApplicationAttemptReport(
+      GetApplicationAttemptReportRequest request)
+      throws YarnException, IOException {
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getApplicationAttemptReport(request);
+  }
+
+  @Override
+  public GetApplicationAttemptsResponse getApplicationAttempts(
+      GetApplicationAttemptsRequest request) throws YarnException, IOException {
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getApplicationAttempts(request);
+  }
+
+  @Override
+  public GetContainerReportResponse getContainerReport(
+      GetContainerReportRequest request) throws YarnException, IOException {
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getContainerReport(request);
+  }
+
+  @Override
+  public GetContainersResponse getContainers(GetContainersRequest request)
+      throws YarnException, IOException {
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getContainers(request);
+  }
+
+  @Override
+  public GetDelegationTokenResponse getDelegationToken(
+      GetDelegationTokenRequest request) throws YarnException, IOException {
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getDelegationToken(request);
+  }
+
+  @Override
+  public RenewDelegationTokenResponse renewDelegationToken(
+      RenewDelegationTokenRequest request) throws YarnException, IOException {
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().renewDelegationToken(request);
+  }
+
+  @Override
+  public CancelDelegationTokenResponse cancelDelegationToken(
+      CancelDelegationTokenRequest request) throws YarnException, IOException {
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().cancelDelegationToken(request);
+  }
+
+  @Override
+  public FailApplicationAttemptResponse failApplicationAttempt(
+      FailApplicationAttemptRequest request) throws YarnException, IOException {
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().failApplicationAttempt(request);
+  }
+
+  @Override
+  public UpdateApplicationPriorityResponse updateApplicationPriority(
+      UpdateApplicationPriorityRequest request)
+      throws YarnException, IOException {
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().updateApplicationPriority(request);
+  }
+
+  @Override
+  public SignalContainerResponse signalToContainer(
+      SignalContainerRequest request) throws YarnException, IOException {
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().signalToContainer(request);
+  }
+
+  @Override
+  public UpdateApplicationTimeoutsResponse updateApplicationTimeouts(
+      UpdateApplicationTimeoutsRequest request)
+      throws YarnException, IOException {
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().updateApplicationTimeouts(request);
+  }
+
+  private RequestInterceptorChainWrapper getInterceptorChain()
+      throws IOException {
+    String user = UserGroupInformation.getCurrentUser().getUserName();
+    if (!userPipelineMap.containsKey(user)) {
+      initializePipeline(user);
+    }
+    return userPipelineMap.get(user);
+  }
+
+  /**
+   * Gets the Request intercepter chains for all the users.
+   *
+   * @return the request intercepter chains.
+   */
+  @VisibleForTesting
+  protected Map<String, RequestInterceptorChainWrapper> getPipelines() {
+    return this.userPipelineMap;
+  }
+
+  /**
+   * This method creates and returns reference of the first intercepter in the
+   * chain of request intercepter instances.
+   *
+   * @return the reference of the first intercepter in the chain
+   */
+  @VisibleForTesting
+  protected ClientRequestInterceptor createRequestInterceptorChain() {
+    Configuration conf = getConfig();
+
+    List<String> interceptorClassNames = getInterceptorClassNames(conf);
+
+    ClientRequestInterceptor pipeline = null;
+    ClientRequestInterceptor current = null;
+    for (String interceptorClassName : interceptorClassNames) {
+      try {
+        Class<?> interceptorClass = conf.getClassByName(interceptorClassName);
+        if (ClientRequestInterceptor.class.isAssignableFrom(interceptorClass)) {
+          ClientRequestInterceptor interceptorInstance =
+              (ClientRequestInterceptor) ReflectionUtils
+                  .newInstance(interceptorClass, conf);
+          if (pipeline == null) {
+            pipeline = interceptorInstance;
+            current = interceptorInstance;
+            continue;
+          } else {
+            current.setNextInterceptor(interceptorInstance);
+            current = interceptorInstance;
+          }
+        } else {
+          throw new YarnRuntimeException(
+              "Class: " + interceptorClassName + " not instance of "
+                  + ClientRequestInterceptor.class.getCanonicalName());
+        }
+      } catch (ClassNotFoundException e) {
+        throw new YarnRuntimeException(
+            "Could not instantiate ApplicationClientRequestInterceptor: "
+                + interceptorClassName,
+            e);
+      }
+    }
+
+    if (pipeline == null) {
+      throw new YarnRuntimeException(
+          "RequestInterceptor pipeline is not configured in the system");
+    }
+    return pipeline;
+  }
+
+  /**
+   * Initializes the request intercepter pipeline for the specified application.
+   *
+   * @param user
+   */
+  private void initializePipeline(String user) {
+    RequestInterceptorChainWrapper chainWrapper = null;
+    synchronized (this.userPipelineMap) {
+      if (this.userPipelineMap.containsKey(user)) {
+        LOG.info("Request to start an already existing user: {}"
+            + " was received, so ignoring.", user);
+        return;
+      }
+
+      chainWrapper = new RequestInterceptorChainWrapper();
+      this.userPipelineMap.put(user, chainWrapper);
+    }
+
+    // We register the pipeline instance in the map first and then initialize it
+    // later because chain initialization can be expensive and we would like to
+    // release the lock as soon as possible to prevent other applications from
+    // blocking when one application's chain is initializing
+    LOG.info("Initializing request processing pipeline for application "
+        + "for the user: {}", user);
+
+    try {
+      ClientRequestInterceptor interceptorChain =
+          this.createRequestInterceptorChain();
+      interceptorChain.init(user);
+      chainWrapper.init(interceptorChain);
+    } catch (Exception e) {
+      synchronized (this.userPipelineMap) {
+        this.userPipelineMap.remove(user);
+      }
+      throw e;
+    }
+  }
+
+  /**
+   * Private structure for encapsulating RequestInterceptor and user instances.
+   *
+   */
+  @Private
+  public static class RequestInterceptorChainWrapper {
+    private ClientRequestInterceptor rootInterceptor;
+
+    /**
+     * Initializes the wrapper with the specified parameters.
+     *
+     * @param interceptor the first interceptor in the pipeline
+     */
+    public synchronized void init(ClientRequestInterceptor interceptor) {
+      this.rootInterceptor = interceptor;
+    }
+
+    /**
+     * Gets the root request intercepter.
+     *
+     * @return the root request intercepter
+     */
+    public synchronized ClientRequestInterceptor getRootInterceptor() {
+      return rootInterceptor;
+    }
+
+    /**
+     * Shutdown the chain of interceptors when the object is destroyed.
+     */
+    @Override
+    protected void finalize() {
+      rootInterceptor.shutdown();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a93a32e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/package-info.java
new file mode 100644
index 0000000..7d1dadd
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/package-info.java
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/** Router ClientRM Proxy Service package. **/
+package org.apache.hadoop.yarn.server.router.clientrm;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a93a32e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/TestRouter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/TestRouter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/TestRouter.java
deleted file mode 100644
index a31d6b9..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/TestRouter.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.server.router;
-
-/**
- * Test class for YARN Router.
- */
-public class TestRouter {
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a93a32e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/BaseRouterClientRMTest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/BaseRouterClientRMTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/BaseRouterClientRMTest.java
new file mode 100644
index 0000000..a283a62
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/BaseRouterClientRMTest.java
@@ -0,0 +1,574 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.router.clientrm;
+
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+import java.util.Collections;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptsRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeLabelsRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeLabelsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerReportRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerReportResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainersRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainersResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetLabelsToNodesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetLabelsToNodesResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.MoveApplicationAcrossQueuesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.MoveApplicationAcrossQueuesResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationResponse;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ReservationDefinition;
+import org.apache.hadoop.yarn.api.records.ReservationId;
+import org.apache.hadoop.yarn.api.records.ReservationRequest;
+import org.apache.hadoop.yarn.api.records.ReservationRequestInterpreter;
+import org.apache.hadoop.yarn.api.records.ReservationRequests;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.Token;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.event.AsyncDispatcher;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.util.Clock;
+import org.apache.hadoop.yarn.util.UTCClock;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+
+/**
+ * Base class for all the RouterClientRMService test cases. It provides utility
+ * methods that can be used by the concrete test case classes.
+ *
+ */
+public abstract class BaseRouterClientRMTest {
+
+  /**
+   * The RouterClientRMService instance that will be used by all the test cases.
+   */
+  private MockRouterClientRMService clientrmService;
+  /**
+   * Thread pool used for asynchronous operations.
+   */
+  private static ExecutorService threadpool = Executors.newCachedThreadPool();
+  private Configuration conf;
+  private AsyncDispatcher dispatcher;
+
+  public final static int TEST_MAX_CACHE_SIZE = 10;
+
+  protected MockRouterClientRMService getRouterClientRMService() {
+    Assert.assertNotNull(this.clientrmService);
+    return this.clientrmService;
+  }
+
+  @Before
+  public void setUp() {
+    this.conf = new YarnConfiguration();
+    String mockPassThroughInterceptorClass =
+        PassThroughClientRequestInterceptor.class.getName();
+
+    // Create a request intercepter pipeline for testing. The last one in the
+    // chain will call the mock resource manager. The others in the chain will
+    // simply forward it to the next one in the chain
+    this.conf.set(YarnConfiguration.ROUTER_CLIENTRM_INTERCEPTOR_CLASS_PIPELINE,
+        mockPassThroughInterceptorClass + "," + mockPassThroughInterceptorClass
+            + "," + mockPassThroughInterceptorClass + ","
+            + MockClientRequestInterceptor.class.getName());
+
+    this.conf.setInt(YarnConfiguration.ROUTER_CLIENTRM_PIPELINE_CACHE_MAX_SIZE,
+        TEST_MAX_CACHE_SIZE);
+
+    this.dispatcher = new AsyncDispatcher();
+    this.dispatcher.init(conf);
+    this.dispatcher.start();
+    this.clientrmService = createAndStartRouterClientRMService();
+  }
+
+  @After
+  public void tearDown() {
+    if (clientrmService != null) {
+      clientrmService.stop();
+      clientrmService = null;
+    }
+    if (this.dispatcher != null) {
+      this.dispatcher.stop();
+    }
+  }
+
+  protected ExecutorService getThreadPool() {
+    return threadpool;
+  }
+
+  protected MockRouterClientRMService createAndStartRouterClientRMService() {
+    MockRouterClientRMService svc = new MockRouterClientRMService();
+    svc.init(conf);
+    svc.start();
+    return svc;
+  }
+
+  protected static class MockRouterClientRMService
+      extends RouterClientRMService {
+    public MockRouterClientRMService() {
+      super();
+    }
+  }
+
+  protected GetNewApplicationResponse getNewApplication(String user)
+      throws YarnException, IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<GetNewApplicationResponse>() {
+          @Override
+          public GetNewApplicationResponse run() throws Exception {
+            GetNewApplicationRequest req =
+                GetNewApplicationRequest.newInstance();
+            GetNewApplicationResponse response =
+                getRouterClientRMService().getNewApplication(req);
+            return response;
+          }
+        });
+  }
+
+  protected SubmitApplicationResponse submitApplication(
+      final ApplicationId appId, String user)
+      throws YarnException, IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<SubmitApplicationResponse>() {
+          @Override
+          public SubmitApplicationResponse run() throws Exception {
+            ApplicationSubmissionContext context =
+                ApplicationSubmissionContext.newInstance(appId, "", "", null,
+                    null, false, false, -1, null, null);
+            SubmitApplicationRequest req =
+                SubmitApplicationRequest.newInstance(context);
+            SubmitApplicationResponse response =
+                getRouterClientRMService().submitApplication(req);
+            return response;
+          }
+        });
+  }
+
+  protected KillApplicationResponse forceKillApplication(
+      final ApplicationId appId, String user)
+      throws YarnException, IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<KillApplicationResponse>() {
+          @Override
+          public KillApplicationResponse run() throws Exception {
+            KillApplicationRequest req =
+                KillApplicationRequest.newInstance(appId);
+            KillApplicationResponse response =
+                getRouterClientRMService().forceKillApplication(req);
+            return response;
+          }
+        });
+  }
+
+  protected GetClusterMetricsResponse getClusterMetrics(String user)
+      throws YarnException, IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<GetClusterMetricsResponse>() {
+          @Override
+          public GetClusterMetricsResponse run() throws Exception {
+            GetClusterMetricsRequest req =
+                GetClusterMetricsRequest.newInstance();
+            GetClusterMetricsResponse response =
+                getRouterClientRMService().getClusterMetrics(req);
+            return response;
+          }
+        });
+  }
+
+  protected GetClusterNodesResponse getClusterNodes(String user)
+      throws YarnException, IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<GetClusterNodesResponse>() {
+          @Override
+          public GetClusterNodesResponse run() throws Exception {
+            GetClusterNodesRequest req = GetClusterNodesRequest.newInstance();
+            GetClusterNodesResponse response =
+                getRouterClientRMService().getClusterNodes(req);
+            return response;
+          }
+        });
+  }
+
+  protected GetQueueInfoResponse getQueueInfo(String user)
+      throws YarnException, IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<GetQueueInfoResponse>() {
+          @Override
+          public GetQueueInfoResponse run() throws Exception {
+            GetQueueInfoRequest req =
+                GetQueueInfoRequest.newInstance("default", false, false, false);
+            GetQueueInfoResponse response =
+                getRouterClientRMService().getQueueInfo(req);
+            return response;
+          }
+        });
+  }
+
+  protected GetQueueUserAclsInfoResponse getQueueUserAcls(String user)
+      throws YarnException, IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<GetQueueUserAclsInfoResponse>() {
+          @Override
+          public GetQueueUserAclsInfoResponse run() throws Exception {
+            GetQueueUserAclsInfoRequest req =
+                GetQueueUserAclsInfoRequest.newInstance();
+            GetQueueUserAclsInfoResponse response =
+                getRouterClientRMService().getQueueUserAcls(req);
+            return response;
+          }
+        });
+  }
+
+  protected MoveApplicationAcrossQueuesResponse moveApplicationAcrossQueues(
+      String user, final ApplicationId appId)
+      throws YarnException, IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user).doAs(
+        new PrivilegedExceptionAction<MoveApplicationAcrossQueuesResponse>() {
+          @Override
+          public MoveApplicationAcrossQueuesResponse run() throws Exception {
+
+            MoveApplicationAcrossQueuesRequest req =
+                MoveApplicationAcrossQueuesRequest.newInstance(appId,
+                    "newQueue");
+            MoveApplicationAcrossQueuesResponse response =
+                getRouterClientRMService().moveApplicationAcrossQueues(req);
+            return response;
+          }
+        });
+  }
+
+  public GetNewReservationResponse getNewReservation(String user)
+      throws YarnException, IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<GetNewReservationResponse>() {
+          @Override
+          public GetNewReservationResponse run() throws Exception {
+            GetNewReservationResponse response = getRouterClientRMService()
+                .getNewReservation(GetNewReservationRequest.newInstance());
+            return response;
+          }
+        });
+  }
+
+  protected ReservationSubmissionResponse submitReservation(String user,
+      final ReservationId reservationId)
+      throws YarnException, IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<ReservationSubmissionResponse>() {
+          @Override
+          public ReservationSubmissionResponse run() throws Exception {
+            Clock clock = new UTCClock();
+            long arrival = clock.getTime();
+            long duration = 60000;
+            long deadline = (long) (arrival + 1.05 * duration);
+
+            ReservationSubmissionRequest req = createSimpleReservationRequest(1,
+                arrival, deadline, duration, reservationId);
+            ReservationSubmissionResponse response =
+                getRouterClientRMService().submitReservation(req);
+            return response;
+          }
+        });
+  }
+
+  protected ReservationUpdateResponse updateReservation(String user,
+      final ReservationId reservationId)
+      throws YarnException, IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<ReservationUpdateResponse>() {
+          @Override
+          public ReservationUpdateResponse run() throws Exception {
+            Clock clock = new UTCClock();
+            long arrival = clock.getTime();
+            long duration = 60000;
+            long deadline = (long) (arrival + 1.05 * duration);
+            ReservationDefinition rDef =
+                createSimpleReservationRequest(1, arrival, deadline, duration,
+                    reservationId).getReservationDefinition();
+
+            ReservationUpdateRequest req =
+                ReservationUpdateRequest.newInstance(rDef, reservationId);
+            ReservationUpdateResponse response =
+                getRouterClientRMService().updateReservation(req);
+            return response;
+          }
+        });
+  }
+
+  protected ReservationDeleteResponse deleteReservation(String user,
+      final ReservationId reservationId)
+      throws YarnException, IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<ReservationDeleteResponse>() {
+          @Override
+          public ReservationDeleteResponse run() throws Exception {
+            ReservationDeleteRequest req =
+                ReservationDeleteRequest.newInstance(reservationId);
+            ReservationDeleteResponse response =
+                getRouterClientRMService().deleteReservation(req);
+            return response;
+          }
+        });
+  }
+
+  protected GetNodesToLabelsResponse getNodeToLabels(String user)
+      throws YarnException, IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<GetNodesToLabelsResponse>() {
+          @Override
+          public GetNodesToLabelsResponse run() throws Exception {
+            GetNodesToLabelsRequest req = GetNodesToLabelsRequest.newInstance();
+            GetNodesToLabelsResponse response =
+                getRouterClientRMService().getNodeToLabels(req);
+            return response;
+          }
+        });
+  }
+
+  protected GetLabelsToNodesResponse getLabelsToNodes(String user)
+      throws YarnException, IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<GetLabelsToNodesResponse>() {
+          @Override
+          public GetLabelsToNodesResponse run() throws Exception {
+            GetLabelsToNodesRequest req = GetLabelsToNodesRequest.newInstance();
+            GetLabelsToNodesResponse response =
+                getRouterClientRMService().getLabelsToNodes(req);
+            return response;
+          }
+        });
+  }
+
+  protected GetClusterNodeLabelsResponse getClusterNodeLabels(String user)
+      throws YarnException, IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<GetClusterNodeLabelsResponse>() {
+          @Override
+          public GetClusterNodeLabelsResponse run() throws Exception {
+            GetClusterNodeLabelsRequest req =
+                GetClusterNodeLabelsRequest.newInstance();
+            GetClusterNodeLabelsResponse response =
+                getRouterClientRMService().getClusterNodeLabels(req);
+            return response;
+          }
+        });
+  }
+
+  protected GetApplicationReportResponse getApplicationReport(String user,
+      final ApplicationId appId)
+      throws YarnException, IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<GetApplicationReportResponse>() {
+          @Override
+          public GetApplicationReportResponse run() throws Exception {
+            GetApplicationReportRequest req =
+                GetApplicationReportRequest.newInstance(appId);
+            GetApplicationReportResponse response =
+                getRouterClientRMService().getApplicationReport(req);
+            return response;
+          }
+        });
+  }
+
+  protected GetApplicationsResponse getApplications(String user)
+      throws YarnException, IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<GetApplicationsResponse>() {
+          @Override
+          public GetApplicationsResponse run() throws Exception {
+            GetApplicationsRequest req = GetApplicationsRequest.newInstance();
+            GetApplicationsResponse response =
+                getRouterClientRMService().getApplications(req);
+            return response;
+          }
+        });
+  }
+
+  protected GetApplicationAttemptReportResponse getApplicationAttemptReport(
+      String user, final ApplicationAttemptId appAttemptId)
+      throws YarnException, IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user).doAs(
+        new PrivilegedExceptionAction<GetApplicationAttemptReportResponse>() {
+          @Override
+          public GetApplicationAttemptReportResponse run() throws Exception {
+            GetApplicationAttemptReportRequest req =
+                GetApplicationAttemptReportRequest.newInstance(appAttemptId);
+            GetApplicationAttemptReportResponse response =
+                getRouterClientRMService().getApplicationAttemptReport(req);
+            return response;
+          }
+        });
+  }
+
+  protected GetApplicationAttemptsResponse getApplicationAttempts(String user,
+      final ApplicationId applicationId)
+      throws YarnException, IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<GetApplicationAttemptsResponse>() {
+          @Override
+          public GetApplicationAttemptsResponse run() throws Exception {
+            GetApplicationAttemptsRequest req =
+                GetApplicationAttemptsRequest.newInstance(applicationId);
+            GetApplicationAttemptsResponse response =
+                getRouterClientRMService().getApplicationAttempts(req);
+            return response;
+          }
+        });
+  }
+
+  protected GetContainerReportResponse getContainerReport(String user,
+      final ContainerId containerId)
+      throws YarnException, IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<GetContainerReportResponse>() {
+          @Override
+          public GetContainerReportResponse run() throws Exception {
+            GetContainerReportRequest req =
+                GetContainerReportRequest.newInstance(containerId);
+            GetContainerReportResponse response =
+                getRouterClientRMService().getContainerReport(req);
+            return response;
+          }
+        });
+  }
+
+  protected GetContainersResponse getContainers(String user,
+      final ApplicationAttemptId appAttemptId)
+      throws YarnException, IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<GetContainersResponse>() {
+          @Override
+          public GetContainersResponse run() throws Exception {
+            GetContainersRequest req =
+                GetContainersRequest.newInstance(appAttemptId);
+            GetContainersResponse response =
+                getRouterClientRMService().getContainers(req);
+            return response;
+          }
+        });
+  }
+
+  protected GetDelegationTokenResponse getDelegationToken(final String user)
+      throws YarnException, IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<GetDelegationTokenResponse>() {
+          @Override
+          public GetDelegationTokenResponse run() throws Exception {
+            GetDelegationTokenRequest req =
+                GetDelegationTokenRequest.newInstance(user);
+            GetDelegationTokenResponse response =
+                getRouterClientRMService().getDelegationToken(req);
+            return response;
+          }
+        });
+  }
+
+  protected RenewDelegationTokenResponse renewDelegationToken(String user,
+      final Token token)
+      throws YarnException, IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<RenewDelegationTokenResponse>() {
+          @Override
+          public RenewDelegationTokenResponse run() throws Exception {
+            RenewDelegationTokenRequest req =
+                RenewDelegationTokenRequest.newInstance(token);
+            RenewDelegationTokenResponse response =
+                getRouterClientRMService().renewDelegationToken(req);
+            return response;
+          }
+        });
+  }
+
+  protected CancelDelegationTokenResponse cancelDelegationToken(String user,
+      final Token token)
+      throws YarnException, IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<CancelDelegationTokenResponse>() {
+          @Override
+          public CancelDelegationTokenResponse run() throws Exception {
+            CancelDelegationTokenRequest req =
+                CancelDelegationTokenRequest.newInstance(token);
+            CancelDelegationTokenResponse response =
+                getRouterClientRMService().cancelDelegationToken(req);
+            return response;
+          }
+        });
+  }
+
+  private ReservationSubmissionRequest createSimpleReservationRequest(
+      int numContainers, long arrival, long deadline, long duration,
+      ReservationId reservationId) {
+    // create a request with a single atomic ask
+    ReservationRequest r = ReservationRequest
+        .newInstance(Resource.newInstance(1024, 1), numContainers, 1, duration);
+    ReservationRequests reqs = ReservationRequests.newInstance(
+        Collections.singletonList(r), ReservationRequestInterpreter.R_ALL);
+    ReservationDefinition rDef = ReservationDefinition.newInstance(arrival,
+        deadline, reqs, "testRouterClientRMService#reservation");
+    ReservationSubmissionRequest request = ReservationSubmissionRequest
+        .newInstance(rDef, "dedicated", reservationId);
+    return request;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a93a32e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/MockClientRequestInterceptor.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/MockClientRequestInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/MockClientRequestInterceptor.java
new file mode 100644
index 0000000..b38703f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/MockClientRequestInterceptor.java
@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.router.clientrm;
+
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.MockResourceManagerFacade;
+
+/**
+ * This class mocks the ClientRequestInterceptor.
+ */
+public class MockClientRequestInterceptor
+    extends DefaultClientRequestInterceptor {
+
+  public void init(String user) {
+    MockResourceManagerFacade mockRM = new MockResourceManagerFacade(
+        new YarnConfiguration(super.getConf()), 0);
+    super.setRMClient(mockRM);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a93a32e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/PassThroughClientRequestInterceptor.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/PassThroughClientRequestInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/PassThroughClientRequestInterceptor.java
new file mode 100644
index 0000000..c403bd5
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/PassThroughClientRequestInterceptor.java
@@ -0,0 +1,267 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.router.clientrm;
+
+import java.io.IOException;
+
+import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.FailApplicationAttemptRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.FailApplicationAttemptResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptsRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeLabelsRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeLabelsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerReportRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerReportResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainersRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainersResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetLabelsToNodesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetLabelsToNodesResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.MoveApplicationAcrossQueuesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.MoveApplicationAcrossQueuesResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationListRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationListResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.SignalContainerRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.SignalContainerResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.UpdateApplicationPriorityRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.UpdateApplicationPriorityResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.UpdateApplicationTimeoutsRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.UpdateApplicationTimeoutsResponse;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+
+/**
+ * Mock intercepter that does not do anything other than forwarding it to the
+ * next intercepter in the chain.
+ */
+public class PassThroughClientRequestInterceptor
+    extends AbstractClientRequestInterceptor {
+
+  @Override
+  public GetNewApplicationResponse getNewApplication(
+      GetNewApplicationRequest request) throws YarnException, IOException {
+    return getNextInterceptor().getNewApplication(request);
+  }
+
+  @Override
+  public SubmitApplicationResponse submitApplication(
+      SubmitApplicationRequest request) throws YarnException, IOException {
+    return getNextInterceptor().submitApplication(request);
+  }
+
+  @Override
+  public KillApplicationResponse forceKillApplication(
+      KillApplicationRequest request) throws YarnException, IOException {
+    return getNextInterceptor().forceKillApplication(request);
+  }
+
+  @Override
+  public GetClusterMetricsResponse getClusterMetrics(
+      GetClusterMetricsRequest request) throws YarnException, IOException {
+    return getNextInterceptor().getClusterMetrics(request);
+  }
+
+  @Override
+  public GetClusterNodesResponse getClusterNodes(GetClusterNodesRequest request)
+      throws YarnException, IOException {
+    return getNextInterceptor().getClusterNodes(request);
+  }
+
+  @Override
+  public GetQueueInfoResponse getQueueInfo(GetQueueInfoRequest request)
+      throws YarnException, IOException {
+    return getNextInterceptor().getQueueInfo(request);
+  }
+
+  @Override
+  public GetQueueUserAclsInfoResponse getQueueUserAcls(
+      GetQueueUserAclsInfoRequest request) throws YarnException, IOException {
+    return getNextInterceptor().getQueueUserAcls(request);
+  }
+
+  @Override
+  public MoveApplicationAcrossQueuesResponse moveApplicationAcrossQueues(
+      MoveApplicationAcrossQueuesRequest request)
+      throws YarnException, IOException {
+    return getNextInterceptor().moveApplicationAcrossQueues(request);
+  }
+
+  @Override
+  public GetNewReservationResponse getNewReservation(
+      GetNewReservationRequest request) throws YarnException, IOException {
+    return getNextInterceptor().getNewReservation(request);
+  }
+
+  @Override
+  public ReservationSubmissionResponse submitReservation(
+      ReservationSubmissionRequest request) throws YarnException, IOException {
+    return getNextInterceptor().submitReservation(request);
+  }
+
+  @Override
+  public ReservationListResponse listReservations(
+      ReservationListRequest request) throws YarnException, IOException {
+    return getNextInterceptor().listReservations(request);
+  }
+
+  @Override
+  public ReservationUpdateResponse updateReservation(
+      ReservationUpdateRequest request) throws YarnException, IOException {
+    return getNextInterceptor().updateReservation(request);
+  }
+
+  @Override
+  public ReservationDeleteResponse deleteReservation(
+      ReservationDeleteRequest request) throws YarnException, IOException {
+    return getNextInterceptor().deleteReservation(request);
+  }
+
+  @Override
+  public GetNodesToLabelsResponse getNodeToLabels(
+      GetNodesToLabelsRequest request) throws YarnException, IOException {
+    return getNextInterceptor().getNodeToLabels(request);
+  }
+
+  @Override
+  public GetLabelsToNodesResponse getLabelsToNodes(
+      GetLabelsToNodesRequest request) throws YarnException, IOException {
+    return getNextInterceptor().getLabelsToNodes(request);
+  }
+
+  @Override
+  public GetClusterNodeLabelsResponse getClusterNodeLabels(
+      GetClusterNodeLabelsRequest request) throws YarnException, IOException {
+    return getNextInterceptor().getClusterNodeLabels(request);
+  }
+
+  @Override
+  public GetApplicationReportResponse getApplicationReport(
+      GetApplicationReportRequest request) throws YarnException, IOException {
+    return getNextInterceptor().getApplicationReport(request);
+  }
+
+  @Override
+  public GetApplicationsResponse getApplications(GetApplicationsRequest request)
+      throws YarnException, IOException {
+    return getNextInterceptor().getApplications(request);
+  }
+
+  @Override
+  public GetApplicationAttemptReportResponse getApplicationAttemptReport(
+      GetApplicationAttemptReportRequest request)
+      throws YarnException, IOException {
+    return getNextInterceptor().getApplicationAttemptReport(request);
+  }
+
+  @Override
+  public GetApplicationAttemptsResponse getApplicationAttempts(
+      GetApplicationAttemptsRequest request) throws YarnException, IOException {
+    return getNextInterceptor().getApplicationAttempts(request);
+  }
+
+  @Override
+  public GetContainerReportResponse getContainerReport(
+      GetContainerReportRequest request) throws YarnException, IOException {
+    return getNextInterceptor().getContainerReport(request);
+  }
+
+  @Override
+  public GetContainersResponse getContainers(GetContainersRequest request)
+      throws YarnException, IOException {
+    return getNextInterceptor().getContainers(request);
+  }
+
+  @Override
+  public GetDelegationTokenResponse getDelegationToken(
+      GetDelegationTokenRequest request) throws YarnException, IOException {
+    return getNextInterceptor().getDelegationToken(request);
+  }
+
+  @Override
+  public RenewDelegationTokenResponse renewDelegationToken(
+      RenewDelegationTokenRequest request) throws YarnException, IOException {
+    return getNextInterceptor().renewDelegationToken(request);
+  }
+
+  @Override
+  public CancelDelegationTokenResponse cancelDelegationToken(
+      CancelDelegationTokenRequest request) throws YarnException, IOException {
+    return getNextInterceptor().cancelDelegationToken(request);
+  }
+
+  @Override
+  public FailApplicationAttemptResponse failApplicationAttempt(
+      FailApplicationAttemptRequest request) throws YarnException, IOException {
+    return getNextInterceptor().failApplicationAttempt(request);
+  }
+
+  @Override
+  public UpdateApplicationPriorityResponse updateApplicationPriority(
+      UpdateApplicationPriorityRequest request)
+      throws YarnException, IOException {
+    return getNextInterceptor().updateApplicationPriority(request);
+  }
+
+  @Override
+  public SignalContainerResponse signalToContainer(
+      SignalContainerRequest request) throws YarnException, IOException {
+    return getNextInterceptor().signalToContainer(request);
+  }
+
+  @Override
+  public UpdateApplicationTimeoutsResponse updateApplicationTimeouts(
+      UpdateApplicationTimeoutsRequest request)
+      throws YarnException, IOException {
+    return getNextInterceptor().updateApplicationTimeouts(request);
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[10/42] hadoop git commit: YARN-5307. Federation Application State Store internal APIs

Posted by su...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/048df39b/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/GetApplicationsHomeSubClusterResponsePBImpl.java
----------------------------------------------------------------------
diff --git a/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/GetApplicationsHomeSubClusterResponsePBImpl.java b/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/GetApplicationsHomeSubClusterResponsePBImpl.java
new file mode 100644
index 0000000..8b72a1e
--- /dev/null
+++ b/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/GetApplicationsHomeSubClusterResponsePBImpl.java
@@ -0,0 +1,190 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records.impl.pb;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.ApplicationHomeSubClusterProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.GetApplicationsHomeSubClusterResponseProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.GetApplicationsHomeSubClusterResponseProtoOrBuilder;
+import org.apache.hadoop.yarn.server.federation.store.records.ApplicationHomeSubCluster;
+import org.apache.hadoop.yarn.server.federation.store.records.GetApplicationsHomeSubClusterResponse;
+
+import com.google.protobuf.TextFormat;
+
+/**
+ * Protocol buffer based implementation of
+ * {@link GetApplicationsHomeSubClusterResponse}.
+ */
+@Private
+@Unstable
+public class GetApplicationsHomeSubClusterResponsePBImpl
+    extends GetApplicationsHomeSubClusterResponse {
+
+  private GetApplicationsHomeSubClusterResponseProto proto =
+      GetApplicationsHomeSubClusterResponseProto.getDefaultInstance();
+  private GetApplicationsHomeSubClusterResponseProto.Builder builder = null;
+  private boolean viaProto = false;
+
+  private List<ApplicationHomeSubCluster> appsHomeSubCluster;
+
+  public GetApplicationsHomeSubClusterResponsePBImpl() {
+    builder = GetApplicationsHomeSubClusterResponseProto.newBuilder();
+  }
+
+  public GetApplicationsHomeSubClusterResponsePBImpl(
+      GetApplicationsHomeSubClusterResponseProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public GetApplicationsHomeSubClusterResponseProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = GetApplicationsHomeSubClusterResponseProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.appsHomeSubCluster != null) {
+      addSubClustersInfoToProto();
+    }
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null) {
+      return false;
+    }
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
+  }
+
+  @Override
+  public List<ApplicationHomeSubCluster> getAppsHomeSubClusters() {
+    initSubClustersInfoList();
+    return appsHomeSubCluster;
+  }
+
+  @Override
+  public void setAppsHomeSubClusters(
+      List<ApplicationHomeSubCluster> appsHomeSubClusters) {
+    maybeInitBuilder();
+    if (appsHomeSubClusters == null) {
+      builder.clearAppSubclusterMap();
+      return;
+    }
+    this.appsHomeSubCluster = appsHomeSubClusters;
+  }
+
+  private void initSubClustersInfoList() {
+    if (this.appsHomeSubCluster != null) {
+      return;
+    }
+    GetApplicationsHomeSubClusterResponseProtoOrBuilder p =
+        viaProto ? proto : builder;
+    List<ApplicationHomeSubClusterProto> subClusterInfosList =
+        p.getAppSubclusterMapList();
+    appsHomeSubCluster = new ArrayList<ApplicationHomeSubCluster>();
+
+    for (ApplicationHomeSubClusterProto r : subClusterInfosList) {
+      appsHomeSubCluster.add(convertFromProtoFormat(r));
+    }
+  }
+
+  private void addSubClustersInfoToProto() {
+    maybeInitBuilder();
+    builder.clearAppSubclusterMap();
+    if (appsHomeSubCluster == null) {
+      return;
+    }
+    Iterable<ApplicationHomeSubClusterProto> iterable =
+        new Iterable<ApplicationHomeSubClusterProto>() {
+          @Override
+          public Iterator<ApplicationHomeSubClusterProto> iterator() {
+            return new Iterator<ApplicationHomeSubClusterProto>() {
+
+              private Iterator<ApplicationHomeSubCluster> iter =
+                  appsHomeSubCluster.iterator();
+
+              @Override
+              public boolean hasNext() {
+                return iter.hasNext();
+              }
+
+              @Override
+              public ApplicationHomeSubClusterProto next() {
+                return convertToProtoFormat(iter.next());
+              }
+
+              @Override
+              public void remove() {
+                throw new UnsupportedOperationException();
+              }
+
+            };
+
+          }
+
+        };
+    builder.addAllAppSubclusterMap(iterable);
+  }
+
+  private ApplicationHomeSubCluster convertFromProtoFormat(
+      ApplicationHomeSubClusterProto sc) {
+    return new ApplicationHomeSubClusterPBImpl(sc);
+  }
+
+  private ApplicationHomeSubClusterProto convertToProtoFormat(
+      ApplicationHomeSubCluster sc) {
+    return ((ApplicationHomeSubClusterPBImpl) sc).getProto();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/048df39b/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/GetSubClustersInfoResponsePBImpl.java
----------------------------------------------------------------------
diff --git a/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/GetSubClustersInfoResponsePBImpl.java b/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/GetSubClustersInfoResponsePBImpl.java
index d39ef7f..92bdf06 100644
--- a/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/GetSubClustersInfoResponsePBImpl.java
+++ b/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/GetSubClustersInfoResponsePBImpl.java
@@ -6,9 +6,9 @@
  * to you under the Apache License, Version 2.0 (the
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/048df39b/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/UpdateApplicationHomeSubClusterRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/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/UpdateApplicationHomeSubClusterRequestPBImpl.java b/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/UpdateApplicationHomeSubClusterRequestPBImpl.java
new file mode 100644
index 0000000..e42eb00
--- /dev/null
+++ b/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/UpdateApplicationHomeSubClusterRequestPBImpl.java
@@ -0,0 +1,132 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records.impl.pb;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.ApplicationHomeSubClusterProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.UpdateApplicationHomeSubClusterRequestProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.UpdateApplicationHomeSubClusterRequestProtoOrBuilder;
+import org.apache.hadoop.yarn.server.federation.store.records.ApplicationHomeSubCluster;
+import org.apache.hadoop.yarn.server.federation.store.records.UpdateApplicationHomeSubClusterRequest;
+
+import com.google.protobuf.TextFormat;
+
+/**
+ * Protocol buffer based implementation of
+ * {@link UpdateApplicationHomeSubClusterRequest} .
+ */
+@Private
+@Unstable
+public class UpdateApplicationHomeSubClusterRequestPBImpl
+    extends UpdateApplicationHomeSubClusterRequest {
+
+  private UpdateApplicationHomeSubClusterRequestProto proto =
+      UpdateApplicationHomeSubClusterRequestProto.getDefaultInstance();
+  private UpdateApplicationHomeSubClusterRequestProto.Builder builder = null;
+  private boolean viaProto = false;
+
+  public UpdateApplicationHomeSubClusterRequestPBImpl() {
+    builder = UpdateApplicationHomeSubClusterRequestProto.newBuilder();
+  }
+
+  public UpdateApplicationHomeSubClusterRequestPBImpl(
+      UpdateApplicationHomeSubClusterRequestProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public UpdateApplicationHomeSubClusterRequestProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = UpdateApplicationHomeSubClusterRequestProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  private void mergeLocalToBuilder() {
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null) {
+      return false;
+    }
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
+  }
+
+  @Override
+  public ApplicationHomeSubCluster getApplicationHomeSubCluster() {
+    UpdateApplicationHomeSubClusterRequestProtoOrBuilder p =
+        viaProto ? proto : builder;
+    if (!p.hasAppSubclusterMap()) {
+      return null;
+    }
+    return convertFromProtoFormat(p.getAppSubclusterMap());
+  }
+
+  @Override
+  public void setApplicationHomeSubCluster(
+      ApplicationHomeSubCluster applicationInfo) {
+    maybeInitBuilder();
+    if (applicationInfo == null) {
+      builder.clearAppSubclusterMap();
+      return;
+    }
+    builder.setAppSubclusterMap(convertToProtoFormat(applicationInfo));
+  }
+
+  private ApplicationHomeSubCluster convertFromProtoFormat(
+      ApplicationHomeSubClusterProto sc) {
+    return new ApplicationHomeSubClusterPBImpl(sc);
+  }
+
+  private ApplicationHomeSubClusterProto convertToProtoFormat(
+      ApplicationHomeSubCluster sc) {
+    return ((ApplicationHomeSubClusterPBImpl) sc).getProto();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/048df39b/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/UpdateApplicationHomeSubClusterResponsePBImpl.java
----------------------------------------------------------------------
diff --git a/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/UpdateApplicationHomeSubClusterResponsePBImpl.java b/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/UpdateApplicationHomeSubClusterResponsePBImpl.java
new file mode 100644
index 0000000..ec31f0b
--- /dev/null
+++ b/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/UpdateApplicationHomeSubClusterResponsePBImpl.java
@@ -0,0 +1,78 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records.impl.pb;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.UpdateApplicationHomeSubClusterResponseProto;
+import org.apache.hadoop.yarn.server.federation.store.records.UpdateApplicationHomeSubClusterResponse;
+
+import com.google.protobuf.TextFormat;
+
+/**
+ * Protocol buffer based implementation of
+ * {@link UpdateApplicationHomeSubClusterResponse}.
+ */
+@Private
+@Unstable
+public class UpdateApplicationHomeSubClusterResponsePBImpl
+    extends UpdateApplicationHomeSubClusterResponse {
+
+  private UpdateApplicationHomeSubClusterResponseProto proto =
+      UpdateApplicationHomeSubClusterResponseProto.getDefaultInstance();
+  private UpdateApplicationHomeSubClusterResponseProto.Builder builder = null;
+  private boolean viaProto = false;
+
+  public UpdateApplicationHomeSubClusterResponsePBImpl() {
+    builder = UpdateApplicationHomeSubClusterResponseProto.newBuilder();
+  }
+
+  public UpdateApplicationHomeSubClusterResponsePBImpl(
+      UpdateApplicationHomeSubClusterResponseProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public UpdateApplicationHomeSubClusterResponseProto getProto() {
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null) {
+      return false;
+    }
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/048df39b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_federation_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_federation_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_federation_protos.proto
index 1b2e53e..b1ad310 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_federation_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_federation_protos.proto
@@ -90,4 +90,47 @@ message GetSubClustersInfoRequestProto {
 
 message GetSubClustersInfoResponseProto {
   repeated SubClusterInfoProto sub_cluster_infos = 1;
-}
\ No newline at end of file
+}
+
+message ApplicationHomeSubClusterProto {
+  optional ApplicationIdProto application_id = 1;
+  optional SubClusterIdProto home_sub_cluster = 2;
+}
+
+message AddApplicationHomeSubClusterRequestProto {
+  optional ApplicationHomeSubClusterProto app_subcluster_map = 1;
+}
+
+message AddApplicationHomeSubClusterResponseProto {
+}
+
+message UpdateApplicationHomeSubClusterRequestProto {
+  optional ApplicationHomeSubClusterProto app_subcluster_map = 1;
+}
+
+message UpdateApplicationHomeSubClusterResponseProto {
+}
+
+message GetApplicationHomeSubClusterRequestProto {
+  optional ApplicationIdProto application_id = 1;
+}
+
+message GetApplicationHomeSubClusterResponseProto {
+  optional ApplicationHomeSubClusterProto app_subcluster_map = 1;
+}
+
+message GetApplicationsHomeSubClusterRequestProto {
+
+}
+
+message GetApplicationsHomeSubClusterResponseProto {
+  repeated ApplicationHomeSubClusterProto app_subcluster_map = 1;
+}
+
+
+message DeleteApplicationHomeSubClusterRequestProto {
+  optional ApplicationIdProto application_id = 1;
+}
+
+message DeleteApplicationHomeSubClusterResponseProto {
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/048df39b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/records/TestFederationProtocolRecords.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/records/TestFederationProtocolRecords.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/records/TestFederationProtocolRecords.java
index 681edb1..210a246 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/records/TestFederationProtocolRecords.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/records/TestFederationProtocolRecords.java
@@ -19,6 +19,14 @@ package org.apache.hadoop.yarn.server.federation.store.records;
 
 import org.apache.hadoop.yarn.api.BasePBImplRecordsTest;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.AddApplicationHomeSubClusterRequestProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.AddApplicationHomeSubClusterResponseProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.DeleteApplicationHomeSubClusterRequestProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.DeleteApplicationHomeSubClusterResponseProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.GetApplicationHomeSubClusterRequestProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.GetApplicationHomeSubClusterResponseProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.GetApplicationsHomeSubClusterRequestProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.GetApplicationsHomeSubClusterResponseProto;
 import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.GetSubClusterInfoRequestProto;
 import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.GetSubClusterInfoResponseProto;
 import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.GetSubClustersInfoRequestProto;
@@ -31,6 +39,16 @@ import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SubClu
 import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SubClusterInfoProto;
 import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SubClusterRegisterRequestProto;
 import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SubClusterRegisterResponseProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.UpdateApplicationHomeSubClusterRequestProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.UpdateApplicationHomeSubClusterResponseProto;
+import org.apache.hadoop.yarn.server.federation.store.records.impl.pb.AddApplicationHomeSubClusterRequestPBImpl;
+import org.apache.hadoop.yarn.server.federation.store.records.impl.pb.AddApplicationHomeSubClusterResponsePBImpl;
+import org.apache.hadoop.yarn.server.federation.store.records.impl.pb.DeleteApplicationHomeSubClusterRequestPBImpl;
+import org.apache.hadoop.yarn.server.federation.store.records.impl.pb.DeleteApplicationHomeSubClusterResponsePBImpl;
+import org.apache.hadoop.yarn.server.federation.store.records.impl.pb.GetApplicationHomeSubClusterRequestPBImpl;
+import org.apache.hadoop.yarn.server.federation.store.records.impl.pb.GetApplicationHomeSubClusterResponsePBImpl;
+import org.apache.hadoop.yarn.server.federation.store.records.impl.pb.GetApplicationsHomeSubClusterRequestPBImpl;
+import org.apache.hadoop.yarn.server.federation.store.records.impl.pb.GetApplicationsHomeSubClusterResponsePBImpl;
 import org.apache.hadoop.yarn.server.federation.store.records.impl.pb.GetSubClusterInfoRequestPBImpl;
 import org.apache.hadoop.yarn.server.federation.store.records.impl.pb.GetSubClusterInfoResponsePBImpl;
 import org.apache.hadoop.yarn.server.federation.store.records.impl.pb.GetSubClustersInfoRequestPBImpl;
@@ -43,6 +61,8 @@ import org.apache.hadoop.yarn.server.federation.store.records.impl.pb.SubCluster
 import org.apache.hadoop.yarn.server.federation.store.records.impl.pb.SubClusterInfoPBImpl;
 import org.apache.hadoop.yarn.server.federation.store.records.impl.pb.SubClusterRegisterRequestPBImpl;
 import org.apache.hadoop.yarn.server.federation.store.records.impl.pb.SubClusterRegisterResponsePBImpl;
+import org.apache.hadoop.yarn.server.federation.store.records.impl.pb.UpdateApplicationHomeSubClusterRequestPBImpl;
+import org.apache.hadoop.yarn.server.federation.store.records.impl.pb.UpdateApplicationHomeSubClusterResponsePBImpl;
 import org.apache.hadoop.yarn.server.records.Version;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -58,6 +78,7 @@ public class TestFederationProtocolRecords extends BasePBImplRecordsTest {
     generateByNewInstance(Version.class);
     generateByNewInstance(SubClusterId.class);
     generateByNewInstance(SubClusterInfo.class);
+    generateByNewInstance(ApplicationHomeSubCluster.class);
   }
 
   @Test
@@ -130,4 +151,64 @@ public class TestFederationProtocolRecords extends BasePBImplRecordsTest {
         GetSubClustersInfoResponseProto.class);
   }
 
+  @Test
+  public void testAddApplicationHomeSubClusterRequest() throws Exception {
+    validatePBImplRecord(AddApplicationHomeSubClusterRequestPBImpl.class,
+        AddApplicationHomeSubClusterRequestProto.class);
+  }
+
+  @Test
+  public void testAddApplicationHomeSubClusterResponse() throws Exception {
+    validatePBImplRecord(AddApplicationHomeSubClusterResponsePBImpl.class,
+        AddApplicationHomeSubClusterResponseProto.class);
+  }
+
+  @Test
+  public void testUpdateApplicationHomeSubClusterRequest() throws Exception {
+    validatePBImplRecord(UpdateApplicationHomeSubClusterRequestPBImpl.class,
+        UpdateApplicationHomeSubClusterRequestProto.class);
+  }
+
+  @Test
+  public void testUpdateApplicationHomeSubClusterResponse() throws Exception {
+    validatePBImplRecord(UpdateApplicationHomeSubClusterResponsePBImpl.class,
+        UpdateApplicationHomeSubClusterResponseProto.class);
+  }
+
+  @Test
+  public void testGetApplicationHomeSubClusterRequest() throws Exception {
+    validatePBImplRecord(GetApplicationHomeSubClusterRequestPBImpl.class,
+        GetApplicationHomeSubClusterRequestProto.class);
+  }
+
+  @Test
+  public void testGetApplicationHomeSubClusterResponse() throws Exception {
+    validatePBImplRecord(GetApplicationHomeSubClusterResponsePBImpl.class,
+        GetApplicationHomeSubClusterResponseProto.class);
+  }
+
+  @Test
+  public void testGetApplicationsHomeSubClusterRequest() throws Exception {
+    validatePBImplRecord(GetApplicationsHomeSubClusterRequestPBImpl.class,
+        GetApplicationsHomeSubClusterRequestProto.class);
+  }
+
+  @Test
+  public void testGetApplicationsHomeSubClusterResponse() throws Exception {
+    validatePBImplRecord(GetApplicationsHomeSubClusterResponsePBImpl.class,
+        GetApplicationsHomeSubClusterResponseProto.class);
+  }
+
+  @Test
+  public void testDeleteApplicationHomeSubClusterRequest() throws Exception {
+    validatePBImplRecord(DeleteApplicationHomeSubClusterRequestPBImpl.class,
+        DeleteApplicationHomeSubClusterRequestProto.class);
+  }
+
+  @Test
+  public void testDeleteApplicationHomeSubClusterResponse() throws Exception {
+    validatePBImplRecord(DeleteApplicationHomeSubClusterResponsePBImpl.class,
+        DeleteApplicationHomeSubClusterResponseProto.class);
+  }
+
 }
\ No newline at end of file


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[18/42] hadoop git commit: YARN-3672. Create Facade for Federation State and Policy Store. Contributed by Subru Krishnan

Posted by su...@apache.org.
YARN-3672. Create Facade for Federation State and Policy Store. Contributed by Subru Krishnan


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/865e61bb
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/865e61bb
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/865e61bb

Branch: refs/heads/YARN-2915
Commit: 865e61bb205b25b43b1e77bfecc71e8b39fc92f8
Parents: 35ffdef
Author: Jian He <ji...@apache.org>
Authored: Wed Aug 17 11:13:19 2016 +0800
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon May 8 16:59:12 2017 -0700

----------------------------------------------------------------------
 hadoop-project/pom.xml                          |  13 +
 .../hadoop/yarn/conf/YarnConfiguration.java     |  13 +
 .../yarn/conf/TestYarnConfigurationFields.java  |   4 +
 .../src/main/resources/yarn-default.xml         |  20 +-
 .../hadoop-yarn-server-common/pom.xml           |  10 +
 .../utils/FederationStateStoreFacade.java       | 532 +++++++++++++++++++
 .../server/federation/utils/package-info.java   |  17 +
 .../utils/FederationStateStoreTestUtil.java     | 149 ++++++
 .../utils/TestFederationStateStoreFacade.java   | 148 ++++++
 9 files changed, 905 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/865e61bb/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index c9b6522..c67150f 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -98,6 +98,9 @@
     <apacheds.version>2.0.0-M21</apacheds.version>
     <ldap-api.version>1.0.0-M33</ldap-api.version>
 
+    <jcache.version>1.0.0</jcache.version>
+    <ehcache.version>3.0.3</ehcache.version>
+
     <!-- define the Java language version used by the compiler -->
     <javac.version>1.8</javac.version>
 
@@ -1262,6 +1265,16 @@
           <artifactId>kerb-simplekdc</artifactId>
           <version>1.0.0-RC2</version>
         </dependency>
+        <dependency>
+          <groupId>javax.cache</groupId>
+          <artifactId>cache-api</artifactId>
+          <version>${jcache.version}</version>
+        </dependency>
+        <dependency>
+          <groupId>org.ehcache</groupId>
+          <artifactId>ehcache</artifactId>
+          <version>${ehcache.version}</version>
+        </dependency>
     </dependencies>
   </dependencyManagement>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/865e61bb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
index a7a9641..25a9786 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
@@ -2538,6 +2538,19 @@ public class YarnConfiguration extends Configuration {
   ////////////////////////////////
 
   public static final String FEDERATION_PREFIX = YARN_PREFIX + "federation.";
+
+  public static final String FEDERATION_STATESTORE_CLIENT_CLASS =
+      FEDERATION_PREFIX + "state-store.class";
+
+  public static final String DEFAULT_FEDERATION_STATESTORE_CLIENT_CLASS =
+      "org.apache.hadoop.yarn.server.federation.store.impl.MemoryFederationStateStore";
+
+  public static final String FEDERATION_CACHE_TIME_TO_LIVE_SECS =
+      FEDERATION_PREFIX + "cache-ttl.secs";
+
+  // 5 minutes
+  public static final int DEFAULT_FEDERATION_CACHE_TIME_TO_LIVE_SECS = 5 * 60;
+
   public static final String FEDERATION_MACHINE_LIST =
       FEDERATION_PREFIX + "machine-list";
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/865e61bb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
index 3da4bab..bfc2534 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
@@ -68,6 +68,10 @@ public class TestYarnConfigurationFields extends TestConfigurationFieldsBase {
             .YARN_SECURITY_SERVICE_AUTHORIZATION_RESOURCETRACKER_PROTOCOL);
     configurationPropsToSkipCompare.add(YarnConfiguration.CURATOR_LEADER_ELECTOR);
 
+    // Federation default configs to be ignored
+    configurationPropsToSkipCompare
+        .add(YarnConfiguration.DEFAULT_FEDERATION_STATESTORE_CLIENT_CLASS);
+
     // Ignore blacklisting nodes for AM failures feature since it is still a
     // "work in progress"
     configurationPropsToSkipCompare.add(YarnConfiguration.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/865e61bb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
index a3a2465..5ba81de 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
@@ -2668,8 +2668,8 @@
     <description>The arguments to pass to the Node label script.</description>
     <name>yarn.nodemanager.node-labels.provider.script.opts</name>
   </property>
-  <!-- Other Configuration -->
 
+  <!-- Federation Configuration -->
   <property>
     <description>
       Machine list file to be loaded by the FederationSubCluster Resolver
@@ -2678,6 +2678,24 @@
   </property>
 
   <property>
+    <description>
+      Store class name for federation state store
+    </description>
+    <name>yarn.federation.state-store.class</name>
+    <value>org.apache.hadoop.yarn.server.federation.store.impl.MemoryFederationStateStore</value>
+  </property>
+
+  <property>
+   <description>
+    The time in seconds after which the federation state store local cache
+    will be refreshed periodically
+   </description>
+   <name>yarn.federation.cache-ttl.secs</name>
+   <value>300</value>
+  </property>
+
+  <!-- Other Configuration -->
+  <property>
     <description>The interval that the yarn client library uses to poll the
     completion status of the asynchronous API of application client protocol.
     </description>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/865e61bb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml
index 6d2fbef..cfb2533 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml
@@ -108,6 +108,16 @@
       <groupId>org.fusesource.leveldbjni</groupId>
       <artifactId>leveldbjni-all</artifactId>
     </dependency>
+    <dependency>
+      <groupId>javax.cache</groupId>
+      <artifactId>cache-api</artifactId>
+      <version>${jcache.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.ehcache</groupId>
+      <artifactId>ehcache</artifactId>
+      <version>${ehcache.version}</version>
+    </dependency>
   </dependencies>
 
   <build>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/865e61bb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java
new file mode 100644
index 0000000..f1c8218
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java
@@ -0,0 +1,532 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.utils;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import javax.cache.Cache;
+import javax.cache.CacheManager;
+import javax.cache.Caching;
+import javax.cache.configuration.CompleteConfiguration;
+import javax.cache.configuration.FactoryBuilder;
+import javax.cache.configuration.MutableConfiguration;
+import javax.cache.expiry.CreatedExpiryPolicy;
+import javax.cache.expiry.Duration;
+import javax.cache.expiry.ExpiryPolicy;
+import javax.cache.integration.CacheLoader;
+import javax.cache.integration.CacheLoaderException;
+import javax.cache.spi.CachingProvider;
+
+import org.apache.commons.lang.NotImplementedException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.retry.RetryPolicies;
+import org.apache.hadoop.io.retry.RetryPolicy;
+import org.apache.hadoop.io.retry.RetryProxy;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.hadoop.yarn.server.federation.store.FederationStateStore;
+import org.apache.hadoop.yarn.server.federation.store.records.AddApplicationHomeSubClusterRequest;
+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.server.federation.store.records.GetSubClusterInfoRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterPoliciesConfigurationsRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterPoliciesConfigurationsResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterPolicyConfigurationRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClustersInfoRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClustersInfoResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyConfiguration;
+import org.apache.hadoop.yarn.server.federation.store.records.UpdateApplicationHomeSubClusterRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ *
+ * The FederationStateStoreFacade is an utility wrapper that provides singleton
+ * access to the Federation state store. It abstracts out retries and in
+ * addition, it also implements the caching for various objects.
+ *
+ */
+public final class FederationStateStoreFacade {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(FederationStateStoreFacade.class);
+
+  private static final String GET_SUBCLUSTERS_CACHEID = "getSubClusters";
+  private static final String GET_POLICIES_CONFIGURATIONS_CACHEID =
+      "getPoliciesConfigurations";
+
+  private static final FederationStateStoreFacade FACADE =
+      new FederationStateStoreFacade();
+
+  private FederationStateStore stateStore;
+  private int cacheTimeToLive;
+  private Configuration conf;
+  private Cache<Object, Object> cache;
+
+  private FederationStateStoreFacade() {
+    initializeFacadeInternal(new Configuration());
+  }
+
+  private void initializeFacadeInternal(Configuration config) {
+    this.conf = config;
+    try {
+      this.stateStore = (FederationStateStore) createRetryInstance(this.conf,
+          YarnConfiguration.FEDERATION_STATESTORE_CLIENT_CLASS,
+          YarnConfiguration.DEFAULT_FEDERATION_STATESTORE_CLIENT_CLASS,
+          FederationStateStore.class, createRetryPolicy(conf));
+      this.stateStore.init(conf);
+
+      initCache();
+
+    } catch (YarnException ex) {
+      LOG.error("Failed to initialize the FederationStateStoreFacade object",
+          ex);
+      throw new RuntimeException(ex);
+    }
+  }
+
+  /**
+   * Delete and re-initialize the cache, to force it to use the given
+   * configuration.
+   *
+   * @param store the {@link FederationStateStore} instance to reinitialize with
+   * @param config the updated configuration to reinitialize with
+   */
+  @VisibleForTesting
+  public synchronized void reinitialize(FederationStateStore store,
+      Configuration config) {
+    this.conf = config;
+    this.stateStore = store;
+    clearCache();
+    initCache();
+  }
+
+  public static RetryPolicy createRetryPolicy(Configuration conf) {
+    // Retry settings for StateStore
+    RetryPolicy retryPolicy = RetryPolicies.exponentialBackoffRetry(
+        conf.getInt(YarnConfiguration.CLIENT_FAILOVER_RETRIES, Integer.SIZE),
+        conf.getLong(YarnConfiguration.CLIENT_FAILOVER_SLEEPTIME_BASE_MS,
+            YarnConfiguration.DEFAULT_RESOURCEMANAGER_CONNECT_RETRY_INTERVAL_MS),
+        TimeUnit.MILLISECONDS);
+
+    return retryPolicy;
+  }
+
+  private boolean isCachingEnabled() {
+    return (cacheTimeToLive > 0);
+  }
+
+  private void initCache() {
+    // Picking the JCache provider from classpath, need to make sure there's
+    // no conflict or pick up a specific one in the future
+    cacheTimeToLive =
+        conf.getInt(YarnConfiguration.FEDERATION_CACHE_TIME_TO_LIVE_SECS,
+            YarnConfiguration.DEFAULT_FEDERATION_CACHE_TIME_TO_LIVE_SECS);
+    if (isCachingEnabled()) {
+      CachingProvider jcacheProvider = Caching.getCachingProvider();
+      CacheManager jcacheManager = jcacheProvider.getCacheManager();
+      this.cache = jcacheManager.getCache(this.getClass().getSimpleName());
+      if (this.cache == null) {
+        LOG.info("Creating a JCache Manager with name "
+            + this.getClass().getSimpleName());
+        Duration cacheExpiry = new Duration(TimeUnit.SECONDS, cacheTimeToLive);
+        CompleteConfiguration<Object, Object> configuration =
+            new MutableConfiguration<Object, Object>().setStoreByValue(false)
+                .setReadThrough(true)
+                .setExpiryPolicyFactory(
+                    new FactoryBuilder.SingletonFactory<ExpiryPolicy>(
+                        new CreatedExpiryPolicy(cacheExpiry)))
+                .setCacheLoaderFactory(
+                    new FactoryBuilder.SingletonFactory<CacheLoader<Object, Object>>(
+                        new CacheLoaderImpl<Object, Object>()));
+        this.cache = jcacheManager.createCache(this.getClass().getSimpleName(),
+            configuration);
+      }
+    }
+  }
+
+  private void clearCache() {
+    CachingProvider jcacheProvider = Caching.getCachingProvider();
+    CacheManager jcacheManager = jcacheProvider.getCacheManager();
+
+    jcacheManager.destroyCache(this.getClass().getSimpleName());
+    this.cache = null;
+  }
+
+  /**
+   * Returns the singleton instance of the FederationStateStoreFacade object.
+   *
+   * @return the singleton {@link FederationStateStoreFacade} instance
+   */
+  public static FederationStateStoreFacade getInstance() {
+    return FACADE;
+  }
+
+  /**
+   * Returns the {@link SubClusterInfo} for the specified {@link SubClusterId}.
+   *
+   * @param subClusterId the identifier of the sub-cluster
+   * @return the sub cluster information
+   * @throws YarnException if the call to the state store is unsuccessful
+   */
+  public SubClusterInfo getSubCluster(final SubClusterId subClusterId)
+      throws YarnException {
+    if (isCachingEnabled()) {
+      return getSubClusters(false).get(subClusterId);
+    } else {
+      return stateStore
+          .getSubCluster(GetSubClusterInfoRequest.newInstance(subClusterId))
+          .getSubClusterInfo();
+    }
+  }
+
+  /**
+   * Updates the cache with the central {@link FederationStateStore} and returns
+   * the {@link SubClusterInfo} for the specified {@link SubClusterId}.
+   *
+   * @param subClusterId the identifier of the sub-cluster
+   * @param flushCache flag to indicate if the cache should be flushed or not
+   * @return the sub cluster information
+   * @throws YarnException if the call to the state store is unsuccessful
+   */
+  public SubClusterInfo getSubCluster(final SubClusterId subClusterId,
+      final boolean flushCache) throws YarnException {
+    if (flushCache && isCachingEnabled()) {
+      LOG.info("Flushing subClusters from cache and rehydrating from store,"
+          + " most likely on account of RM failover.");
+      cache.remove(buildGetSubClustersCacheRequest(false));
+    }
+    return getSubCluster(subClusterId);
+  }
+
+  /**
+   * Returns the {@link SubClusterInfo} of all active sub cluster(s).
+   *
+   * @param filterInactiveSubClusters whether to filter out inactive
+   *          sub-clusters
+   * @return the information of all active sub cluster(s)
+   * @throws YarnException if the call to the state store is unsuccessful
+   */
+  @SuppressWarnings("unchecked")
+  public Map<SubClusterId, SubClusterInfo> getSubClusters(
+      final boolean filterInactiveSubClusters) throws YarnException {
+    try {
+      if (isCachingEnabled()) {
+        return (Map<SubClusterId, SubClusterInfo>) cache
+            .get(buildGetSubClustersCacheRequest(filterInactiveSubClusters));
+      } else {
+        return buildSubClusterInfoMap(stateStore.getSubClusters(
+            GetSubClustersInfoRequest.newInstance(filterInactiveSubClusters)));
+      }
+    } catch (Throwable ex) {
+      throw new YarnException(ex);
+    }
+  }
+
+  /**
+   * Returns the {@link SubClusterPolicyConfiguration} for the specified queue.
+   *
+   * @param queue the queue whose policy is required
+   * @return the corresponding configured policy
+   * @throws YarnException if the call to the state store is unsuccessful
+   */
+  public SubClusterPolicyConfiguration getPolicyConfiguration(
+      final String queue) throws YarnException {
+    if (isCachingEnabled()) {
+      return getPoliciesConfigurations().get(queue);
+    } else {
+      return stateStore
+          .getPolicyConfiguration(
+              GetSubClusterPolicyConfigurationRequest.newInstance(queue))
+          .getPolicyConfiguration();
+    }
+
+  }
+
+  /**
+   * Get the policies that is represented as
+   * {@link SubClusterPolicyConfiguration} for all currently active queues in
+   * the system.
+   *
+   * @return the policies for all currently active queues in the system
+   * @throws YarnException if the call to the state store is unsuccessful
+   */
+  @SuppressWarnings("unchecked")
+  public Map<String, SubClusterPolicyConfiguration> getPoliciesConfigurations()
+      throws YarnException {
+    try {
+      if (isCachingEnabled()) {
+        return (Map<String, SubClusterPolicyConfiguration>) cache
+            .get(buildGetPoliciesConfigurationsCacheRequest());
+      } else {
+        return buildPolicyConfigMap(stateStore.getPoliciesConfigurations(
+            GetSubClusterPoliciesConfigurationsRequest.newInstance()));
+      }
+    } catch (Throwable ex) {
+      throw new YarnException(ex);
+    }
+  }
+
+  /**
+   * Adds the home {@link SubClusterId} for the specified {@link ApplicationId}.
+   *
+   * @param appHomeSubCluster the mapping of the application to it's home
+   *          sub-cluster
+   * @throws YarnException if the call to the state store is unsuccessful
+   */
+  public void addApplicationHomeSubCluster(
+      ApplicationHomeSubCluster appHomeSubCluster) throws YarnException {
+    stateStore.addApplicationHomeSubCluster(
+        AddApplicationHomeSubClusterRequest.newInstance(appHomeSubCluster));
+    return;
+  }
+
+  /**
+   * Updates the home {@link SubClusterId} for the specified
+   * {@link ApplicationId}.
+   *
+   * @param appHomeSubCluster the mapping of the application to it's home
+   *          sub-cluster
+   * @throws YarnException if the call to the state store is unsuccessful
+   */
+  public void updateApplicationHomeSubCluster(
+      ApplicationHomeSubCluster appHomeSubCluster) throws YarnException {
+    stateStore.updateApplicationHomeSubCluster(
+        UpdateApplicationHomeSubClusterRequest.newInstance(appHomeSubCluster));
+    return;
+  }
+
+  /**
+   * Returns the home {@link SubClusterId} for the specified
+   * {@link ApplicationId}.
+   *
+   * @param appId the identifier of the application
+   * @return the home sub cluster identifier
+   * @throws YarnException if the call to the state store is unsuccessful
+   */
+  public SubClusterId getApplicationHomeSubCluster(ApplicationId appId)
+      throws YarnException {
+    GetApplicationHomeSubClusterResponse response =
+        stateStore.getApplicationHomeSubCluster(
+            GetApplicationHomeSubClusterRequest.newInstance(appId));
+    return response.getApplicationHomeSubCluster().getHomeSubCluster();
+  }
+
+  /**
+   * Helper method to create instances of Object using the class name defined in
+   * the configuration object. The instances creates {@link RetryProxy} using
+   * the specific {@link RetryPolicy}.
+   *
+   * @param conf the yarn configuration
+   * @param configuredClassName the configuration provider key
+   * @param defaultValue the default implementation for fallback
+   * @param type the class for which a retry proxy is required
+   * @param retryPolicy the policy for retrying method call failures
+   * @return a retry proxy for the specified interface
+   */
+  @SuppressWarnings("unchecked")
+  public static <T> Object createRetryInstance(Configuration conf,
+      String configuredClassName, String defaultValue, Class<T> type,
+      RetryPolicy retryPolicy) {
+
+    String className = conf.get(configuredClassName, defaultValue);
+    try {
+      Class<?> clusterResolverClass = conf.getClassByName(className);
+      if (type.isAssignableFrom(clusterResolverClass)) {
+        return RetryProxy.create(type,
+            (T) ReflectionUtils.newInstance(clusterResolverClass, conf),
+            retryPolicy);
+      } else {
+        throw new YarnRuntimeException(
+            "Class: " + className + " not instance of " + type.getSimpleName());
+      }
+    } catch (Exception e) {
+      throw new YarnRuntimeException("Could not instantiate : " + className, e);
+    }
+  }
+
+  private Map<SubClusterId, SubClusterInfo> buildSubClusterInfoMap(
+      final GetSubClustersInfoResponse response) {
+    List<SubClusterInfo> subClusters = response.getSubClusters();
+    Map<SubClusterId, SubClusterInfo> subClustersMap =
+        new HashMap<>(subClusters.size());
+    for (SubClusterInfo subCluster : subClusters) {
+      subClustersMap.put(subCluster.getSubClusterId(), subCluster);
+    }
+    return subClustersMap;
+  }
+
+  private Object buildGetSubClustersCacheRequest(
+      final boolean filterInactiveSubClusters) {
+    final String cacheKey = buildCacheKey(getClass().getSimpleName(),
+        GET_SUBCLUSTERS_CACHEID, null);
+    CacheRequest<String, Map<SubClusterId, SubClusterInfo>> cacheRequest =
+        new CacheRequest<String, Map<SubClusterId, SubClusterInfo>>(cacheKey,
+            new Func<String, Map<SubClusterId, SubClusterInfo>>() {
+              @Override
+              public Map<SubClusterId, SubClusterInfo> invoke(String key)
+                  throws Exception {
+                GetSubClustersInfoResponse subClusters =
+                    stateStore.getSubClusters(GetSubClustersInfoRequest
+                        .newInstance(filterInactiveSubClusters));
+                return buildSubClusterInfoMap(subClusters);
+              }
+            });
+    return cacheRequest;
+  }
+
+  private Map<String, SubClusterPolicyConfiguration> buildPolicyConfigMap(
+      GetSubClusterPoliciesConfigurationsResponse response) {
+    List<SubClusterPolicyConfiguration> policyConfigs =
+        response.getPoliciesConfigs();
+    Map<String, SubClusterPolicyConfiguration> queuePolicyConfigs =
+        new HashMap<>();
+    for (SubClusterPolicyConfiguration policyConfig : policyConfigs) {
+      queuePolicyConfigs.put(policyConfig.getQueue(), policyConfig);
+    }
+    return queuePolicyConfigs;
+  }
+
+  private Object buildGetPoliciesConfigurationsCacheRequest() {
+    final String cacheKey = buildCacheKey(getClass().getSimpleName(),
+        GET_POLICIES_CONFIGURATIONS_CACHEID, null);
+    CacheRequest<String, Map<String, SubClusterPolicyConfiguration>> cacheRequest =
+        new CacheRequest<String, Map<String, SubClusterPolicyConfiguration>>(
+            cacheKey,
+            new Func<String, Map<String, SubClusterPolicyConfiguration>>() {
+              @Override
+              public Map<String, SubClusterPolicyConfiguration> invoke(
+                  String key) throws Exception {
+                GetSubClusterPoliciesConfigurationsResponse policyConfigs =
+                    stateStore.getPoliciesConfigurations(
+                        GetSubClusterPoliciesConfigurationsRequest
+                            .newInstance());
+                return buildPolicyConfigMap(policyConfigs);
+              }
+            });
+    return cacheRequest;
+  }
+
+  protected String buildCacheKey(String typeName, String methodName,
+      String argName) {
+    StringBuilder buffer = new StringBuilder();
+    buffer.append(typeName).append(".");
+    buffer.append(methodName);
+    if (argName != null) {
+      buffer.append("::");
+      buffer.append(argName);
+    }
+    return buffer.toString();
+  }
+
+  /**
+   * Internal class that implements the CacheLoader interface that can be
+   * plugged into the CacheManager to load objects into the cache for specified
+   * keys.
+   */
+  private static class CacheLoaderImpl<K, V> implements CacheLoader<K, V> {
+    @SuppressWarnings("unchecked")
+    @Override
+    public V load(K key) throws CacheLoaderException {
+      try {
+        CacheRequest<K, V> query = (CacheRequest<K, V>) key;
+        assert query != null;
+        return query.getValue();
+      } catch (Throwable ex) {
+        throw new CacheLoaderException(ex);
+      }
+    }
+
+    @Override
+    public Map<K, V> loadAll(Iterable<? extends K> keys)
+        throws CacheLoaderException {
+      // The FACADE does not use the Cache's getAll API. Hence this is not
+      // required to be implemented
+      throw new NotImplementedException();
+    }
+  }
+
+  /**
+   * Internal class that encapsulates the cache key and a function that returns
+   * the value for the specified key.
+   */
+  private static class CacheRequest<K, V> {
+    private K key;
+    private Func<K, V> func;
+
+    public CacheRequest(K key, Func<K, V> func) {
+      this.key = key;
+      this.func = func;
+    }
+
+    public V getValue() throws Exception {
+      return func.invoke(key);
+    }
+
+    @Override
+    public int hashCode() {
+      final int prime = 31;
+      int result = 1;
+      result = prime * result + ((key == null) ? 0 : key.hashCode());
+      return result;
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public boolean equals(Object obj) {
+      if (this == obj) {
+        return true;
+      }
+      if (obj == null) {
+        return false;
+      }
+      if (getClass() != obj.getClass()) {
+        return false;
+      }
+      CacheRequest<K, V> other = (CacheRequest<K, V>) obj;
+      if (key == null) {
+        if (other.key != null) {
+          return false;
+        }
+      } else if (!key.equals(other.key)) {
+        return false;
+      }
+
+      return true;
+    }
+  }
+
+  /**
+   * Encapsulates a method that has one parameter and returns a value of the
+   * type specified by the TResult parameter.
+   */
+  protected interface Func<T, TResult> {
+    TResult invoke(T input) throws Exception;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/865e61bb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/package-info.java
new file mode 100644
index 0000000..39a46ec
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/package-info.java
@@ -0,0 +1,17 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.yarn.server.federation.utils;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/865e61bb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreTestUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreTestUtil.java
new file mode 100644
index 0000000..c179521
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreTestUtil.java
@@ -0,0 +1,149 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.utils;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.federation.store.FederationStateStore;
+import org.apache.hadoop.yarn.server.federation.store.records.AddApplicationHomeSubClusterRequest;
+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.server.federation.store.records.GetSubClusterInfoRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterPolicyConfigurationRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterPolicyConfigurationResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.SetSubClusterPolicyConfigurationRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyConfiguration;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterRegisterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterState;
+import org.apache.hadoop.yarn.util.MonotonicClock;
+
+/**
+ * Utility class for FederationStateStore unit tests.
+ */
+public class FederationStateStoreTestUtil {
+
+  private static final MonotonicClock CLOCK = new MonotonicClock();
+
+  public static final String SC_PREFIX = "SC-";
+  public static final String Q_PREFIX = "queue-";
+  public static final String POLICY_PREFIX = "policy-";
+
+  private FederationStateStore stateStore;
+
+  public FederationStateStoreTestUtil(FederationStateStore stateStore) {
+    this.stateStore = stateStore;
+  }
+
+  private SubClusterInfo createSubClusterInfo(SubClusterId subClusterId) {
+
+    String amRMAddress = "1.2.3.4:1";
+    String clientRMAddress = "1.2.3.4:2";
+    String rmAdminAddress = "1.2.3.4:3";
+    String webAppAddress = "1.2.3.4:4";
+
+    return SubClusterInfo.newInstance(subClusterId, amRMAddress,
+        clientRMAddress, rmAdminAddress, webAppAddress, SubClusterState.SC_NEW,
+        CLOCK.getTime(), "capability");
+  }
+
+  private void registerSubCluster(SubClusterId subClusterId)
+      throws YarnException {
+
+    SubClusterInfo subClusterInfo = createSubClusterInfo(subClusterId);
+    stateStore.registerSubCluster(
+        SubClusterRegisterRequest.newInstance(subClusterInfo));
+  }
+
+  public void registerSubClusters(int numSubClusters) throws YarnException {
+
+    for (int i = 0; i < numSubClusters; i++) {
+      registerSubCluster(SubClusterId.newInstance(SC_PREFIX + i));
+    }
+  }
+
+  private void addApplicationHomeSC(ApplicationId appId,
+      SubClusterId subClusterId) throws YarnException {
+    ApplicationHomeSubCluster ahsc =
+        ApplicationHomeSubCluster.newInstance(appId, subClusterId);
+    AddApplicationHomeSubClusterRequest request =
+        AddApplicationHomeSubClusterRequest.newInstance(ahsc);
+    stateStore.addApplicationHomeSubCluster(request);
+  }
+
+  public void addAppsHomeSC(long clusterTs, int numApps) throws YarnException {
+    for (int i = 0; i < numApps; i++) {
+      addApplicationHomeSC(ApplicationId.newInstance(clusterTs, i),
+          SubClusterId.newInstance(SC_PREFIX + i));
+    }
+  }
+
+  private SubClusterPolicyConfiguration createSCPolicyConf(String queueName,
+      String policyType) {
+    return SubClusterPolicyConfiguration.newInstance(queueName, policyType,
+        ByteBuffer.allocate(1));
+  }
+
+  private void setPolicyConf(String queue, String policyType)
+      throws YarnException {
+    SetSubClusterPolicyConfigurationRequest request =
+        SetSubClusterPolicyConfigurationRequest
+            .newInstance(createSCPolicyConf(queue, policyType));
+    stateStore.setPolicyConfiguration(request);
+  }
+
+  public void addPolicyConfigs(int numQueues) throws YarnException {
+
+    for (int i = 0; i < numQueues; i++) {
+      setPolicyConf(Q_PREFIX + i, POLICY_PREFIX + i);
+    }
+  }
+
+  public SubClusterInfo querySubClusterInfo(SubClusterId subClusterId)
+      throws YarnException {
+    GetSubClusterInfoRequest request =
+        GetSubClusterInfoRequest.newInstance(subClusterId);
+    return stateStore.getSubCluster(request).getSubClusterInfo();
+  }
+
+  public SubClusterId queryApplicationHomeSC(ApplicationId appId)
+      throws YarnException {
+    GetApplicationHomeSubClusterRequest request =
+        GetApplicationHomeSubClusterRequest.newInstance(appId);
+
+    GetApplicationHomeSubClusterResponse response =
+        stateStore.getApplicationHomeSubCluster(request);
+
+    return response.getApplicationHomeSubCluster().getHomeSubCluster();
+  }
+
+  public SubClusterPolicyConfiguration queryPolicyConfiguration(String queue)
+      throws YarnException {
+    GetSubClusterPolicyConfigurationRequest request =
+        GetSubClusterPolicyConfigurationRequest.newInstance(queue);
+
+    GetSubClusterPolicyConfigurationResponse result =
+        stateStore.getPolicyConfiguration(request);
+    return result.getPolicyConfiguration();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/865e61bb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/TestFederationStateStoreFacade.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/TestFederationStateStoreFacade.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/TestFederationStateStoreFacade.java
new file mode 100644
index 0000000..53f4f84
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/TestFederationStateStoreFacade.java
@@ -0,0 +1,148 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.utils;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.federation.store.FederationStateStore;
+import org.apache.hadoop.yarn.server.federation.store.impl.MemoryFederationStateStore;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyConfiguration;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+/**
+ * Unit tests for FederationStateStoreFacade.
+ */
+@RunWith(Parameterized.class)
+public class TestFederationStateStoreFacade {
+
+  @Parameters
+  public static Collection<Boolean[]> getParameters() {
+    return Arrays
+        .asList(new Boolean[][] {{Boolean.FALSE }, {Boolean.TRUE } });
+  }
+
+  private final long clusterTs = System.currentTimeMillis();
+  private final int numSubClusters = 3;
+  private final int numApps = 5;
+  private final int numQueues = 2;
+
+  private Configuration conf;
+  private FederationStateStore stateStore;
+  private FederationStateStoreTestUtil stateStoreTestUtil;
+  private FederationStateStoreFacade facade =
+      FederationStateStoreFacade.getInstance();
+
+  public TestFederationStateStoreFacade(Boolean isCachingEnabled) {
+    conf = new Configuration();
+    if (!(isCachingEnabled.booleanValue())) {
+      conf.setInt(YarnConfiguration.FEDERATION_CACHE_TIME_TO_LIVE_SECS, 0);
+    }
+  }
+
+  @Before
+  public void setUp() throws IOException, YarnException {
+    stateStore = new MemoryFederationStateStore();
+    stateStore.init(conf);
+    facade.reinitialize(stateStore, conf);
+    // hydrate the store
+    stateStoreTestUtil = new FederationStateStoreTestUtil(stateStore);
+    stateStoreTestUtil.registerSubClusters(numSubClusters);
+    stateStoreTestUtil.addAppsHomeSC(clusterTs, numApps);
+    stateStoreTestUtil.addPolicyConfigs(numQueues);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    stateStore.close();
+    stateStore = null;
+  }
+
+  @Test
+  public void testGetSubCluster() throws YarnException {
+    for (int i = 0; i < numSubClusters; i++) {
+      SubClusterId subClusterId =
+          SubClusterId.newInstance(FederationStateStoreTestUtil.SC_PREFIX + i);
+      Assert.assertEquals(stateStoreTestUtil.querySubClusterInfo(subClusterId),
+          facade.getSubCluster(subClusterId));
+    }
+  }
+
+  @Test
+  public void testGetSubClusterFlushCache() throws YarnException {
+    for (int i = 0; i < numSubClusters; i++) {
+      SubClusterId subClusterId =
+          SubClusterId.newInstance(FederationStateStoreTestUtil.SC_PREFIX + i);
+      Assert.assertEquals(stateStoreTestUtil.querySubClusterInfo(subClusterId),
+          facade.getSubCluster(subClusterId, true));
+    }
+  }
+
+  @Test
+  public void testGetSubClusters() throws YarnException {
+    Map<SubClusterId, SubClusterInfo> subClusters =
+        facade.getSubClusters(false);
+    for (SubClusterId subClusterId : subClusters.keySet()) {
+      Assert.assertEquals(stateStoreTestUtil.querySubClusterInfo(subClusterId),
+          subClusters.get(subClusterId));
+    }
+  }
+
+  @Test
+  public void testGetPolicyConfiguration() throws YarnException {
+    for (int i = 0; i < numQueues; i++) {
+      String queue = FederationStateStoreTestUtil.Q_PREFIX + i;
+      Assert.assertEquals(stateStoreTestUtil.queryPolicyConfiguration(queue),
+          facade.getPolicyConfiguration(queue));
+    }
+  }
+
+  @Test
+  public void testGetPoliciesConfigurations() throws YarnException {
+    Map<String, SubClusterPolicyConfiguration> queuePolicies =
+        facade.getPoliciesConfigurations();
+    for (String queue : queuePolicies.keySet()) {
+      Assert.assertEquals(stateStoreTestUtil.queryPolicyConfiguration(queue),
+          queuePolicies.get(queue));
+    }
+  }
+
+  @Test
+  public void testGetHomeSubClusterForApp() throws YarnException {
+    for (int i = 0; i < numApps; i++) {
+      ApplicationId appId = ApplicationId.newInstance(clusterTs, i);
+      Assert.assertEquals(stateStoreTestUtil.queryApplicationHomeSC(appId),
+          facade.getApplicationHomeSubCluster(appId));
+    }
+  }
+
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[23/42] hadoop git commit: YARN-3673. Create a FailoverProxy for Federation services. Contributed by Subru Krishnan

Posted by su...@apache.org.
YARN-3673. Create a FailoverProxy for Federation services. Contributed by Subru Krishnan


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/b96be3da
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/b96be3da
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/b96be3da

Branch: refs/heads/YARN-2915
Commit: b96be3da037ae42c590c3bc32f415a0916f58eb1
Parents: 8aa6bda
Author: Jian He <ji...@apache.org>
Authored: Mon Aug 22 14:43:07 2016 +0800
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon May 8 16:59:12 2017 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/yarn/conf/HAUtil.java     |  30 ++-
 .../hadoop/yarn/conf/YarnConfiguration.java     |  10 +
 .../yarn/conf/TestYarnConfigurationFields.java  |   4 +
 .../TestFederationRMFailoverProxyProvider.java  | 154 ++++++++++++++
 .../hadoop/yarn/client/ClientRMProxy.java       |   4 +-
 .../org/apache/hadoop/yarn/client/RMProxy.java  |  23 +-
 .../src/main/resources/yarn-default.xml         |   7 +
 .../hadoop-yarn-server-common/pom.xml           |   2 -
 .../hadoop/yarn/server/api/ServerRMProxy.java   |   4 +-
 .../failover/FederationProxyProviderUtil.java   | 163 ++++++++++++++
 .../FederationRMFailoverProxyProvider.java      | 211 +++++++++++++++++++
 .../federation/failover/package-info.java       |  17 ++
 12 files changed, 613 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b96be3da/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/HAUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/HAUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/HAUtil.java
index 133b377..528b642 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/HAUtil.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/HAUtil.java
@@ -18,7 +18,9 @@
 
 package org.apache.hadoop.yarn.conf;
 
-import com.google.common.annotations.VisibleForTesting;
+import java.net.InetSocketAddress;
+import java.util.Collection;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.HadoopIllegalArgumentException;
@@ -27,8 +29,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 
-import java.net.InetSocketAddress;
-import java.util.Collection;
+import com.google.common.annotations.VisibleForTesting;
 
 @InterfaceAudience.Private
 public class HAUtil {
@@ -45,6 +46,29 @@ public class HAUtil {
   }
 
   /**
+   * Returns true if Federation is configured.
+   *
+   * @param conf Configuration
+   * @return true if federation is configured in the configuration; else false.
+   */
+  public static boolean isFederationEnabled(Configuration conf) {
+    return conf.getBoolean(YarnConfiguration.FEDERATION_ENABLED,
+        YarnConfiguration.DEFAULT_FEDERATION_ENABLED);
+  }
+
+  /**
+   * Returns true if RM failover is enabled in a Federation setting.
+   *
+   * @param conf Configuration
+   * @return if RM failover is enabled in conjunction with Federation in the
+   *         configuration; else false.
+   */
+  public static boolean isFederationFailoverEnabled(Configuration conf) {
+    return conf.getBoolean(YarnConfiguration.FEDERATION_FAILOVER_ENABLED,
+        YarnConfiguration.DEFAULT_FEDERATION_FAILOVER_ENABLED);
+  }
+
+  /**
    * Returns true if Resource Manager HA is configured.
    *
    * @param conf Configuration

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b96be3da/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
index 25a9786..05321e1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
@@ -2539,6 +2539,16 @@ public class YarnConfiguration extends Configuration {
 
   public static final String FEDERATION_PREFIX = YARN_PREFIX + "federation.";
 
+  public static final String FEDERATION_ENABLED = FEDERATION_PREFIX + "enabled";
+  public static final boolean DEFAULT_FEDERATION_ENABLED = false;
+
+  public static final String FEDERATION_FAILOVER_ENABLED =
+      FEDERATION_PREFIX + "failover.enabled";
+  public static final boolean DEFAULT_FEDERATION_FAILOVER_ENABLED = true;
+
+  public static final String FEDERATION_SUBCLUSTER_ID =
+      FEDERATION_PREFIX + "sub-cluster.id";
+
   public static final String FEDERATION_STATESTORE_CLIENT_CLASS =
       FEDERATION_PREFIX + "state-store.class";
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b96be3da/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
index bfc2534..c4d8f38 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
@@ -71,6 +71,10 @@ public class TestYarnConfigurationFields extends TestConfigurationFieldsBase {
     // Federation default configs to be ignored
     configurationPropsToSkipCompare
         .add(YarnConfiguration.DEFAULT_FEDERATION_STATESTORE_CLIENT_CLASS);
+    configurationPropsToSkipCompare
+        .add(YarnConfiguration.FEDERATION_SUBCLUSTER_ID);
+    configurationPropsToSkipCompare
+        .add(YarnConfiguration.FEDERATION_FAILOVER_ENABLED);
 
     // Ignore blacklisting nodes for AM failures feature since it is still a
     // "work in progress"

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b96be3da/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestFederationRMFailoverProxyProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestFederationRMFailoverProxyProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestFederationRMFailoverProxyProvider.java
new file mode 100644
index 0000000..fa3523c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestFederationRMFailoverProxyProvider.java
@@ -0,0 +1,154 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.client;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ha.HAServiceProtocol;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsResponse;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.MiniYARNCluster;
+import org.apache.hadoop.yarn.server.federation.failover.FederationProxyProviderUtil;
+import org.apache.hadoop.yarn.server.federation.store.FederationStateStore;
+import org.apache.hadoop.yarn.server.federation.store.impl.MemoryFederationStateStore;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterRegisterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterState;
+import org.apache.hadoop.yarn.server.federation.utils.FederationStateStoreFacade;
+import org.apache.hadoop.yarn.server.resourcemanager.HATestUtil;
+import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Unit tests for FederationRMFailoverProxyProvider.
+ */
+public class TestFederationRMFailoverProxyProvider {
+
+  private Configuration conf;
+  private FederationStateStore stateStore;
+  private final String dummyCapability = "cap";
+
+  @Before
+  public void setUp() throws IOException, YarnException {
+    conf = new YarnConfiguration();
+    stateStore = new MemoryFederationStateStore();
+    stateStore.init(conf);
+    FederationStateStoreFacade.getInstance().reinitialize(stateStore, conf);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    stateStore.close();
+    stateStore = null;
+  }
+
+  @Test
+  public void testFederationRMFailoverProxyProvider() throws Exception {
+    final SubClusterId subClusterId = SubClusterId.newInstance("SC-1");
+    final MiniYARNCluster cluster = new MiniYARNCluster(
+        "testFederationRMFailoverProxyProvider", 3, 0, 1, 1);
+
+    conf.setBoolean(YarnConfiguration.RM_HA_ENABLED, true);
+    conf.setBoolean(YarnConfiguration.AUTO_FAILOVER_ENABLED, false);
+    conf.set(YarnConfiguration.RM_CLUSTER_ID, "cluster1");
+    conf.set(YarnConfiguration.RM_HA_IDS, "rm1,rm2,rm3");
+
+    conf.setLong(YarnConfiguration.RESOURCEMANAGER_CONNECT_RETRY_INTERVAL_MS,
+        2000);
+
+    HATestUtil.setRpcAddressForRM("rm1", 10000, conf);
+    HATestUtil.setRpcAddressForRM("rm2", 20000, conf);
+    HATestUtil.setRpcAddressForRM("rm3", 30000, conf);
+    conf.setBoolean(YarnConfiguration.YARN_MINICLUSTER_FIXED_PORTS, true);
+
+    cluster.init(conf);
+    cluster.start();
+
+    // Transition rm3 to active;
+    makeRMActive(subClusterId, cluster, 2);
+
+    ApplicationClientProtocol client = FederationProxyProviderUtil
+        .createRMProxy(conf, ApplicationClientProtocol.class, subClusterId,
+            UserGroupInformation.getCurrentUser());
+
+    // client will retry until the rm becomes active.
+    GetClusterMetricsResponse response =
+        client.getClusterMetrics(GetClusterMetricsRequest.newInstance());
+
+    // validate response
+    checkResponse(response);
+
+    // transition rm3 to standby
+    cluster.getResourceManager(2).getRMContext().getRMAdminService()
+        .transitionToStandby(new HAServiceProtocol.StateChangeRequestInfo(
+            HAServiceProtocol.RequestSource.REQUEST_BY_USER));
+
+    // Transition rm2 to active;
+    makeRMActive(subClusterId, cluster, 1);
+    response = client.getClusterMetrics(GetClusterMetricsRequest.newInstance());
+
+    // validate response
+    checkResponse(response);
+
+    cluster.stop();
+  }
+
+  private void checkResponse(GetClusterMetricsResponse response) {
+    Assert.assertNotNull(response.getClusterMetrics());
+    Assert.assertEquals(0,
+        response.getClusterMetrics().getNumActiveNodeManagers());
+  }
+
+  private void makeRMActive(final SubClusterId subClusterId,
+      final MiniYARNCluster cluster, final int index) {
+    try {
+      System.out.println("Transition rm" + (index + 1) + " to active");
+      String dummyAddress = "host:" + index;
+      cluster.getResourceManager(index).getRMContext().getRMAdminService()
+          .transitionToActive(new HAServiceProtocol.StateChangeRequestInfo(
+              HAServiceProtocol.RequestSource.REQUEST_BY_USER));
+      ResourceManager rm = cluster.getResourceManager(index);
+      InetSocketAddress amRMAddress =
+          rm.getApplicationMasterService().getBindAddress();
+      InetSocketAddress clientRMAddress =
+          rm.getClientRMService().getBindAddress();
+      SubClusterRegisterRequest request = SubClusterRegisterRequest
+          .newInstance(SubClusterInfo.newInstance(subClusterId,
+              amRMAddress.getAddress().getHostAddress() + ":"
+                  + amRMAddress.getPort(),
+              clientRMAddress.getAddress().getHostAddress() + ":"
+                  + clientRMAddress.getPort(),
+              dummyAddress, dummyAddress, SubClusterState.SC_NEW, 1,
+              dummyCapability));
+      stateStore.registerSubCluster(request);
+    } catch (Exception e) {
+      e.printStackTrace();
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b96be3da/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/ClientRMProxy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/ClientRMProxy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/ClientRMProxy.java
index b29263e..6365662 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/ClientRMProxy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/ClientRMProxy.java
@@ -84,7 +84,7 @@ public class ClientRMProxy<T> extends RMProxy<T>  {
 
   @Private
   @Override
-  protected InetSocketAddress getRMAddress(YarnConfiguration conf,
+  public InetSocketAddress getRMAddress(YarnConfiguration conf,
       Class<?> protocol) throws IOException {
     if (protocol == ApplicationClientProtocol.class) {
       return conf.getSocketAddr(YarnConfiguration.RM_ADDRESS,
@@ -111,7 +111,7 @@ public class ClientRMProxy<T> extends RMProxy<T>  {
 
   @Private
   @Override
-  protected void checkAllowedProtocols(Class<?> protocol) {
+  public void checkAllowedProtocols(Class<?> protocol) {
     Preconditions.checkArgument(
         protocol.isAssignableFrom(ClientRMProtocols.class),
         "RM does not support this client protocol");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b96be3da/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/RMProxy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/RMProxy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/RMProxy.java
index 3ab06bd..f93a182 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/RMProxy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/RMProxy.java
@@ -33,8 +33,8 @@ import java.util.concurrent.TimeUnit;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.retry.RetryPolicies;
 import org.apache.hadoop.io.retry.RetryPolicy;
@@ -64,14 +64,14 @@ public class RMProxy<T> {
    * Verify the passed protocol is supported.
    */
   @Private
-  protected void checkAllowedProtocols(Class<?> protocol) {}
+  public void checkAllowedProtocols(Class<?> protocol) {}
 
   /**
    * Get the ResourceManager address from the provided Configuration for the
    * given protocol.
    */
   @Private
-  protected InetSocketAddress getRMAddress(
+  public InetSocketAddress getRMAddress(
       YarnConfiguration conf, Class<?> protocol) throws IOException {
     throw new UnsupportedOperationException("This method should be invoked " +
         "from an instance of ClientRMProxy or ServerRMProxy");
@@ -90,7 +90,8 @@ public class RMProxy<T> {
     YarnConfiguration conf = (configuration instanceof YarnConfiguration)
         ? (YarnConfiguration) configuration
         : new YarnConfiguration(configuration);
-    RetryPolicy retryPolicy = createRetryPolicy(conf, HAUtil.isHAEnabled(conf));
+    RetryPolicy retryPolicy = createRetryPolicy(conf,
+        (HAUtil.isHAEnabled(conf) || HAUtil.isFederationFailoverEnabled(conf)));
     return newProxyInstance(conf, protocol, instance, retryPolicy);
   }
 
@@ -116,7 +117,7 @@ public class RMProxy<T> {
   private static <T> T newProxyInstance(final YarnConfiguration conf,
       final Class<T> protocol, RMProxy instance, RetryPolicy retryPolicy)
           throws IOException{
-    if (HAUtil.isHAEnabled(conf)) {
+    if (HAUtil.isHAEnabled(conf) || HAUtil.isFederationEnabled(conf)) {
       RMFailoverProxyProvider<T> provider =
           instance.createRMFailoverProxyProvider(conf, protocol);
       return (T) RetryProxy.create(protocol, provider, retryPolicy);
@@ -146,7 +147,8 @@ public class RMProxy<T> {
   @Deprecated
   public static <T> T createRMProxy(final Configuration conf,
       final Class<T> protocol, InetSocketAddress rmAddress) throws IOException {
-    RetryPolicy retryPolicy = createRetryPolicy(conf, HAUtil.isHAEnabled(conf));
+    RetryPolicy retryPolicy = createRetryPolicy(conf,
+        (HAUtil.isHAEnabled(conf) || HAUtil.isFederationFailoverEnabled(conf)));
     T proxy = RMProxy.<T>getProxy(conf, protocol, rmAddress);
     LOG.info("Connecting to ResourceManager at " + rmAddress);
     return (T) RetryProxy.create(protocol, proxy, retryPolicy);
@@ -155,9 +157,16 @@ public class RMProxy<T> {
   /**
    * Get a proxy to the RM at the specified address. To be used to create a
    * RetryProxy.
+   *
+   * @param conf Configuration to generate retry policy
+   * @param protocol Protocol for the proxy
+   * @param rmAddress Address of the ResourceManager
+   * @param <T> Type information of the proxy
+   * @return Proxy to the RM
+   * @throws IOException on failure
    */
   @Private
-  static <T> T getProxy(final Configuration conf,
+  public static <T> T getProxy(final Configuration conf,
       final Class<T> protocol, final InetSocketAddress rmAddress)
       throws IOException {
     return UserGroupInformation.getCurrentUser().doAs(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b96be3da/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
index 5ba81de..0d40482 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
@@ -2672,6 +2672,13 @@
   <!-- Federation Configuration -->
   <property>
     <description>
+      Flag to indicate whether the RM is participating in Federation or not.
+    </description>
+    <name>yarn.federation.enabled</name>
+    <value>false</value>
+  </property>
+  <property>
+    <description>
       Machine list file to be loaded by the FederationSubCluster Resolver
     </description>
     <name>yarn.federation.machine-list</name>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b96be3da/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml
index cfb2533..5ae8889 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml
@@ -111,12 +111,10 @@
     <dependency>
       <groupId>javax.cache</groupId>
       <artifactId>cache-api</artifactId>
-      <version>${jcache.version}</version>
     </dependency>
     <dependency>
       <groupId>org.ehcache</groupId>
       <artifactId>ehcache</artifactId>
-      <version>${ehcache.version}</version>
     </dependency>
   </dependencies>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b96be3da/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/ServerRMProxy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/ServerRMProxy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/ServerRMProxy.java
index 8555fc3..b3038e9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/ServerRMProxy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/ServerRMProxy.java
@@ -71,7 +71,7 @@ public class ServerRMProxy<T> extends RMProxy<T> {
 
   @InterfaceAudience.Private
   @Override
-  protected InetSocketAddress getRMAddress(YarnConfiguration conf,
+  public InetSocketAddress getRMAddress(YarnConfiguration conf,
                                            Class<?> protocol) {
     if (protocol == ResourceTracker.class) {
       return conf.getSocketAddr(
@@ -93,7 +93,7 @@ public class ServerRMProxy<T> extends RMProxy<T> {
 
   @InterfaceAudience.Private
   @Override
-  protected void checkAllowedProtocols(Class<?> protocol) {
+  public void checkAllowedProtocols(Class<?> protocol) {
     Preconditions.checkArgument(
         protocol.isAssignableFrom(ResourceTracker.class),
         "ResourceManager does not support this protocol");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b96be3da/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/failover/FederationProxyProviderUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/failover/FederationProxyProviderUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/failover/FederationProxyProviderUtil.java
new file mode 100644
index 0000000..a986008
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/failover/FederationProxyProviderUtil.java
@@ -0,0 +1,163 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.failover;
+
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.security.SaslRpcServer;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.yarn.client.ClientRMProxy;
+import org.apache.hadoop.yarn.client.RMFailoverProxyProvider;
+import org.apache.hadoop.yarn.conf.HAUtil;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Utility class that creates proxy for specified protocols when federation is
+ * enabled. The class creates a federation aware failover provider, i.e. the
+ * failover provider uses the {@code FederationStateStore} to determine the
+ * current active ResourceManager
+ */
+@Private
+@Unstable
+public final class FederationProxyProviderUtil {
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(FederationProxyProviderUtil.class);
+
+  /**
+   * Create a proxy for the specified protocol. For non-HA, this is a direct
+   * connection to the ResourceManager address. When HA is enabled, the proxy
+   * handles the failover between the ResourceManagers as well.
+   *
+   * @param configuration Configuration to generate {@link ClientRMProxy}
+   * @param protocol Protocol for the proxy
+   * @param subClusterId the unique identifier or the sub-cluster
+   * @param user the user on whose behalf the proxy is being created
+   * @param <T> Type information of the proxy
+   * @return Proxy to the RM
+   * @throws IOException on failure
+   */
+  @Public
+  @Unstable
+  public static <T> T createRMProxy(Configuration configuration,
+      final Class<T> protocol, SubClusterId subClusterId,
+      UserGroupInformation user) throws IOException {
+    return createRMProxy(configuration, protocol, subClusterId, user, null);
+  }
+
+  /**
+   * Create a proxy for the specified protocol. For non-HA, this is a direct
+   * connection to the ResourceManager address. When HA is enabled, the proxy
+   * handles the failover between the ResourceManagers as well.
+   *
+   * @param configuration Configuration to generate {@link ClientRMProxy}
+   * @param protocol Protocol for the proxy
+   * @param subClusterId the unique identifier or the sub-cluster
+   * @param user the user on whose behalf the proxy is being created
+   * @param token the auth token to use for connection
+   * @param <T> Type information of the proxy
+   * @return Proxy to the RM
+   * @throws IOException on failure
+   */
+  @Public
+  @Unstable
+  @SuppressWarnings("unchecked")
+  public static <T> T createRMProxy(final Configuration configuration,
+      final Class<T> protocol, SubClusterId subClusterId,
+      UserGroupInformation user, final Token token) throws IOException {
+    try {
+      final YarnConfiguration conf = new YarnConfiguration(configuration);
+      updateConf(conf, subClusterId);
+      if (token != null) {
+        LOG.info(
+            "Creating RMProxy with a token: {} to subcluster: {}"
+                + " for protocol: {}",
+            token, subClusterId, protocol.getSimpleName());
+        user.addToken(token);
+        setAuthModeInConf(conf);
+      } else {
+        LOG.info("Creating RMProxy without a token to subcluster: {}"
+            + " for protocol: {}", subClusterId, protocol.getSimpleName());
+      }
+      final T proxyConnection = user.doAs(new PrivilegedExceptionAction<T>() {
+        @Override
+        public T run() throws Exception {
+          return ClientRMProxy.createRMProxy(conf, protocol);
+        }
+      });
+
+      return proxyConnection;
+    } catch (IOException e) {
+      String message =
+          "Error while creating of RM application master service proxy for"
+              + " appAttemptId: " + user;
+      LOG.info(message);
+      throw new YarnRuntimeException(message, e);
+    } catch (InterruptedException e) {
+      throw new YarnRuntimeException(e);
+    }
+  }
+
+  private static void setAuthModeInConf(Configuration conf) {
+    conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION,
+        SaslRpcServer.AuthMethod.TOKEN.toString());
+  }
+
+  // updating the conf with the refreshed RM addresses as proxy creations
+  // are based out of conf
+  private static void updateConf(Configuration conf,
+      SubClusterId subClusterId) {
+    conf.set(YarnConfiguration.FEDERATION_SUBCLUSTER_ID, subClusterId.getId());
+    // In a Federation setting, we will connect to not just the local cluster RM
+    // but also multiple external RMs. The membership information of all the RMs
+    // that are currently
+    // participating in Federation is available in the central
+    // FederationStateStore.
+    // So we will:
+    // 1. obtain the RM service addresses from FederationStateStore using the
+    // FederationRMFailoverProxyProvider.
+    // 2. disable traditional HA as that depends on local configuration lookup
+    // for RMs using indexes.
+    // 3. we will enable federation failover IF traditional HA is enabled so
+    // that the appropriate failover RetryPolicy is initialized.
+    conf.setBoolean(YarnConfiguration.FEDERATION_ENABLED, true);
+    conf.setClass(YarnConfiguration.CLIENT_FAILOVER_PROXY_PROVIDER,
+        FederationRMFailoverProxyProvider.class, RMFailoverProxyProvider.class);
+    if (HAUtil.isHAEnabled(conf)) {
+      conf.setBoolean(YarnConfiguration.FEDERATION_FAILOVER_ENABLED, true);
+      conf.setBoolean(YarnConfiguration.RM_HA_ENABLED, false);
+    }
+  }
+
+  // disable instantiation
+  private FederationProxyProviderUtil() {
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b96be3da/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/failover/FederationRMFailoverProxyProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/failover/FederationRMFailoverProxyProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/failover/FederationRMFailoverProxyProvider.java
new file mode 100644
index 0000000..90a9239
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/failover/FederationRMFailoverProxyProvider.java
@@ -0,0 +1,211 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.failover;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.Collection;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
+import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
+import org.apache.hadoop.yarn.client.RMFailoverProxyProvider;
+import org.apache.hadoop.yarn.client.RMProxy;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.api.ResourceManagerAdministrationProtocol;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
+import org.apache.hadoop.yarn.server.federation.utils.FederationStateStoreFacade;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * A FailoverProxyProvider implementation that uses the
+ * {@code FederationStateStore} to determine the ResourceManager to connect to.
+ * This supports both HA and regular mode which is controlled by configuration.
+ */
+@Private
+@Unstable
+public class FederationRMFailoverProxyProvider<T>
+    implements RMFailoverProxyProvider<T> {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(FederationRMFailoverProxyProvider.class);
+
+  private RMProxy<T> rmProxy;
+  private Class<T> protocol;
+  private T current;
+  private YarnConfiguration conf;
+  private FederationStateStoreFacade facade;
+  private SubClusterId subClusterId;
+  private Collection<Token<? extends TokenIdentifier>> originalTokens;
+  private boolean federationFailoverEnabled = false;
+
+  @Override
+  public void init(Configuration configuration, RMProxy<T> proxy,
+      Class<T> proto) {
+    this.rmProxy = proxy;
+    this.protocol = proto;
+    this.rmProxy.checkAllowedProtocols(this.protocol);
+    String clusterId =
+        configuration.get(YarnConfiguration.FEDERATION_SUBCLUSTER_ID);
+    Preconditions.checkNotNull(clusterId, "Missing Federation SubClusterId");
+    this.subClusterId = SubClusterId.newInstance(clusterId);
+    this.facade = facade.getInstance();
+    if (configuration instanceof YarnConfiguration) {
+      this.conf = (YarnConfiguration) configuration;
+    }
+    federationFailoverEnabled =
+        conf.getBoolean(YarnConfiguration.FEDERATION_FAILOVER_ENABLED,
+            YarnConfiguration.DEFAULT_FEDERATION_FAILOVER_ENABLED);
+
+    conf.setInt(
+        CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY,
+        conf.getInt(YarnConfiguration.CLIENT_FAILOVER_RETRIES,
+            YarnConfiguration.DEFAULT_CLIENT_FAILOVER_RETRIES));
+
+    conf.setInt(
+        CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SOCKET_TIMEOUTS_KEY,
+        conf.getInt(
+            YarnConfiguration.CLIENT_FAILOVER_RETRIES_ON_SOCKET_TIMEOUTS,
+            YarnConfiguration.DEFAULT_CLIENT_FAILOVER_RETRIES_ON_SOCKET_TIMEOUTS));
+
+    try {
+      UserGroupInformation currentUser = UserGroupInformation.getCurrentUser();
+      originalTokens = currentUser.getTokens();
+      LOG.info("Initialized Federation proxy for user: {}",
+          currentUser.getUserName());
+    } catch (IOException e) {
+      LOG.warn("Could not get information of requester, ignoring for now.");
+    }
+
+  }
+
+  private void addOriginalTokens(UserGroupInformation currentUser) {
+    if (originalTokens == null || originalTokens.isEmpty()) {
+      return;
+    }
+    for (Token<? extends TokenIdentifier> token : originalTokens) {
+      currentUser.addToken(token);
+    }
+  }
+
+  private T getProxyInternal(boolean isFailover) {
+    SubClusterInfo subClusterInfo;
+    UserGroupInformation currentUser = null;
+    try {
+      LOG.info("Failing over to the ResourceManager for SubClusterId: {}",
+          subClusterId);
+      subClusterInfo = facade.getSubCluster(subClusterId, isFailover);
+      // updating the conf with the refreshed RM addresses as proxy
+      // creations
+      // are based out of conf
+      updateRMAddress(subClusterInfo);
+      currentUser = UserGroupInformation.getCurrentUser();
+      addOriginalTokens(currentUser);
+    } catch (YarnException e) {
+      LOG.error("Exception while trying to create proxy to the ResourceManager"
+          + " for SubClusterId: {}", subClusterId, e);
+      return null;
+    } catch (IOException e) {
+      LOG.warn("Could not get information of requester, ignoring for now.");
+    }
+    try {
+      final InetSocketAddress rmAddress = rmProxy.getRMAddress(conf, protocol);
+      LOG.info("Connecting to {} with protocol {} as user: {}", rmAddress,
+          protocol.getSimpleName(), currentUser);
+      LOG.info("Failed over to the RM at {} for SubClusterId: {}", rmAddress,
+          subClusterId);
+      return RMProxy.getProxy(conf, protocol, rmAddress);
+    } catch (IOException ioe) {
+      LOG.error(
+          "IOException while trying to create proxy to the ResourceManager"
+              + " for SubClusterId: {}",
+          subClusterId, ioe);
+      return null;
+    }
+  }
+
+  private void updateRMAddress(SubClusterInfo subClusterInfo) {
+    if (subClusterInfo != null) {
+      if (protocol == ApplicationClientProtocol.class) {
+        conf.set(YarnConfiguration.RM_ADDRESS,
+            subClusterInfo.getClientRMServiceAddress());
+      } else if (protocol == ApplicationMasterProtocol.class) {
+        conf.set(YarnConfiguration.RM_SCHEDULER_ADDRESS,
+            subClusterInfo.getAMRMServiceAddress());
+      } else if (protocol == ResourceManagerAdministrationProtocol.class) {
+        conf.set(YarnConfiguration.RM_ADMIN_ADDRESS,
+            subClusterInfo.getRMAdminServiceAddress());
+      }
+    }
+  }
+
+  @Override
+  public synchronized ProxyInfo<T> getProxy() {
+    if (current == null) {
+      current = getProxyInternal(false);
+    }
+    return new ProxyInfo<T>(current, subClusterId.getId());
+  }
+
+  @Override
+  public synchronized void performFailover(T currentProxy) {
+    closeInternal(currentProxy);
+    current = getProxyInternal(federationFailoverEnabled);
+  }
+
+  @Override
+  public Class<T> getInterface() {
+    return protocol;
+  }
+
+  private void closeInternal(T currentProxy) {
+    if ((currentProxy != null) && (currentProxy instanceof Closeable)) {
+      try {
+        ((Closeable) currentProxy).close();
+      } catch (IOException e) {
+        LOG.warn("Exception while trying to close proxy", e);
+      }
+    } else {
+      RPC.stopProxy(currentProxy);
+    }
+
+  }
+
+  /**
+   * Close all the proxy objects which have been opened over the lifetime of
+   * this proxy provider.
+   */
+  @Override
+  public synchronized void close() throws IOException {
+    closeInternal(current);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b96be3da/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/failover/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/failover/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/failover/package-info.java
new file mode 100644
index 0000000..b1baa0c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/failover/package-info.java
@@ -0,0 +1,17 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.yarn.server.federation.failover;


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[09/42] hadoop git commit: YARN-5406. In-memory based implementation of the FederationMembershipStateStore. Contributed by Ellen Hui.

Posted by su...@apache.org.
YARN-5406. In-memory based implementation of the FederationMembershipStateStore. Contributed by Ellen Hui.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/38ddea03
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/38ddea03
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/38ddea03

Branch: refs/heads/YARN-2915
Commit: 38ddea031bfc3f99ec8bd48e388430bea54fbd7d
Parents: abe79cb
Author: Subru Krishnan <su...@apache.org>
Authored: Thu Aug 4 15:54:38 2016 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon May 8 16:59:12 2017 -0700

----------------------------------------------------------------------
 .../store/impl/MemoryFederationStateStore.java  | 138 ++++++++++++
 .../federation/store/impl/package-info.java     |  17 ++
 .../records/GetSubClustersInfoRequest.java      |   4 +
 .../store/records/SubClusterState.java          |   4 +
 .../impl/FederationStateStoreBaseTest.java      | 221 +++++++++++++++++++
 .../impl/TestMemoryFederationStateStore.java    |  49 ++++
 6 files changed, 433 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/38ddea03/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/MemoryFederationStateStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/MemoryFederationStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/MemoryFederationStateStore.java
new file mode 100644
index 0000000..7fdc4a9
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/MemoryFederationStateStore.java
@@ -0,0 +1,138 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.impl;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
+import org.apache.hadoop.yarn.server.federation.store.FederationMembershipStateStore;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterDeregisterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterDeregisterResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterInfoRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterInfoResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClustersInfoRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClustersInfoResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterHeartbeatRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterHeartbeatResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterRegisterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterRegisterResponse;
+import org.apache.hadoop.yarn.server.records.Version;
+import org.apache.hadoop.yarn.util.MonotonicClock;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * In-memory implementation of FederationMembershipStateStore.
+ */
+public class MemoryFederationStateStore
+    implements FederationMembershipStateStore {
+
+  private final Map<SubClusterId, SubClusterInfo> membership =
+      new ConcurrentHashMap<SubClusterId, SubClusterInfo>();
+  private final MonotonicClock clock = new MonotonicClock();
+
+  @Override
+  public Version getMembershipStateStoreVersion() {
+    return null;
+  }
+
+  @Override
+  public SubClusterRegisterResponse registerSubCluster(
+      SubClusterRegisterRequest request) throws YarnException {
+    SubClusterInfo subClusterInfo = request.getSubClusterInfo();
+    subClusterInfo.setLastStartTime(clock.getTime());
+    membership.put(subClusterInfo.getSubClusterId(), subClusterInfo);
+    return SubClusterRegisterResponse.newInstance();
+  }
+
+  @Override
+  public SubClusterDeregisterResponse deregisterSubCluster(
+      SubClusterDeregisterRequest request) throws YarnException {
+    SubClusterInfo subClusterInfo = membership.get(request.getSubClusterId());
+    if (subClusterInfo == null) {
+      throw new YarnException(
+          "SubCluster " + request.getSubClusterId().toString() + " not found");
+    } else {
+      subClusterInfo.setState(request.getState());
+    }
+
+    return SubClusterDeregisterResponse.newInstance();
+  }
+
+  @Override
+  public SubClusterHeartbeatResponse subClusterHeartbeat(
+      SubClusterHeartbeatRequest request) throws YarnException {
+
+    SubClusterId subClusterId = request.getSubClusterId();
+    SubClusterInfo subClusterInfo = membership.get(subClusterId);
+
+    if (subClusterInfo == null) {
+      throw new YarnException("Subcluster " + subClusterId.toString()
+          + " does not exist; cannot heartbeat");
+    }
+
+    subClusterInfo.setLastHeartBeat(clock.getTime());
+    subClusterInfo.setState(request.getState());
+    subClusterInfo.setCapability(request.getCapability());
+
+    return SubClusterHeartbeatResponse.newInstance();
+  }
+
+  @Override
+  public GetSubClusterInfoResponse getSubCluster(
+      GetSubClusterInfoRequest request) throws YarnException {
+    SubClusterId subClusterId = request.getSubClusterId();
+    if (!membership.containsKey(subClusterId)) {
+      throw new YarnException(
+          "Subcluster " + subClusterId.toString() + " does not exist");
+    }
+
+    return GetSubClusterInfoResponse.newInstance(membership.get(subClusterId));
+  }
+
+  @Override
+  public GetSubClustersInfoResponse getSubClusters(
+      GetSubClustersInfoRequest request) throws YarnException {
+    List<SubClusterInfo> result = new ArrayList<SubClusterInfo>();
+
+    for (SubClusterInfo info : membership.values()) {
+      if (!request.getFilterInactiveSubClusters()
+          || info.getState().isActive()) {
+        result.add(info);
+      }
+    }
+    return GetSubClustersInfoResponse.newInstance(result);
+
+  }
+
+  @VisibleForTesting
+  public Map<SubClusterId, SubClusterInfo> getMembershipTable() {
+    return membership;
+  }
+
+  @VisibleForTesting
+  public void clearMembershipTable() {
+    membership.clear();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38ddea03/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/package-info.java
new file mode 100644
index 0000000..56e1274
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/package-info.java
@@ -0,0 +1,17 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.yarn.server.federation.store.impl;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38ddea03/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetSubClustersInfoRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetSubClustersInfoRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetSubClustersInfoRequest.java
index 3264d81..90d2f99 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetSubClustersInfoRequest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetSubClustersInfoRequest.java
@@ -26,6 +26,10 @@ import org.apache.hadoop.yarn.util.Records;
 /**
  * Request class to obtain information about all sub-clusters that are
  * participating in federation.
+ *
+ * If filterInactiveSubClusters is set to true, only active sub-clusters will be
+ * returned; otherwise, all sub-clusters will be returned regardless of state.
+ * By default, filterInactiveSubClusters is true.
  */
 @Private
 @Unstable

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38ddea03/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterState.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterState.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterState.java
index 22cec99..ff49aaa 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterState.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterState.java
@@ -53,6 +53,10 @@ public enum SubClusterState {
     return (this != SC_RUNNING && this != SC_NEW);
   }
 
+  public boolean isActive() {
+    return this == SC_RUNNING;
+  }
+
   public boolean isFinal() {
     return (this == SC_UNREGISTERED || this == SC_DECOMMISSIONED
         || this == SC_LOST);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38ddea03/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
----------------------------------------------------------------------
diff --git a/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 b/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
new file mode 100644
index 0000000..7eb1c86
--- /dev/null
+++ b/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
@@ -0,0 +1,221 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.impl;
+
+import java.io.IOException;
+
+import org.junit.Before;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterState;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterDeregisterRequest;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.federation.store.FederationMembershipStateStore;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterInfoRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClustersInfoRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterHeartbeatRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterRegisterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterRegisterResponse;
+import org.apache.hadoop.yarn.util.MonotonicClock;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Base class for FederationMembershipStateStore implementations.
+ */
+public abstract class FederationStateStoreBaseTest {
+
+  static final Logger LOG =
+      LoggerFactory.getLogger(FederationStateStoreBaseTest.class);
+  private static final MonotonicClock CLOCK = new MonotonicClock();
+
+  private FederationMembershipStateStore stateStore = getStateStore();
+
+  @Before
+  public void before() throws IOException {
+    clearMembership();
+  }
+
+  @Test
+  public void testRegisterSubCluster() throws Exception {
+    SubClusterId subClusterId = SubClusterId.newInstance("SC");
+    SubClusterInfo subClusterInfo = createSubClusterInfo(subClusterId);
+
+    SubClusterRegisterResponse result = stateStore.registerSubCluster(
+        SubClusterRegisterRequest.newInstance(subClusterInfo));
+    Map<SubClusterId, SubClusterInfo> membership = getMembership();
+
+    Assert.assertNotNull(membership.get(subClusterId));
+    Assert.assertNotNull(result);
+    Assert.assertEquals(subClusterInfo, membership.get(subClusterId));
+  }
+
+  @Test
+  public void testDeregisterSubCluster() throws Exception {
+    SubClusterId subClusterId = SubClusterId.newInstance("SC");
+    SubClusterInfo subClusterInfo = createSubClusterInfo(subClusterId);
+
+    stateStore.registerSubCluster(
+        SubClusterRegisterRequest.newInstance(subClusterInfo));
+
+    SubClusterDeregisterRequest deregisterRequest = SubClusterDeregisterRequest
+        .newInstance(subClusterId, SubClusterState.SC_UNREGISTERED);
+
+    stateStore.deregisterSubCluster(deregisterRequest);
+
+    Map<SubClusterId, SubClusterInfo> membership = getMembership();
+    Assert.assertNotNull(membership.get(subClusterId));
+    Assert.assertEquals(membership.get(subClusterId).getState(),
+        SubClusterState.SC_UNREGISTERED);
+  }
+
+  @Test
+  public void testDeregisterSubClusterUnknownSubCluster() throws Exception {
+    SubClusterId subClusterId = SubClusterId.newInstance("SC");
+
+    SubClusterDeregisterRequest deregisterRequest = SubClusterDeregisterRequest
+        .newInstance(subClusterId, SubClusterState.SC_UNREGISTERED);
+    try {
+      stateStore.deregisterSubCluster(deregisterRequest);
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertTrue(e.getMessage().startsWith("SubCluster SC not found"));
+    }
+  }
+
+  @Test
+  public void testGetSubClusterInfo() throws Exception {
+
+    SubClusterId subClusterId = SubClusterId.newInstance("SC");
+    SubClusterInfo subClusterInfo = createSubClusterInfo(subClusterId);
+
+    stateStore.registerSubCluster(
+        SubClusterRegisterRequest.newInstance(subClusterInfo));
+
+    GetSubClusterInfoRequest request =
+        GetSubClusterInfoRequest.newInstance(subClusterId);
+    Assert.assertEquals(subClusterInfo,
+        stateStore.getSubCluster(request).getSubClusterInfo());
+  }
+
+  @Test
+  public void testGetSubClusterInfoUnknownSubCluster() throws Exception {
+    SubClusterId subClusterId = SubClusterId.newInstance("SC");
+    GetSubClusterInfoRequest request =
+        GetSubClusterInfoRequest.newInstance(subClusterId);
+
+    try {
+      stateStore.getSubCluster(request).getSubClusterInfo();
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertTrue(
+          e.getMessage().startsWith("Subcluster SC does not exist"));
+    }
+  }
+
+  @Test
+  public void testGetAllSubClustersInfo() throws Exception {
+
+    SubClusterId subClusterId1 = SubClusterId.newInstance("SC1");
+    SubClusterInfo subClusterInfo1 = createSubClusterInfo(subClusterId1);
+
+    SubClusterId subClusterId2 = SubClusterId.newInstance("SC2");
+    SubClusterInfo subClusterInfo2 = createSubClusterInfo(subClusterId2);
+
+    stateStore.registerSubCluster(
+        SubClusterRegisterRequest.newInstance(subClusterInfo1));
+    stateStore.registerSubCluster(
+        SubClusterRegisterRequest.newInstance(subClusterInfo2));
+
+    stateStore.subClusterHeartbeat(SubClusterHeartbeatRequest
+        .newInstance(subClusterId1, SubClusterState.SC_RUNNING, ""));
+    stateStore.subClusterHeartbeat(SubClusterHeartbeatRequest
+        .newInstance(subClusterId2, SubClusterState.SC_UNHEALTHY, ""));
+
+    Assert.assertTrue(
+        stateStore.getSubClusters(GetSubClustersInfoRequest.newInstance(true))
+            .getSubClusters().contains(subClusterInfo1));
+    Assert.assertFalse(
+        stateStore.getSubClusters(GetSubClustersInfoRequest.newInstance(true))
+            .getSubClusters().contains(subClusterInfo2));
+
+    Assert.assertTrue(
+        stateStore.getSubClusters(GetSubClustersInfoRequest.newInstance(false))
+            .getSubClusters().contains(subClusterInfo1));
+    Assert.assertTrue(
+        stateStore.getSubClusters(GetSubClustersInfoRequest.newInstance(false))
+            .getSubClusters().contains(subClusterInfo2));
+  }
+
+  @Test
+  public void testSubClusterHeartbeat() throws Exception {
+    SubClusterId subClusterId = SubClusterId.newInstance("SC");
+    SubClusterInfo subClusterInfo = createSubClusterInfo(subClusterId);
+
+    stateStore.registerSubCluster(
+        SubClusterRegisterRequest.newInstance(subClusterInfo));
+
+    SubClusterHeartbeatRequest heartbeatRequest = SubClusterHeartbeatRequest
+        .newInstance(subClusterId, SubClusterState.SC_RUNNING, "cabability");
+    stateStore.subClusterHeartbeat(heartbeatRequest);
+
+    Map<SubClusterId, SubClusterInfo> membership = getMembership();
+    Assert.assertEquals(membership.get(subClusterId).getState(),
+        SubClusterState.SC_RUNNING);
+    Assert.assertNotNull(membership.get(subClusterId).getLastHeartBeat());
+  }
+
+  @Test
+  public void testSubClusterHeartbeatUnknownSubCluster() throws Exception {
+    SubClusterId subClusterId = SubClusterId.newInstance("SC");
+    SubClusterHeartbeatRequest heartbeatRequest = SubClusterHeartbeatRequest
+        .newInstance(subClusterId, SubClusterState.SC_RUNNING, "cabability");
+
+    try {
+      stateStore.subClusterHeartbeat(heartbeatRequest);
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertTrue(e.getMessage()
+          .startsWith("Subcluster SC does not exist; cannot heartbeat"));
+    }
+  }
+
+  private SubClusterInfo createSubClusterInfo(SubClusterId subClusterId) {
+
+    String amRMAddress = "1.2.3.4:1";
+    String clientRMAddress = "1.2.3.4:2";
+    String rmAdminAddress = "1.2.3.4:3";
+    String webAppAddress = "1.2.3.4:4";
+
+    return SubClusterInfo.newInstance(subClusterId, amRMAddress,
+        clientRMAddress, rmAdminAddress, webAppAddress, SubClusterState.SC_NEW,
+        CLOCK.getTime(), "cabability");
+  }
+
+  protected abstract Map<SubClusterId, SubClusterInfo> getMembership();
+
+  protected abstract void clearMembership();
+
+  protected abstract FederationMembershipStateStore getStateStore();
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38ddea03/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
----------------------------------------------------------------------
diff --git a/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 b/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
new file mode 100644
index 0000000..b74ffbd
--- /dev/null
+++ b/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
@@ -0,0 +1,49 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.impl;
+
+import java.util.Map;
+
+import org.apache.hadoop.yarn.server.federation.store.FederationMembershipStateStore;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
+
+/**
+ * Unit tests for MemoryFederationStateStore.
+ */
+public class TestMemoryFederationStateStore
+    extends FederationStateStoreBaseTest {
+
+  private static final MemoryFederationStateStore STATESTORE =
+      new MemoryFederationStateStore();
+
+  @Override
+  protected Map<SubClusterId, SubClusterInfo> getMembership() {
+    return STATESTORE.getMembershipTable();
+  }
+
+  @Override
+  protected void clearMembership() {
+    STATESTORE.clearMembershipTable();
+  }
+
+  @Override
+  protected FederationMembershipStateStore getStateStore() {
+    return STATESTORE;
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[30/42] hadoop git commit: YARN-5410. Bootstrap Router server module. (Giovanni Matteo Fumarola via Subru).

Posted by su...@apache.org.
YARN-5410. Bootstrap Router server module. (Giovanni Matteo Fumarola via Subru).


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/c2aab006
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/c2aab006
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/c2aab006

Branch: refs/heads/YARN-2915
Commit: c2aab0062a486d7abef206320dc76d5a975459d5
Parents: 8f6f1c4
Author: Subru Krishnan <su...@apache.org>
Authored: Fri Feb 24 12:08:53 2017 -0800
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon May 8 16:59:13 2017 -0700

----------------------------------------------------------------------
 hadoop-project/pom.xml                          |  6 ++
 .../hadoop-yarn-server-router/pom.xml           | 62 ++++++++++++++++++++
 .../hadoop/yarn/server/router/Router.java       | 38 ++++++++++++
 .../hadoop/yarn/server/router/package-info.java | 20 +++++++
 .../hadoop/yarn/server/router/TestRouter.java   | 26 ++++++++
 .../hadoop-yarn/hadoop-yarn-server/pom.xml      |  1 +
 hadoop-yarn-project/pom.xml                     |  4 ++
 7 files changed, 157 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c2aab006/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
old mode 100644
new mode 100755
index c67150f..b07c510
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -408,6 +408,12 @@
 
       <dependency>
         <groupId>org.apache.hadoop</groupId>
+        <artifactId>hadoop-yarn-server-router</artifactId>
+        <version>${project.version}</version>
+      </dependency>
+
+      <dependency>
+        <groupId>org.apache.hadoop</groupId>
          <artifactId>hadoop-mapreduce-client-jobclient</artifactId>
         <version>${project.version}</version>
         <type>test-jar</type>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c2aab006/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/pom.xml
new file mode 100644
index 0000000..25afa5c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/pom.xml
@@ -0,0 +1,62 @@
+<?xml version="1.0"?>
+<!--
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License. See accompanying LICENSE file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/POM/4.0.0
+                      http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <parent>
+    <artifactId>hadoop-yarn-server</artifactId>
+    <groupId>org.apache.hadoop</groupId>
+    <version>3.0.0-alpha3-SNAPSHOT</version>
+  </parent>
+  <modelVersion>4.0.0</modelVersion>
+  <groupId>org.apache.hadoop</groupId>
+  <artifactId>hadoop-yarn-server-router</artifactId>
+  <version>3.0.0-alpha3-SNAPSHOT</version>
+  <name>Apache Hadoop YARN Router</name>
+
+  <properties>
+    <!-- Needed for generating FindBugs warnings using parent pom -->
+    <yarn.basedir>${project.parent.parent.basedir}</yarn.basedir>
+  </properties>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-api</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-common</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-server-common</artifactId>
+    </dependency>
+  </dependencies>
+
+  <build>
+    <plugins>
+    </plugins>
+  </build>
+
+</project>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c2aab006/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/Router.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/Router.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/Router.java
new file mode 100644
index 0000000..7be8a59
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/Router.java
@@ -0,0 +1,38 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.router;
+
+/**
+ * The router is a stateless YARN component which is the entry point to the
+ * cluster. It can be deployed on multiple nodes behind a Virtual IP (VIP) with
+ * a LoadBalancer.
+ *
+ * The Router exposes the ApplicationClientProtocol (RPC and REST) to the
+ * outside world, transparently hiding the presence of ResourceManager(s), which
+ * allows users to request and update reservations, submit and kill
+ * applications, and request status on running applications.
+ *
+ * In addition, it exposes the ResourceManager Admin API.
+ *
+ * This provides a placeholder for throttling mis-behaving clients (YARN-1546)
+ * and masks the access to multiple RMs (YARN-3659).
+ */
+public class Router{
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c2aab006/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/package-info.java
new file mode 100644
index 0000000..bca1f64
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/package-info.java
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/** Router Server package. **/
+package org.apache.hadoop.yarn.server.router;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c2aab006/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/TestRouter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/TestRouter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/TestRouter.java
new file mode 100644
index 0000000..a31d6b9
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/TestRouter.java
@@ -0,0 +1,26 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.router;
+
+/**
+ * Test class for YARN Router.
+ */
+public class TestRouter {
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c2aab006/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/pom.xml
index 384ced4..37209e0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/pom.xml
@@ -45,5 +45,6 @@
     <module>hadoop-yarn-server-timelineservice</module>
     <module>hadoop-yarn-server-timelineservice-hbase</module>
     <module>hadoop-yarn-server-timelineservice-hbase-tests</module>
+    <module>hadoop-yarn-server-router</module>
   </modules>
 </project>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c2aab006/hadoop-yarn-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/pom.xml b/hadoop-yarn-project/pom.xml
index a64f25a..0f06d60 100644
--- a/hadoop-yarn-project/pom.xml
+++ b/hadoop-yarn-project/pom.xml
@@ -78,6 +78,10 @@
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-yarn-server-timelineservice-hbase</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-server-router</artifactId>
+    </dependency>
   </dependencies>
 
   <build>


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[33/42] hadoop git commit: YARN-5411. Create a proxy chain for ApplicationClientProtocol in the Router. (Giovanni Matteo Fumarola via Subru).

Posted by su...@apache.org.
YARN-5411. Create a proxy chain for ApplicationClientProtocol in the Router. (Giovanni Matteo Fumarola via Subru).


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/0a93a32e
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/0a93a32e
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/0a93a32e

Branch: refs/heads/YARN-2915
Commit: 0a93a32e615b97d2a819efa5cb8fd8acc7b12aed
Parents: 5599b91
Author: Subru Krishnan <su...@apache.org>
Authored: Wed May 3 18:26:15 2017 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon May 8 16:59:13 2017 -0700

----------------------------------------------------------------------
 hadoop-project/pom.xml                          |   7 +
 .../hadoop/yarn/conf/YarnConfiguration.java     |  21 +
 .../hadoop/yarn/util/LRUCacheHashMap.java       |  49 ++
 .../src/main/resources/yarn-default.xml         |  18 +
 .../hadoop/yarn/util/TestLRUCacheHashMap.java   |  74 +++
 .../hadoop-yarn-server-common/pom.xml           |  11 +
 .../yarn/server/MockResourceManagerFacade.java  | 511 +++++++++++++++++
 .../hadoop-yarn-server-nodemanager/pom.xml      |   7 +
 .../amrmproxy/MockRequestInterceptor.java       |  14 +-
 .../amrmproxy/MockResourceManagerFacade.java    | 514 -----------------
 .../hadoop-yarn-server-router/pom.xml           |  19 +
 .../hadoop/yarn/server/router/Router.java       |  98 +++-
 .../AbstractClientRequestInterceptor.java       |  89 +++
 .../clientrm/ClientRequestInterceptor.java      |  65 +++
 .../DefaultClientRequestInterceptor.java        | 334 +++++++++++
 .../router/clientrm/RouterClientRMService.java  | 544 ++++++++++++++++++
 .../server/router/clientrm/package-info.java    |  20 +
 .../hadoop/yarn/server/router/TestRouter.java   |  26 -
 .../router/clientrm/BaseRouterClientRMTest.java | 574 +++++++++++++++++++
 .../clientrm/MockClientRequestInterceptor.java  |  36 ++
 .../PassThroughClientRequestInterceptor.java    | 267 +++++++++
 .../clientrm/TestRouterClientRMService.java     | 210 +++++++
 22 files changed, 2960 insertions(+), 548 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a93a32e/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 61112b2..7f68f45 100755
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -325,6 +325,13 @@
 
       <dependency>
         <groupId>org.apache.hadoop</groupId>
+        <artifactId>hadoop-yarn-server-common</artifactId>
+        <version>${project.version}</version>
+        <type>test-jar</type>
+      </dependency>
+
+      <dependency>
+        <groupId>org.apache.hadoop</groupId>
          <artifactId>hadoop-yarn-server-tests</artifactId>
         <version>${project.version}</version>
         <type>test-jar</type>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a93a32e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
index 7139394..d745903 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
@@ -2615,6 +2615,27 @@ public class YarnConfiguration extends Configuration {
 
   public static final int DEFAULT_FEDERATION_STATESTORE_SQL_MAXCONNECTIONS = 1;
 
+  public static final String ROUTER_PREFIX = YARN_PREFIX + "router.";
+
+  public static final String ROUTER_CLIENTRM_PREFIX =
+      ROUTER_PREFIX + "clientrm.";
+
+  public static final String ROUTER_CLIENTRM_ADDRESS =
+      ROUTER_CLIENTRM_PREFIX + ".address";
+  public static final int DEFAULT_ROUTER_CLIENTRM_PORT = 8050;
+  public static final String DEFAULT_ROUTER_CLIENTRM_ADDRESS =
+      "0.0.0.0:" + DEFAULT_ROUTER_CLIENTRM_PORT;
+
+  public static final String ROUTER_CLIENTRM_INTERCEPTOR_CLASS_PIPELINE =
+      ROUTER_CLIENTRM_PREFIX + "interceptor-class.pipeline";
+  public static final String DEFAULT_ROUTER_CLIENTRM_INTERCEPTOR_CLASS =
+      "org.apache.hadoop.yarn.server.router.clientrm."
+          + "DefaultClientRequestInterceptor";
+
+  public static final String ROUTER_CLIENTRM_PIPELINE_CACHE_MAX_SIZE =
+      ROUTER_CLIENTRM_PREFIX + "cache-max-size";
+  public static final int DEFAULT_ROUTER_CLIENTRM_PIPELINE_CACHE_MAX_SIZE = 25;
+
   ////////////////////////////////
   // Other Configs
   ////////////////////////////////

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a93a32e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/LRUCacheHashMap.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/LRUCacheHashMap.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/LRUCacheHashMap.java
new file mode 100644
index 0000000..7cb4e1b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/LRUCacheHashMap.java
@@ -0,0 +1,49 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.hadoop.yarn.util;
+
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+/**
+ * LRU cache with a configurable maximum cache size and access order.
+ */
+public class LRUCacheHashMap<K, V> extends LinkedHashMap<K, V> {
+
+  private static final long serialVersionUID = 1L;
+
+  // Maximum size of the cache
+  private int maxSize;
+
+  /**
+   * Constructor.
+   *
+   * @param maxSize max size of the cache
+   * @param accessOrder true for access-order, false for insertion-order
+   */
+  public LRUCacheHashMap(int maxSize, boolean accessOrder) {
+    super(maxSize, 0.75f, accessOrder);
+    this.maxSize = maxSize;
+  }
+
+  @Override
+  protected boolean removeEldestEntry(Map.Entry<K, V> eldest) {
+    return size() > maxSize;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a93a32e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
index 75b32ce..03559dc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
@@ -3147,4 +3147,22 @@
     <value>false</value>
   </property>
 
+  <property>
+    <description>
+      The comma separated list of class names that implement the
+      RequestInterceptor interface. This is used by the RouterClientRMService
+      to create the request processing pipeline for users.
+    </description>
+    <name>yarn.router.clientrm.interceptor-class.pipeline</name>
+    <value>org.apache.hadoop.yarn.server.router.clientrm.DefaultClientRequestInterceptor</value>
+  </property>
+
+  <property>
+    <description>
+      Size of LRU cache for Router ClientRM Service.
+    </description>
+    <name>yarn.router.clientrm.cache-max-size</name>
+    <value>25</value>
+  </property>
+
 </configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a93a32e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestLRUCacheHashMap.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestLRUCacheHashMap.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestLRUCacheHashMap.java
new file mode 100644
index 0000000..1cbb56c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestLRUCacheHashMap.java
@@ -0,0 +1,74 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+package org.apache.hadoop.yarn.util;
+
+import java.io.IOException;
+
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test class to validate the correctness of the LRUCacheHashMap.
+ *
+ */
+public class TestLRUCacheHashMap {
+
+  /**
+   * Test if the different entries are generated, and LRU cache is working as
+   * expected.
+   */
+  @Test
+  public void testLRUCache()
+      throws YarnException, IOException, InterruptedException {
+
+    int mapSize = 5;
+
+    LRUCacheHashMap<String, Integer> map =
+        new LRUCacheHashMap<String, Integer>(mapSize, true);
+
+    map.put("1", 1);
+    map.put("2", 2);
+    map.put("3", 3);
+    map.put("4", 4);
+    map.put("5", 5);
+
+    Assert.assertEquals(mapSize, map.size());
+
+    // Check if all the elements in the map are from 1 to 5
+    for (int i = 1; i < mapSize; i++) {
+      Assert.assertTrue(map.containsKey(Integer.toString(i)));
+    }
+
+    map.put("6", 6);
+    map.put("3", 3);
+    map.put("7", 7);
+    map.put("8", 8);
+
+    Assert.assertEquals(mapSize, map.size());
+
+    // Check if all the elements in the map are from 5 to 8 and the 3
+    for (int i = 5; i < mapSize; i++) {
+      Assert.assertTrue(map.containsKey(Integer.toString(i)));
+    }
+
+    Assert.assertTrue(map.containsKey("3"));
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a93a32e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml
index 3bf1b88..ae284ce 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml
@@ -215,6 +215,17 @@
           </excludes>
         </configuration>
       </plugin>
+      <plugin>
+        <artifactId>maven-jar-plugin</artifactId>
+        <executions>
+          <execution>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+            <phase>test-compile</phase>
+          </execution>
+        </executions>
+      </plugin>
     </plugins>
   </build>
 </project>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a93a32e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java
new file mode 100644
index 0000000..e302c70
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java
@@ -0,0 +1,511 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
+import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.FailApplicationAttemptRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.FailApplicationAttemptResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptsRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeLabelsRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeLabelsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerReportRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerReportResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainersRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainersResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetLabelsToNodesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetLabelsToNodesResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.MoveApplicationAcrossQueuesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.MoveApplicationAcrossQueuesResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationListRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationListResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.SignalContainerRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.SignalContainerResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.UpdateApplicationPriorityRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.UpdateApplicationPriorityResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.UpdateApplicationTimeoutsRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.UpdateApplicationTimeoutsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SignalContainerResponsePBImpl;
+import org.apache.hadoop.yarn.api.records.AMCommand;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.hadoop.yarn.api.records.NMToken;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.NodeLabel;
+import org.apache.hadoop.yarn.api.records.NodeReport;
+import org.apache.hadoop.yarn.api.records.ReservationAllocationState;
+import org.apache.hadoop.yarn.api.records.ReservationId;
+import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.api.records.Token;
+import org.apache.hadoop.yarn.api.records.UpdatedContainer;
+import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
+import org.apache.hadoop.yarn.util.Records;
+import org.junit.Assert;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Strings;
+
+/**
+ * Mock Resource Manager facade implementation that exposes all the methods
+ * implemented by the YARN RM. The behavior and the values returned by this mock
+ * implementation is expected by the Router/AMRMProxy unit test cases. So please
+ * change the implementation with care.
+ */
+public class MockResourceManagerFacade
+    implements ApplicationClientProtocol, ApplicationMasterProtocol {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(MockResourceManagerFacade.class);
+
+  private HashMap<String, List<ContainerId>> applicationContainerIdMap =
+      new HashMap<String, List<ContainerId>>();
+  private HashMap<ContainerId, Container> allocatedContainerMap =
+      new HashMap<ContainerId, Container>();
+  private AtomicInteger containerIndex = new AtomicInteger(0);
+  private Configuration conf;
+
+  public MockResourceManagerFacade(Configuration conf,
+      int startContainerIndex) {
+    this.conf = conf;
+    this.containerIndex.set(startContainerIndex);
+  }
+
+  private static String getAppIdentifier() throws IOException {
+    AMRMTokenIdentifier result = null;
+    UserGroupInformation remoteUgi = UserGroupInformation.getCurrentUser();
+    Set<TokenIdentifier> tokenIds = remoteUgi.getTokenIdentifiers();
+    for (TokenIdentifier tokenId : tokenIds) {
+      if (tokenId instanceof AMRMTokenIdentifier) {
+        result = (AMRMTokenIdentifier) tokenId;
+        break;
+      }
+    }
+    return result != null ? result.getApplicationAttemptId().toString() : "";
+  }
+
+  @Override
+  public RegisterApplicationMasterResponse registerApplicationMaster(
+      RegisterApplicationMasterRequest request)
+      throws YarnException, IOException {
+    String amrmToken = getAppIdentifier();
+    LOG.info("Registering application attempt: " + amrmToken);
+
+    synchronized (applicationContainerIdMap) {
+      Assert.assertFalse(
+          "The application id is already registered: " + amrmToken,
+          applicationContainerIdMap.containsKey(amrmToken));
+      // Keep track of the containers that are returned to this application
+      applicationContainerIdMap.put(amrmToken, new ArrayList<ContainerId>());
+    }
+
+    return RegisterApplicationMasterResponse.newInstance(null, null, null, null,
+        null, request.getHost(), null);
+  }
+
+  @Override
+  public FinishApplicationMasterResponse finishApplicationMaster(
+      FinishApplicationMasterRequest request)
+      throws YarnException, IOException {
+    String amrmToken = getAppIdentifier();
+    LOG.info("Finishing application attempt: " + amrmToken);
+
+    synchronized (applicationContainerIdMap) {
+      // Remove the containers that were being tracked for this application
+      Assert.assertTrue("The application id is NOT registered: " + amrmToken,
+          applicationContainerIdMap.containsKey(amrmToken));
+      List<ContainerId> ids = applicationContainerIdMap.remove(amrmToken);
+      for (ContainerId c : ids) {
+        allocatedContainerMap.remove(c);
+      }
+    }
+
+    return FinishApplicationMasterResponse.newInstance(
+        request.getFinalApplicationStatus() == FinalApplicationStatus.SUCCEEDED
+            ? true : false);
+  }
+
+  protected ApplicationId getApplicationId(int id) {
+    return ApplicationId.newInstance(12345, id);
+  }
+
+  protected ApplicationAttemptId getApplicationAttemptId(int id) {
+    return ApplicationAttemptId.newInstance(getApplicationId(id), 1);
+  }
+
+  @SuppressWarnings("deprecation")
+  @Override
+  public AllocateResponse allocate(AllocateRequest request)
+      throws YarnException, IOException {
+    if (request.getAskList() != null && request.getAskList().size() > 0
+        && request.getReleaseList() != null
+        && request.getReleaseList().size() > 0) {
+      Assert.fail("The mock RM implementation does not support receiving "
+          + "askList and releaseList in the same heartbeat");
+    }
+
+    String amrmToken = getAppIdentifier();
+
+    ArrayList<Container> containerList = new ArrayList<Container>();
+    if (request.getAskList() != null) {
+      for (ResourceRequest rr : request.getAskList()) {
+        for (int i = 0; i < rr.getNumContainers(); i++) {
+          ContainerId containerId = ContainerId.newInstance(
+              getApplicationAttemptId(1), containerIndex.incrementAndGet());
+          Container container = Records.newRecord(Container.class);
+          container.setId(containerId);
+          container.setPriority(rr.getPriority());
+
+          // We don't use the node for running containers in the test cases. So
+          // it is OK to hard code it to some dummy value
+          NodeId nodeId =
+              NodeId.newInstance(!Strings.isNullOrEmpty(rr.getResourceName())
+                  ? rr.getResourceName() : "dummy", 1000);
+          container.setNodeId(nodeId);
+          container.setResource(rr.getCapability());
+          containerList.add(container);
+
+          synchronized (applicationContainerIdMap) {
+            // Keep track of the containers returned to this application. We
+            // will need it in future
+            Assert.assertTrue(
+                "The application id is Not registered before allocate(): "
+                    + amrmToken,
+                applicationContainerIdMap.containsKey(amrmToken));
+            List<ContainerId> ids = applicationContainerIdMap.get(amrmToken);
+            ids.add(containerId);
+            this.allocatedContainerMap.put(containerId, container);
+          }
+        }
+      }
+    }
+
+    if (request.getReleaseList() != null
+        && request.getReleaseList().size() > 0) {
+      LOG.info("Releasing containers: " + request.getReleaseList().size());
+      synchronized (applicationContainerIdMap) {
+        Assert
+            .assertTrue(
+                "The application id is not registered before allocate(): "
+                    + amrmToken,
+                applicationContainerIdMap.containsKey(amrmToken));
+        List<ContainerId> ids = applicationContainerIdMap.get(amrmToken);
+
+        for (ContainerId id : request.getReleaseList()) {
+          boolean found = false;
+          for (ContainerId c : ids) {
+            if (c.equals(id)) {
+              found = true;
+              break;
+            }
+          }
+
+          Assert.assertTrue("ContainerId " + id
+              + " being released is not valid for application: "
+              + conf.get("AMRMTOKEN"), found);
+
+          ids.remove(id);
+
+          // Return the released container back to the AM with new fake Ids. The
+          // test case does not care about the IDs. The IDs are faked because
+          // otherwise the LRM will throw duplication identifier exception. This
+          // returning of fake containers is ONLY done for testing purpose - for
+          // the test code to get confirmation that the sub-cluster resource
+          // managers received the release request
+          ContainerId fakeContainerId = ContainerId.newInstance(
+              getApplicationAttemptId(1), containerIndex.incrementAndGet());
+          Container fakeContainer = allocatedContainerMap.get(id);
+          fakeContainer.setId(fakeContainerId);
+          containerList.add(fakeContainer);
+        }
+      }
+    }
+
+    LOG.info("Allocating containers: " + containerList.size()
+        + " for application attempt: " + conf.get("AMRMTOKEN"));
+
+    // Always issue a new AMRMToken as if RM rolled master key
+    Token newAMRMToken = Token.newInstance(new byte[0], "", new byte[0], "");
+
+    return AllocateResponse.newInstance(0, new ArrayList<ContainerStatus>(),
+        containerList, new ArrayList<NodeReport>(), null, AMCommand.AM_RESYNC,
+        1, null, new ArrayList<NMToken>(), newAMRMToken,
+        new ArrayList<UpdatedContainer>());
+  }
+
+  @Override
+  public GetApplicationReportResponse getApplicationReport(
+      GetApplicationReportRequest request) throws YarnException, IOException {
+
+    GetApplicationReportResponse response =
+        Records.newRecord(GetApplicationReportResponse.class);
+    ApplicationReport report = Records.newRecord(ApplicationReport.class);
+    report.setYarnApplicationState(YarnApplicationState.ACCEPTED);
+    report.setApplicationId(request.getApplicationId());
+    report.setCurrentApplicationAttemptId(
+        ApplicationAttemptId.newInstance(request.getApplicationId(), 1));
+    response.setApplicationReport(report);
+    return response;
+  }
+
+  @Override
+  public GetApplicationAttemptReportResponse getApplicationAttemptReport(
+      GetApplicationAttemptReportRequest request)
+      throws YarnException, IOException {
+
+    GetApplicationAttemptReportResponse response =
+        Records.newRecord(GetApplicationAttemptReportResponse.class);
+    ApplicationAttemptReport report =
+        Records.newRecord(ApplicationAttemptReport.class);
+    report.setApplicationAttemptId(request.getApplicationAttemptId());
+    report.setYarnApplicationAttemptState(YarnApplicationAttemptState.LAUNCHED);
+    response.setApplicationAttemptReport(report);
+    return response;
+  }
+
+  @Override
+  public GetNewApplicationResponse getNewApplication(
+      GetNewApplicationRequest request) throws YarnException, IOException {
+    return GetNewApplicationResponse.newInstance(null, null, null);
+  }
+
+  @Override
+  public SubmitApplicationResponse submitApplication(
+      SubmitApplicationRequest request) throws YarnException, IOException {
+    return SubmitApplicationResponse.newInstance();
+  }
+
+  @Override
+  public KillApplicationResponse forceKillApplication(
+      KillApplicationRequest request) throws YarnException, IOException {
+    return KillApplicationResponse.newInstance(true);
+  }
+
+  @Override
+  public GetClusterMetricsResponse getClusterMetrics(
+      GetClusterMetricsRequest request) throws YarnException, IOException {
+    return GetClusterMetricsResponse.newInstance(null);
+  }
+
+  @Override
+  public GetApplicationsResponse getApplications(GetApplicationsRequest request)
+      throws YarnException, IOException {
+    return GetApplicationsResponse.newInstance(null);
+  }
+
+  @Override
+  public GetClusterNodesResponse getClusterNodes(GetClusterNodesRequest request)
+      throws YarnException, IOException {
+    return GetClusterNodesResponse.newInstance(null);
+  }
+
+  @Override
+  public GetQueueInfoResponse getQueueInfo(GetQueueInfoRequest request)
+      throws YarnException, IOException {
+    return GetQueueInfoResponse.newInstance(null);
+  }
+
+  @Override
+  public GetQueueUserAclsInfoResponse getQueueUserAcls(
+      GetQueueUserAclsInfoRequest request) throws YarnException, IOException {
+    return GetQueueUserAclsInfoResponse.newInstance(null);
+  }
+
+  @Override
+  public GetDelegationTokenResponse getDelegationToken(
+      GetDelegationTokenRequest request) throws YarnException, IOException {
+    return GetDelegationTokenResponse.newInstance(null);
+  }
+
+  @Override
+  public RenewDelegationTokenResponse renewDelegationToken(
+      RenewDelegationTokenRequest request) throws YarnException, IOException {
+    return RenewDelegationTokenResponse.newInstance(0);
+  }
+
+  @Override
+  public CancelDelegationTokenResponse cancelDelegationToken(
+      CancelDelegationTokenRequest request) throws YarnException, IOException {
+    return CancelDelegationTokenResponse.newInstance();
+  }
+
+  @Override
+  public MoveApplicationAcrossQueuesResponse moveApplicationAcrossQueues(
+      MoveApplicationAcrossQueuesRequest request)
+      throws YarnException, IOException {
+    return MoveApplicationAcrossQueuesResponse.newInstance();
+  }
+
+  @Override
+  public GetApplicationAttemptsResponse getApplicationAttempts(
+      GetApplicationAttemptsRequest request) throws YarnException, IOException {
+    return GetApplicationAttemptsResponse.newInstance(null);
+  }
+
+  @Override
+  public GetContainerReportResponse getContainerReport(
+      GetContainerReportRequest request) throws YarnException, IOException {
+    return GetContainerReportResponse.newInstance(null);
+  }
+
+  @Override
+  public GetContainersResponse getContainers(GetContainersRequest request)
+      throws YarnException, IOException {
+    return GetContainersResponse.newInstance(null);
+  }
+
+  @Override
+  public ReservationSubmissionResponse submitReservation(
+      ReservationSubmissionRequest request) throws YarnException, IOException {
+    return ReservationSubmissionResponse.newInstance();
+  }
+
+  @Override
+  public ReservationListResponse listReservations(
+      ReservationListRequest request) throws YarnException, IOException {
+    return ReservationListResponse
+        .newInstance(new ArrayList<ReservationAllocationState>());
+  }
+
+  @Override
+  public ReservationUpdateResponse updateReservation(
+      ReservationUpdateRequest request) throws YarnException, IOException {
+    return ReservationUpdateResponse.newInstance();
+  }
+
+  @Override
+  public ReservationDeleteResponse deleteReservation(
+      ReservationDeleteRequest request) throws YarnException, IOException {
+    return ReservationDeleteResponse.newInstance();
+  }
+
+  @Override
+  public GetNodesToLabelsResponse getNodeToLabels(
+      GetNodesToLabelsRequest request) throws YarnException, IOException {
+    return GetNodesToLabelsResponse
+        .newInstance(new HashMap<NodeId, Set<String>>());
+  }
+
+  @Override
+  public GetClusterNodeLabelsResponse getClusterNodeLabels(
+      GetClusterNodeLabelsRequest request) throws YarnException, IOException {
+    return GetClusterNodeLabelsResponse.newInstance(new ArrayList<NodeLabel>());
+  }
+
+  @Override
+  public GetLabelsToNodesResponse getLabelsToNodes(
+      GetLabelsToNodesRequest request) throws YarnException, IOException {
+    return GetLabelsToNodesResponse.newInstance(null);
+  }
+
+  @Override
+  public GetNewReservationResponse getNewReservation(
+      GetNewReservationRequest request) throws YarnException, IOException {
+    return GetNewReservationResponse
+        .newInstance(ReservationId.newInstance(0, 0));
+  }
+
+  @Override
+  public FailApplicationAttemptResponse failApplicationAttempt(
+      FailApplicationAttemptRequest request) throws YarnException, IOException {
+    return FailApplicationAttemptResponse.newInstance();
+  }
+
+  @Override
+  public UpdateApplicationPriorityResponse updateApplicationPriority(
+      UpdateApplicationPriorityRequest request)
+      throws YarnException, IOException {
+    return UpdateApplicationPriorityResponse.newInstance(null);
+  }
+
+  @Override
+  public SignalContainerResponse signalToContainer(
+      SignalContainerRequest request) throws YarnException, IOException {
+    return new SignalContainerResponsePBImpl();
+  }
+
+  @Override
+  public UpdateApplicationTimeoutsResponse updateApplicationTimeouts(
+      UpdateApplicationTimeoutsRequest request)
+      throws YarnException, IOException {
+    return UpdateApplicationTimeoutsResponse.newInstance();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a93a32e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml
index 1445d2a..0d2daea 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml
@@ -168,6 +168,13 @@
       <groupId>org.fusesource.leveldbjni</groupId>
       <artifactId>leveldbjni-all</artifactId>
     </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-server-common</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 
   <profiles>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a93a32e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/MockRequestInterceptor.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/MockRequestInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/MockRequestInterceptor.java
index c962f97..1cbb237 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/MockRequestInterceptor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/MockRequestInterceptor.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterReque
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.MockResourceManagerFacade;
 
 public class MockRequestInterceptor extends AbstractRequestInterceptor {
 
@@ -38,22 +39,21 @@ public class MockRequestInterceptor extends AbstractRequestInterceptor {
 
   public void init(AMRMProxyApplicationContext appContext) {
     super.init(appContext);
-    mockRM =
-        new MockResourceManagerFacade(new YarnConfiguration(
-            super.getConf()), 0);
+    mockRM = new MockResourceManagerFacade(
+        new YarnConfiguration(super.getConf()), 0);
   }
 
   @Override
   public RegisterApplicationMasterResponse registerApplicationMaster(
-      RegisterApplicationMasterRequest request) throws YarnException,
-      IOException {
+      RegisterApplicationMasterRequest request)
+      throws YarnException, IOException {
     return mockRM.registerApplicationMaster(request);
   }
 
   @Override
   public FinishApplicationMasterResponse finishApplicationMaster(
-      FinishApplicationMasterRequest request) throws YarnException,
-      IOException {
+      FinishApplicationMasterRequest request)
+      throws YarnException, IOException {
     return mockRM.finishApplicationMaster(request);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a93a32e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/MockResourceManagerFacade.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/MockResourceManagerFacade.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/MockResourceManagerFacade.java
deleted file mode 100644
index f584c94..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/MockResourceManagerFacade.java
+++ /dev/null
@@ -1,514 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.server.nodemanager.amrmproxy;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicInteger;
-import com.google.common.base.Strings;
-import org.apache.commons.lang.NotImplementedException;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.token.TokenIdentifier;
-import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.FailApplicationAttemptRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.FailApplicationAttemptResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptsRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptsResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeLabelsRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeLabelsResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.GetContainerReportRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.GetContainerReportResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.GetContainersRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.GetContainersResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.GetLabelsToNodesRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.GetLabelsToNodesResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.MoveApplicationAcrossQueuesRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.MoveApplicationAcrossQueuesResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.ReservationListRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.ReservationListResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.SignalContainerRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.SignalContainerResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.UpdateApplicationPriorityRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.UpdateApplicationPriorityResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.UpdateApplicationTimeoutsRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.UpdateApplicationTimeoutsResponse;
-import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
-import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
-import org.apache.hadoop.yarn.api.records.AMCommand;
-import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.ApplicationReport;
-import org.apache.hadoop.yarn.api.records.Container;
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.ContainerStatus;
-import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
-import org.apache.hadoop.yarn.api.records.NMToken;
-import org.apache.hadoop.yarn.api.records.NodeId;
-import org.apache.hadoop.yarn.api.records.NodeReport;
-import org.apache.hadoop.yarn.api.records.ResourceRequest;
-import org.apache.hadoop.yarn.api.records.Token;
-import org.apache.hadoop.yarn.api.records.UpdatedContainer;
-import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState;
-import org.apache.hadoop.yarn.api.records.YarnApplicationState;
-import org.apache.hadoop.yarn.exceptions.YarnException;
-import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
-import org.apache.hadoop.yarn.util.Records;
-import org.junit.Assert;
-import org.eclipse.jetty.util.log.Log;
-
-/**
- * Mock Resource Manager facade implementation that exposes all the methods
- * implemented by the YARN RM. The behavior and the values returned by this mock
- * implementation is expected by the unit test cases. So please change the
- * implementation with care.
- */
-public class MockResourceManagerFacade implements
-    ApplicationMasterProtocol, ApplicationClientProtocol {
-
-  private HashMap<String, List<ContainerId>> applicationContainerIdMap =
-      new HashMap<String, List<ContainerId>>();
-  private HashMap<ContainerId, Container> allocatedContainerMap =
-      new HashMap<ContainerId, Container>();
-  private AtomicInteger containerIndex = new AtomicInteger(0);
-  private Configuration conf;
-
-  public MockResourceManagerFacade(Configuration conf,
-      int startContainerIndex) {
-    this.conf = conf;
-    this.containerIndex.set(startContainerIndex);
-  }
-
-  private static String getAppIdentifier() throws IOException {
-    AMRMTokenIdentifier result = null;
-    UserGroupInformation remoteUgi = UserGroupInformation.getCurrentUser();
-    Set<TokenIdentifier> tokenIds = remoteUgi.getTokenIdentifiers();
-    for (TokenIdentifier tokenId : tokenIds) {
-      if (tokenId instanceof AMRMTokenIdentifier) {
-        result = (AMRMTokenIdentifier) tokenId;
-        break;
-      }
-    }
-    return result != null ? result.getApplicationAttemptId().toString()
-        : "";
-  }
-
-  @Override
-  public RegisterApplicationMasterResponse registerApplicationMaster(
-      RegisterApplicationMasterRequest request) throws YarnException,
-      IOException {
-    String amrmToken = getAppIdentifier();
-    Log.getLog().info("Registering application attempt: " + amrmToken);
-
-    synchronized (applicationContainerIdMap) {
-      Assert.assertFalse("The application id is already registered: "
-          + amrmToken, applicationContainerIdMap.containsKey(amrmToken));
-      // Keep track of the containers that are returned to this application
-      applicationContainerIdMap.put(amrmToken,
-          new ArrayList<ContainerId>());
-    }
-
-    return RegisterApplicationMasterResponse.newInstance(null, null, null,
-        null, null, request.getHost(), null);
-  }
-
-  @Override
-  public FinishApplicationMasterResponse finishApplicationMaster(
-      FinishApplicationMasterRequest request) throws YarnException,
-      IOException {
-    String amrmToken = getAppIdentifier();
-    Log.getLog().info("Finishing application attempt: " + amrmToken);
-
-    synchronized (applicationContainerIdMap) {
-      // Remove the containers that were being tracked for this application
-      Assert.assertTrue("The application id is NOT registered: "
-          + amrmToken, applicationContainerIdMap.containsKey(amrmToken));
-      List<ContainerId> ids = applicationContainerIdMap.remove(amrmToken);
-      for (ContainerId c : ids) {
-        allocatedContainerMap.remove(c);
-      }
-    }
-
-    return FinishApplicationMasterResponse
-        .newInstance(request.getFinalApplicationStatus() == FinalApplicationStatus.SUCCEEDED ? true
-            : false);
-  }
-
-  protected ApplicationId getApplicationId(int id) {
-    return ApplicationId.newInstance(12345, id);
-  }
-
-  protected ApplicationAttemptId getApplicationAttemptId(int id) {
-    return ApplicationAttemptId.newInstance(getApplicationId(id), 1);
-  }
-
-  @SuppressWarnings("deprecation")
-  @Override
-  public AllocateResponse allocate(AllocateRequest request)
-      throws YarnException, IOException {
-    if (request.getAskList() != null && request.getAskList().size() > 0
-        && request.getReleaseList() != null
-        && request.getReleaseList().size() > 0) {
-      Assert.fail("The mock RM implementation does not support receiving "
-          + "askList and releaseList in the same heartbeat");
-    }
-
-    String amrmToken = getAppIdentifier();
-
-    ArrayList<Container> containerList = new ArrayList<Container>();
-    if (request.getAskList() != null) {
-      for (ResourceRequest rr : request.getAskList()) {
-        for (int i = 0; i < rr.getNumContainers(); i++) {
-          ContainerId containerId =
-              ContainerId.newInstance(getApplicationAttemptId(1),
-                  containerIndex.incrementAndGet());
-          Container container = Records.newRecord(Container.class);
-          container.setId(containerId);
-          container.setPriority(rr.getPriority());
-
-          // We don't use the node for running containers in the test cases. So
-          // it is OK to hard code it to some dummy value
-          NodeId nodeId =
-              NodeId.newInstance(
-                  !Strings.isNullOrEmpty(rr.getResourceName()) ? rr
-                      .getResourceName() : "dummy", 1000);
-          container.setNodeId(nodeId);
-          container.setResource(rr.getCapability());
-          containerList.add(container);
-
-          synchronized (applicationContainerIdMap) {
-            // Keep track of the containers returned to this application. We
-            // will need it in future
-            Assert.assertTrue(
-                "The application id is Not registered before allocate(): "
-                    + amrmToken,
-                applicationContainerIdMap.containsKey(amrmToken));
-            List<ContainerId> ids =
-                applicationContainerIdMap.get(amrmToken);
-            ids.add(containerId);
-            this.allocatedContainerMap.put(containerId, container);
-          }
-        }
-      }
-    }
-
-    if (request.getReleaseList() != null
-        && request.getReleaseList().size() > 0) {
-      Log.getLog().info("Releasing containers: "
-          + request.getReleaseList().size());
-      synchronized (applicationContainerIdMap) {
-        Assert.assertTrue(
-            "The application id is not registered before allocate(): "
-                + amrmToken,
-            applicationContainerIdMap.containsKey(amrmToken));
-        List<ContainerId> ids = applicationContainerIdMap.get(amrmToken);
-
-        for (ContainerId id : request.getReleaseList()) {
-          boolean found = false;
-          for (ContainerId c : ids) {
-            if (c.equals(id)) {
-              found = true;
-              break;
-            }
-          }
-
-          Assert.assertTrue(
-              "ContainerId " + id
-                  + " being released is not valid for application: "
-                  + conf.get("AMRMTOKEN"), found);
-
-          ids.remove(id);
-
-          // Return the released container back to the AM with new fake Ids. The
-          // test case does not care about the IDs. The IDs are faked because
-          // otherwise the LRM will throw duplication identifier exception. This
-          // returning of fake containers is ONLY done for testing purpose - for
-          // the test code to get confirmation that the sub-cluster resource
-          // managers received the release request
-          ContainerId fakeContainerId =
-              ContainerId.newInstance(getApplicationAttemptId(1),
-                  containerIndex.incrementAndGet());
-          Container fakeContainer = allocatedContainerMap.get(id);
-          fakeContainer.setId(fakeContainerId);
-          containerList.add(fakeContainer);
-        }
-      }
-    }
-
-    Log.getLog().info("Allocating containers: " + containerList.size()
-        + " for application attempt: " + conf.get("AMRMTOKEN"));
-
-    // Always issue a new AMRMToken as if RM rolled master key
-    Token newAMRMToken = Token.newInstance(new byte[0], "", new byte[0], "");
-
-    return AllocateResponse.newInstance(0,
-        new ArrayList<ContainerStatus>(), containerList,
-        new ArrayList<NodeReport>(), null, AMCommand.AM_RESYNC, 1, null,
-        new ArrayList<NMToken>(), newAMRMToken,
-        new ArrayList<UpdatedContainer>());
-  }
-
-  @Override
-  public GetApplicationReportResponse getApplicationReport(
-      GetApplicationReportRequest request) throws YarnException,
-      IOException {
-
-    GetApplicationReportResponse response =
-        Records.newRecord(GetApplicationReportResponse.class);
-    ApplicationReport report = Records.newRecord(ApplicationReport.class);
-    report.setYarnApplicationState(YarnApplicationState.ACCEPTED);
-    report.setApplicationId(request.getApplicationId());
-    report.setCurrentApplicationAttemptId(ApplicationAttemptId
-        .newInstance(request.getApplicationId(), 1));
-    response.setApplicationReport(report);
-    return response;
-  }
-
-  @Override
-  public GetApplicationAttemptReportResponse getApplicationAttemptReport(
-      GetApplicationAttemptReportRequest request) throws YarnException,
-      IOException {
-    GetApplicationAttemptReportResponse response =
-        Records.newRecord(GetApplicationAttemptReportResponse.class);
-    ApplicationAttemptReport report =
-        Records.newRecord(ApplicationAttemptReport.class);
-    report.setApplicationAttemptId(request.getApplicationAttemptId());
-    report
-        .setYarnApplicationAttemptState(YarnApplicationAttemptState.LAUNCHED);
-    response.setApplicationAttemptReport(report);
-    return response;
-  }
-
-  @Override
-  public GetNewApplicationResponse getNewApplication(
-      GetNewApplicationRequest request) throws YarnException, IOException {
-    return null;
-  }
-
-  @Override
-  public SubmitApplicationResponse submitApplication(
-      SubmitApplicationRequest request) throws YarnException, IOException {
-    return null;
-  }
-
-  @Override
-  public KillApplicationResponse forceKillApplication(
-      KillApplicationRequest request) throws YarnException, IOException {
-    throw new NotImplementedException();
-  }
-
-  @Override
-  public GetClusterMetricsResponse getClusterMetrics(
-      GetClusterMetricsRequest request) throws YarnException, IOException {
-    throw new NotImplementedException();
-  }
-
-  @Override
-  public GetApplicationsResponse getApplications(
-      GetApplicationsRequest request) throws YarnException, IOException {
-    throw new NotImplementedException();
-  }
-
-  @Override
-  public GetClusterNodesResponse getClusterNodes(
-      GetClusterNodesRequest request) throws YarnException, IOException {
-    throw new NotImplementedException();
-  }
-
-  @Override
-  public GetQueueInfoResponse getQueueInfo(GetQueueInfoRequest request)
-      throws YarnException, IOException {
-    throw new NotImplementedException();
-  }
-
-  @Override
-  public GetQueueUserAclsInfoResponse getQueueUserAcls(
-      GetQueueUserAclsInfoRequest request) throws YarnException,
-      IOException {
-    throw new NotImplementedException();
-  }
-
-  @Override
-  public GetDelegationTokenResponse getDelegationToken(
-      GetDelegationTokenRequest request) throws YarnException, IOException {
-    throw new NotImplementedException();
-  }
-
-  @Override
-  public RenewDelegationTokenResponse renewDelegationToken(
-      RenewDelegationTokenRequest request) throws YarnException,
-      IOException {
-    throw new NotImplementedException();
-  }
-
-  @Override
-  public CancelDelegationTokenResponse cancelDelegationToken(
-      CancelDelegationTokenRequest request) throws YarnException,
-      IOException {
-    throw new NotImplementedException();
-  }
-
-  @Override
-  public MoveApplicationAcrossQueuesResponse moveApplicationAcrossQueues(
-      MoveApplicationAcrossQueuesRequest request) throws YarnException,
-      IOException {
-    throw new NotImplementedException();
-  }
-
-  @Override
-  public GetApplicationAttemptsResponse getApplicationAttempts(
-      GetApplicationAttemptsRequest request) throws YarnException,
-      IOException {
-    throw new NotImplementedException();
-  }
-
-  @Override
-  public GetContainerReportResponse getContainerReport(
-      GetContainerReportRequest request) throws YarnException, IOException {
-    throw new NotImplementedException();
-  }
-
-  @Override
-  public GetContainersResponse getContainers(GetContainersRequest request)
-      throws YarnException, IOException {
-    throw new NotImplementedException();
-  }
-
-  @Override
-  public GetNewReservationResponse getNewReservation(
-      GetNewReservationRequest request) throws YarnException, IOException {
-    throw new NotImplementedException();
-  }
-
-  @Override
-  public ReservationSubmissionResponse submitReservation(
-      ReservationSubmissionRequest request) throws YarnException,
-      IOException {
-    throw new NotImplementedException();
-  }
-
-  @Override
-  public ReservationListResponse listReservations(
-          ReservationListRequest request) throws YarnException,
-          IOException {
-      throw new NotImplementedException();
-  }
-
-  @Override
-  public ReservationUpdateResponse updateReservation(
-      ReservationUpdateRequest request) throws YarnException, IOException {
-    throw new NotImplementedException();
-  }
-
-  @Override
-  public ReservationDeleteResponse deleteReservation(
-      ReservationDeleteRequest request) throws YarnException, IOException {
-    throw new NotImplementedException();
-  }
-
-  @Override
-  public GetNodesToLabelsResponse getNodeToLabels(
-      GetNodesToLabelsRequest request) throws YarnException, IOException {
-    throw new NotImplementedException();
-  }
-
-  @Override
-  public GetClusterNodeLabelsResponse getClusterNodeLabels(
-      GetClusterNodeLabelsRequest request) throws YarnException,
-      IOException {
-    throw new NotImplementedException();
-  }
-
-  @Override
-  public GetLabelsToNodesResponse getLabelsToNodes(
-      GetLabelsToNodesRequest request) throws YarnException, IOException {
-    return null;
-  }
-
-  @Override
-  public UpdateApplicationPriorityResponse updateApplicationPriority(
-      UpdateApplicationPriorityRequest request) throws YarnException,
-      IOException {
-    return null;
-  }
-
-  @Override
-  public SignalContainerResponse signalToContainer(
-      SignalContainerRequest request) throws IOException {
-return null;
-}
-
-  @Override
-  public FailApplicationAttemptResponse failApplicationAttempt(
-      FailApplicationAttemptRequest request) throws YarnException, IOException {
-    throw new NotImplementedException();
-  }
-
-  @Override
-  public UpdateApplicationTimeoutsResponse updateApplicationTimeouts(
-      UpdateApplicationTimeoutsRequest request)
-      throws YarnException, IOException {
-    throw new NotImplementedException();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a93a32e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/pom.xml
index 25afa5c..89813de 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/pom.xml
@@ -50,12 +50,31 @@
 
     <dependency>
       <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-yarn-server-common</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
     </dependency>
   </dependencies>
 
   <build>
     <plugins>
+      <plugin>
+        <groupId>org.apache.rat</groupId>
+        <artifactId>apache-rat-plugin</artifactId>
+      </plugin>
     </plugins>
   </build>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a93a32e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/Router.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/Router.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/Router.java
index 7be8a59..7cfabf5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/Router.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/Router.java
@@ -18,6 +18,20 @@
 
 package org.apache.hadoop.yarn.server.router;
 
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.CompositeService;
+import org.apache.hadoop.util.ShutdownHookManager;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.YarnUncaughtExceptionHandler;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.hadoop.yarn.server.router.clientrm.RouterClientRMService;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 /**
  * The router is a stateless YARN component which is the entry point to the
  * cluster. It can be deployed on multiple nodes behind a Virtual IP (VIP) with
@@ -33,6 +47,88 @@ package org.apache.hadoop.yarn.server.router;
  * This provides a placeholder for throttling mis-behaving clients (YARN-1546)
  * and masks the access to multiple RMs (YARN-3659).
  */
-public class Router{
+public class Router extends CompositeService {
+
+  private static final Logger LOG = LoggerFactory.getLogger(Router.class);
+  private static CompositeServiceShutdownHook routerShutdownHook;
+  private Configuration conf;
+  private AtomicBoolean isStopping = new AtomicBoolean(false);
+  private RouterClientRMService clientRMProxyService;
+
+  /**
+   * Priority of the Router shutdown hook.
+   */
+  public static final int SHUTDOWN_HOOK_PRIORITY = 30;
+
+  public Router() {
+    super(Router.class.getName());
+  }
+
+  protected void doSecureLogin() throws IOException {
+    // TODO YARN-6539 Create SecureLogin inside Router
+  }
+
+  @Override
+  protected void serviceInit(Configuration config) throws Exception {
+    this.conf = config;
+    clientRMProxyService = createClientRMProxyService();
+    addService(clientRMProxyService);
+    super.serviceInit(conf);
+  }
+
+  @Override
+  protected void serviceStart() throws Exception {
+    try {
+      doSecureLogin();
+    } catch (IOException e) {
+      throw new YarnRuntimeException("Failed Router login", e);
+    }
+    super.serviceStart();
+  }
+
+  @Override
+  protected void serviceStop() throws Exception {
+    if (isStopping.getAndSet(true)) {
+      return;
+    }
+    super.serviceStop();
+  }
+
+  protected void shutDown() {
+    new Thread() {
+      @Override
+      public void run() {
+        Router.this.stop();
+      }
+    }.start();
+  }
+
+  protected RouterClientRMService createClientRMProxyService() {
+    return new RouterClientRMService();
+  }
+
+  public static void main(String[] argv) {
+    Configuration conf = new YarnConfiguration();
+    Thread
+        .setDefaultUncaughtExceptionHandler(new YarnUncaughtExceptionHandler());
+    StringUtils.startupShutdownMessage(Router.class, argv, LOG);
+    Router router = new Router();
+    try {
+
+      // Remove the old hook if we are rebooting.
+      if (null != routerShutdownHook) {
+        ShutdownHookManager.get().removeShutdownHook(routerShutdownHook);
+      }
+
+      routerShutdownHook = new CompositeServiceShutdownHook(router);
+      ShutdownHookManager.get().addShutdownHook(routerShutdownHook,
+          SHUTDOWN_HOOK_PRIORITY);
 
+      router.init(conf);
+      router.start();
+    } catch (Throwable t) {
+      LOG.error("Error starting Router", t);
+      System.exit(-1);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a93a32e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/AbstractClientRequestInterceptor.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/AbstractClientRequestInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/AbstractClientRequestInterceptor.java
new file mode 100644
index 0000000..fc6a118
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/AbstractClientRequestInterceptor.java
@@ -0,0 +1,89 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.router.clientrm;
+
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Implements the RequestInterceptor interface and provides common functionality
+ * which can can be used and/or extended by other concrete intercepter classes.
+ *
+ */
+public abstract class AbstractClientRequestInterceptor
+    implements ClientRequestInterceptor {
+  private Configuration conf;
+  private ClientRequestInterceptor nextInterceptor;
+
+  /**
+   * Sets the {@code RequestInterceptor} in the chain.
+   */
+  @Override
+  public void setNextInterceptor(ClientRequestInterceptor nextInterceptor) {
+    this.nextInterceptor = nextInterceptor;
+  }
+
+  /**
+   * Sets the {@link Configuration}.
+   */
+
+  @Override
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+    if (this.nextInterceptor != null) {
+      this.nextInterceptor.setConf(conf);
+    }
+  }
+
+  /**
+   * Gets the {@link Configuration}.
+   */
+  @Override
+  public Configuration getConf() {
+    return this.conf;
+  }
+
+  /**
+   * Initializes the {@code ClientRequestInterceptor}.
+   */
+  @Override
+  public void init(String user) {
+    if (this.nextInterceptor != null) {
+      this.nextInterceptor.init(user);
+    }
+  }
+
+  /**
+   * Disposes the {@code ClientRequestInterceptor}.
+   */
+  @Override
+  public void shutdown() {
+    if (this.nextInterceptor != null) {
+      this.nextInterceptor.shutdown();
+    }
+  }
+
+  /**
+   * Gets the next {@link ClientRequestInterceptor} in the chain.
+   */
+  @Override
+  public ClientRequestInterceptor getNextInterceptor() {
+    return this.nextInterceptor;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a93a32e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/ClientRequestInterceptor.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/ClientRequestInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/ClientRequestInterceptor.java
new file mode 100644
index 0000000..2f8fb93
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/ClientRequestInterceptor.java
@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.router.clientrm;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
+
+/**
+ * Defines the contract to be implemented by the request intercepter classes,
+ * that can be used to intercept and inspect messages sent from the client to
+ * the resource manager.
+ */
+public interface ClientRequestInterceptor
+    extends ApplicationClientProtocol, Configurable {
+  /**
+   * This method is called for initializing the intercepter. This is guaranteed
+   * to be called only once in the lifetime of this instance.
+   *
+   * @param user the name of the client
+   */
+  void init(String user);
+
+  /**
+   * This method is called to release the resources held by the intercepter.
+   * This will be called when the application pipeline is being destroyed. The
+   * concrete implementations should dispose the resources and forward the
+   * request to the next intercepter, if any.
+   */
+  void shutdown();
+
+  /**
+   * Sets the next intercepter in the pipeline. The concrete implementation of
+   * this interface should always pass the request to the nextInterceptor after
+   * inspecting the message. The last intercepter in the chain is responsible to
+   * send the messages to the resource manager service and so the last
+   * intercepter will not receive this method call.
+   *
+   * @param nextInterceptor the ClientRequestInterceptor to set in the pipeline
+   */
+  void setNextInterceptor(ClientRequestInterceptor nextInterceptor);
+
+  /**
+   * Returns the next intercepter in the chain.
+   *
+   * @return the next intercepter in the chain
+   */
+  ClientRequestInterceptor getNextInterceptor();
+
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[41/42] hadoop git commit: YARN-3663. Federation State and Policy Store (DBMS implementation). (Giovanni Matteo Fumarola via curino).

Posted by su...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/5599b916/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
----------------------------------------------------------------------
diff --git a/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 b/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
index 80b00ef..db04592 100644
--- a/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
+++ b/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
@@ -19,13 +19,14 @@ package org.apache.hadoop.yarn.server.federation.store.impl;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.util.Calendar;
 import java.util.List;
+import java.util.TimeZone;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.federation.store.FederationStateStore;
-import org.apache.hadoop.yarn.server.federation.store.exception.FederationStateStoreErrorCode;
 import org.apache.hadoop.yarn.server.federation.store.exception.FederationStateStoreException;
 import org.apache.hadoop.yarn.server.federation.store.records.AddApplicationHomeSubClusterRequest;
 import org.apache.hadoop.yarn.server.federation.store.records.AddApplicationHomeSubClusterResponse;
@@ -87,13 +88,26 @@ public abstract class FederationStateStoreBaseTest {
   @Test
   public void testRegisterSubCluster() throws Exception {
     SubClusterId subClusterId = SubClusterId.newInstance("SC");
+
     SubClusterInfo subClusterInfo = createSubClusterInfo(subClusterId);
 
+    long previousTimeStamp =
+        Calendar.getInstance(TimeZone.getTimeZone("UTC")).getTimeInMillis();
+
     SubClusterRegisterResponse result = stateStore.registerSubCluster(
         SubClusterRegisterRequest.newInstance(subClusterInfo));
 
+    long currentTimeStamp =
+        Calendar.getInstance(TimeZone.getTimeZone("UTC")).getTimeInMillis();
+
     Assert.assertNotNull(result);
     Assert.assertEquals(subClusterInfo, querySubClusterInfo(subClusterId));
+
+    // The saved heartbeat is between the old one and the current timestamp
+    Assert.assertTrue(querySubClusterInfo(subClusterId)
+        .getLastHeartBeat() <= currentTimeStamp);
+    Assert.assertTrue(querySubClusterInfo(subClusterId)
+        .getLastHeartBeat() >= previousTimeStamp);
   }
 
   @Test
@@ -120,9 +134,7 @@ public abstract class FederationStateStoreBaseTest {
       stateStore.deregisterSubCluster(deregisterRequest);
       Assert.fail();
     } catch (FederationStateStoreException e) {
-      Assert.assertEquals(
-          FederationStateStoreErrorCode.MEMBERSHIP_UPDATE_DEREGISTER_FAIL,
-          e.getCode());
+      Assert.assertTrue(e.getMessage().startsWith("SubCluster SC not found"));
     }
   }
 
@@ -149,9 +161,8 @@ public abstract class FederationStateStoreBaseTest {
       stateStore.getSubCluster(request).getSubClusterInfo();
       Assert.fail();
     } catch (FederationStateStoreException e) {
-      Assert.assertEquals(
-          FederationStateStoreErrorCode.MEMBERSHIP_SINGLE_SELECT_FAIL,
-          e.getCode());
+      Assert.assertTrue(
+          e.getMessage().startsWith("SubCluster SC does not exist"));
     }
   }
 
@@ -200,13 +211,24 @@ public abstract class FederationStateStoreBaseTest {
     SubClusterId subClusterId = SubClusterId.newInstance("SC");
     registerSubCluster(createSubClusterInfo(subClusterId));
 
+    long previousHeartBeat =
+        querySubClusterInfo(subClusterId).getLastHeartBeat();
+
     SubClusterHeartbeatRequest heartbeatRequest = SubClusterHeartbeatRequest
         .newInstance(subClusterId, SubClusterState.SC_RUNNING, "capability");
     stateStore.subClusterHeartbeat(heartbeatRequest);
 
+    long currentTimeStamp =
+        Calendar.getInstance(TimeZone.getTimeZone("UTC")).getTimeInMillis();
+
     Assert.assertEquals(SubClusterState.SC_RUNNING,
         querySubClusterInfo(subClusterId).getState());
-    Assert.assertNotNull(querySubClusterInfo(subClusterId).getLastHeartBeat());
+
+    // The saved heartbeat is between the old one and the current timestamp
+    Assert.assertTrue(querySubClusterInfo(subClusterId)
+        .getLastHeartBeat() <= currentTimeStamp);
+    Assert.assertTrue(querySubClusterInfo(subClusterId)
+        .getLastHeartBeat() >= previousHeartBeat);
   }
 
   @Test
@@ -219,9 +241,8 @@ public abstract class FederationStateStoreBaseTest {
       stateStore.subClusterHeartbeat(heartbeatRequest);
       Assert.fail();
     } catch (FederationStateStoreException e) {
-      Assert.assertEquals(
-          FederationStateStoreErrorCode.MEMBERSHIP_UPDATE_HEARTBEAT_FAIL,
-          e.getCode());
+      Assert.assertTrue(e.getMessage()
+          .startsWith("SubCluster SC does not exist; cannot heartbeat"));
     }
   }
 
@@ -281,9 +302,8 @@ public abstract class FederationStateStoreBaseTest {
       queryApplicationHomeSC(appId);
       Assert.fail();
     } catch (FederationStateStoreException e) {
-      Assert.assertEquals(
-          FederationStateStoreErrorCode.APPLICATIONS_SINGLE_SELECT_FAIL,
-          e.getCode());
+      Assert.assertTrue(e.getMessage()
+          .startsWith("Application " + appId + " does not exist"));
     }
 
   }
@@ -298,8 +318,8 @@ public abstract class FederationStateStoreBaseTest {
       stateStore.deleteApplicationHomeSubCluster(delRequest);
       Assert.fail();
     } catch (FederationStateStoreException e) {
-      Assert.assertEquals(
-          FederationStateStoreErrorCode.APPLICATIONS_DELETE_FAIL, e.getCode());
+      Assert.assertTrue(e.getMessage()
+          .startsWith("Application " + appId.toString() + " does not exist"));
     }
   }
 
@@ -331,9 +351,8 @@ public abstract class FederationStateStoreBaseTest {
       stateStore.getApplicationHomeSubCluster(request);
       Assert.fail();
     } catch (FederationStateStoreException e) {
-      Assert.assertEquals(
-          FederationStateStoreErrorCode.APPLICATIONS_SINGLE_SELECT_FAIL,
-          e.getCode());
+      Assert.assertTrue(e.getMessage()
+          .startsWith("Application " + appId.toString() + " does not exist"));
     }
   }
 
@@ -397,8 +416,8 @@ public abstract class FederationStateStoreBaseTest {
       stateStore.updateApplicationHomeSubCluster((updateRequest));
       Assert.fail();
     } catch (FederationStateStoreException e) {
-      Assert.assertEquals(
-          FederationStateStoreErrorCode.APPLICATIONS_UPDATE_FAIL, e.getCode());
+      Assert.assertTrue(e.getMessage()
+          .startsWith("Application " + appId.toString() + " does not exist"));
     }
   }
 
@@ -458,8 +477,8 @@ public abstract class FederationStateStoreBaseTest {
       stateStore.getPolicyConfiguration(request);
       Assert.fail();
     } catch (FederationStateStoreException e) {
-      Assert.assertEquals(
-          FederationStateStoreErrorCode.POLICY_SINGLE_SELECT_FAIL, e.getCode());
+      Assert.assertTrue(
+          e.getMessage().startsWith("Policy for queue Queue does not exist"));
     }
   }
 
@@ -499,8 +518,9 @@ public abstract class FederationStateStoreBaseTest {
 
   private SubClusterPolicyConfiguration createSCPolicyConf(String queueName,
       String policyType) {
-    return SubClusterPolicyConfiguration.newInstance(queueName, policyType,
-        ByteBuffer.allocate(1));
+    ByteBuffer bb = ByteBuffer.allocate(100);
+    bb.put((byte) 0x02);
+    return SubClusterPolicyConfiguration.newInstance(queueName, policyType, bb);
   }
 
   private void addApplicationHomeSC(ApplicationId appId,
@@ -558,4 +578,8 @@ public abstract class FederationStateStoreBaseTest {
     this.conf = conf;
   }
 
+  protected Configuration getConf() {
+    return conf;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5599b916/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
----------------------------------------------------------------------
diff --git a/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 b/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
new file mode 100644
index 0000000..289a3a6
--- /dev/null
+++ b/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
@@ -0,0 +1,252 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.impl;
+
+import java.sql.Connection;
+import java.sql.SQLException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.federation.store.FederationStateStore;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * HSQLDB implementation of {@link FederationStateStore}.
+ */
+public class HSQLDBFederationStateStore extends SQLFederationStateStore {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(HSQLDBFederationStateStore.class);
+
+  private Connection conn;
+
+  private static final String TABLE_APPLICATIONSHOMESUBCLUSTER =
+      " CREATE TABLE applicationsHomeSubCluster ("
+          + " applicationId varchar(64) NOT NULL,"
+          + " homeSubCluster varchar(256) NOT NULL,"
+          + " CONSTRAINT pk_applicationId PRIMARY KEY (applicationId))";
+
+  private static final String TABLE_MEMBERSHIP =
+      "CREATE TABLE membership ( subClusterId varchar(256) NOT NULL,"
+          + " amRMServiceAddress varchar(256) NOT NULL,"
+          + " clientRMServiceAddress varchar(256) NOT NULL,"
+          + " rmAdminServiceAddress varchar(256) NOT NULL,"
+          + " rmWebServiceAddress varchar(256) NOT NULL,"
+          + " lastHeartBeat datetime NOT NULL, state varchar(32) NOT NULL,"
+          + " lastStartTime bigint NULL, capability varchar(6000) NOT NULL,"
+          + " CONSTRAINT pk_subClusterId PRIMARY KEY (subClusterId))";
+
+  private static final String TABLE_POLICIES =
+      "CREATE TABLE policies ( queue varchar(256) NOT NULL,"
+          + " policyType varchar(256) NOT NULL, params varbinary(512),"
+          + " CONSTRAINT pk_queue PRIMARY KEY (queue))";
+
+  private static final String SP_REGISTERSUBCLUSTER =
+      "CREATE PROCEDURE sp_registerSubCluster("
+          + " IN subClusterId_IN varchar(256),"
+          + " IN amRMServiceAddress_IN varchar(256),"
+          + " IN clientRMServiceAddress_IN varchar(256),"
+          + " IN rmAdminServiceAddress_IN varchar(256),"
+          + " IN rmWebServiceAddress_IN varchar(256),"
+          + " IN state_IN varchar(256),"
+          + " IN lastStartTime_IN bigint, IN capability_IN varchar(6000),"
+          + " OUT rowCount_OUT int)MODIFIES SQL DATA BEGIN ATOMIC"
+          + " DELETE FROM membership WHERE (subClusterId = subClusterId_IN);"
+          + " INSERT INTO membership ( subClusterId,"
+          + " amRMServiceAddress, clientRMServiceAddress,"
+          + " rmAdminServiceAddress, rmWebServiceAddress,"
+          + " lastHeartBeat, state, lastStartTime,"
+          + " capability) VALUES ( subClusterId_IN,"
+          + " amRMServiceAddress_IN, clientRMServiceAddress_IN,"
+          + " rmAdminServiceAddress_IN, rmWebServiceAddress_IN,"
+          + " NOW() AT TIME ZONE INTERVAL '0:00' HOUR TO MINUTE,"
+          + " state_IN, lastStartTime_IN, capability_IN);"
+          + " GET DIAGNOSTICS rowCount_OUT = ROW_COUNT; END";
+
+  private static final String SP_DEREGISTERSUBCLUSTER =
+      "CREATE PROCEDURE sp_deregisterSubCluster("
+          + " IN subClusterId_IN varchar(256),"
+          + " IN state_IN varchar(64), OUT rowCount_OUT int)"
+          + " MODIFIES SQL DATA BEGIN ATOMIC"
+          + " UPDATE membership SET state = state_IN WHERE ("
+          + " subClusterId = subClusterId_IN AND state != state_IN);"
+          + " GET DIAGNOSTICS rowCount_OUT = ROW_COUNT; END";
+
+  private static final String SP_SUBCLUSTERHEARTBEAT =
+      "CREATE PROCEDURE sp_subClusterHeartbeat("
+          + " IN subClusterId_IN varchar(256), IN state_IN varchar(64),"
+          + " IN capability_IN varchar(6000), OUT rowCount_OUT int)"
+          + " MODIFIES SQL DATA BEGIN ATOMIC UPDATE membership"
+          + " SET capability = capability_IN, state = state_IN,"
+          + " lastHeartBeat = NOW() AT TIME ZONE INTERVAL '0:00'"
+          + " HOUR TO MINUTE WHERE subClusterId = subClusterId_IN;"
+          + " GET DIAGNOSTICS rowCount_OUT = ROW_COUNT; END";
+
+  private static final String SP_GETSUBCLUSTER =
+      "CREATE PROCEDURE sp_getSubCluster( IN subClusterId_IN varchar(256),"
+          + " OUT amRMServiceAddress_OUT varchar(256),"
+          + " OUT clientRMServiceAddress_OUT varchar(256),"
+          + " OUT rmAdminServiceAddress_OUT varchar(256),"
+          + " OUT rmWebServiceAddress_OUT varchar(256),"
+          + " OUT lastHeartBeat_OUT datetime, OUT state_OUT varchar(64),"
+          + " OUT lastStartTime_OUT bigint,"
+          + " OUT capability_OUT varchar(6000))"
+          + " MODIFIES SQL DATA BEGIN ATOMIC SELECT amRMServiceAddress,"
+          + " clientRMServiceAddress,"
+          + " rmAdminServiceAddress, rmWebServiceAddress,"
+          + " lastHeartBeat, state, lastStartTime, capability"
+          + " INTO amRMServiceAddress_OUT, clientRMServiceAddress_OUT,"
+          + " rmAdminServiceAddress_OUT,"
+          + " rmWebServiceAddress_OUT, lastHeartBeat_OUT,"
+          + " state_OUT, lastStartTime_OUT, capability_OUT"
+          + " FROM membership WHERE subClusterId = subClusterId_IN; END";
+
+  private static final String SP_GETSUBCLUSTERS =
+      "CREATE PROCEDURE sp_getSubClusters()"
+          + " MODIFIES SQL DATA DYNAMIC RESULT SETS 1 BEGIN ATOMIC"
+          + " DECLARE result CURSOR FOR"
+          + " SELECT subClusterId, amRMServiceAddress, clientRMServiceAddress,"
+          + " rmAdminServiceAddress, rmWebServiceAddress, lastHeartBeat,"
+          + " state, lastStartTime, capability"
+          + " FROM membership; OPEN result; END";
+
+  private static final String SP_ADDAPPLICATIONHOMESUBCLUSTER =
+      "CREATE PROCEDURE sp_addApplicationHomeSubCluster("
+          + " IN applicationId_IN varchar(64),"
+          + " IN homeSubCluster_IN varchar(256),"
+          + " OUT storedHomeSubCluster_OUT varchar(256), OUT rowCount_OUT int)"
+          + " MODIFIES SQL DATA BEGIN ATOMIC"
+          + " INSERT INTO applicationsHomeSubCluster "
+          + " (applicationId,homeSubCluster) "
+          + " (SELECT applicationId_IN, homeSubCluster_IN"
+          + " FROM applicationsHomeSubCluster"
+          + " WHERE applicationId = applicationId_IN"
+          + " HAVING COUNT(*) = 0 );"
+          + " GET DIAGNOSTICS rowCount_OUT = ROW_COUNT;"
+          + " SELECT homeSubCluster INTO storedHomeSubCluster_OUT"
+          + " FROM applicationsHomeSubCluster"
+          + " WHERE applicationId = applicationID_IN; END";
+
+  private static final String SP_UPDATEAPPLICATIONHOMESUBCLUSTER =
+      "CREATE PROCEDURE sp_updateApplicationHomeSubCluster("
+          + " IN applicationId_IN varchar(64),"
+          + " IN homeSubCluster_IN varchar(256), OUT rowCount_OUT int)"
+          + " MODIFIES SQL DATA BEGIN ATOMIC"
+          + " UPDATE applicationsHomeSubCluster"
+          + " SET homeSubCluster = homeSubCluster_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))"
+          + " MODIFIES SQL DATA BEGIN ATOMIC"
+          + " SELECT homeSubCluster INTO homeSubCluster_OUT"
+          + " FROM applicationsHomeSubCluster"
+          + " WHERE applicationId = applicationID_IN; END";
+
+  private static final String SP_GETAPPLICATIONSHOMESUBCLUSTER =
+      "CREATE PROCEDURE sp_getApplicationsHomeSubCluster()"
+          + " MODIFIES SQL DATA DYNAMIC RESULT SETS 1 BEGIN ATOMIC"
+          + " DECLARE result CURSOR FOR"
+          + " SELECT applicationId, homeSubCluster"
+          + " FROM applicationsHomeSubCluster; OPEN result; END";
+
+  private static final String SP_DELETEAPPLICATIONHOMESUBCLUSTER =
+      "CREATE PROCEDURE sp_deleteApplicationHomeSubCluster("
+          + " IN applicationId_IN varchar(64), OUT rowCount_OUT int)"
+          + " MODIFIES SQL DATA BEGIN ATOMIC"
+          + " DELETE FROM applicationsHomeSubCluster"
+          + " WHERE applicationId = applicationId_IN;"
+          + " GET DIAGNOSTICS rowCount_OUT = ROW_COUNT; END";
+
+  private static final String SP_SETPOLICYCONFIGURATION =
+      "CREATE PROCEDURE sp_setPolicyConfiguration("
+          + " IN queue_IN varchar(256), IN policyType_IN varchar(256),"
+          + " IN params_IN varbinary(512), OUT rowCount_OUT int)"
+          + " MODIFIES SQL DATA BEGIN ATOMIC"
+          + " DELETE FROM policies WHERE queue = queue_IN;"
+          + " INSERT INTO policies (queue, policyType, params)"
+          + " VALUES (queue_IN, policyType_IN, params_IN);"
+          + " GET DIAGNOSTICS rowCount_OUT = ROW_COUNT; END";
+
+  private static final String SP_GETPOLICYCONFIGURATION =
+      "CREATE PROCEDURE sp_getPolicyConfiguration("
+          + " IN queue_IN varchar(256), OUT policyType_OUT varchar(256),"
+          + " OUT params_OUT varbinary(512)) MODIFIES SQL DATA BEGIN ATOMIC"
+          + " SELECT policyType, params INTO policyType_OUT, params_OUT"
+          + " FROM policies WHERE queue = queue_IN; END";
+
+  private static final String SP_GETPOLICIESCONFIGURATIONS =
+      "CREATE PROCEDURE sp_getPoliciesConfigurations()"
+          + " MODIFIES SQL DATA DYNAMIC RESULT SETS 1 BEGIN ATOMIC"
+          + " DECLARE result CURSOR FOR"
+          + " SELECT * FROM policies; OPEN result; END";
+
+  @Override
+  public void init(Configuration conf) {
+    try {
+      super.init(conf);
+    } catch (YarnException e1) {
+      LOG.error("ERROR: failed to init HSQLDB " + e1.getMessage());
+    }
+    try {
+      conn = getConnection();
+
+      LOG.info("Database Init: Start");
+
+      conn.prepareStatement(TABLE_APPLICATIONSHOMESUBCLUSTER).execute();
+      conn.prepareStatement(TABLE_MEMBERSHIP).execute();
+      conn.prepareStatement(TABLE_POLICIES).execute();
+
+      conn.prepareStatement(SP_REGISTERSUBCLUSTER).execute();
+      conn.prepareStatement(SP_DEREGISTERSUBCLUSTER).execute();
+      conn.prepareStatement(SP_SUBCLUSTERHEARTBEAT).execute();
+      conn.prepareStatement(SP_GETSUBCLUSTER).execute();
+      conn.prepareStatement(SP_GETSUBCLUSTERS).execute();
+
+      conn.prepareStatement(SP_ADDAPPLICATIONHOMESUBCLUSTER).execute();
+      conn.prepareStatement(SP_UPDATEAPPLICATIONHOMESUBCLUSTER).execute();
+      conn.prepareStatement(SP_GETAPPLICATIONHOMESUBCLUSTER).execute();
+      conn.prepareStatement(SP_GETAPPLICATIONSHOMESUBCLUSTER).execute();
+      conn.prepareStatement(SP_DELETEAPPLICATIONHOMESUBCLUSTER).execute();
+
+      conn.prepareStatement(SP_SETPOLICYCONFIGURATION).execute();
+      conn.prepareStatement(SP_GETPOLICYCONFIGURATION).execute();
+      conn.prepareStatement(SP_GETPOLICIESCONFIGURATIONS).execute();
+
+      LOG.info("Database Init: Complete");
+      conn.close();
+    } catch (SQLException e) {
+      LOG.error("ERROR: failed to inizialize HSQLDB " + e.getMessage());
+    }
+  }
+
+  public void closeConnection() {
+    try {
+      conn.close();
+    } catch (SQLException e) {
+      LOG.error(
+          "ERROR: failed to close connection to HSQLDB DB " + e.getMessage());
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5599b916/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
----------------------------------------------------------------------
diff --git a/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 b/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
index 64adab8..c29fc03 100644
--- a/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
+++ b/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
@@ -28,7 +28,8 @@ public class TestMemoryFederationStateStore
 
   @Override
   protected FederationStateStore createStateStore() {
-    super.setConf(new Configuration());
+    Configuration conf = new Configuration();
+    super.setConf(conf);
     return new MemoryFederationStateStore();
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5599b916/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/TestSQLFederationStateStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/TestSQLFederationStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/TestSQLFederationStateStore.java
new file mode 100644
index 0000000..d4e6cc5
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/TestSQLFederationStateStore.java
@@ -0,0 +1,49 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.impl;
+
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.federation.store.FederationStateStore;
+
+/**
+ * Unit tests for SQLFederationStateStore.
+ */
+public class TestSQLFederationStateStore extends FederationStateStoreBaseTest {
+
+  private static final String HSQLDB_DRIVER = "org.hsqldb.jdbc.JDBCDataSource";
+  private static final String DATABASE_URL = "jdbc:hsqldb:mem:state";
+  private static final String DATABASE_USERNAME = "SA";
+  private static final String DATABASE_PASSWORD = "";
+
+  @Override
+  protected FederationStateStore createStateStore() {
+
+    YarnConfiguration conf = new YarnConfiguration();
+
+    conf.set(YarnConfiguration.FEDERATION_STATESTORE_SQL_JDBC_CLASS,
+        HSQLDB_DRIVER);
+    conf.set(YarnConfiguration.FEDERATION_STATESTORE_SQL_USERNAME,
+        DATABASE_USERNAME);
+    conf.set(YarnConfiguration.FEDERATION_STATESTORE_SQL_PASSWORD,
+        DATABASE_PASSWORD);
+    conf.set(YarnConfiguration.FEDERATION_STATESTORE_SQL_URL,
+        DATABASE_URL + System.currentTimeMillis());
+    super.setConf(conf);
+    return new HSQLDBFederationStateStore();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5599b916/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/utils/TestFederationStateStoreInputValidator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/utils/TestFederationStateStoreInputValidator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/utils/TestFederationStateStoreInputValidator.java
index 8ac5e81..5a5703e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/utils/TestFederationStateStoreInputValidator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/utils/TestFederationStateStoreInputValidator.java
@@ -145,7 +145,7 @@ public class TestFederationStateStoreInputValidator {
       SubClusterRegisterRequest request =
           SubClusterRegisterRequest.newInstance(subClusterInfo);
       FederationMembershipStateStoreInputValidator
-          .validateSubClusterRegisterRequest(request);
+          .validate(request);
     } catch (FederationStateStoreInvalidInputException e) {
       Assert.fail(e.getMessage());
     }
@@ -155,7 +155,7 @@ public class TestFederationStateStoreInputValidator {
     try {
       SubClusterRegisterRequest request = null;
       FederationMembershipStateStoreInputValidator
-          .validateSubClusterRegisterRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       LOG.info(e.getMessage());
@@ -170,7 +170,7 @@ public class TestFederationStateStoreInputValidator {
       SubClusterRegisterRequest request =
           SubClusterRegisterRequest.newInstance(subClusterInfo);
       FederationMembershipStateStoreInputValidator
-          .validateSubClusterRegisterRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       LOG.info(e.getMessage());
@@ -188,7 +188,7 @@ public class TestFederationStateStoreInputValidator {
       SubClusterRegisterRequest request =
           SubClusterRegisterRequest.newInstance(subClusterInfo);
       FederationMembershipStateStoreInputValidator
-          .validateSubClusterRegisterRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       LOG.info(e.getMessage());
@@ -206,7 +206,7 @@ public class TestFederationStateStoreInputValidator {
       SubClusterRegisterRequest request =
           SubClusterRegisterRequest.newInstance(subClusterInfo);
       FederationMembershipStateStoreInputValidator
-          .validateSubClusterRegisterRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       LOG.info(e.getMessage());
@@ -224,7 +224,7 @@ public class TestFederationStateStoreInputValidator {
       SubClusterRegisterRequest request =
           SubClusterRegisterRequest.newInstance(subClusterInfo);
       FederationMembershipStateStoreInputValidator
-          .validateSubClusterRegisterRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       LOG.info(e.getMessage());
@@ -242,7 +242,7 @@ public class TestFederationStateStoreInputValidator {
       SubClusterRegisterRequest request =
           SubClusterRegisterRequest.newInstance(subClusterInfo);
       FederationMembershipStateStoreInputValidator
-          .validateSubClusterRegisterRequest(request);
+          .validate(request);
     } catch (FederationStateStoreInvalidInputException e) {
       Assert.fail(e.getMessage());
     }
@@ -257,7 +257,7 @@ public class TestFederationStateStoreInputValidator {
       SubClusterRegisterRequest request =
           SubClusterRegisterRequest.newInstance(subClusterInfo);
       FederationMembershipStateStoreInputValidator
-          .validateSubClusterRegisterRequest(request);
+          .validate(request);
     } catch (FederationStateStoreInvalidInputException e) {
       Assert.fail(e.getMessage());
     }
@@ -276,7 +276,7 @@ public class TestFederationStateStoreInputValidator {
       SubClusterRegisterRequest request =
           SubClusterRegisterRequest.newInstance(subClusterInfo);
       FederationMembershipStateStoreInputValidator
-          .validateSubClusterRegisterRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       LOG.info(e.getMessage());
@@ -294,7 +294,7 @@ public class TestFederationStateStoreInputValidator {
       SubClusterRegisterRequest request =
           SubClusterRegisterRequest.newInstance(subClusterInfo);
       FederationMembershipStateStoreInputValidator
-          .validateSubClusterRegisterRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       LOG.info(e.getMessage());
@@ -315,7 +315,7 @@ public class TestFederationStateStoreInputValidator {
       SubClusterRegisterRequest request =
           SubClusterRegisterRequest.newInstance(subClusterInfo);
       FederationMembershipStateStoreInputValidator
-          .validateSubClusterRegisterRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       LOG.info(e.getMessage());
@@ -332,7 +332,7 @@ public class TestFederationStateStoreInputValidator {
       SubClusterRegisterRequest request =
           SubClusterRegisterRequest.newInstance(subClusterInfo);
       FederationMembershipStateStoreInputValidator
-          .validateSubClusterRegisterRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       LOG.info(e.getMessage());
@@ -350,7 +350,7 @@ public class TestFederationStateStoreInputValidator {
       SubClusterRegisterRequest request =
           SubClusterRegisterRequest.newInstance(subClusterInfo);
       FederationMembershipStateStoreInputValidator
-          .validateSubClusterRegisterRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       LOG.info(e.getMessage());
@@ -368,7 +368,7 @@ public class TestFederationStateStoreInputValidator {
       SubClusterRegisterRequest request =
           SubClusterRegisterRequest.newInstance(subClusterInfo);
       FederationMembershipStateStoreInputValidator
-          .validateSubClusterRegisterRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       LOG.info(e.getMessage());
@@ -386,7 +386,7 @@ public class TestFederationStateStoreInputValidator {
       SubClusterRegisterRequest request =
           SubClusterRegisterRequest.newInstance(subClusterInfo);
       FederationMembershipStateStoreInputValidator
-          .validateSubClusterRegisterRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       LOG.info(e.getMessage());
@@ -404,7 +404,7 @@ public class TestFederationStateStoreInputValidator {
       SubClusterRegisterRequest request =
           SubClusterRegisterRequest.newInstance(subClusterInfo);
       FederationMembershipStateStoreInputValidator
-          .validateSubClusterRegisterRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       LOG.info(e.getMessage());
@@ -421,7 +421,7 @@ public class TestFederationStateStoreInputValidator {
       SubClusterRegisterRequest request =
           SubClusterRegisterRequest.newInstance(subClusterInfo);
       FederationMembershipStateStoreInputValidator
-          .validateSubClusterRegisterRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       LOG.info(e.getMessage());
@@ -438,7 +438,7 @@ public class TestFederationStateStoreInputValidator {
       SubClusterRegisterRequest request =
           SubClusterRegisterRequest.newInstance(subClusterInfo);
       FederationMembershipStateStoreInputValidator
-          .validateSubClusterRegisterRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       LOG.info(e.getMessage());
@@ -460,7 +460,7 @@ public class TestFederationStateStoreInputValidator {
       SubClusterRegisterRequest request =
           SubClusterRegisterRequest.newInstance(subClusterInfo);
       FederationMembershipStateStoreInputValidator
-          .validateSubClusterRegisterRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       LOG.info(e.getMessage());
@@ -477,7 +477,7 @@ public class TestFederationStateStoreInputValidator {
       SubClusterRegisterRequest request =
           SubClusterRegisterRequest.newInstance(subClusterInfo);
       FederationMembershipStateStoreInputValidator
-          .validateSubClusterRegisterRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       LOG.info(e.getMessage());
@@ -494,7 +494,7 @@ public class TestFederationStateStoreInputValidator {
       SubClusterRegisterRequest request =
           SubClusterRegisterRequest.newInstance(subClusterInfo);
       FederationMembershipStateStoreInputValidator
-          .validateSubClusterRegisterRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       LOG.info(e.getMessage());
@@ -510,7 +510,7 @@ public class TestFederationStateStoreInputValidator {
       SubClusterRegisterRequest request =
           SubClusterRegisterRequest.newInstance(subClusterInfo);
       FederationMembershipStateStoreInputValidator
-          .validateSubClusterRegisterRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       LOG.info(e.getMessage());
@@ -526,7 +526,7 @@ public class TestFederationStateStoreInputValidator {
       SubClusterRegisterRequest request =
           SubClusterRegisterRequest.newInstance(subClusterInfo);
       FederationMembershipStateStoreInputValidator
-          .validateSubClusterRegisterRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       LOG.info(e.getMessage());
@@ -543,7 +543,7 @@ public class TestFederationStateStoreInputValidator {
       SubClusterRegisterRequest request =
           SubClusterRegisterRequest.newInstance(subClusterInfo);
       FederationMembershipStateStoreInputValidator
-          .validateSubClusterRegisterRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       LOG.info(e.getMessage());
@@ -560,7 +560,7 @@ public class TestFederationStateStoreInputValidator {
       SubClusterRegisterRequest request =
           SubClusterRegisterRequest.newInstance(subClusterInfo);
       FederationMembershipStateStoreInputValidator
-          .validateSubClusterRegisterRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       LOG.info(e.getMessage());
@@ -576,7 +576,7 @@ public class TestFederationStateStoreInputValidator {
       SubClusterRegisterRequest request =
           SubClusterRegisterRequest.newInstance(subClusterInfo);
       FederationMembershipStateStoreInputValidator
-          .validateSubClusterRegisterRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       LOG.info(e.getMessage());
@@ -594,7 +594,7 @@ public class TestFederationStateStoreInputValidator {
       SubClusterDeregisterRequest request =
           SubClusterDeregisterRequest.newInstance(subClusterId, stateLost);
       FederationMembershipStateStoreInputValidator
-          .validateSubClusterDeregisterRequest(request);
+          .validate(request);
     } catch (FederationStateStoreInvalidInputException e) {
       Assert.fail(e.getMessage());
     }
@@ -604,7 +604,7 @@ public class TestFederationStateStoreInputValidator {
     try {
       SubClusterDeregisterRequest request = null;
       FederationMembershipStateStoreInputValidator
-          .validateSubClusterDeregisterRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       LOG.info(e.getMessage());
@@ -618,7 +618,7 @@ public class TestFederationStateStoreInputValidator {
       SubClusterDeregisterRequest request =
           SubClusterDeregisterRequest.newInstance(subClusterIdNull, stateLost);
       FederationMembershipStateStoreInputValidator
-          .validateSubClusterDeregisterRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       LOG.info(e.getMessage());
@@ -632,7 +632,7 @@ public class TestFederationStateStoreInputValidator {
       SubClusterDeregisterRequest request = SubClusterDeregisterRequest
           .newInstance(subClusterIdInvalid, stateLost);
       FederationMembershipStateStoreInputValidator
-          .validateSubClusterDeregisterRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       LOG.info(e.getMessage());
@@ -646,7 +646,7 @@ public class TestFederationStateStoreInputValidator {
       SubClusterDeregisterRequest request =
           SubClusterDeregisterRequest.newInstance(subClusterId, stateNull);
       FederationMembershipStateStoreInputValidator
-          .validateSubClusterDeregisterRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       LOG.info(e.getMessage());
@@ -660,7 +660,7 @@ public class TestFederationStateStoreInputValidator {
       SubClusterDeregisterRequest request =
           SubClusterDeregisterRequest.newInstance(subClusterId, stateNew);
       FederationMembershipStateStoreInputValidator
-          .validateSubClusterDeregisterRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       LOG.info(e.getMessage());
@@ -677,7 +677,7 @@ public class TestFederationStateStoreInputValidator {
       SubClusterHeartbeatRequest request = SubClusterHeartbeatRequest
           .newInstance(subClusterId, lastHeartBeat, stateLost, capability);
       FederationMembershipStateStoreInputValidator
-          .validateSubClusterHeartbeatRequest(request);
+          .validate(request);
     } catch (FederationStateStoreInvalidInputException e) {
       Assert.fail(e.getMessage());
     }
@@ -687,7 +687,7 @@ public class TestFederationStateStoreInputValidator {
     try {
       SubClusterHeartbeatRequest request = null;
       FederationMembershipStateStoreInputValidator
-          .validateSubClusterHeartbeatRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       LOG.info(e.getMessage());
@@ -701,7 +701,7 @@ public class TestFederationStateStoreInputValidator {
       SubClusterHeartbeatRequest request = SubClusterHeartbeatRequest
           .newInstance(subClusterIdNull, lastHeartBeat, stateLost, capability);
       FederationMembershipStateStoreInputValidator
-          .validateSubClusterHeartbeatRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       LOG.info(e.getMessage());
@@ -716,7 +716,7 @@ public class TestFederationStateStoreInputValidator {
           SubClusterHeartbeatRequest.newInstance(subClusterIdInvalid,
               lastHeartBeat, stateLost, capability);
       FederationMembershipStateStoreInputValidator
-          .validateSubClusterHeartbeatRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       LOG.info(e.getMessage());
@@ -730,7 +730,7 @@ public class TestFederationStateStoreInputValidator {
       SubClusterHeartbeatRequest request = SubClusterHeartbeatRequest
           .newInstance(subClusterId, lastHeartBeat, stateNull, capability);
       FederationMembershipStateStoreInputValidator
-          .validateSubClusterHeartbeatRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       LOG.info(e.getMessage());
@@ -745,7 +745,7 @@ public class TestFederationStateStoreInputValidator {
           SubClusterHeartbeatRequest.newInstance(subClusterId,
               lastHeartBeatNegative, stateLost, capability);
       FederationMembershipStateStoreInputValidator
-          .validateSubClusterHeartbeatRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       LOG.info(e.getMessage());
@@ -759,7 +759,7 @@ public class TestFederationStateStoreInputValidator {
       SubClusterHeartbeatRequest request = SubClusterHeartbeatRequest
           .newInstance(subClusterId, lastHeartBeat, stateLost, capabilityNull);
       FederationMembershipStateStoreInputValidator
-          .validateSubClusterHeartbeatRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       LOG.info(e.getMessage());
@@ -773,7 +773,7 @@ public class TestFederationStateStoreInputValidator {
       SubClusterHeartbeatRequest request = SubClusterHeartbeatRequest
           .newInstance(subClusterId, lastHeartBeat, stateLost, capabilityEmpty);
       FederationMembershipStateStoreInputValidator
-          .validateSubClusterHeartbeatRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       LOG.info(e.getMessage());
@@ -791,7 +791,7 @@ public class TestFederationStateStoreInputValidator {
       GetSubClusterInfoRequest request =
           GetSubClusterInfoRequest.newInstance(subClusterId);
       FederationMembershipStateStoreInputValidator
-          .validateGetSubClusterInfoRequest(request);
+          .validate(request);
     } catch (FederationStateStoreInvalidInputException e) {
       Assert.fail(e.getMessage());
     }
@@ -801,7 +801,7 @@ public class TestFederationStateStoreInputValidator {
     try {
       GetSubClusterInfoRequest request = null;
       FederationMembershipStateStoreInputValidator
-          .validateGetSubClusterInfoRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       LOG.info(e.getMessage());
@@ -815,7 +815,7 @@ public class TestFederationStateStoreInputValidator {
       GetSubClusterInfoRequest request =
           GetSubClusterInfoRequest.newInstance(subClusterIdNull);
       FederationMembershipStateStoreInputValidator
-          .validateGetSubClusterInfoRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       LOG.info(e.getMessage());
@@ -829,7 +829,7 @@ public class TestFederationStateStoreInputValidator {
       GetSubClusterInfoRequest request =
           GetSubClusterInfoRequest.newInstance(subClusterIdInvalid);
       FederationMembershipStateStoreInputValidator
-          .validateGetSubClusterInfoRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       LOG.info(e.getMessage());
@@ -850,7 +850,7 @@ public class TestFederationStateStoreInputValidator {
           AddApplicationHomeSubClusterRequest
               .newInstance(applicationHomeSubCluster);
       FederationApplicationHomeSubClusterStoreInputValidator
-          .validateAddApplicationHomeSubClusterRequest(request);
+          .validate(request);
     } catch (FederationStateStoreInvalidInputException e) {
       Assert.fail(e.getMessage());
     }
@@ -860,7 +860,7 @@ public class TestFederationStateStoreInputValidator {
     try {
       AddApplicationHomeSubClusterRequest request = null;
       FederationApplicationHomeSubClusterStoreInputValidator
-          .validateAddApplicationHomeSubClusterRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       Assert.assertTrue(e.getMessage()
@@ -875,7 +875,7 @@ public class TestFederationStateStoreInputValidator {
           AddApplicationHomeSubClusterRequest
               .newInstance(applicationHomeSubCluster);
       FederationApplicationHomeSubClusterStoreInputValidator
-          .validateAddApplicationHomeSubClusterRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       Assert.assertTrue(
@@ -891,7 +891,7 @@ public class TestFederationStateStoreInputValidator {
           AddApplicationHomeSubClusterRequest
               .newInstance(applicationHomeSubCluster);
       FederationApplicationHomeSubClusterStoreInputValidator
-          .validateAddApplicationHomeSubClusterRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       LOG.info(e.getMessage());
@@ -908,7 +908,7 @@ public class TestFederationStateStoreInputValidator {
           AddApplicationHomeSubClusterRequest
               .newInstance(applicationHomeSubCluster);
       FederationApplicationHomeSubClusterStoreInputValidator
-          .validateAddApplicationHomeSubClusterRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       LOG.info(e.getMessage());
@@ -925,7 +925,7 @@ public class TestFederationStateStoreInputValidator {
           AddApplicationHomeSubClusterRequest
               .newInstance(applicationHomeSubCluster);
       FederationApplicationHomeSubClusterStoreInputValidator
-          .validateAddApplicationHomeSubClusterRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       Assert.assertTrue(e.getMessage().startsWith("Missing Application Id."));
@@ -944,7 +944,7 @@ public class TestFederationStateStoreInputValidator {
           UpdateApplicationHomeSubClusterRequest
               .newInstance(applicationHomeSubCluster);
       FederationApplicationHomeSubClusterStoreInputValidator
-          .validateUpdateApplicationHomeSubClusterRequest(request);
+          .validate(request);
     } catch (FederationStateStoreInvalidInputException e) {
       Assert.fail(e.getMessage());
     }
@@ -954,7 +954,7 @@ public class TestFederationStateStoreInputValidator {
     try {
       UpdateApplicationHomeSubClusterRequest request = null;
       FederationApplicationHomeSubClusterStoreInputValidator
-          .validateUpdateApplicationHomeSubClusterRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       Assert.assertTrue(e.getMessage()
@@ -969,7 +969,7 @@ public class TestFederationStateStoreInputValidator {
           UpdateApplicationHomeSubClusterRequest
               .newInstance(applicationHomeSubCluster);
       FederationApplicationHomeSubClusterStoreInputValidator
-          .validateUpdateApplicationHomeSubClusterRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       Assert.assertTrue(
@@ -985,7 +985,7 @@ public class TestFederationStateStoreInputValidator {
           UpdateApplicationHomeSubClusterRequest
               .newInstance(applicationHomeSubCluster);
       FederationApplicationHomeSubClusterStoreInputValidator
-          .validateUpdateApplicationHomeSubClusterRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       LOG.info(e.getMessage());
@@ -1002,7 +1002,7 @@ public class TestFederationStateStoreInputValidator {
           UpdateApplicationHomeSubClusterRequest
               .newInstance(applicationHomeSubCluster);
       FederationApplicationHomeSubClusterStoreInputValidator
-          .validateUpdateApplicationHomeSubClusterRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       LOG.info(e.getMessage());
@@ -1019,7 +1019,7 @@ public class TestFederationStateStoreInputValidator {
           UpdateApplicationHomeSubClusterRequest
               .newInstance(applicationHomeSubCluster);
       FederationApplicationHomeSubClusterStoreInputValidator
-          .validateUpdateApplicationHomeSubClusterRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       Assert.assertTrue(e.getMessage().startsWith("Missing Application Id."));
@@ -1035,7 +1035,7 @@ public class TestFederationStateStoreInputValidator {
       GetApplicationHomeSubClusterRequest request =
           GetApplicationHomeSubClusterRequest.newInstance(appId);
       FederationApplicationHomeSubClusterStoreInputValidator
-          .validateGetApplicationHomeSubClusterRequest(request);
+          .validate(request);
     } catch (FederationStateStoreInvalidInputException e) {
       Assert.fail(e.getMessage());
     }
@@ -1045,7 +1045,7 @@ public class TestFederationStateStoreInputValidator {
     try {
       GetApplicationHomeSubClusterRequest request = null;
       FederationApplicationHomeSubClusterStoreInputValidator
-          .validateGetApplicationHomeSubClusterRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       Assert.assertTrue(e.getMessage()
@@ -1058,7 +1058,7 @@ public class TestFederationStateStoreInputValidator {
       GetApplicationHomeSubClusterRequest request =
           GetApplicationHomeSubClusterRequest.newInstance(appIdNull);
       FederationApplicationHomeSubClusterStoreInputValidator
-          .validateGetApplicationHomeSubClusterRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       Assert.assertTrue(e.getMessage().startsWith("Missing Application Id."));
@@ -1075,7 +1075,7 @@ public class TestFederationStateStoreInputValidator {
       DeleteApplicationHomeSubClusterRequest request =
           DeleteApplicationHomeSubClusterRequest.newInstance(appId);
       FederationApplicationHomeSubClusterStoreInputValidator
-          .validateDeleteApplicationHomeSubClusterRequest(request);
+          .validate(request);
     } catch (FederationStateStoreInvalidInputException e) {
       Assert.fail(e.getMessage());
     }
@@ -1085,7 +1085,7 @@ public class TestFederationStateStoreInputValidator {
     try {
       DeleteApplicationHomeSubClusterRequest request = null;
       FederationApplicationHomeSubClusterStoreInputValidator
-          .validateDeleteApplicationHomeSubClusterRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       Assert.assertTrue(e.getMessage()
@@ -1098,7 +1098,7 @@ public class TestFederationStateStoreInputValidator {
       DeleteApplicationHomeSubClusterRequest request =
           DeleteApplicationHomeSubClusterRequest.newInstance(appIdNull);
       FederationApplicationHomeSubClusterStoreInputValidator
-          .validateDeleteApplicationHomeSubClusterRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       Assert.assertTrue(e.getMessage().startsWith("Missing Application Id."));
@@ -1115,7 +1115,7 @@ public class TestFederationStateStoreInputValidator {
       GetSubClusterPolicyConfigurationRequest request =
           GetSubClusterPolicyConfigurationRequest.newInstance(queue);
       FederationPolicyStoreInputValidator
-          .validateGetSubClusterPolicyConfigurationRequest(request);
+          .validate(request);
     } catch (FederationStateStoreInvalidInputException e) {
       Assert.fail(e.getMessage());
     }
@@ -1125,7 +1125,7 @@ public class TestFederationStateStoreInputValidator {
     try {
       GetSubClusterPolicyConfigurationRequest request = null;
       FederationPolicyStoreInputValidator
-          .validateGetSubClusterPolicyConfigurationRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       Assert.assertTrue(e.getMessage()
@@ -1138,7 +1138,7 @@ public class TestFederationStateStoreInputValidator {
       GetSubClusterPolicyConfigurationRequest request =
           GetSubClusterPolicyConfigurationRequest.newInstance(queueNull);
       FederationPolicyStoreInputValidator
-          .validateGetSubClusterPolicyConfigurationRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       Assert.assertTrue(e.getMessage().startsWith("Missing Queue."));
@@ -1150,7 +1150,7 @@ public class TestFederationStateStoreInputValidator {
       GetSubClusterPolicyConfigurationRequest request =
           GetSubClusterPolicyConfigurationRequest.newInstance(queueEmpty);
       FederationPolicyStoreInputValidator
-          .validateGetSubClusterPolicyConfigurationRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       Assert.assertTrue(e.getMessage().startsWith("Missing Queue."));
@@ -1169,7 +1169,7 @@ public class TestFederationStateStoreInputValidator {
       SetSubClusterPolicyConfigurationRequest request =
           SetSubClusterPolicyConfigurationRequest.newInstance(policy);
       FederationPolicyStoreInputValidator
-          .validateSetSubClusterPolicyConfigurationRequest(request);
+          .validate(request);
     } catch (FederationStateStoreInvalidInputException e) {
       Assert.fail(e.getMessage());
     }
@@ -1179,7 +1179,7 @@ public class TestFederationStateStoreInputValidator {
     try {
       SetSubClusterPolicyConfigurationRequest request = null;
       FederationPolicyStoreInputValidator
-          .validateSetSubClusterPolicyConfigurationRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       Assert.assertTrue(e.getMessage()
@@ -1193,7 +1193,7 @@ public class TestFederationStateStoreInputValidator {
       SetSubClusterPolicyConfigurationRequest request =
           SetSubClusterPolicyConfigurationRequest.newInstance(policy);
       FederationPolicyStoreInputValidator
-          .validateSetSubClusterPolicyConfigurationRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       Assert.assertTrue(
@@ -1208,7 +1208,7 @@ public class TestFederationStateStoreInputValidator {
       SetSubClusterPolicyConfigurationRequest request =
           SetSubClusterPolicyConfigurationRequest.newInstance(policy);
       FederationPolicyStoreInputValidator
-          .validateSetSubClusterPolicyConfigurationRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       Assert.assertTrue(e.getMessage().startsWith("Missing Queue."));
@@ -1222,7 +1222,7 @@ public class TestFederationStateStoreInputValidator {
       SetSubClusterPolicyConfigurationRequest request =
           SetSubClusterPolicyConfigurationRequest.newInstance(policy);
       FederationPolicyStoreInputValidator
-          .validateSetSubClusterPolicyConfigurationRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       Assert.assertTrue(e.getMessage().startsWith("Missing Queue."));
@@ -1236,7 +1236,7 @@ public class TestFederationStateStoreInputValidator {
       SetSubClusterPolicyConfigurationRequest request =
           SetSubClusterPolicyConfigurationRequest.newInstance(policy);
       FederationPolicyStoreInputValidator
-          .validateSetSubClusterPolicyConfigurationRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       Assert.assertTrue(e.getMessage().startsWith("Missing Policy Type."));
@@ -1250,7 +1250,7 @@ public class TestFederationStateStoreInputValidator {
       SetSubClusterPolicyConfigurationRequest request =
           SetSubClusterPolicyConfigurationRequest.newInstance(policy);
       FederationPolicyStoreInputValidator
-          .validateSetSubClusterPolicyConfigurationRequest(request);
+          .validate(request);
       Assert.fail();
     } catch (FederationStateStoreInvalidInputException e) {
       Assert.assertTrue(e.getMessage().startsWith("Missing Policy Type."));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5599b916/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/TestFederationStateStoreFacadeRetry.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/TestFederationStateStoreFacadeRetry.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/TestFederationStateStoreFacadeRetry.java
index 632e865..304910e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/TestFederationStateStoreFacadeRetry.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/TestFederationStateStoreFacadeRetry.java
@@ -24,7 +24,6 @@ import org.apache.hadoop.io.retry.RetryPolicy;
 import org.apache.hadoop.io.retry.RetryPolicy.RetryAction;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
-import org.apache.hadoop.yarn.server.federation.store.exception.FederationStateStoreErrorCode;
 import org.apache.hadoop.yarn.server.federation.store.exception.FederationStateStoreException;
 import org.apache.hadoop.yarn.server.federation.store.exception.FederationStateStoreInvalidInputException;
 import org.apache.hadoop.yarn.server.federation.store.exception.FederationStateStoreRetriableException;
@@ -82,10 +81,8 @@ public class TestFederationStateStoreFacadeRetry {
     conf = new Configuration();
     conf.setInt(YarnConfiguration.CLIENT_FAILOVER_RETRIES, maxRetries);
     RetryPolicy policy = FederationStateStoreFacade.createRetryPolicy(conf);
-    RetryAction action = policy.shouldRetry(
-        new FederationStateStoreException(
-            FederationStateStoreErrorCode.APPLICATIONS_INSERT_FAIL),
-        0, 0, false);
+    RetryAction action = policy
+        .shouldRetry(new FederationStateStoreException("Error"), 0, 0, false);
     Assert.assertEquals(RetryAction.FAIL.action, action.action);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5599b916/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/resources/FederationStateStore/SQLServer/FederationStateStoreStoreProcs.sql
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/resources/FederationStateStore/SQLServer/FederationStateStoreStoreProcs.sql b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/resources/FederationStateStore/SQLServer/FederationStateStoreStoreProcs.sql
new file mode 100644
index 0000000..66d6f0e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/resources/FederationStateStore/SQLServer/FederationStateStoreStoreProcs.sql
@@ -0,0 +1,511 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+USE [FederationStateStore]
+GO
+
+IF OBJECT_ID ( '[sp_addApplicationHomeSubCluster]', 'P' ) IS NOT NULL
+    DROP PROCEDURE [sp_addApplicationHomeSubCluster];
+GO
+
+CREATE PROCEDURE [dbo].[sp_addApplicationHomeSubCluster]
+    @applicationId VARCHAR(64),
+    @homeSubCluster VARCHAR(256),
+    @storedHomeSubCluster VARCHAR(256) OUTPUT,
+    @rowCount int OUTPUT
+AS BEGIN
+    DECLARE @errorMessage nvarchar(4000)
+
+    BEGIN TRY
+        BEGIN TRAN
+            -- If application to sub-cluster map doesn't exist, insert it.
+            -- Otherwise don't change the current mapping.
+            IF NOT EXISTS (SELECT TOP 1 *
+                       FROM [dbo].[applicationsHomeSubCluster]
+                       WHERE [applicationId] = @applicationId)
+
+                INSERT INTO [dbo].[applicationsHomeSubCluster] (
+                    [applicationId],
+                    [homeSubCluster])
+                VALUES (
+                    @applicationId,
+                    @homeSubCluster);
+            -- End of the IF block
+
+            SELECT @rowCount = @@ROWCOUNT;
+
+            SELECT @storedHomeSubCluster = [homeSubCluster]
+            FROM [dbo].[applicationsHomeSubCluster]
+            WHERE [applicationId] = @applicationId;
+
+        COMMIT TRAN
+    END TRY
+
+    BEGIN CATCH
+        ROLLBACK TRAN
+
+        SET @errorMessage = dbo.func_FormatErrorMessage(ERROR_MESSAGE(), ERROR_LINE())
+
+        /*  raise error and terminate the execution */
+        RAISERROR(@errorMessage, --- Error Message
+            1, -- Severity
+            -1 -- State
+        ) WITH log
+    END CATCH
+END;
+GO
+
+IF OBJECT_ID ( '[sp_updateApplicationHomeSubCluster]', 'P' ) IS NOT NULL
+    DROP PROCEDURE [sp_updateApplicationHomeSubCluster];
+GO
+
+CREATE PROCEDURE [dbo].[sp_updateApplicationHomeSubCluster]
+    @applicationId VARCHAR(64),
+    @homeSubCluster VARCHAR(256),
+    @rowCount int OUTPUT
+AS BEGIN
+    DECLARE @errorMessage nvarchar(4000)
+
+    BEGIN TRY
+        BEGIN TRAN
+
+            UPDATE [dbo].[applicationsHomeSubCluster]
+            SET [homeSubCluster] = @homeSubCluster
+            WHERE [applicationId] = @applicationid;
+            SELECT @rowCount = @@ROWCOUNT;
+
+        COMMIT TRAN
+    END TRY
+
+    BEGIN CATCH
+        ROLLBACK TRAN
+
+        SET @errorMessage = dbo.func_FormatErrorMessage(ERROR_MESSAGE(), ERROR_LINE())
+
+        /*  raise error and terminate the execution */
+        RAISERROR(@errorMessage, --- Error Message
+            1, -- Severity
+            -1 -- State
+        ) WITH log
+    END CATCH
+END;
+GO
+
+IF OBJECT_ID ( '[sp_getApplicationsHomeSubCluster]', 'P' ) IS NOT NULL
+    DROP PROCEDURE [sp_getApplicationsHomeSubCluster];
+GO
+
+CREATE PROCEDURE [dbo].[sp_getApplicationsHomeSubCluster]
+AS BEGIN
+    DECLARE @errorMessage nvarchar(4000)
+
+    BEGIN TRY
+        SELECT [applicationId], [homeSubCluster], [createTime]
+        FROM [dbo].[applicationsHomeSubCluster]
+    END TRY
+
+    BEGIN CATCH
+        SET @errorMessage = dbo.func_FormatErrorMessage(ERROR_MESSAGE(), ERROR_LINE())
+
+        /*  raise error and terminate the execution */
+        RAISERROR(@errorMessage, --- Error Message
+            1, -- Severity
+            -1 -- State
+        ) WITH log
+    END CATCH
+END;
+GO
+
+IF OBJECT_ID ( '[sp_getApplicationHomeSubCluster]', 'P' ) IS NOT NULL
+    DROP PROCEDURE [sp_getApplicationHomeSubCluster];
+GO
+
+CREATE PROCEDURE [dbo].[sp_getApplicationHomeSubCluster]
+    @applicationId VARCHAR(64),
+    @homeSubCluster VARCHAR(256) OUTPUT
+AS BEGIN
+    DECLARE @errorMessage nvarchar(4000)
+
+    BEGIN TRY
+
+        SELECT @homeSubCluster = [homeSubCluster]
+        FROM [dbo].[applicationsHomeSubCluster]
+        WHERE [applicationId] = @applicationid;
+
+    END TRY
+
+    BEGIN CATCH
+
+        SET @errorMessage = dbo.func_FormatErrorMessage(ERROR_MESSAGE(), ERROR_LINE())
+
+        /*  raise error and terminate the execution */
+        RAISERROR(@errorMessage, --- Error Message
+            1, -- Severity
+            -1 -- State
+        ) WITH log
+    END CATCH
+END;
+GO
+
+IF OBJECT_ID ( '[sp_deleteApplicationHomeSubCluster]', 'P' ) IS NOT NULL
+    DROP PROCEDURE [sp_deleteApplicationHomeSubCluster];
+GO
+
+CREATE PROCEDURE [dbo].[sp_deleteApplicationHomeSubCluster]
+    @applicationId VARCHAR(64),
+    @rowCount int OUTPUT
+AS BEGIN
+    DECLARE @errorMessage nvarchar(4000)
+
+    BEGIN TRY
+        BEGIN TRAN
+
+            DELETE FROM [dbo].[applicationsHomeSubCluster]
+            WHERE [applicationId] = @applicationId;
+            SELECT @rowCount = @@ROWCOUNT;
+
+        COMMIT TRAN
+    END TRY
+
+    BEGIN CATCH
+        ROLLBACK TRAN
+
+        SET @errorMessage = dbo.func_FormatErrorMessage(ERROR_MESSAGE(), ERROR_LINE())
+
+        /*  raise error and terminate the execution */
+        RAISERROR(@errorMessage, --- Error Message
+            1, -- Severity
+            -1 -- State
+        ) WITH log
+    END CATCH
+END;
+GO
+
+IF OBJECT_ID ( '[sp_registerSubCluster]', 'P' ) IS NOT NULL
+    DROP PROCEDURE [sp_registerSubCluster];
+GO
+
+CREATE PROCEDURE [dbo].[sp_registerSubCluster]
+    @subClusterId VARCHAR(256),
+    @amRMServiceAddress VARCHAR(256),
+    @clientRMServiceAddress VARCHAR(256),
+    @rmAdminServiceAddress VARCHAR(256),
+    @rmWebServiceAddress VARCHAR(256),
+    @state VARCHAR(32),
+    @lastStartTime BIGINT,
+    @capability VARCHAR(6000),
+    @rowCount int OUTPUT
+AS BEGIN
+    DECLARE @errorMessage nvarchar(4000)
+
+    BEGIN TRY
+        BEGIN TRAN
+
+            DELETE FROM [dbo].[membership]
+            WHERE [subClusterId] = @subClusterId;
+            INSERT INTO [dbo].[membership] (
+                [subClusterId],
+                [amRMServiceAddress],
+                [clientRMServiceAddress],
+                [rmAdminServiceAddress],
+                [rmWebServiceAddress],
+                [lastHeartBeat],
+                [state],
+                [lastStartTime],
+                [capability] )
+            VALUES (
+                @subClusterId,
+                @amRMServiceAddress,
+                @clientRMServiceAddress,
+                @rmAdminServiceAddress,
+                @rmWebServiceAddress,
+                GETUTCDATE(),
+                @state,
+                @lastStartTime,
+                @capability);
+            SELECT @rowCount = @@ROWCOUNT;
+
+        COMMIT TRAN
+    END TRY
+
+    BEGIN CATCH
+        ROLLBACK TRAN
+
+        SET @errorMessage = dbo.func_FormatErrorMessage(ERROR_MESSAGE(), ERROR_LINE())
+
+        /*  raise error and terminate the execution */
+        RAISERROR(@errorMessage, --- Error Message
+            1, -- Severity
+            -1 -- State
+        ) WITH log
+    END CATCH
+END;
+GO
+
+IF OBJECT_ID ( '[sp_getSubClusters]', 'P' ) IS NOT NULL
+    DROP PROCEDURE [sp_getSubClusters];
+GO
+
+CREATE PROCEDURE [dbo].[sp_getSubClusters]
+AS BEGIN
+    DECLARE @errorMessage nvarchar(4000)
+
+    BEGIN TRY
+        SELECT [subClusterId], [amRMServiceAddress], [clientRMServiceAddress],
+               [rmAdminServiceAddress], [rmWebServiceAddress], [lastHeartBeat],
+               [state], [lastStartTime], [capability]
+        FROM [dbo].[membership]
+    END TRY
+
+    BEGIN CATCH
+        SET @errorMessage = dbo.func_FormatErrorMessage(ERROR_MESSAGE(), ERROR_LINE())
+
+        /*  raise error and terminate the execution */
+        RAISERROR(@errorMessage, --- Error Message
+            1, -- Severity
+            -1 -- State
+        ) WITH log
+    END CATCH
+END;
+GO
+
+IF OBJECT_ID ( '[sp_getSubCluster]', 'P' ) IS NOT NULL
+    DROP PROCEDURE [sp_getSubCluster];
+GO
+
+CREATE PROCEDURE [dbo].[sp_getSubCluster]
+    @subClusterId VARCHAR(256),
+    @amRMServiceAddress VARCHAR(256) OUTPUT,
+    @clientRMServiceAddress VARCHAR(256) OUTPUT,
+    @rmAdminServiceAddress VARCHAR(256) OUTPUT,
+    @rmWebServiceAddress VARCHAR(256) OUTPUT,
+    @lastHeartbeat DATETIME2 OUTPUT,
+    @state VARCHAR(256) OUTPUT,
+    @lastStartTime BIGINT OUTPUT,
+    @capability VARCHAR(6000) OUTPUT
+AS BEGIN
+    DECLARE @errorMessage nvarchar(4000)
+
+    BEGIN TRY
+        BEGIN TRAN
+
+            SELECT @subClusterId = [subClusterId],
+                   @amRMServiceAddress = [amRMServiceAddress],
+                   @clientRMServiceAddress = [clientRMServiceAddress],
+                   @rmAdminServiceAddress = [rmAdminServiceAddress],
+                   @rmWebServiceAddress = [rmWebServiceAddress],
+                   @lastHeartBeat = [lastHeartBeat],
+                   @state = [state],
+                   @lastStartTime = [lastStartTime],
+                   @capability = [capability]
+            FROM [dbo].[membership]
+            WHERE [subClusterId] = @subClusterId
+
+        COMMIT TRAN
+    END TRY
+
+    BEGIN CATCH
+        ROLLBACK TRAN
+
+        SET @errorMessage = dbo.func_FormatErrorMessage(ERROR_MESSAGE(), ERROR_LINE())
+
+        /*  raise error and terminate the execution */
+        RAISERROR(@errorMessage, --- Error Message
+            1, -- Severity
+            -1 -- State
+        ) WITH log
+    END CATCH
+END;
+GO
+
+
+IF OBJECT_ID ( '[sp_subClusterHeartbeat]', 'P' ) IS NOT NULL
+    DROP PROCEDURE [sp_subClusterHeartbeat];
+GO
+
+CREATE PROCEDURE [dbo].[sp_subClusterHeartbeat]
+    @subClusterId VARCHAR(256),
+    @state VARCHAR(256),
+    @capability VARCHAR(6000),
+    @rowCount int OUTPUT
+AS BEGIN
+    DECLARE @errorMessage nvarchar(4000)
+
+    BEGIN TRY
+        BEGIN TRAN
+
+            UPDATE [dbo].[membership]
+            SET [state] = @state,
+                [lastHeartbeat] = GETUTCDATE(),
+                [capability] = @capability
+            WHERE [subClusterId] = @subClusterId;
+            SELECT @rowCount = @@ROWCOUNT;
+
+        COMMIT TRAN
+    END TRY
+
+    BEGIN CATCH
+        ROLLBACK TRAN
+
+        SET @errorMessage = dbo.func_FormatErrorMessage(ERROR_MESSAGE(), ERROR_LINE())
+
+        /*  raise error and terminate the execution */
+        RAISERROR(@errorMessage, --- Error Message
+            1, -- Severity
+            -1 -- State
+        ) WITH log
+    END CATCH
+END;
+GO
+
+IF OBJECT_ID ( '[sp_deregisterSubCluster]', 'P' ) IS NOT NULL
+    DROP PROCEDURE [sp_deregisterSubCluster];
+GO
+
+CREATE PROCEDURE [dbo].[sp_deregisterSubCluster]
+    @subClusterId VARCHAR(256),
+    @state VARCHAR(256),
+    @rowCount int OUTPUT
+AS BEGIN
+    DECLARE @errorMessage nvarchar(4000)
+
+    BEGIN TRY
+        BEGIN TRAN
+
+            UPDATE [dbo].[membership]
+            SET [state] = @state
+            WHERE [subClusterId] = @subClusterId;
+            SELECT @rowCount = @@ROWCOUNT;
+
+        COMMIT TRAN
+    END TRY
+
+    BEGIN CATCH
+        ROLLBACK TRAN
+
+        SET @errorMessage = dbo.func_FormatErrorMessage(ERROR_MESSAGE(), ERROR_LINE())
+
+        /*  raise error and terminate the execution */
+        RAISERROR(@errorMessage, --- Error Message
+            1, -- Severity
+            -1 -- State
+        ) WITH log
+    END CATCH
+END;
+GO
+
+IF OBJECT_ID ( '[sp_setPolicyConfiguration]', 'P' ) IS NOT NULL
+    DROP PROCEDURE [sp_setPolicyConfiguration];
+GO
+
+CREATE PROCEDURE [dbo].[sp_setPolicyConfiguration]
+    @queue VARCHAR(256),
+    @policyType VARCHAR(256),
+    @params VARBINARY(512),
+    @rowCount int OUTPUT
+AS BEGIN
+    DECLARE @errorMessage nvarchar(4000)
+
+    BEGIN TRY
+        BEGIN TRAN
+
+            DELETE FROM [dbo].[policies]
+            WHERE [queue] = @queue;
+            INSERT INTO [dbo].[policies] (
+                [queue],
+                [policyType],
+                [params])
+            VALUES (
+                @queue,
+                @policyType,
+                @params);
+            SELECT @rowCount = @@ROWCOUNT;
+
+        COMMIT TRAN
+    END TRY
+
+    BEGIN CATCH
+        ROLLBACK TRAN
+
+        SET @errorMessage = dbo.func_FormatErrorMessage(ERROR_MESSAGE(), ERROR_LINE())
+
+        /*  raise error and terminate the execution */
+        RAISERROR(@errorMessage, --- Error Message
+            1, -- Severity
+            -1 -- State
+        ) WITH log
+    END CATCH
+END;
+GO
+
+IF OBJECT_ID ( '[sp_getPolicyConfiguration]', 'P' ) IS NOT NULL
+    DROP PROCEDURE [sp_getPolicyConfiguration];
+GO
+
+CREATE PROCEDURE [dbo].[sp_getPolicyConfiguration]
+    @queue VARCHAR(256),
+    @policyType VARCHAR(256) OUTPUT,
+    @params VARBINARY(6000) OUTPUT
+AS BEGIN
+    DECLARE @errorMessage nvarchar(4000)
+
+    BEGIN TRY
+
+        SELECT @policyType = [policyType],
+               @params = [params]
+        FROM [dbo].[policies]
+        WHERE [queue] = @queue
+
+    END TRY
+
+    BEGIN CATCH
+
+        SET @errorMessage = dbo.func_FormatErrorMessage(ERROR_MESSAGE(), ERROR_LINE())
+
+        /*  raise error and terminate the execution */
+        RAISERROR(@errorMessage, --- Error Message
+            1, -- Severity
+            -1 -- State
+        ) WITH log
+    END CATCH
+END;
+GO
+
+IF OBJECT_ID ( '[sp_getPoliciesConfigurations]', 'P' ) IS NOT NULL
+    DROP PROCEDURE [sp_getPoliciesConfigurations];
+GO
+
+CREATE PROCEDURE [dbo].[sp_getPoliciesConfigurations]
+AS BEGIN
+    DECLARE @errorMessage nvarchar(4000)
+
+    BEGIN TRY
+        SELECT  [queue], [policyType], [params] FROM [dbo].[policies]
+    END TRY
+
+    BEGIN CATCH
+        SET @errorMessage = dbo.func_FormatErrorMessage(ERROR_MESSAGE(), ERROR_LINE())
+
+        /*  raise error and terminate the execution */
+        RAISERROR(@errorMessage, --- Error Message
+            1, -- Severity
+            -1 -- State
+        ) WITH log
+    END CATCH
+END;
+GO
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5599b916/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/resources/FederationStateStore/SQLServer/FederationStateStoreTables.sql
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/resources/FederationStateStore/SQLServer/FederationStateStoreTables.sql b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/resources/FederationStateStore/SQLServer/FederationStateStoreTables.sql
new file mode 100644
index 0000000..a97385b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/resources/FederationStateStore/SQLServer/FederationStateStoreTables.sql
@@ -0,0 +1,122 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+USE [FederationStateStore]
+GO
+
+IF NOT EXISTS ( SELECT * FROM [FederationStateStore].sys.tables
+    WHERE name = 'applicationsHomeSubCluster'
+    AND schema_id = SCHEMA_ID('dbo'))
+    BEGIN
+        PRINT 'Table applicationsHomeSubCluster does not exist, create it...'
+
+        SET ANSI_NULLS ON
+
+        SET QUOTED_IDENTIFIER ON
+
+        SET ANSI_PADDING ON
+
+        CREATE TABLE [dbo].[applicationsHomeSubCluster](
+            applicationId   VARCHAR(64) COLLATE Latin1_General_100_BIN2 NOT NULL,
+            homeSubCluster  VARCHAR(256) NOT NULL,
+            createTime      DATETIME2 NOT NULL CONSTRAINT ts_createAppTime DEFAULT GETUTCDATE(),
+
+            CONSTRAINT [pk_applicationId] PRIMARY KEY
+            (
+                [applicationId]
+            )
+        )
+
+        SET ANSI_PADDING OFF
+
+        PRINT 'Table applicationsHomeSubCluster created.'
+    END
+ELSE
+    PRINT 'Table applicationsHomeSubCluster exists, no operation required...'
+    GO
+GO
+
+IF NOT EXISTS ( SELECT * FROM [FederationStateStore].sys.tables
+    WHERE name = 'membership'
+    AND schema_id = SCHEMA_ID('dbo'))
+    BEGIN
+        PRINT 'Table membership does not exist, create it...'
+
+        SET ANSI_NULLS ON
+
+        SET QUOTED_IDENTIFIER ON
+
+        SET ANSI_PADDING ON
+
+        CREATE TABLE [dbo].[membership](
+            [subClusterId]            VARCHAR(256) COLLATE Latin1_General_100_BIN2 NOT NULL,
+            [amRMServiceAddress]      VARCHAR(256) NOT NULL,
+            [clientRMServiceAddress]  VARCHAR(256) NOT NULL,
+            [rmAdminServiceAddress]   VARCHAR(256) NOT NULL,
+            [rmWebServiceAddress]     VARCHAR(256) NOT NULL,
+            [lastHeartBeat]           DATETIME2 NOT NULL,
+            [state]                   VARCHAR(32) NOT NULL,
+            [lastStartTime]           BIGINT NOT NULL,
+            [capability]              VARCHAR(6000) NOT NULL,
+
+            CONSTRAINT [pk_subClusterId] PRIMARY KEY
+            (
+                [subClusterId]
+            )
+        )
+
+        SET ANSI_PADDING OFF
+
+        PRINT 'Table membership created.'
+    END
+ELSE
+    PRINT 'Table membership exists, no operation required...'
+    GO
+GO
+
+IF NOT EXISTS ( SELECT * FROM [FederationStateStore].sys.tables
+    WHERE name = 'policies'
+    AND schema_id = SCHEMA_ID('dbo'))
+    BEGIN
+        PRINT 'Table policies does not exist, create it...'
+
+        SET ANSI_NULLS ON
+
+        SET QUOTED_IDENTIFIER ON
+
+        SET ANSI_PADDING ON
+
+        CREATE TABLE [dbo].[policies](
+            queue       VARCHAR(256) COLLATE Latin1_General_100_BIN2 NOT NULL,
+            policyType  VARCHAR(256) NOT NULL,
+            params      VARBINARY(6000) NOT NULL,
+
+            CONSTRAINT [pk_queue] PRIMARY KEY
+            (
+                [queue]
+            )
+        )
+
+        SET ANSI_PADDING OFF
+
+        PRINT 'Table policies created.'
+    END
+ELSE
+    PRINT 'Table policies exists, no operation required...'
+    GO
+GO


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[29/42] hadoop git commit: YARN-5323. Policies APIs for Federation Router and AMRMProxy policies. (Carlo Curino via Subru).

Posted by su...@apache.org.
YARN-5323. Policies APIs for Federation Router and AMRMProxy policies. (Carlo Curino via Subru).


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/77e39857
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/77e39857
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/77e39857

Branch: refs/heads/YARN-2915
Commit: 77e39857857de8fdf1e6cbf90e990c685557dc43
Parents: 7d17701
Author: Subru Krishnan <su...@apache.org>
Authored: Wed Sep 7 17:33:34 2016 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon May 8 16:59:12 2017 -0700

----------------------------------------------------------------------
 .../policies/ConfigurableFederationPolicy.java  |  44 +++++++
 .../policies/FederationPolicyConfigurator.java  |  91 +++++++++++++
 .../FederationPolicyInitializationContext.java  | 109 ++++++++++++++++
 ...ionPolicyInitializationContextValidator.java |  82 ++++++++++++
 .../policies/FederationPolicyWriter.java        |  45 +++++++
 .../amrmproxy/FederationAMRMProxyPolicy.java    |  66 ++++++++++
 .../policies/amrmproxy/package-info.java        |  20 +++
 .../exceptions/FederationPolicyException.java   |  33 +++++
 ...FederationPolicyInitializationException.java |  33 +++++
 .../NoActiveSubclustersException.java           |  27 ++++
 .../exceptions/UnknownSubclusterException.java  |  28 ++++
 .../policies/exceptions/package-info.java       |  20 +++
 .../federation/policies/package-info.java       |  20 +++
 .../policies/router/FederationRouterPolicy.java |  45 +++++++
 .../policies/router/package-info.java           |  20 +++
 ...ionPolicyInitializationContextValidator.java | 128 +++++++++++++++++++
 .../utils/FederationPoliciesTestUtil.java       |  83 ++++++++++++
 17 files changed, 894 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/77e39857/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/ConfigurableFederationPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/ConfigurableFederationPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/ConfigurableFederationPolicy.java
new file mode 100644
index 0000000..fd6ceea
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/ConfigurableFederationPolicy.java
@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies;
+
+import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
+
+/**
+ * This interface provides a general method to reinitialize a policy. The
+ * semantics are try-n-swap, so in case of an exception is thrown the
+ * implmentation must ensure the previous state and configuration is preserved.
+ */
+public interface ConfigurableFederationPolicy {
+
+  /**
+   * This method is invoked to initialize of update the configuration of
+   * policies. The implementor should provide try-n-swap semantics, and retain
+   * state if possible.
+   *
+   * @param federationPolicyInitializationContext the new context to provide to
+   *                                              implementor.
+   *
+   * @throws FederationPolicyInitializationException in case the initialization
+   *                                                 fails.
+   */
+  void reinitialize(
+      FederationPolicyInitializationContext
+          federationPolicyInitializationContext)
+      throws FederationPolicyInitializationException;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/77e39857/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyConfigurator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyConfigurator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyConfigurator.java
new file mode 100644
index 0000000..fdc3857
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyConfigurator.java
@@ -0,0 +1,91 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies;
+
+import org.apache.hadoop.yarn.server.federation.policies.amrmproxy.FederationAMRMProxyPolicy;
+
+
+import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
+
+import org.apache.hadoop.yarn.server.federation.policies.router
+    .FederationRouterPolicy;
+
+/**
+ * Implementors of this interface are capable to instantiate and (re)initalize
+ * {@link FederationAMRMProxyPolicy} and {@link FederationRouterPolicy} based on
+ * a {@link FederationPolicyInitializationContext}. The reason to bind these two
+ * policies together is to make sure we remain consistent across the router and
+ * amrmproxy policy decisions.
+ */
+public interface FederationPolicyConfigurator {
+
+  /**
+   * If the current instance is compatible, this method returns the same
+   * instance of {@link FederationAMRMProxyPolicy} reinitialized with the
+   * current context, otherwise a new instance initialized with the current
+   * context is provided. If the instance is compatible with the current class
+   * the implementors should attempt to reinitalize (retaining state). To affect
+   * a complete policy reset oldInstance should be null.
+   *
+   * @param federationPolicyInitializationContext the current context
+   * @param oldInstance                           the existing (possibly null)
+   *                                              instance.
+   *
+   * @return an updated {@link FederationAMRMProxyPolicy
+  }.
+   *
+   * @throws FederationPolicyInitializationException if the initialization
+   *                                                 cannot be completed
+   *                                                 properly. The oldInstance
+   *                                                 should be still valid in
+   *                                                 case of failed
+   *                                                 initialization.
+   */
+  FederationAMRMProxyPolicy getAMRMPolicy(
+      FederationPolicyInitializationContext
+          federationPolicyInitializationContext,
+      FederationAMRMProxyPolicy oldInstance)
+      throws FederationPolicyInitializationException;
+
+  /**
+   * If the current instance is compatible, this method returns the same
+   * instance of {@link FederationRouterPolicy} reinitialized with the current
+   * context, otherwise a new instance initialized with the current context is
+   * provided. If the instance is compatible with the current class the
+   * implementors should attempt to reinitalize (retaining state). To affect a
+   * complete policy reset oldInstance shoulb be set to null.
+   *
+   * @param federationPolicyInitializationContext the current context
+   * @param oldInstance                           the existing (possibly null)
+   *                                              instance.
+   *
+   * @return an updated {@link FederationRouterPolicy}.
+   *
+   * @throws FederationPolicyInitializationException if the initalization cannot
+   *                                                 be completed properly. The
+   *                                                 oldInstance should be still
+   *                                                 valid in case of failed
+   *                                                 initialization.
+   */
+  FederationRouterPolicy getRouterPolicy(
+      FederationPolicyInitializationContext
+          federationPolicyInitializationContext,
+      FederationRouterPolicy oldInstance)
+      throws FederationPolicyInitializationException;
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/77e39857/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyInitializationContext.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyInitializationContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyInitializationContext.java
new file mode 100644
index 0000000..879ccee
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyInitializationContext.java
@@ -0,0 +1,109 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies;
+
+import org.apache.hadoop.yarn.server.federation.resolver.SubClusterResolver;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyConfiguration;
+import org.apache.hadoop.yarn.server.federation.utils.FederationStateStoreFacade;
+
+/**
+ * Context to (re)initialize a {@code FederationAMRMProxyPolicy} and {@code
+ * FederationRouterPolicy}.
+ */
+public class FederationPolicyInitializationContext {
+
+  private SubClusterPolicyConfiguration federationPolicyConfiguration;
+  private SubClusterResolver federationSubclusterResolver;
+  private FederationStateStoreFacade federationStateStoreFacade;
+
+  public FederationPolicyInitializationContext() {
+    federationPolicyConfiguration = null;
+    federationSubclusterResolver = null;
+    federationStateStoreFacade = null;
+  }
+
+  public FederationPolicyInitializationContext(SubClusterPolicyConfiguration
+      policy, SubClusterResolver resolver, FederationStateStoreFacade
+      storeFacade) {
+    this.federationPolicyConfiguration = policy;
+    this.federationSubclusterResolver = resolver;
+    this.federationStateStoreFacade = storeFacade;
+  }
+
+
+  /**
+   * Getter for the {@link SubClusterPolicyConfiguration}.
+   *
+   * @return the {@link SubClusterPolicyConfiguration} to be used for
+   * initialization.
+   */
+  public SubClusterPolicyConfiguration getSubClusterPolicyConfiguration() {
+    return federationPolicyConfiguration;
+  }
+
+  /**
+   * Setter for the {@link SubClusterPolicyConfiguration}.
+   *
+   * @param federationPolicyConfiguration the
+   * {@link SubClusterPolicyConfiguration}
+   *                                      to be used for initialization.
+   */
+  public void setFederationPolicyConfiguration(
+      SubClusterPolicyConfiguration federationPolicyConfiguration) {
+    this.federationPolicyConfiguration = federationPolicyConfiguration;
+  }
+
+  /**
+   * Getter for the {@link SubClusterResolver}.
+   *
+   * @return the {@link SubClusterResolver} to be used for initialization.
+   */
+  public SubClusterResolver getFederationSubclusterResolver() {
+    return federationSubclusterResolver;
+  }
+
+  /**
+   * Setter for the {@link SubClusterResolver}.
+   *
+   * @param federationSubclusterResolver the {@link SubClusterResolver} to be
+   *                                     used for initialization.
+   */
+  public void setFederationSubclusterResolver(
+      SubClusterResolver federationSubclusterResolver) {
+    this.federationSubclusterResolver = federationSubclusterResolver;
+  }
+
+  /**
+   * Getter for the {@link FederationStateStoreFacade}.
+   *
+   * @return the facade.
+   */
+  public FederationStateStoreFacade getFederationStateStoreFacade() {
+    return federationStateStoreFacade;
+  }
+
+  /**
+   * Setter for the {@link FederationStateStoreFacade}.
+   *
+   * @param federationStateStoreFacade the facade.
+   */
+  public void setFederationStateStoreFacade(
+      FederationStateStoreFacade federationStateStoreFacade) {
+    this.federationStateStoreFacade = federationStateStoreFacade;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/77e39857/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyInitializationContextValidator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyInitializationContextValidator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyInitializationContextValidator.java
new file mode 100644
index 0000000..31f83d4
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyInitializationContextValidator.java
@@ -0,0 +1,82 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies;
+
+import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
+
+/**
+ * Helper class used to factor out common validation steps for policies.
+ */
+public final class FederationPolicyInitializationContextValidator {
+
+  private FederationPolicyInitializationContextValidator() {
+    //disable constructor per checkstyle
+  }
+
+  public static void validate(
+      FederationPolicyInitializationContext
+          federationPolicyInitializationContext,
+      String myType) throws FederationPolicyInitializationException {
+
+    if (myType == null) {
+      throw new FederationPolicyInitializationException("The myType parameter"
+          + " should not be null.");
+    }
+
+    if (federationPolicyInitializationContext == null) {
+      throw new FederationPolicyInitializationException(
+          "The FederationPolicyInitializationContext provided is null. Cannot"
+              + " reinitalize "
+              + "successfully.");
+    }
+
+    if (federationPolicyInitializationContext.getFederationStateStoreFacade()
+        == null) {
+      throw new FederationPolicyInitializationException(
+          "The FederationStateStoreFacade provided is null. Cannot"
+              + " reinitalize successfully.");
+    }
+
+    if (federationPolicyInitializationContext.getFederationSubclusterResolver()
+        == null) {
+      throw new FederationPolicyInitializationException(
+          "The FederationStateStoreFacase provided is null. Cannot"
+              + " reinitalize successfully.");
+    }
+
+    if (federationPolicyInitializationContext.getSubClusterPolicyConfiguration()
+        == null) {
+      throw new FederationPolicyInitializationException(
+          "The FederationSubclusterResolver provided is null. Cannot "
+              + "reinitalize successfully.");
+    }
+
+    String intendedType =
+        federationPolicyInitializationContext.getSubClusterPolicyConfiguration()
+            .getType();
+
+    if (!myType.equals(intendedType)) {
+      throw new FederationPolicyInitializationException(
+          "The FederationPolicyConfiguration carries a type (" + intendedType
+              + ") different then mine (" + myType
+              + "). Cannot reinitialize successfully.");
+    }
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/77e39857/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyWriter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyWriter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyWriter.java
new file mode 100644
index 0000000..5034b7e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyWriter.java
@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies;
+
+import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyConfiguration;
+
+/**
+ * Implementors of this class are able to serializeConf the configuraiton of a
+ * policy as a {@link SubClusterPolicyConfiguration}. This is used during the
+ * lifetime of a policy from the admin APIs or policy engine to serializeConf
+ * the policy into the policy store.
+ */
+public interface FederationPolicyWriter {
+
+  /**
+   /**
+   * This method is invoked to derive a {@link SubClusterPolicyConfiguration}.
+   * This is to be used when writing a policy object in the federation policy
+   * store.
+   *
+   * @return a valid policy configuration representing this object
+   * parametrization.
+   *
+   * @throws FederationPolicyInitializationException if the current state cannot
+   *                                                 be serialized properly
+   */
+  SubClusterPolicyConfiguration serializeConf()
+      throws FederationPolicyInitializationException;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/77e39857/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/FederationAMRMProxyPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/FederationAMRMProxyPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/FederationAMRMProxyPolicy.java
new file mode 100644
index 0000000..4a3305c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/FederationAMRMProxyPolicy.java
@@ -0,0 +1,66 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies.amrmproxy;
+
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
+import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.federation.policies.ConfigurableFederationPolicy;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Implementors of this interface provide logic to split the list of {@link
+ * ResourceRequest}s received by the AM among various RMs.
+ */
+public interface FederationAMRMProxyPolicy
+    extends ConfigurableFederationPolicy {
+
+  /**
+   * Splits the {@link ResourceRequest}s from the client across one or more
+   * sub-clusters based on the policy semantics (e.g., broadcast, load-based).
+   *
+   * @param resourceRequests the list of {@link ResourceRequest}s from the
+   *                         AM to be split
+   *
+   * @return map of sub-cluster as identified by {@link SubClusterId} to the
+   * list of {@link ResourceRequest}s that should be forwarded to it
+   *
+   * @throws YarnException in case the request is malformed or no viable
+   *                       sub-clusters can be found.
+   */
+  Map<SubClusterId, List<ResourceRequest>> splitResourceRequests(
+      List<ResourceRequest> resourceRequests)
+      throws YarnException;
+
+  /**
+   * This method should be invoked to notify the policy about responses being
+   * received. This is useful for stateful policies that make decisions based on
+   * previous responses being received.
+   *
+   * @param subClusterId the id of the subcluster sending the notification
+   * @param response the response received from one of the RMs
+   *
+   * @throws YarnException in case the response is not valid
+   */
+  void notifyOfResponse(SubClusterId subClusterId,
+      AllocateResponse response) throws YarnException;
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/77e39857/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/package-info.java
new file mode 100644
index 0000000..99da20b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/package-info.java
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/** AMRMPRoxy policies. **/
+package org.apache.hadoop.yarn.server.federation.policies.amrmproxy;
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/77e39857/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/exceptions/FederationPolicyException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/exceptions/FederationPolicyException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/exceptions/FederationPolicyException.java
new file mode 100644
index 0000000..24fe421
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/exceptions/FederationPolicyException.java
@@ -0,0 +1,33 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies.exceptions;
+
+import org.apache.hadoop.yarn.exceptions.YarnException;
+
+/**
+ * Generic policy exception.
+ */
+public class FederationPolicyException extends YarnException {
+  public FederationPolicyException(String s) {
+    super(s);
+  }
+
+  public FederationPolicyException(Throwable t) {
+    super(t);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/77e39857/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/exceptions/FederationPolicyInitializationException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/exceptions/FederationPolicyInitializationException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/exceptions/FederationPolicyInitializationException.java
new file mode 100644
index 0000000..fcc09c2
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/exceptions/FederationPolicyInitializationException.java
@@ -0,0 +1,33 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies.exceptions;
+
+/**
+ * This exception is thrown when the initialization of a federation policy is
+ * not successful.
+ */
+public class FederationPolicyInitializationException
+    extends FederationPolicyException {
+  public FederationPolicyInitializationException(String message) {
+    super(message);
+  }
+
+  public FederationPolicyInitializationException(Throwable j) {
+    super(j);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/77e39857/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/exceptions/NoActiveSubclustersException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/exceptions/NoActiveSubclustersException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/exceptions/NoActiveSubclustersException.java
new file mode 100644
index 0000000..a427944
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/exceptions/NoActiveSubclustersException.java
@@ -0,0 +1,27 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies.exceptions;
+
+/**
+ * This exception is thrown when policies cannot locate any active cluster.
+ */
+public class NoActiveSubclustersException extends FederationPolicyException {
+  public NoActiveSubclustersException(String s) {
+    super(s);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/77e39857/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/exceptions/UnknownSubclusterException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/exceptions/UnknownSubclusterException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/exceptions/UnknownSubclusterException.java
new file mode 100644
index 0000000..8a0fb4f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/exceptions/UnknownSubclusterException.java
@@ -0,0 +1,28 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies.exceptions;
+
+/**
+ * This exception is thrown whenever a policy is given a {@code SubClusterId}
+ * that is unknown.
+ */
+public class UnknownSubclusterException extends FederationPolicyException {
+  public UnknownSubclusterException(String s) {
+    super(s);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/77e39857/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/exceptions/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/exceptions/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/exceptions/package-info.java
new file mode 100644
index 0000000..3318da9
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/exceptions/package-info.java
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/** Exceptions for policies. **/
+package org.apache.hadoop.yarn.server.federation.policies.exceptions;
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/77e39857/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/package-info.java
new file mode 100644
index 0000000..7d9a121
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/package-info.java
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/** Federation Policies. **/
+package org.apache.hadoop.yarn.server.federation.policies;
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/77e39857/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/FederationRouterPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/FederationRouterPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/FederationRouterPolicy.java
new file mode 100644
index 0000000..42c86cc
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/FederationRouterPolicy.java
@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies.router;
+
+import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.federation.policies.ConfigurableFederationPolicy;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+
+/**
+ * Implements the logic for determining the routing of an application submission
+ * based on a policy.
+ */
+public interface FederationRouterPolicy extends ConfigurableFederationPolicy {
+
+  /**
+   * Determines the sub-cluster that the user application submision should be
+   * routed to.
+   *
+   * @param appSubmissionContext the context for the app being submitted.
+   *
+   * @return the sub-cluster as identified by {@link SubClusterId} to route the
+   * request to.
+   *
+   * @throws YarnException if the policy cannot determine a viable subcluster.
+   */
+  SubClusterId getHomeSubcluster(
+      ApplicationSubmissionContext appSubmissionContext)
+      throws YarnException;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/77e39857/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/package-info.java
new file mode 100644
index 0000000..5d0fcb6
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/package-info.java
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/** Router policies. **/
+package org.apache.hadoop.yarn.server.federation.policies.router;
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/77e39857/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestFederationPolicyInitializationContextValidator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestFederationPolicyInitializationContextValidator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestFederationPolicyInitializationContextValidator.java
new file mode 100644
index 0000000..4ec04d5
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestFederationPolicyInitializationContextValidator.java
@@ -0,0 +1,128 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.hadoop.yarn.server.federation.policies;
+
+import org.apache.hadoop.yarn.server.federation.policies.amrmproxy.FederationAMRMProxyPolicy;
+import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
+import org.apache.hadoop.yarn.server.federation.policies.router.FederationRouterPolicy;
+import org.apache.hadoop.yarn.server.federation.resolver.SubClusterResolver;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyConfiguration;
+
+import org.apache.hadoop.yarn.server.federation.utils.FederationPoliciesTestUtil;
+import org.apache.hadoop.yarn.server.federation.utils.FederationStateStoreFacade;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.nio.ByteBuffer;
+
+/**
+ * Test class for {@link FederationPolicyInitializationContextValidator}.
+ */
+public class TestFederationPolicyInitializationContextValidator {
+
+  private SubClusterPolicyConfiguration goodConfig;
+  private SubClusterResolver goodSR;
+  private FederationStateStoreFacade goodFacade;
+  private FederationPolicyInitializationContext context;
+
+  @Before
+  public void setUp() throws Exception {
+    goodFacade = FederationPoliciesTestUtil.initFacade();
+    goodConfig =
+        new MockPolicyManager().serializeConf();
+    goodSR =FederationPoliciesTestUtil.initResolver();
+    context = new
+        FederationPolicyInitializationContext(goodConfig, goodSR, goodFacade);
+  }
+
+  @Test
+  public void correcInit() throws Exception {
+    FederationPolicyInitializationContextValidator.validate(context,
+        MockPolicyManager.class.getCanonicalName());
+  }
+
+  @Test(expected = FederationPolicyInitializationException.class)
+  public void nullContext() throws Exception {
+    FederationPolicyInitializationContextValidator.validate(null,
+        MockPolicyManager.class.getCanonicalName());
+  }
+
+  @Test(expected = FederationPolicyInitializationException.class)
+  public void nullType() throws Exception {
+    FederationPolicyInitializationContextValidator.validate(context, null);
+  }
+
+  @Test(expected = FederationPolicyInitializationException.class)
+  public void wrongType() throws Exception {
+    FederationPolicyInitializationContextValidator.validate(context,
+        "WrongType");
+  }
+
+  @Test(expected = FederationPolicyInitializationException.class)
+  public void nullConf() throws Exception {
+    context.setFederationPolicyConfiguration(null);
+    FederationPolicyInitializationContextValidator.validate(context,
+        MockPolicyManager.class.getCanonicalName());
+  }
+
+  @Test(expected = FederationPolicyInitializationException.class)
+  public void nullResolver() throws Exception {
+    context.setFederationSubclusterResolver(null);
+    FederationPolicyInitializationContextValidator.validate(context,
+        MockPolicyManager.class.getCanonicalName());
+  }
+
+  @Test(expected = FederationPolicyInitializationException.class)
+  public void nullFacade() throws Exception {
+    context.setFederationStateStoreFacade(null);
+    FederationPolicyInitializationContextValidator.validate(context,
+        MockPolicyManager.class.getCanonicalName());
+  }
+
+  private class MockPolicyManager
+      implements FederationPolicyWriter, FederationPolicyConfigurator {
+    @Override
+    public FederationAMRMProxyPolicy getAMRMPolicy(
+        FederationPolicyInitializationContext
+            federationPolicyInitializationContext,
+        FederationAMRMProxyPolicy oldInstance)
+        throws FederationPolicyInitializationException {
+      return null;
+    }
+
+    @Override
+    public FederationRouterPolicy getRouterPolicy(
+        FederationPolicyInitializationContext
+            federationPolicyInitializationContext,
+        FederationRouterPolicy oldInstance)
+        throws FederationPolicyInitializationException {
+      return null;
+    }
+
+    @Override
+    public SubClusterPolicyConfiguration serializeConf()
+        throws FederationPolicyInitializationException {
+      ByteBuffer buf = ByteBuffer.allocate(0);
+      return SubClusterPolicyConfiguration
+          .newInstance("queue1", this.getClass().getCanonicalName(), buf);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/77e39857/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/FederationPoliciesTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/FederationPoliciesTestUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/FederationPoliciesTestUtil.java
new file mode 100644
index 0000000..8c2115b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/FederationPoliciesTestUtil.java
@@ -0,0 +1,83 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.yarn.server.federation.utils;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.federation.resolver.DefaultSubClusterResolverImpl;
+import org.apache.hadoop.yarn.server.federation.resolver.SubClusterResolver;
+import org.apache.hadoop.yarn.server.federation.store.FederationStateStore;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClustersInfoResponse;
+
+import java.net.URL;
+import java.util.ArrayList;
+
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * Support class providing common initialization methods to test federation
+ * policies.
+ */
+public final class FederationPoliciesTestUtil {
+
+  private FederationPoliciesTestUtil() {
+    // disabled.
+  }
+
+  /**
+   * Initialize a {@link SubClusterResolver}.
+   *
+   * @return a subcluster resolver for tests.
+   */
+  public static SubClusterResolver initResolver() {
+    YarnConfiguration conf = new YarnConfiguration();
+    SubClusterResolver resolver =
+        new DefaultSubClusterResolverImpl();
+    URL url =
+        Thread.currentThread().getContextClassLoader().getResource("nodes");
+    if (url == null) {
+      throw new RuntimeException(
+          "Could not find 'nodes' dummy file in classpath");
+    }
+    conf.set(YarnConfiguration.FEDERATION_MACHINE_LIST, url.getPath());
+    resolver.setConf(conf);
+    resolver.load();
+    return resolver;
+  }
+
+  /**
+   * Initialiaze a main-memory {@link FederationStateStoreFacade} used for
+   * testing, wiht a mock resolver.
+   *
+   * @return the facade.
+   *
+   * @throws YarnException in case the initialization is not successful.
+   */
+  public static FederationStateStoreFacade initFacade() throws YarnException {
+    FederationStateStoreFacade goodFacade = FederationStateStoreFacade
+        .getInstance();
+    FederationStateStore fss = mock(FederationStateStore.class);
+    GetSubClustersInfoResponse response = GetSubClustersInfoResponse
+        .newInstance(new ArrayList<>());
+    when(fss.getSubClusters(any())).thenReturn(response);
+    goodFacade.reinitialize(fss, new Configuration());
+    return goodFacade;
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[15/42] hadoop git commit: YARN-3664. Federation PolicyStore internal APIs

Posted by su...@apache.org.
YARN-3664. Federation PolicyStore internal APIs


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/33dfdb70
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/33dfdb70
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/33dfdb70

Branch: refs/heads/YARN-2915
Commit: 33dfdb70e30556f1e25b28699e534de990d36650
Parents: 048df39
Author: Subru Krishnan <su...@apache.org>
Authored: Fri Aug 5 12:34:58 2016 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon May 8 16:59:12 2017 -0700

----------------------------------------------------------------------
 .../federation/store/FederationPolicyStore.java |  76 ++++++++
 ...SubClusterPoliciesConfigurationsRequest.java |  35 ++++
 ...ubClusterPoliciesConfigurationsResponse.java |  66 +++++++
 ...GetSubClusterPolicyConfigurationRequest.java |  62 ++++++
 ...etSubClusterPolicyConfigurationResponse.java |  65 +++++++
 ...SetSubClusterPolicyConfigurationRequest.java |  79 ++++++++
 ...etSubClusterPolicyConfigurationResponse.java |  36 ++++
 .../records/SubClusterPolicyConfiguration.java  | 130 +++++++++++++
 ...sterPoliciesConfigurationsRequestPBImpl.java |  95 +++++++++
 ...terPoliciesConfigurationsResponsePBImpl.java | 191 +++++++++++++++++++
 ...ClusterPolicyConfigurationRequestPBImpl.java | 103 ++++++++++
 ...lusterPolicyConfigurationResponsePBImpl.java | 143 ++++++++++++++
 .../pb/GetSubClustersInfoResponsePBImpl.java    |   4 +-
 ...ClusterPolicyConfigurationRequestPBImpl.java | 159 +++++++++++++++
 ...lusterPolicyConfigurationResponsePBImpl.java |  93 +++++++++
 .../pb/SubClusterPolicyConfigurationPBImpl.java | 121 ++++++++++++
 .../proto/yarn_server_federation_protos.proto   |  28 +++
 .../records/TestFederationProtocolRecords.java  |  53 ++++-
 18 files changed, 1536 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/33dfdb70/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/FederationPolicyStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/FederationPolicyStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/FederationPolicyStore.java
new file mode 100644
index 0000000..9d9bd9b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/FederationPolicyStore.java
@@ -0,0 +1,76 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterPoliciesConfigurationsRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterPoliciesConfigurationsResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterPolicyConfigurationRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterPolicyConfigurationResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.SetSubClusterPolicyConfigurationRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.SetSubClusterPolicyConfigurationResponse;
+
+/**
+ * The FederationPolicyStore provides a key-value interface to access the
+ * policies configured for the system. The key is a "queue" name, i.e., the
+ * system allows to configure a different policy for each queue in the system
+ * (though each policy can make dynamic run-time decisions on a per-job/per-task
+ * basis). The value is a {@code SubClusterPolicyConfiguration}, a serialized
+ * representation of the policy type and its parameters.
+ */
+@Private
+@Unstable
+public interface FederationPolicyStore {
+
+  /**
+   * Get the policy configuration for a given queue.
+   *
+   * @param request the queue whose {@code SubClusterPolicyConfiguration} is
+   *          required
+   * @return the {@code SubClusterPolicyConfiguration} for the specified queue
+   * @throws YarnException if the request is invalid/fails
+   */
+  GetSubClusterPolicyConfigurationResponse getPolicyConfiguration(
+      GetSubClusterPolicyConfigurationRequest request) throws YarnException;
+
+  /**
+   * Set the policy configuration for a given queue.
+   *
+   * @param request the {@code SubClusterPolicyConfiguration} with the
+   *          corresponding queue
+   * @return response empty on successfully updating the
+   *         {@code SubClusterPolicyConfiguration} for the specified queue
+   * @throws YarnException if the request is invalid/fails
+   */
+  SetSubClusterPolicyConfigurationResponse setPolicyConfiguration(
+      SetSubClusterPolicyConfigurationRequest request) throws YarnException;
+
+  /**
+   * Get a map of all queue-to-policy configurations.
+   *
+   * @param request empty to represent all configured queues in the system
+   * @return the policies for all currently active queues in the system
+   * @throws YarnException if the request is invalid/fails
+   */
+  GetSubClusterPoliciesConfigurationsResponse getPoliciesConfigurations(
+      GetSubClusterPoliciesConfigurationsRequest request) throws YarnException;
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/33dfdb70/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetSubClusterPoliciesConfigurationsRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetSubClusterPoliciesConfigurationsRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetSubClusterPoliciesConfigurationsRequest.java
new file mode 100644
index 0000000..404521b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetSubClusterPoliciesConfigurationsRequest.java
@@ -0,0 +1,35 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * GetSubClusterPoliciesConfigurationsRequest is a request to the
+ * {@code FederationPolicyStore} to obtain all policy configurations.
+ */
+@Private
+@Unstable
+public abstract class GetSubClusterPoliciesConfigurationsRequest {
+  public GetSubClusterPoliciesConfigurationsRequest newInstance() {
+    return Records.newRecord(GetSubClusterPoliciesConfigurationsRequest.class);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/33dfdb70/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetSubClusterPoliciesConfigurationsResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetSubClusterPoliciesConfigurationsResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetSubClusterPoliciesConfigurationsResponse.java
new file mode 100644
index 0000000..6554d68
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetSubClusterPoliciesConfigurationsResponse.java
@@ -0,0 +1,66 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records;
+
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * GetSubClusterPolicyConfigurationResponse contains the answer from the {@code
+ * FederationPolicyStore} to a request to get all the policies configured in the
+ * system via a {@link SubClusterPolicyConfiguration}.
+ */
+@Private
+@Unstable
+public abstract class GetSubClusterPoliciesConfigurationsResponse {
+
+  @Private
+  @Unstable
+  public GetSubClusterPoliciesConfigurationsResponse newInstance(
+      List<SubClusterPolicyConfiguration> policyConfigurations) {
+    GetSubClusterPoliciesConfigurationsResponse response =
+        Records.newRecord(GetSubClusterPoliciesConfigurationsResponse.class);
+    response.setPoliciesConfigs(policyConfigurations);
+    return response;
+  }
+
+  /**
+   * Get all the policies configured in the system.
+   *
+   * @return all the policies configured in the system
+   */
+  @Public
+  @Unstable
+  public abstract List<SubClusterPolicyConfiguration> getPoliciesConfigs();
+
+  /**
+   * Sets all the policies configured in the system.
+   *
+   * @param policyConfigurations all the policies configured in the system
+   */
+  @Private
+  @Unstable
+  public abstract void setPoliciesConfigs(
+      List<SubClusterPolicyConfiguration> policyConfigurations);
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/33dfdb70/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetSubClusterPolicyConfigurationRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetSubClusterPolicyConfigurationRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetSubClusterPolicyConfigurationRequest.java
new file mode 100644
index 0000000..7b7d8c4
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetSubClusterPolicyConfigurationRequest.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * GetSubClusterPolicyConfigurationRequest is a request to the
+ * {@code FederationPolicyStore} to get the configuration of a policy for a
+ * given queue.
+ */
+@Private
+@Unstable
+public abstract class GetSubClusterPolicyConfigurationRequest {
+
+  @Private
+  @Unstable
+  public GetSubClusterPolicyConfigurationRequest newInstance(String queueName) {
+    GetSubClusterPolicyConfigurationRequest request =
+        Records.newRecord(GetSubClusterPolicyConfigurationRequest.class);
+    request.setQueue(queueName);
+    return request;
+  }
+
+  /**
+   * Get the name of the queue for which we are requesting a policy
+   * configuration.
+   *
+   * @return the name of the queue
+   */
+  @Public
+  @Unstable
+  public abstract String getQueue();
+
+  /**
+   * Sets the name of the queue for which we are requesting a policy
+   * configuration.
+   *
+   * @param queueName the name of the queue
+   */
+  @Private
+  @Unstable
+  public abstract void setQueue(String queueName);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/33dfdb70/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetSubClusterPolicyConfigurationResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetSubClusterPolicyConfigurationResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetSubClusterPolicyConfigurationResponse.java
new file mode 100644
index 0000000..11a46e0
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetSubClusterPolicyConfigurationResponse.java
@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * GetSubClusterPolicyConfigurationResponse contains the answer from the {@code
+ * FederationPolicyStore} to a request to get the information about how a policy
+ * should be configured via a {@link SubClusterPolicyConfiguration}.
+ */
+@Private
+@Unstable
+public abstract class GetSubClusterPolicyConfigurationResponse {
+
+  @Private
+  @Unstable
+  public GetSubClusterPolicyConfigurationResponse newInstance(
+      SubClusterPolicyConfiguration policy) {
+    GetSubClusterPolicyConfigurationResponse response =
+        Records.newRecord(GetSubClusterPolicyConfigurationResponse.class);
+    response.setPolicyConfiguration(policy);
+    return response;
+  }
+
+  /**
+   * Get the policy configuration.
+   *
+   * @return the policy configuration for the specified queue
+   */
+  @Public
+  @Unstable
+  public abstract SubClusterPolicyConfiguration getPolicyConfiguration();
+
+  /**
+   * Sets the policyConfiguration configuration.
+   *
+   * @param policyConfiguration the policyConfiguration configuration for the
+   *          specified queue
+   */
+  @Private
+  @Unstable
+  public abstract void setPolicyConfiguration(
+      SubClusterPolicyConfiguration policyConfiguration);
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/33dfdb70/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SetSubClusterPolicyConfigurationRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SetSubClusterPolicyConfigurationRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SetSubClusterPolicyConfigurationRequest.java
new file mode 100644
index 0000000..06d5399
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SetSubClusterPolicyConfigurationRequest.java
@@ -0,0 +1,79 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * SetSubClusterPolicyConfigurationRequest is a request to the
+ * {@code FederationPolicyStore} to set the policy configuration corresponding
+ * to a queue.
+ */
+@Private
+@Unstable
+public abstract class SetSubClusterPolicyConfigurationRequest {
+  @Private
+  @Unstable
+  public SetSubClusterPolicyConfigurationRequest newInstance(
+      SubClusterPolicyConfiguration policy) {
+    SetSubClusterPolicyConfigurationRequest request =
+        Records.newRecord(SetSubClusterPolicyConfigurationRequest.class);
+    request.setPolicyConfiguration(policy);
+    return request;
+  }
+
+  /**
+   * Get the name of the queue for which we are configuring a policy.
+   *
+   * @return the name of the queue
+   */
+  @Public
+  @Unstable
+  public abstract String getQueue();
+
+  /**
+   * Sets the name of the queue for which we are configuring a policy.
+   *
+   * @param queueName the name of the queue
+   */
+  @Private
+  @Unstable
+  public abstract void setQueue(String queueName);
+
+  /**
+   * Get the policy configuration assigned to the queue.
+   *
+   * @return the policy for the specified queue
+   */
+  @Public
+  @Unstable
+  public abstract SubClusterPolicyConfiguration getPolicyConfiguration();
+
+  /**
+   * Set the policyConfiguration configuration for the queue.
+   *
+   * @param policyConfiguration the policyConfiguration for the specified queue
+   */
+  @Private
+  @Unstable
+  public abstract void setPolicyConfiguration(
+      SubClusterPolicyConfiguration policyConfiguration);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/33dfdb70/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SetSubClusterPolicyConfigurationResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SetSubClusterPolicyConfigurationResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SetSubClusterPolicyConfigurationResponse.java
new file mode 100644
index 0000000..33c4043
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SetSubClusterPolicyConfigurationResponse.java
@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * SetSubClusterPolicyConfigurationResponse contains the answer from the
+ * {@code FederationPolicyStore} to a request to set for a policy configuration
+ * for a given queue.
+ */
+@Private
+@Unstable
+public abstract class SetSubClusterPolicyConfigurationResponse {
+  public SetSubClusterPolicyConfigurationResponse newInstance() {
+    return Records.newRecord(SetSubClusterPolicyConfigurationResponse.class);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/33dfdb70/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterPolicyConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterPolicyConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterPolicyConfiguration.java
new file mode 100644
index 0000000..bc12acb
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterPolicyConfiguration.java
@@ -0,0 +1,130 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+import java.nio.ByteBuffer;
+
+// used in javadoc
+
+/**
+ * {@link SubClusterPolicyConfiguration} is a class that represents a
+ * configuration of a policy. It contains a policy type (resolve to a class
+ * name) and its params as an opaque {@link ByteBuffer}.
+ *
+ * Note: by design the params are an opaque ByteBuffer, this allows for enough
+ * flexibility to evolve the policies without impacting the protocols to/from
+ * the federation state store.
+ */
+@Private
+@Unstable
+public abstract class SubClusterPolicyConfiguration {
+
+  @Private
+  @Unstable
+  public static SubClusterPolicyConfiguration newInstance(String policyType,
+      ByteBuffer policyParams) {
+    SubClusterPolicyConfiguration policy =
+        Records.newRecord(SubClusterPolicyConfiguration.class);
+    policy.setType(policyType);
+    policy.setParams(policyParams);
+    return policy;
+  }
+
+  /**
+   * Get the type of the policy. This could be random, round-robin, load-based,
+   * etc.
+   *
+   * @return the type of the policy
+   */
+  @Public
+  @Unstable
+  public abstract String getType();
+
+  /**
+   * Sets the type of the policy. This could be random, round-robin, load-based,
+   * etc.
+   *
+   * @param policyType the type of the policy
+   */
+  @Private
+  @Unstable
+  public abstract void setType(String policyType);
+
+  /**
+   * Get the policy parameters. This affects how the policy behaves and an
+   * example could be weight distribution of queues across multiple
+   * sub-clusters.
+   *
+   * @return the byte array that contains the parameters
+   */
+  @Public
+  @Unstable
+  public abstract ByteBuffer getParams();
+
+  /**
+   * Set the policy parameters. This affects how the policy behaves and an
+   * example could be weight distribution of queues across multiple
+   * sub-clusters.
+   *
+   * @param policyParams byte array that describes the policy
+   */
+  @Private
+  @Unstable
+  public abstract void setParams(ByteBuffer policyParams);
+
+  @Override
+  public int hashCode() {
+    return 31 * getParams().hashCode() + getType().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (getClass() != obj.getClass()) {
+      return false;
+    }
+    SubClusterPolicyConfiguration other = (SubClusterPolicyConfiguration) obj;
+    if (!this.getType().equals(other.getType())) {
+      return false;
+    }
+    if (!this.getParams().equals(other.getParams())) {
+      return false;
+    }
+    return true;
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append(getType());
+    sb.append(" : ");
+    sb.append(getParams());
+    return sb.toString();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/33dfdb70/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/GetSubClusterPoliciesConfigurationsRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/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/GetSubClusterPoliciesConfigurationsRequestPBImpl.java b/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/GetSubClusterPoliciesConfigurationsRequestPBImpl.java
new file mode 100644
index 0000000..3cb6601
--- /dev/null
+++ b/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/GetSubClusterPoliciesConfigurationsRequestPBImpl.java
@@ -0,0 +1,95 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records.impl.pb;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.GetSubClusterPoliciesConfigurationsRequestProto;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterPoliciesConfigurationsRequest;
+
+import com.google.protobuf.TextFormat;
+
+/**
+ * Protocol buffer based implementation of
+ * {@link GetSubClusterPoliciesConfigurationsRequest}.
+ */
+@Private
+@Unstable
+public class GetSubClusterPoliciesConfigurationsRequestPBImpl
+    extends GetSubClusterPoliciesConfigurationsRequest {
+
+  private GetSubClusterPoliciesConfigurationsRequestProto proto =
+      GetSubClusterPoliciesConfigurationsRequestProto.getDefaultInstance();
+  private GetSubClusterPoliciesConfigurationsRequestProto.Builder builder =
+      null;
+  private boolean viaProto = false;
+
+  public GetSubClusterPoliciesConfigurationsRequestPBImpl() {
+    builder = GetSubClusterPoliciesConfigurationsRequestProto.newBuilder();
+  }
+
+  public GetSubClusterPoliciesConfigurationsRequestPBImpl(
+      GetSubClusterPoliciesConfigurationsRequestProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public GetSubClusterPoliciesConfigurationsRequestProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder =
+          GetSubClusterPoliciesConfigurationsRequestProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null) {
+      return false;
+    }
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/33dfdb70/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/GetSubClusterPoliciesConfigurationsResponsePBImpl.java
----------------------------------------------------------------------
diff --git a/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/GetSubClusterPoliciesConfigurationsResponsePBImpl.java b/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/GetSubClusterPoliciesConfigurationsResponsePBImpl.java
new file mode 100644
index 0000000..67c3654
--- /dev/null
+++ b/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/GetSubClusterPoliciesConfigurationsResponsePBImpl.java
@@ -0,0 +1,191 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records.impl.pb;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.GetSubClusterPoliciesConfigurationsResponseProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.GetSubClusterPoliciesConfigurationsResponseProtoOrBuilder;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SubClusterPolicyConfigurationProto;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterPoliciesConfigurationsResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyConfiguration;
+
+import com.google.protobuf.TextFormat;
+
+/**
+ * Protocol buffer based implementation of
+ * {@link GetSubClusterPoliciesConfigurationsResponse}.
+ */
+@Private
+@Unstable
+public class GetSubClusterPoliciesConfigurationsResponsePBImpl
+    extends GetSubClusterPoliciesConfigurationsResponse {
+
+  private GetSubClusterPoliciesConfigurationsResponseProto proto =
+      GetSubClusterPoliciesConfigurationsResponseProto.getDefaultInstance();
+  private GetSubClusterPoliciesConfigurationsResponseProto.Builder builder =
+      null;
+  private boolean viaProto = false;
+
+  private List<SubClusterPolicyConfiguration> subClusterPolicies = null;
+
+  public GetSubClusterPoliciesConfigurationsResponsePBImpl() {
+    builder = GetSubClusterPoliciesConfigurationsResponseProto.newBuilder();
+  }
+
+  public GetSubClusterPoliciesConfigurationsResponsePBImpl(
+      GetSubClusterPoliciesConfigurationsResponseProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public GetSubClusterPoliciesConfigurationsResponseProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder =
+          GetSubClusterPoliciesConfigurationsResponseProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.subClusterPolicies != null) {
+      addSubClusterPoliciesConfigurationsToProto();
+    }
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null) {
+      return false;
+    }
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
+  }
+
+  @Override
+  public List<SubClusterPolicyConfiguration> getPoliciesConfigs() {
+    initSubClusterPoliciesConfigurationsList();
+    return this.subClusterPolicies;
+  }
+
+  @Override
+  public void setPoliciesConfigs(
+      List<SubClusterPolicyConfiguration> policyConfigurations) {
+    maybeInitBuilder();
+    if (policyConfigurations == null) {
+      builder.clearPoliciesConfigurations();
+    }
+    this.subClusterPolicies = policyConfigurations;
+  }
+
+  private void initSubClusterPoliciesConfigurationsList() {
+    if (this.subClusterPolicies != null) {
+      return;
+    }
+    GetSubClusterPoliciesConfigurationsResponseProtoOrBuilder p =
+        viaProto ? proto : builder;
+    List<SubClusterPolicyConfigurationProto> subClusterPoliciesList =
+        p.getPoliciesConfigurationsList();
+    subClusterPolicies = new ArrayList<SubClusterPolicyConfiguration>();
+
+    for (SubClusterPolicyConfigurationProto r : subClusterPoliciesList) {
+      subClusterPolicies.add(convertFromProtoFormat(r));
+    }
+  }
+
+  private void addSubClusterPoliciesConfigurationsToProto() {
+    maybeInitBuilder();
+    builder.clearPoliciesConfigurations();
+    if (subClusterPolicies == null) {
+      return;
+    }
+    Iterable<SubClusterPolicyConfigurationProto> iterable =
+        new Iterable<SubClusterPolicyConfigurationProto>() {
+          @Override
+          public Iterator<SubClusterPolicyConfigurationProto> iterator() {
+            return new Iterator<SubClusterPolicyConfigurationProto>() {
+
+              private Iterator<SubClusterPolicyConfiguration> iter =
+                  subClusterPolicies.iterator();
+
+              @Override
+              public boolean hasNext() {
+                return iter.hasNext();
+              }
+
+              @Override
+              public SubClusterPolicyConfigurationProto next() {
+                return convertToProtoFormat(iter.next());
+              }
+
+              @Override
+              public void remove() {
+                throw new UnsupportedOperationException();
+              }
+
+            };
+
+          }
+
+        };
+    builder.addAllPoliciesConfigurations(iterable);
+  }
+
+  private SubClusterPolicyConfiguration convertFromProtoFormat(
+      SubClusterPolicyConfigurationProto policy) {
+    return new SubClusterPolicyConfigurationPBImpl(policy);
+  }
+
+  private SubClusterPolicyConfigurationProto convertToProtoFormat(
+      SubClusterPolicyConfiguration policy) {
+    return ((SubClusterPolicyConfigurationPBImpl) policy).getProto();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/33dfdb70/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/GetSubClusterPolicyConfigurationRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/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/GetSubClusterPolicyConfigurationRequestPBImpl.java b/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/GetSubClusterPolicyConfigurationRequestPBImpl.java
new file mode 100644
index 0000000..35aff17
--- /dev/null
+++ b/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/GetSubClusterPolicyConfigurationRequestPBImpl.java
@@ -0,0 +1,103 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records.impl.pb;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.GetSubClusterPolicyConfigurationRequestProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.GetSubClusterPolicyConfigurationRequestProtoOrBuilder;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterPolicyConfigurationRequest;
+
+import com.google.protobuf.TextFormat;
+
+/**
+ * Protocol buffer based implementation of
+ * {@link GetSubClusterPolicyConfigurationRequest}.
+ */
+@Private
+@Unstable
+public class GetSubClusterPolicyConfigurationRequestPBImpl
+    extends GetSubClusterPolicyConfigurationRequest {
+
+  private GetSubClusterPolicyConfigurationRequestProto proto =
+      GetSubClusterPolicyConfigurationRequestProto.getDefaultInstance();
+  private GetSubClusterPolicyConfigurationRequestProto.Builder builder = null;
+  private boolean viaProto = false;
+
+  public GetSubClusterPolicyConfigurationRequestPBImpl() {
+    builder = GetSubClusterPolicyConfigurationRequestProto.newBuilder();
+  }
+
+  public GetSubClusterPolicyConfigurationRequestPBImpl(
+      GetSubClusterPolicyConfigurationRequestProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public GetSubClusterPolicyConfigurationRequestProto getProto() {
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = GetSubClusterPolicyConfigurationRequestProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null) {
+      return false;
+    }
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
+  }
+
+  @Override
+  public String getQueue() {
+    GetSubClusterPolicyConfigurationRequestProtoOrBuilder p =
+        viaProto ? proto : builder;
+    return p.getQueue();
+  }
+
+  @Override
+  public void setQueue(String queueName) {
+    maybeInitBuilder();
+    if (queueName == null) {
+      builder.clearQueue();
+      return;
+    }
+    builder.setQueue(queueName);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/33dfdb70/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/GetSubClusterPolicyConfigurationResponsePBImpl.java
----------------------------------------------------------------------
diff --git a/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/GetSubClusterPolicyConfigurationResponsePBImpl.java b/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/GetSubClusterPolicyConfigurationResponsePBImpl.java
new file mode 100644
index 0000000..96b88ae
--- /dev/null
+++ b/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/GetSubClusterPolicyConfigurationResponsePBImpl.java
@@ -0,0 +1,143 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records.impl.pb;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.GetSubClusterPolicyConfigurationResponseProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.GetSubClusterPolicyConfigurationResponseProtoOrBuilder;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SubClusterPolicyConfigurationProto;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterPolicyConfigurationResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyConfiguration;
+
+import com.google.protobuf.TextFormat;
+
+/**
+ * Protocol buffer based implementation of
+ * {@link GetSubClusterPolicyConfigurationResponse}.
+ */
+@Private
+@Unstable
+public class GetSubClusterPolicyConfigurationResponsePBImpl
+    extends GetSubClusterPolicyConfigurationResponse {
+
+  private GetSubClusterPolicyConfigurationResponseProto proto =
+      GetSubClusterPolicyConfigurationResponseProto.getDefaultInstance();
+  private GetSubClusterPolicyConfigurationResponseProto.Builder builder = null;
+  private boolean viaProto = false;
+
+  private SubClusterPolicyConfiguration subClusterPolicy = null;
+
+  public GetSubClusterPolicyConfigurationResponsePBImpl() {
+    builder = GetSubClusterPolicyConfigurationResponseProto.newBuilder();
+  }
+
+  public GetSubClusterPolicyConfigurationResponsePBImpl(
+      GetSubClusterPolicyConfigurationResponseProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public GetSubClusterPolicyConfigurationResponseProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = GetSubClusterPolicyConfigurationResponseProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.subClusterPolicy != null
+        && !((SubClusterPolicyConfigurationPBImpl) this.subClusterPolicy)
+            .getProto().equals(builder.getPolicyConfiguration())) {
+      builder
+          .setPolicyConfiguration(convertToProtoFormat(this.subClusterPolicy));
+    }
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null) {
+      return false;
+    }
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
+  }
+
+  @Override
+  public SubClusterPolicyConfiguration getPolicyConfiguration() {
+    GetSubClusterPolicyConfigurationResponseProtoOrBuilder p =
+        viaProto ? proto : builder;
+    if (this.subClusterPolicy != null) {
+      return this.subClusterPolicy;
+    }
+    if (!p.hasPolicyConfiguration()) {
+      return null;
+    }
+    this.subClusterPolicy = convertFromProtoFormat(p.getPolicyConfiguration());
+    return this.subClusterPolicy;
+  }
+
+  @Override
+  public void setPolicyConfiguration(
+      SubClusterPolicyConfiguration policyConfiguration) {
+    maybeInitBuilder();
+    if (policyConfiguration == null) {
+      builder.clearPolicyConfiguration();
+    }
+    this.subClusterPolicy = policyConfiguration;
+  }
+
+  private SubClusterPolicyConfiguration convertFromProtoFormat(
+      SubClusterPolicyConfigurationProto policy) {
+    return new SubClusterPolicyConfigurationPBImpl(policy);
+  }
+
+  private SubClusterPolicyConfigurationProto convertToProtoFormat(
+      SubClusterPolicyConfiguration policy) {
+    return ((SubClusterPolicyConfigurationPBImpl) policy).getProto();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/33dfdb70/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/GetSubClustersInfoResponsePBImpl.java
----------------------------------------------------------------------
diff --git a/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/GetSubClustersInfoResponsePBImpl.java b/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/GetSubClustersInfoResponsePBImpl.java
index 92bdf06..2efa3b7 100644
--- a/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/GetSubClustersInfoResponsePBImpl.java
+++ b/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/GetSubClustersInfoResponsePBImpl.java
@@ -66,7 +66,7 @@ public class GetSubClustersInfoResponsePBImpl
 
   private void mergeLocalToBuilder() {
     if (this.subClusterInfos != null) {
-      addReservationResourcesToProto();
+      addSubClusterInfosToProto();
     }
   }
 
@@ -114,7 +114,7 @@ public class GetSubClustersInfoResponsePBImpl
     }
   }
 
-  private void addReservationResourcesToProto() {
+  private void addSubClusterInfosToProto() {
     maybeInitBuilder();
     builder.clearSubClusterInfos();
     if (subClusterInfos == null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/33dfdb70/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/SetSubClusterPolicyConfigurationRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/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/SetSubClusterPolicyConfigurationRequestPBImpl.java b/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/SetSubClusterPolicyConfigurationRequestPBImpl.java
new file mode 100644
index 0000000..5e29bd5
--- /dev/null
+++ b/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/SetSubClusterPolicyConfigurationRequestPBImpl.java
@@ -0,0 +1,159 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records.impl.pb;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SetSubClusterPolicyConfigurationRequestProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SetSubClusterPolicyConfigurationRequestProtoOrBuilder;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SubClusterPolicyConfigurationProto;
+import org.apache.hadoop.yarn.server.federation.store.records.SetSubClusterPolicyConfigurationRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyConfiguration;
+
+import com.google.protobuf.TextFormat;
+
+/**
+ * Protocol buffer based implementation of
+ * {@link SetSubClusterPolicyConfigurationRequest}.
+ */
+@Private
+@Unstable
+public class SetSubClusterPolicyConfigurationRequestPBImpl
+    extends SetSubClusterPolicyConfigurationRequest {
+
+  private SetSubClusterPolicyConfigurationRequestProto proto =
+      SetSubClusterPolicyConfigurationRequestProto.getDefaultInstance();
+  private SetSubClusterPolicyConfigurationRequestProto.Builder builder = null;
+  private boolean viaProto = false;
+
+  private SubClusterPolicyConfiguration subClusterPolicy = null;
+
+  public SetSubClusterPolicyConfigurationRequestPBImpl() {
+    builder = SetSubClusterPolicyConfigurationRequestProto.newBuilder();
+  }
+
+  public SetSubClusterPolicyConfigurationRequestPBImpl(
+      SetSubClusterPolicyConfigurationRequestProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public SetSubClusterPolicyConfigurationRequestProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = SetSubClusterPolicyConfigurationRequestProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.subClusterPolicy != null
+        && !((SubClusterPolicyConfigurationPBImpl) this.subClusterPolicy)
+            .getProto().equals(builder.getPolicyConfiguration())) {
+      builder
+          .setPolicyConfiguration(convertToProtoFormat(this.subClusterPolicy));
+    }
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null) {
+      return false;
+    }
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
+  }
+
+  @Override
+  public String getQueue() {
+    SetSubClusterPolicyConfigurationRequestProtoOrBuilder p =
+        viaProto ? proto : builder;
+    return p.getQueue();
+  }
+
+  @Override
+  public void setQueue(String queueName) {
+    maybeInitBuilder();
+    if (queueName == null) {
+      builder.clearQueue();
+      return;
+    }
+    builder.setQueue(queueName);
+  }
+
+  @Override
+  public SubClusterPolicyConfiguration getPolicyConfiguration() {
+    SetSubClusterPolicyConfigurationRequestProtoOrBuilder p =
+        viaProto ? proto : builder;
+    if (this.subClusterPolicy != null) {
+      return this.subClusterPolicy;
+    }
+    if (!p.hasPolicyConfiguration()) {
+      return null;
+    }
+    this.subClusterPolicy = convertFromProtoFormat(p.getPolicyConfiguration());
+    return this.subClusterPolicy;
+  }
+
+  @Override
+  public void setPolicyConfiguration(
+      SubClusterPolicyConfiguration policyConfiguration) {
+    maybeInitBuilder();
+    if (policyConfiguration == null) {
+      builder.clearPolicyConfiguration();
+    }
+    this.subClusterPolicy = policyConfiguration;
+  }
+
+  private SubClusterPolicyConfiguration convertFromProtoFormat(
+      SubClusterPolicyConfigurationProto policy) {
+    return new SubClusterPolicyConfigurationPBImpl(policy);
+  }
+
+  private SubClusterPolicyConfigurationProto convertToProtoFormat(
+      SubClusterPolicyConfiguration policy) {
+    return ((SubClusterPolicyConfigurationPBImpl) policy).getProto();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/33dfdb70/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/SetSubClusterPolicyConfigurationResponsePBImpl.java
----------------------------------------------------------------------
diff --git a/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/SetSubClusterPolicyConfigurationResponsePBImpl.java b/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/SetSubClusterPolicyConfigurationResponsePBImpl.java
new file mode 100644
index 0000000..9d79215
--- /dev/null
+++ b/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/SetSubClusterPolicyConfigurationResponsePBImpl.java
@@ -0,0 +1,93 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records.impl.pb;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SetSubClusterPolicyConfigurationResponseProto;
+import org.apache.hadoop.yarn.server.federation.store.records.SetSubClusterPolicyConfigurationResponse;
+
+import com.google.protobuf.TextFormat;
+
+/**
+ * Protocol buffer based implementation of
+ * {@link SetSubClusterPolicyConfigurationResponse}.
+ */
+@Private
+@Unstable
+public class SetSubClusterPolicyConfigurationResponsePBImpl
+    extends SetSubClusterPolicyConfigurationResponse {
+
+  private SetSubClusterPolicyConfigurationResponseProto proto =
+      SetSubClusterPolicyConfigurationResponseProto.getDefaultInstance();
+  private SetSubClusterPolicyConfigurationResponseProto.Builder builder = null;
+  private boolean viaProto = false;
+
+  public SetSubClusterPolicyConfigurationResponsePBImpl() {
+    builder = SetSubClusterPolicyConfigurationResponseProto.newBuilder();
+  }
+
+  public SetSubClusterPolicyConfigurationResponsePBImpl(
+      SetSubClusterPolicyConfigurationResponseProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public SetSubClusterPolicyConfigurationResponseProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = SetSubClusterPolicyConfigurationResponseProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null) {
+      return false;
+    }
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/33dfdb70/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/SubClusterPolicyConfigurationPBImpl.java
----------------------------------------------------------------------
diff --git a/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/SubClusterPolicyConfigurationPBImpl.java b/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/SubClusterPolicyConfigurationPBImpl.java
new file mode 100644
index 0000000..fe9d9db
--- /dev/null
+++ b/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/SubClusterPolicyConfigurationPBImpl.java
@@ -0,0 +1,121 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records.impl.pb;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SubClusterPolicyConfigurationProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SubClusterPolicyConfigurationProtoOrBuilder;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyConfiguration;
+
+import com.google.protobuf.TextFormat;
+
+/**
+ * Protobuf based implementation of {@link SubClusterPolicyConfiguration}.
+ *
+ */
+@Private
+@Unstable
+public class SubClusterPolicyConfigurationPBImpl
+    extends SubClusterPolicyConfiguration {
+
+  private SubClusterPolicyConfigurationProto proto =
+      SubClusterPolicyConfigurationProto.getDefaultInstance();
+  private SubClusterPolicyConfigurationProto.Builder builder = null;
+  private boolean viaProto = false;
+
+  public SubClusterPolicyConfigurationPBImpl() {
+    builder = SubClusterPolicyConfigurationProto.newBuilder();
+  }
+
+  public SubClusterPolicyConfigurationPBImpl(
+      SubClusterPolicyConfigurationProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public SubClusterPolicyConfigurationProto getProto() {
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = SubClusterPolicyConfigurationProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null) {
+      return false;
+    }
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
+  }
+
+  @Override
+  public String getType() {
+    SubClusterPolicyConfigurationProtoOrBuilder p = viaProto ? proto : builder;
+    return p.getType();
+  }
+
+  @Override
+  public void setType(String policyType) {
+    maybeInitBuilder();
+    if (policyType == null) {
+      builder.clearType();
+      return;
+    }
+    builder.setType(policyType);
+  }
+
+  @Override
+  public ByteBuffer getParams() {
+    SubClusterPolicyConfigurationProtoOrBuilder p = viaProto ? proto : builder;
+    return ProtoUtils.convertFromProtoFormat(p.getParams());
+  }
+
+  @Override
+  public void setParams(ByteBuffer policyParams) {
+    maybeInitBuilder();
+    if (policyParams == null) {
+      builder.clearParams();
+      return;
+    }
+    builder.setParams(ProtoUtils.convertToProtoFormat(policyParams));
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/33dfdb70/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_federation_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_federation_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_federation_protos.proto
index b1ad310..3f1cee9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_federation_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_federation_protos.proto
@@ -134,3 +134,31 @@ message DeleteApplicationHomeSubClusterRequestProto {
 
 message DeleteApplicationHomeSubClusterResponseProto {
 }
+
+message SubClusterPolicyConfigurationProto {
+  optional string type = 1;
+  optional bytes params = 2;
+}
+
+message GetSubClusterPolicyConfigurationRequestProto {
+  optional string queue = 1;
+}
+
+message GetSubClusterPolicyConfigurationResponseProto {
+  optional SubClusterPolicyConfigurationProto policy_configuration = 1;
+}
+
+message SetSubClusterPolicyConfigurationRequestProto {
+  optional string queue = 1;
+  optional SubClusterPolicyConfigurationProto policy_configuration = 2;
+}
+
+message SetSubClusterPolicyConfigurationResponseProto {
+}
+
+message GetSubClusterPoliciesConfigurationsRequestProto {
+}
+
+message GetSubClusterPoliciesConfigurationsResponseProto {
+  repeated SubClusterPolicyConfigurationProto policies_configurations = 1;
+ }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/33dfdb70/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/records/TestFederationProtocolRecords.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/records/TestFederationProtocolRecords.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/records/TestFederationProtocolRecords.java
index 210a246..cf8cf71 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/records/TestFederationProtocolRecords.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/records/TestFederationProtocolRecords.java
@@ -29,8 +29,14 @@ import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.GetApp
 import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.GetApplicationsHomeSubClusterResponseProto;
 import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.GetSubClusterInfoRequestProto;
 import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.GetSubClusterInfoResponseProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.GetSubClusterPoliciesConfigurationsRequestProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.GetSubClusterPoliciesConfigurationsResponseProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.GetSubClusterPolicyConfigurationRequestProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.GetSubClusterPolicyConfigurationResponseProto;
 import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.GetSubClustersInfoRequestProto;
 import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.GetSubClustersInfoResponseProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SetSubClusterPolicyConfigurationRequestProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SetSubClusterPolicyConfigurationResponseProto;
 import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SubClusterDeregisterRequestProto;
 import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SubClusterDeregisterResponseProto;
 import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SubClusterHeartbeatRequestProto;
@@ -51,8 +57,14 @@ import org.apache.hadoop.yarn.server.federation.store.records.impl.pb.GetApplica
 import org.apache.hadoop.yarn.server.federation.store.records.impl.pb.GetApplicationsHomeSubClusterResponsePBImpl;
 import org.apache.hadoop.yarn.server.federation.store.records.impl.pb.GetSubClusterInfoRequestPBImpl;
 import org.apache.hadoop.yarn.server.federation.store.records.impl.pb.GetSubClusterInfoResponsePBImpl;
+import org.apache.hadoop.yarn.server.federation.store.records.impl.pb.GetSubClusterPoliciesConfigurationsRequestPBImpl;
+import org.apache.hadoop.yarn.server.federation.store.records.impl.pb.GetSubClusterPoliciesConfigurationsResponsePBImpl;
+import org.apache.hadoop.yarn.server.federation.store.records.impl.pb.GetSubClusterPolicyConfigurationRequestPBImpl;
+import org.apache.hadoop.yarn.server.federation.store.records.impl.pb.GetSubClusterPolicyConfigurationResponsePBImpl;
 import org.apache.hadoop.yarn.server.federation.store.records.impl.pb.GetSubClustersInfoRequestPBImpl;
 import org.apache.hadoop.yarn.server.federation.store.records.impl.pb.GetSubClustersInfoResponsePBImpl;
+import org.apache.hadoop.yarn.server.federation.store.records.impl.pb.SetSubClusterPolicyConfigurationRequestPBImpl;
+import org.apache.hadoop.yarn.server.federation.store.records.impl.pb.SetSubClusterPolicyConfigurationResponsePBImpl;
 import org.apache.hadoop.yarn.server.federation.store.records.impl.pb.SubClusterDeregisterRequestPBImpl;
 import org.apache.hadoop.yarn.server.federation.store.records.impl.pb.SubClusterDeregisterResponsePBImpl;
 import org.apache.hadoop.yarn.server.federation.store.records.impl.pb.SubClusterHeartbeatRequestPBImpl;
@@ -79,6 +91,7 @@ public class TestFederationProtocolRecords extends BasePBImplRecordsTest {
     generateByNewInstance(SubClusterId.class);
     generateByNewInstance(SubClusterInfo.class);
     generateByNewInstance(ApplicationHomeSubCluster.class);
+    generateByNewInstance(SubClusterPolicyConfiguration.class);
   }
 
   @Test
@@ -211,4 +224,42 @@ public class TestFederationProtocolRecords extends BasePBImplRecordsTest {
         DeleteApplicationHomeSubClusterResponseProto.class);
   }
 
-}
\ No newline at end of file
+  @Test
+  public void testGetSubClusterPolicyConfigurationRequest() throws Exception {
+    validatePBImplRecord(GetSubClusterPolicyConfigurationRequestPBImpl.class,
+        GetSubClusterPolicyConfigurationRequestProto.class);
+  }
+
+  @Test
+  public void testGetSubClusterPolicyConfigurationResponse() throws Exception {
+    validatePBImplRecord(GetSubClusterPolicyConfigurationResponsePBImpl.class,
+        GetSubClusterPolicyConfigurationResponseProto.class);
+  }
+
+  @Test
+  public void testSetSubClusterPolicyConfigurationRequest() throws Exception {
+    validatePBImplRecord(SetSubClusterPolicyConfigurationRequestPBImpl.class,
+        SetSubClusterPolicyConfigurationRequestProto.class);
+  }
+
+  @Test
+  public void testSetSubClusterPolicyConfigurationResponse() throws Exception {
+    validatePBImplRecord(SetSubClusterPolicyConfigurationResponsePBImpl.class,
+        SetSubClusterPolicyConfigurationResponseProto.class);
+  }
+
+  @Test
+  public void testGetSubClusterPoliciesConfigurationsRequest()
+      throws Exception {
+    validatePBImplRecord(GetSubClusterPoliciesConfigurationsRequestPBImpl.class,
+        GetSubClusterPoliciesConfigurationsRequestProto.class);
+  }
+
+  @Test
+  public void testGetSubClusterPoliciesConfigurationsResponse()
+      throws Exception {
+    validatePBImplRecord(
+        GetSubClusterPoliciesConfigurationsResponsePBImpl.class,
+        GetSubClusterPoliciesConfigurationsResponseProto.class);
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[11/42] hadoop git commit: YARN-5307. Federation Application State Store internal APIs

Posted by su...@apache.org.
YARN-5307. Federation Application State Store internal APIs


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/048df39b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/048df39b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/048df39b

Branch: refs/heads/YARN-2915
Commit: 048df39b28776ba67eadc02b911656eb37757b78
Parents: 7986923
Author: Subru Krishnan <su...@apache.org>
Authored: Fri Aug 5 11:52:44 2016 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon May 8 16:59:12 2017 -0700

----------------------------------------------------------------------
 ...ederationApplicationHomeSubClusterStore.java | 126 ++++++++++++
 .../AddApplicationHomeSubClusterRequest.java    |  72 +++++++
 .../AddApplicationHomeSubClusterResponse.java   |  44 +++++
 .../records/ApplicationHomeSubCluster.java      | 124 ++++++++++++
 .../DeleteApplicationHomeSubClusterRequest.java |  65 +++++++
 ...DeleteApplicationHomeSubClusterResponse.java |  43 +++++
 .../GetApplicationHomeSubClusterRequest.java    |  64 +++++++
 .../GetApplicationHomeSubClusterResponse.java   |  73 +++++++
 .../GetApplicationsHomeSubClusterRequest.java   |  40 ++++
 .../GetApplicationsHomeSubClusterResponse.java  |  75 ++++++++
 .../UpdateApplicationHomeSubClusterRequest.java |  74 ++++++++
 ...UpdateApplicationHomeSubClusterResponse.java |  43 +++++
 ...dApplicationHomeSubClusterRequestPBImpl.java | 132 +++++++++++++
 ...ApplicationHomeSubClusterResponsePBImpl.java |  78 ++++++++
 .../pb/ApplicationHomeSubClusterPBImpl.java     | 167 ++++++++++++++++
 ...eApplicationHomeSubClusterRequestPBImpl.java | 130 +++++++++++++
 ...ApplicationHomeSubClusterResponsePBImpl.java |  78 ++++++++
 ...tApplicationHomeSubClusterRequestPBImpl.java | 135 +++++++++++++
 ...ApplicationHomeSubClusterResponsePBImpl.java | 132 +++++++++++++
 ...ApplicationsHomeSubClusterRequestPBImpl.java |  78 ++++++++
 ...pplicationsHomeSubClusterResponsePBImpl.java | 190 +++++++++++++++++++
 .../pb/GetSubClustersInfoResponsePBImpl.java    |   6 +-
 ...eApplicationHomeSubClusterRequestPBImpl.java | 132 +++++++++++++
 ...ApplicationHomeSubClusterResponsePBImpl.java |  78 ++++++++
 .../proto/yarn_server_federation_protos.proto   |  45 ++++-
 .../records/TestFederationProtocolRecords.java  |  81 ++++++++
 26 files changed, 2301 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/048df39b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/FederationApplicationHomeSubClusterStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/FederationApplicationHomeSubClusterStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/FederationApplicationHomeSubClusterStore.java
new file mode 100644
index 0000000..217ee2e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/FederationApplicationHomeSubClusterStore.java
@@ -0,0 +1,126 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.federation.store.records.AddApplicationHomeSubClusterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.AddApplicationHomeSubClusterResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.DeleteApplicationHomeSubClusterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.DeleteApplicationHomeSubClusterResponse;
+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.server.federation.store.records.GetApplicationsHomeSubClusterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.GetApplicationsHomeSubClusterResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.UpdateApplicationHomeSubClusterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.UpdateApplicationHomeSubClusterResponse;
+import org.apache.hadoop.yarn.server.records.Version;
+
+/**
+ * FederationApplicationHomeSubClusterStore maintains the state of all
+ * <em>Applications</em> that have been submitted to the federated cluster.
+ *
+ * *
+ * <p>
+ * The mapping details contains:
+ * <ul>
+ * <li>{@code ApplicationId}</li>
+ * <li>{@code SubClusterId}</li>
+ * </ul>
+ *
+ */
+@Private
+@Unstable
+public interface FederationApplicationHomeSubClusterStore {
+
+  /**
+   * Get the {@link Version} of the underlying federation application state
+   * store.
+   *
+   * @return the {@link Version} of the underlying federation application state
+   *         store
+   */
+  Version getApplicationStateStoreVersion();
+
+  /**
+   * Register the home {@code SubClusterId} of the newly submitted
+   * {@code ApplicationId}. Currently response is empty if the operation was
+   * successful, if not an exception reporting reason for a failure.
+   *
+   * @param request the request to register a new application with its home
+   *          sub-cluster
+   * @return empty on successful registration of the application in the
+   *         StateStore, if not an exception reporting reason for a failure
+   * @throws YarnException if the request is invalid/fails
+   */
+  AddApplicationHomeSubClusterResponse addApplicationHomeSubClusterMap(
+      AddApplicationHomeSubClusterRequest request) throws YarnException;
+
+  /**
+   * Update the home {@code SubClusterId} of a previously submitted
+   * {@code ApplicationId}. Currently response is empty if the operation was
+   * successful, if not an exception reporting reason for a failure.
+   *
+   * @param request the request to update the home sub-cluster of an
+   *          application.
+   * @return empty on successful update of the application in the StateStore, if
+   *         not an exception reporting reason for a failure
+   * @throws YarnException if the request is invalid/fails
+   */
+  UpdateApplicationHomeSubClusterResponse updateApplicationHomeSubClusterMap(
+      UpdateApplicationHomeSubClusterRequest request) throws YarnException;
+
+  /**
+   * Get information about the application identified by the input
+   * {@code ApplicationId}.
+   *
+   * @param request contains the application queried
+   * @return {@code ApplicationHomeSubCluster} containing the application's
+   *         home subcluster
+   * @throws YarnException if the request is invalid/fails
+   */
+  GetApplicationHomeSubClusterResponse getApplicationHomeSubClusterMap(
+      GetApplicationHomeSubClusterRequest request) throws YarnException;
+
+  /**
+   * Get the {@code ApplicationHomeSubCluster} list representing the mapping
+   * of all submitted applications to it's home sub-cluster.
+   *
+   * @param request empty representing all applications
+   * @return the mapping of all submitted application to it's home sub-cluster
+   * @throws YarnException if the request is invalid/fails
+   */
+  GetApplicationsHomeSubClusterResponse getApplicationsHomeSubClusterMap(
+      GetApplicationsHomeSubClusterRequest request) throws YarnException;
+
+  /**
+   * Delete the mapping of home {@code SubClusterId} of a previously submitted
+   * {@code ApplicationId}. Currently response is empty if the operation was
+   * successful, if not an exception reporting reason for a failure.
+   *
+   * @param request the request to delete the home sub-cluster of an
+   *          application.
+   * @return empty on successful update of the application in the StateStore, if
+   *         not an exception reporting reason for a failure
+   * @throws YarnException if the request is invalid/fails
+   */
+  DeleteApplicationHomeSubClusterResponse deleteApplicationHomeSubClusterMap(
+      DeleteApplicationHomeSubClusterRequest request) throws YarnException;
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/048df39b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/AddApplicationHomeSubClusterRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/AddApplicationHomeSubClusterRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/AddApplicationHomeSubClusterRequest.java
new file mode 100644
index 0000000..9cb0589
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/AddApplicationHomeSubClusterRequest.java
@@ -0,0 +1,72 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * <p>
+ * The request sent by the <code>Router</code> to <code>Federation state
+ * store</code> to map the home subcluster of a newly submitted application.
+ *
+ * <p>
+ * The request includes the mapping details, i.e.:
+ * <ul>
+ * <li>{@code ApplicationId}</li>
+ * <li>{@code SubClusterId}</li>
+ * </ul>
+ */
+@Private
+@Unstable
+public abstract class AddApplicationHomeSubClusterRequest {
+
+  @Private
+  @Unstable
+  public static AddApplicationHomeSubClusterRequest newInstance(
+      ApplicationHomeSubCluster applicationHomeSubCluster) {
+    AddApplicationHomeSubClusterRequest mapRequest =
+        Records.newRecord(AddApplicationHomeSubClusterRequest.class);
+    mapRequest.setApplicationHomeSubCluster(applicationHomeSubCluster);
+    return mapRequest;
+  }
+
+  /**
+   * Get the {@link ApplicationHomeSubCluster} representing the mapping of the
+   * application to it's home sub-cluster.
+   *
+   * @return the mapping of the application to it's home sub-cluster.
+   */
+  @Public
+  @Unstable
+  public abstract ApplicationHomeSubCluster getApplicationHomeSubCluster();
+
+  /**
+   * Set the {@link ApplicationHomeSubCluster} representing the mapping of the
+   * application to it's home sub-cluster.
+   *
+   * @param applicationHomeSubCluster the mapping of the application to it's
+   *          home sub-cluster.
+   */
+  @Private
+  @Unstable
+  public abstract void setApplicationHomeSubCluster(
+      ApplicationHomeSubCluster applicationHomeSubCluster);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/048df39b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/AddApplicationHomeSubClusterResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/AddApplicationHomeSubClusterResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/AddApplicationHomeSubClusterResponse.java
new file mode 100644
index 0000000..2145dd1
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/AddApplicationHomeSubClusterResponse.java
@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * AddApplicationHomeSubClusterResponse contains the answer from the
+ * {@code FederationApplicationHomeSubClusterStore} to a request to insert a
+ * newly generated applicationId and its owner. Currently response is empty if
+ * the operation was successful, if not an exception reporting reason for a
+ * failure.
+ *
+ */
+@Private
+@Unstable
+public abstract class AddApplicationHomeSubClusterResponse {
+
+  @Private
+  @Unstable
+  public static AddApplicationHomeSubClusterResponse newInstance() {
+    AddApplicationHomeSubClusterResponse response =
+        Records.newRecord(AddApplicationHomeSubClusterResponse.class);
+    return response;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/048df39b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/ApplicationHomeSubCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/ApplicationHomeSubCluster.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/ApplicationHomeSubCluster.java
new file mode 100644
index 0000000..5e4c7cc
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/ApplicationHomeSubCluster.java
@@ -0,0 +1,124 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * <p>
+ * ApplicationHomeSubCluster is a report of the runtime information of the
+ * application that is running in the federated cluster.
+ *
+ * <p>
+ * It includes information such as:
+ * <ul>
+ * <li>{@link ApplicationId}</li>
+ * <li>{@link SubClusterId}</li>
+ * </ul>
+ *
+ */
+@Private
+@Unstable
+public abstract class ApplicationHomeSubCluster {
+
+  @Private
+  @Unstable
+  public static ApplicationHomeSubCluster newInstance(ApplicationId appId,
+      SubClusterId homeSubCluster) {
+    ApplicationHomeSubCluster appMapping =
+        Records.newRecord(ApplicationHomeSubCluster.class);
+    appMapping.setApplicationId(appId);
+    appMapping.setHomeSubCluster(homeSubCluster);
+    return appMapping;
+  }
+
+  /**
+   * Get the {@link ApplicationId} representing the unique identifier of the
+   * application.
+   *
+   * @return the application identifier
+   */
+  @Public
+  @Unstable
+  public abstract ApplicationId getApplicationId();
+
+  /**
+   * Set the {@link ApplicationId} representing the unique identifier of the
+   * application.
+   *
+   * @param applicationId the application identifier
+   */
+  @Private
+  @Unstable
+  public abstract void setApplicationId(ApplicationId applicationId);
+
+  /**
+   * Get the {@link SubClusterId} representing the unique identifier of the home
+   * subcluster in which the ApplicationMaster of the application is running.
+   *
+   * @return the home subcluster identifier
+   */
+  @Public
+  @Unstable
+  public abstract SubClusterId getHomeSubCluster();
+
+  /**
+   * Set the {@link SubClusterId} representing the unique identifier of the home
+   * subcluster in which the ApplicationMaster of the application is running.
+   *
+   * @param homeSubCluster the home subcluster identifier
+   */
+  @Private
+  @Unstable
+  public abstract void setHomeSubCluster(SubClusterId homeSubCluster);
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (getClass() != obj.getClass()) {
+      return false;
+    }
+    ApplicationHomeSubCluster other = (ApplicationHomeSubCluster) obj;
+    if (!this.getApplicationId().equals(other.getApplicationId())) {
+      return false;
+    }
+    return this.getHomeSubCluster().equals(other.getHomeSubCluster());
+  }
+
+  @Override
+  public int hashCode() {
+    return getApplicationId().hashCode() * 31 + getHomeSubCluster().hashCode();
+  }
+
+  @Override
+  public String toString() {
+    return "ApplicationHomeSubCluster [getApplicationId()="
+        + getApplicationId() + ", getHomeSubCluster()=" + getHomeSubCluster()
+        + "]";
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/048df39b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/DeleteApplicationHomeSubClusterRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/DeleteApplicationHomeSubClusterRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/DeleteApplicationHomeSubClusterRequest.java
new file mode 100644
index 0000000..f678aee
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/DeleteApplicationHomeSubClusterRequest.java
@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * The request to <code>Federation state store</code> to delete the mapping of
+ * home subcluster of a submitted application.
+ */
+@Private
+@Unstable
+public abstract class DeleteApplicationHomeSubClusterRequest {
+
+  @Private
+  @Unstable
+  public static DeleteApplicationHomeSubClusterRequest newInstance(
+      ApplicationId applicationId) {
+    DeleteApplicationHomeSubClusterRequest deleteApplicationRequest =
+        Records.newRecord(DeleteApplicationHomeSubClusterRequest.class);
+    deleteApplicationRequest.setApplicationId(applicationId);
+    return deleteApplicationRequest;
+  }
+
+  /**
+   * Get the identifier of the {@link ApplicationId} to be removed from
+   * <code>Federation state store</code> .
+   *
+   * @return the identifier of the application to be removed from Federation
+   *         State Store.
+   */
+  @Public
+  @Unstable
+  public abstract ApplicationId getApplicationId();
+
+  /**
+   * Set the identifier of the {@link ApplicationId} to be removed from
+   * <code>Federation state store</code> .
+   *
+   * @param applicationId the identifier of the application to be removed from
+   *          Federation State Store.
+   */
+  @Private
+  @Unstable
+  public abstract void setApplicationId(ApplicationId applicationId);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/048df39b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/DeleteApplicationHomeSubClusterResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/DeleteApplicationHomeSubClusterResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/DeleteApplicationHomeSubClusterResponse.java
new file mode 100644
index 0000000..fb1bef9
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/DeleteApplicationHomeSubClusterResponse.java
@@ -0,0 +1,43 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * DeleteApplicationHomeSubClusterResponse contains the answer from the {@code
+ * FederationApplicationHomeSubClusterStore} to a request to delete the mapping
+ * of home subcluster of a submitted application. Currently response is empty if
+ * the operation was successful, if not an exception reporting reason for a
+ * failure.
+ */
+@Private
+@Unstable
+public abstract class DeleteApplicationHomeSubClusterResponse {
+
+  @Private
+  @Unstable
+  public static DeleteApplicationHomeSubClusterResponse newInstance() {
+    DeleteApplicationHomeSubClusterResponse response =
+        Records.newRecord(DeleteApplicationHomeSubClusterResponse.class);
+    return response;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/048df39b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetApplicationHomeSubClusterRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetApplicationHomeSubClusterRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetApplicationHomeSubClusterRequest.java
new file mode 100644
index 0000000..a64d22e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetApplicationHomeSubClusterRequest.java
@@ -0,0 +1,64 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * Request class to obtain the home sub-cluster for the specified
+ * {@link ApplicationId}.
+ */
+@Private
+@Unstable
+public abstract class GetApplicationHomeSubClusterRequest {
+
+  @Private
+  @Unstable
+  public static GetApplicationHomeSubClusterRequest newInstance(
+      ApplicationId appId) {
+    GetApplicationHomeSubClusterRequest appMapping =
+        Records.newRecord(GetApplicationHomeSubClusterRequest.class);
+    appMapping.setApplicationId(appId);
+    return appMapping;
+  }
+
+  /**
+   * Get the {@link ApplicationId} representing the unique identifier of the
+   * application.
+   *
+   * @return the application identifier
+   */
+  @Public
+  @Unstable
+  public abstract ApplicationId getApplicationId();
+
+  /**
+   * Set the {@link ApplicationId} representing the unique identifier of the
+   * application.
+   *
+   * @param applicationId the application identifier
+   */
+  @Private
+  @Unstable
+  public abstract void setApplicationId(ApplicationId applicationId);
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/048df39b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetApplicationHomeSubClusterResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetApplicationHomeSubClusterResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetApplicationHomeSubClusterResponse.java
new file mode 100644
index 0000000..60735b3
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetApplicationHomeSubClusterResponse.java
@@ -0,0 +1,73 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * <p>
+ * The response sent by <code>Federation state
+ * store</code> to a query for the home subcluster of a newly submitted
+ * application.
+ *
+ * <p>
+ * The request includes the mapping details, i.e.:
+ * <ul>
+ * <li>{@code ApplicationId}</li>
+ * <li>{@code SubClusterId}</li>
+ * </ul>
+ */
+@Private
+@Unstable
+public abstract class GetApplicationHomeSubClusterResponse {
+
+  @Private
+  @Unstable
+  public static GetApplicationHomeSubClusterResponse newInstance(
+      ApplicationHomeSubCluster applicationHomeSubCluster) {
+    GetApplicationHomeSubClusterResponse mapResponse =
+        Records.newRecord(GetApplicationHomeSubClusterResponse.class);
+    mapResponse.setApplicationHomeSubCluster(applicationHomeSubCluster);
+    return mapResponse;
+  }
+
+  /**
+   * Get the {@link ApplicationHomeSubCluster} representing the mapping of the
+   * application to it's home sub-cluster.
+   *
+   * @return the mapping of the application to it's home sub-cluster.
+   */
+  @Public
+  @Unstable
+  public abstract ApplicationHomeSubCluster getApplicationHomeSubCluster();
+
+  /**
+   * Set the {@link ApplicationHomeSubCluster} representing the mapping of the
+   * application to it's home sub-cluster.
+   *
+   * @param applicationHomeSubCluster the mapping of the application to it's
+   *          home sub-cluster.
+   */
+  @Private
+  @Unstable
+  public abstract void setApplicationHomeSubCluster(
+      ApplicationHomeSubCluster applicationHomeSubCluster);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/048df39b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetApplicationsHomeSubClusterRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetApplicationsHomeSubClusterRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetApplicationsHomeSubClusterRequest.java
new file mode 100644
index 0000000..6054972
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetApplicationsHomeSubClusterRequest.java
@@ -0,0 +1,40 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * Request class to obtain the home sub-cluster mapping of all active
+ * applications.
+ */
+@Private
+@Unstable
+public abstract class GetApplicationsHomeSubClusterRequest {
+
+  @Private
+  @Unstable
+  public static GetApplicationsHomeSubClusterRequest newInstance() {
+    GetApplicationsHomeSubClusterRequest request =
+        Records.newRecord(GetApplicationsHomeSubClusterRequest.class);
+    return request;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/048df39b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetApplicationsHomeSubClusterResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetApplicationsHomeSubClusterResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetApplicationsHomeSubClusterResponse.java
new file mode 100644
index 0000000..ba3d2c6
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/GetApplicationsHomeSubClusterResponse.java
@@ -0,0 +1,75 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records;
+
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * <p>
+ * The response sent by <code>Federation state
+ * store</code> to a query for the home subcluster of all submitted
+ * applications.
+ *
+ * <p>
+ * The response includes the mapping details, i.e.:
+ * <ul>
+ * <li>{@code ApplicationId}</li>
+ * <li>{@code SubClusterId}</li>
+ * </ul>
+ */
+@Private
+@Unstable
+public abstract class GetApplicationsHomeSubClusterResponse {
+
+  @Private
+  @Unstable
+  public static GetApplicationsHomeSubClusterResponse newInstance(
+      List<ApplicationHomeSubCluster> appsHomeSubClusters) {
+    GetApplicationsHomeSubClusterResponse mapResponse =
+        Records.newRecord(GetApplicationsHomeSubClusterResponse.class);
+    mapResponse.setAppsHomeSubClusters(appsHomeSubClusters);
+    return mapResponse;
+  }
+
+  /**
+   * Get the {@link ApplicationHomeSubCluster} list representing the mapping of
+   * all submitted applications to it's home sub-cluster.
+   *
+   * @return the mapping of all submitted application to it's home sub-cluster.
+   */
+  @Public
+  @Unstable
+  public abstract List<ApplicationHomeSubCluster> getAppsHomeSubClusters();
+
+  /**
+   * Set the {@link ApplicationHomeSubCluster} list representing the mapping of
+   * all submitted applications to it's home sub-cluster.
+   *
+   * @param appsHomeSubClusters the mapping of all submitted application to it's
+   *          home sub-cluster.
+   */
+  @Private
+  @Unstable
+  public abstract void setAppsHomeSubClusters(
+      List<ApplicationHomeSubCluster> appsHomeSubClusters);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/048df39b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/UpdateApplicationHomeSubClusterRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/UpdateApplicationHomeSubClusterRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/UpdateApplicationHomeSubClusterRequest.java
new file mode 100644
index 0000000..eaa9252
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/UpdateApplicationHomeSubClusterRequest.java
@@ -0,0 +1,74 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * <p>
+ * The request sent by the <code>Router</code> to
+ * <code>Federation state store</code> to update the home subcluster of a newly
+ * submitted application.
+ *
+ * <p>
+ * The request includes the mapping details, i.e.:
+ * <ul>
+ * <li>{@code ApplicationId}</li>
+ * <li>{@code SubClusterId}</li>
+ * </ul>
+ */
+@Private
+@Unstable
+public abstract class UpdateApplicationHomeSubClusterRequest {
+
+  @Private
+  @Unstable
+  public static UpdateApplicationHomeSubClusterRequest newInstance(
+      ApplicationHomeSubCluster applicationHomeSubCluster) {
+    UpdateApplicationHomeSubClusterRequest updateApplicationRequest =
+        Records.newRecord(UpdateApplicationHomeSubClusterRequest.class);
+    updateApplicationRequest
+        .setApplicationHomeSubCluster(applicationHomeSubCluster);
+    return updateApplicationRequest;
+  }
+
+  /**
+   * Get the {@link ApplicationHomeSubCluster} representing the mapping of the
+   * application to it's home sub-cluster.
+   *
+   * @return the mapping of the application to it's home sub-cluster.
+   */
+  @Public
+  @Unstable
+  public abstract ApplicationHomeSubCluster getApplicationHomeSubCluster();
+
+  /**
+   * Set the {@link ApplicationHomeSubCluster} representing the mapping of the
+   * application to it's home sub-cluster.
+   *
+   * @param applicationHomeSubCluster the mapping of the application to it's
+   *          home sub-cluster.
+   */
+  @Private
+  @Unstable
+  public abstract void setApplicationHomeSubCluster(
+      ApplicationHomeSubCluster applicationHomeSubCluster);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/048df39b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/UpdateApplicationHomeSubClusterResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/UpdateApplicationHomeSubClusterResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/UpdateApplicationHomeSubClusterResponse.java
new file mode 100644
index 0000000..7434335
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/UpdateApplicationHomeSubClusterResponse.java
@@ -0,0 +1,43 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * UpdateApplicationHomeSubClusterResponse contains the answer from the
+ * {@code FederationApplicationHomeSubClusterStore} to a request to register the
+ * home subcluster of a submitted application. Currently response is empty if
+ * the operation was successful, if not an exception reporting reason for a
+ * failure.
+ */
+@Private
+@Unstable
+public abstract class UpdateApplicationHomeSubClusterResponse {
+
+  @Private
+  @Unstable
+  public static UpdateApplicationHomeSubClusterResponse newInstance() {
+    UpdateApplicationHomeSubClusterResponse response =
+        Records.newRecord(UpdateApplicationHomeSubClusterResponse.class);
+    return response;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/048df39b/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/AddApplicationHomeSubClusterRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/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/AddApplicationHomeSubClusterRequestPBImpl.java b/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/AddApplicationHomeSubClusterRequestPBImpl.java
new file mode 100644
index 0000000..2387cde
--- /dev/null
+++ b/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/AddApplicationHomeSubClusterRequestPBImpl.java
@@ -0,0 +1,132 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records.impl.pb;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.AddApplicationHomeSubClusterRequestProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.AddApplicationHomeSubClusterRequestProtoOrBuilder;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.ApplicationHomeSubClusterProto;
+import org.apache.hadoop.yarn.server.federation.store.records.AddApplicationHomeSubClusterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.ApplicationHomeSubCluster;
+
+import com.google.protobuf.TextFormat;
+
+/**
+ * Protocol buffer based implementation of
+ * {@link AddApplicationHomeSubClusterRequest}.
+ */
+@Private
+@Unstable
+public class AddApplicationHomeSubClusterRequestPBImpl
+    extends AddApplicationHomeSubClusterRequest {
+
+  private AddApplicationHomeSubClusterRequestProto proto =
+      AddApplicationHomeSubClusterRequestProto.getDefaultInstance();
+  private AddApplicationHomeSubClusterRequestProto.Builder builder = null;
+  private boolean viaProto = false;
+
+  public AddApplicationHomeSubClusterRequestPBImpl() {
+    builder = AddApplicationHomeSubClusterRequestProto.newBuilder();
+  }
+
+  public AddApplicationHomeSubClusterRequestPBImpl(
+      AddApplicationHomeSubClusterRequestProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public AddApplicationHomeSubClusterRequestProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = AddApplicationHomeSubClusterRequestProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  private void mergeLocalToBuilder() {
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null) {
+      return false;
+    }
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
+  }
+
+  @Override
+  public ApplicationHomeSubCluster getApplicationHomeSubCluster() {
+    AddApplicationHomeSubClusterRequestProtoOrBuilder p =
+        viaProto ? proto : builder;
+    if (!p.hasAppSubclusterMap()) {
+      return null;
+    }
+    return convertFromProtoFormat(p.getAppSubclusterMap());
+  }
+
+  @Override
+  public void setApplicationHomeSubCluster(
+      ApplicationHomeSubCluster applicationInfo) {
+    maybeInitBuilder();
+    if (applicationInfo == null) {
+      builder.clearAppSubclusterMap();
+      return;
+    }
+    builder.setAppSubclusterMap(convertToProtoFormat(applicationInfo));
+  }
+
+  private ApplicationHomeSubCluster convertFromProtoFormat(
+      ApplicationHomeSubClusterProto sc) {
+    return new ApplicationHomeSubClusterPBImpl(sc);
+  }
+
+  private ApplicationHomeSubClusterProto convertToProtoFormat(
+      ApplicationHomeSubCluster sc) {
+    return ((ApplicationHomeSubClusterPBImpl) sc).getProto();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/048df39b/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/AddApplicationHomeSubClusterResponsePBImpl.java
----------------------------------------------------------------------
diff --git a/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/AddApplicationHomeSubClusterResponsePBImpl.java b/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/AddApplicationHomeSubClusterResponsePBImpl.java
new file mode 100644
index 0000000..1415981
--- /dev/null
+++ b/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/AddApplicationHomeSubClusterResponsePBImpl.java
@@ -0,0 +1,78 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records.impl.pb;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.AddApplicationHomeSubClusterResponseProto;
+import org.apache.hadoop.yarn.server.federation.store.records.AddApplicationHomeSubClusterResponse;
+
+import com.google.protobuf.TextFormat;
+
+/**
+ * Protocol buffer based implementation of
+ * {@link AddApplicationHomeSubClusterResponse}.
+ */
+@Private
+@Unstable
+public class AddApplicationHomeSubClusterResponsePBImpl
+    extends AddApplicationHomeSubClusterResponse {
+
+  private AddApplicationHomeSubClusterResponseProto proto =
+      AddApplicationHomeSubClusterResponseProto.getDefaultInstance();
+  private AddApplicationHomeSubClusterResponseProto.Builder builder = null;
+  private boolean viaProto = false;
+
+  public AddApplicationHomeSubClusterResponsePBImpl() {
+    builder = AddApplicationHomeSubClusterResponseProto.newBuilder();
+  }
+
+  public AddApplicationHomeSubClusterResponsePBImpl(
+      AddApplicationHomeSubClusterResponseProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public AddApplicationHomeSubClusterResponseProto getProto() {
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null) {
+      return false;
+    }
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/048df39b/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
----------------------------------------------------------------------
diff --git a/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 b/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
new file mode 100644
index 0000000..7e6a564
--- /dev/null
+++ b/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
@@ -0,0 +1,167 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records.impl.pb;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.ApplicationHomeSubClusterProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.ApplicationHomeSubClusterProtoOrBuilder;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SubClusterIdProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto;
+import org.apache.hadoop.yarn.server.federation.store.records.ApplicationHomeSubCluster;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+
+import com.google.protobuf.TextFormat;
+
+/**
+ * Protocol buffer based implementation of {@link ApplicationHomeSubCluster}.
+ */
+@Private
+@Unstable
+public class ApplicationHomeSubClusterPBImpl extends ApplicationHomeSubCluster {
+
+  private ApplicationHomeSubClusterProto proto =
+      ApplicationHomeSubClusterProto.getDefaultInstance();
+  private ApplicationHomeSubClusterProto.Builder builder = null;
+  private boolean viaProto = false;
+
+  private ApplicationId applicationId = null;
+  private SubClusterId homeSubCluster = null;
+
+  public ApplicationHomeSubClusterPBImpl() {
+    builder = ApplicationHomeSubClusterProto.newBuilder();
+  }
+
+  public ApplicationHomeSubClusterPBImpl(ApplicationHomeSubClusterProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public ApplicationHomeSubClusterProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = ApplicationHomeSubClusterProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.applicationId != null) {
+      builder.setApplicationId(convertToProtoFormat(this.applicationId));
+    }
+    if (this.homeSubCluster != null) {
+      builder.setHomeSubCluster(convertToProtoFormat(this.homeSubCluster));
+    }
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null) {
+      return false;
+    }
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
+  }
+
+  @Override
+  public ApplicationId getApplicationId() {
+    ApplicationHomeSubClusterProtoOrBuilder p = viaProto ? proto : builder;
+    if (!p.hasApplicationId()) {
+      return null;
+    }
+    this.applicationId = convertFromProtoFormat(p.getApplicationId());
+    return applicationId;
+  }
+
+  @Override
+  public void setApplicationId(ApplicationId applicationId) {
+    maybeInitBuilder();
+    if (applicationId == null) {
+      builder.clearApplicationId();
+      return;
+    }
+    this.applicationId = applicationId;
+  }
+
+  @Override
+  public SubClusterId getHomeSubCluster() {
+    ApplicationHomeSubClusterProtoOrBuilder p = viaProto ? proto : builder;
+    if (this.homeSubCluster != null) {
+      return this.homeSubCluster;
+    }
+    if (!p.hasHomeSubCluster()) {
+      return null;
+    }
+    this.homeSubCluster = convertFromProtoFormat(p.getHomeSubCluster());
+    return this.homeSubCluster;
+  }
+
+  @Override
+  public void setHomeSubCluster(SubClusterId homeSubCluster) {
+    maybeInitBuilder();
+    if (homeSubCluster == null) {
+      builder.clearHomeSubCluster();
+    }
+    this.homeSubCluster = homeSubCluster;
+  }
+
+  private SubClusterId convertFromProtoFormat(SubClusterIdProto subClusterId) {
+    return new SubClusterIdPBImpl(subClusterId);
+  }
+
+  private SubClusterIdProto convertToProtoFormat(SubClusterId subClusterId) {
+    return ((SubClusterIdPBImpl) subClusterId).getProto();
+  }
+
+  private ApplicationId convertFromProtoFormat(ApplicationIdProto appId) {
+    return new ApplicationIdPBImpl(appId);
+  }
+
+  private ApplicationIdProto convertToProtoFormat(ApplicationId appId) {
+    return ((ApplicationIdPBImpl) appId).getProto();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/048df39b/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/DeleteApplicationHomeSubClusterRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/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/DeleteApplicationHomeSubClusterRequestPBImpl.java b/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/DeleteApplicationHomeSubClusterRequestPBImpl.java
new file mode 100644
index 0000000..b4ef680
--- /dev/null
+++ b/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/DeleteApplicationHomeSubClusterRequestPBImpl.java
@@ -0,0 +1,130 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records.impl.pb;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.DeleteApplicationHomeSubClusterRequestProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.DeleteApplicationHomeSubClusterRequestProtoOrBuilder;
+import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto;
+import org.apache.hadoop.yarn.server.federation.store.records.DeleteApplicationHomeSubClusterRequest;
+
+import com.google.protobuf.TextFormat;
+
+/**
+ * Protocol buffer based implementation of
+ * {@link DeleteApplicationHomeSubClusterRequest}.
+ */
+@Private
+@Unstable
+public class DeleteApplicationHomeSubClusterRequestPBImpl
+    extends DeleteApplicationHomeSubClusterRequest {
+
+  private DeleteApplicationHomeSubClusterRequestProto proto =
+      DeleteApplicationHomeSubClusterRequestProto.getDefaultInstance();
+  private DeleteApplicationHomeSubClusterRequestProto.Builder builder = null;
+  private boolean viaProto = false;
+
+  public DeleteApplicationHomeSubClusterRequestPBImpl() {
+    builder = DeleteApplicationHomeSubClusterRequestProto.newBuilder();
+  }
+
+  public DeleteApplicationHomeSubClusterRequestPBImpl(
+      DeleteApplicationHomeSubClusterRequestProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public DeleteApplicationHomeSubClusterRequestProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = DeleteApplicationHomeSubClusterRequestProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  private void mergeLocalToBuilder() {
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null) {
+      return false;
+    }
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
+  }
+
+  @Override
+  public ApplicationId getApplicationId() {
+    DeleteApplicationHomeSubClusterRequestProtoOrBuilder p =
+        viaProto ? proto : builder;
+    if (!p.hasApplicationId()) {
+      return null;
+    }
+    return convertFromProtoFormat(p.getApplicationId());
+  }
+
+  @Override
+  public void setApplicationId(ApplicationId applicationId) {
+    maybeInitBuilder();
+    if (applicationId == null) {
+      builder.clearApplicationId();
+      return;
+    }
+    builder.setApplicationId(convertToProtoFormat(applicationId));
+  }
+
+  private ApplicationId convertFromProtoFormat(ApplicationIdProto appId) {
+    return new ApplicationIdPBImpl(appId);
+  }
+
+  private ApplicationIdProto convertToProtoFormat(ApplicationId appId) {
+    return ((ApplicationIdPBImpl) appId).getProto();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/048df39b/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/DeleteApplicationHomeSubClusterResponsePBImpl.java
----------------------------------------------------------------------
diff --git a/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/DeleteApplicationHomeSubClusterResponsePBImpl.java b/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/DeleteApplicationHomeSubClusterResponsePBImpl.java
new file mode 100644
index 0000000..8a37b3c
--- /dev/null
+++ b/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/DeleteApplicationHomeSubClusterResponsePBImpl.java
@@ -0,0 +1,78 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records.impl.pb;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.DeleteApplicationHomeSubClusterResponseProto;
+import org.apache.hadoop.yarn.server.federation.store.records.DeleteApplicationHomeSubClusterResponse;
+
+import com.google.protobuf.TextFormat;
+
+/**
+ * Protocol buffer based implementation of
+ * {@link DeleteApplicationHomeSubClusterResponse}.
+ */
+@Private
+@Unstable
+public class DeleteApplicationHomeSubClusterResponsePBImpl
+    extends DeleteApplicationHomeSubClusterResponse {
+
+  private DeleteApplicationHomeSubClusterResponseProto proto =
+      DeleteApplicationHomeSubClusterResponseProto.getDefaultInstance();
+  private DeleteApplicationHomeSubClusterResponseProto.Builder builder = null;
+  private boolean viaProto = false;
+
+  public DeleteApplicationHomeSubClusterResponsePBImpl() {
+    builder = DeleteApplicationHomeSubClusterResponseProto.newBuilder();
+  }
+
+  public DeleteApplicationHomeSubClusterResponsePBImpl(
+      DeleteApplicationHomeSubClusterResponseProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public DeleteApplicationHomeSubClusterResponseProto getProto() {
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null) {
+      return false;
+    }
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/048df39b/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/GetApplicationHomeSubClusterRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/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/GetApplicationHomeSubClusterRequestPBImpl.java b/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/GetApplicationHomeSubClusterRequestPBImpl.java
new file mode 100644
index 0000000..865d0c4
--- /dev/null
+++ b/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/GetApplicationHomeSubClusterRequestPBImpl.java
@@ -0,0 +1,135 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records.impl.pb;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.GetApplicationHomeSubClusterRequestProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.GetApplicationHomeSubClusterRequestProtoOrBuilder;
+import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto;
+import org.apache.hadoop.yarn.server.federation.store.records.GetApplicationHomeSubClusterRequest;
+
+import com.google.protobuf.TextFormat;
+
+/**
+ * Protocol buffer based implementation of
+ * {@link GetApplicationHomeSubClusterRequest}.
+ */
+@Private
+@Unstable
+public class GetApplicationHomeSubClusterRequestPBImpl
+    extends GetApplicationHomeSubClusterRequest {
+
+  private GetApplicationHomeSubClusterRequestProto proto =
+      GetApplicationHomeSubClusterRequestProto.getDefaultInstance();
+  private GetApplicationHomeSubClusterRequestProto.Builder builder = null;
+  private boolean viaProto = false;
+
+  private ApplicationId applicationId = null;
+
+  public GetApplicationHomeSubClusterRequestPBImpl() {
+    builder = GetApplicationHomeSubClusterRequestProto.newBuilder();
+  }
+
+  public GetApplicationHomeSubClusterRequestPBImpl(
+      GetApplicationHomeSubClusterRequestProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public GetApplicationHomeSubClusterRequestProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = GetApplicationHomeSubClusterRequestProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.applicationId != null) {
+      builder.setApplicationId(convertToProtoFormat(this.applicationId));
+    }
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null) {
+      return false;
+    }
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
+  }
+
+  @Override
+  public ApplicationId getApplicationId() {
+    GetApplicationHomeSubClusterRequestProtoOrBuilder p =
+        viaProto ? proto : builder;
+    if (!p.hasApplicationId()) {
+      return null;
+    }
+    this.applicationId = convertFromProtoFormat(p.getApplicationId());
+    return applicationId;
+  }
+
+  @Override
+  public void setApplicationId(ApplicationId applicationId) {
+    maybeInitBuilder();
+    if (applicationId == null) {
+      builder.clearApplicationId();
+      return;
+    }
+    this.applicationId = applicationId;
+  }
+
+  private ApplicationId convertFromProtoFormat(ApplicationIdProto appId) {
+    return new ApplicationIdPBImpl(appId);
+  }
+
+  private ApplicationIdProto convertToProtoFormat(ApplicationId appId) {
+    return ((ApplicationIdPBImpl) appId).getProto();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/048df39b/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/GetApplicationHomeSubClusterResponsePBImpl.java
----------------------------------------------------------------------
diff --git a/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/GetApplicationHomeSubClusterResponsePBImpl.java b/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/GetApplicationHomeSubClusterResponsePBImpl.java
new file mode 100644
index 0000000..1180488
--- /dev/null
+++ b/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/GetApplicationHomeSubClusterResponsePBImpl.java
@@ -0,0 +1,132 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records.impl.pb;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.ApplicationHomeSubClusterProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.GetApplicationHomeSubClusterResponseProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.GetApplicationHomeSubClusterResponseProtoOrBuilder;
+import org.apache.hadoop.yarn.server.federation.store.records.ApplicationHomeSubCluster;
+import org.apache.hadoop.yarn.server.federation.store.records.GetApplicationHomeSubClusterResponse;
+
+import com.google.protobuf.TextFormat;
+
+/**
+ * Protocol buffer based implementation of
+ * {@link GetApplicationHomeSubClusterResponse}.
+ */
+@Private
+@Unstable
+public class GetApplicationHomeSubClusterResponsePBImpl
+    extends GetApplicationHomeSubClusterResponse {
+
+  private GetApplicationHomeSubClusterResponseProto proto =
+      GetApplicationHomeSubClusterResponseProto.getDefaultInstance();
+  private GetApplicationHomeSubClusterResponseProto.Builder builder = null;
+  private boolean viaProto = false;
+
+  public GetApplicationHomeSubClusterResponsePBImpl() {
+    builder = GetApplicationHomeSubClusterResponseProto.newBuilder();
+  }
+
+  public GetApplicationHomeSubClusterResponsePBImpl(
+      GetApplicationHomeSubClusterResponseProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public GetApplicationHomeSubClusterResponseProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = GetApplicationHomeSubClusterResponseProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  private void mergeLocalToBuilder() {
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null) {
+      return false;
+    }
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
+  }
+
+  @Override
+  public ApplicationHomeSubCluster getApplicationHomeSubCluster() {
+    GetApplicationHomeSubClusterResponseProtoOrBuilder p =
+        viaProto ? proto : builder;
+    if (!p.hasAppSubclusterMap()) {
+      return null;
+    }
+    return convertFromProtoFormat(p.getAppSubclusterMap());
+  }
+
+  @Override
+  public void setApplicationHomeSubCluster(
+      ApplicationHomeSubCluster applicationInfo) {
+    maybeInitBuilder();
+    if (applicationInfo == null) {
+      builder.clearAppSubclusterMap();
+      return;
+    }
+    builder.setAppSubclusterMap(convertToProtoFormat(applicationInfo));
+  }
+
+  private ApplicationHomeSubCluster convertFromProtoFormat(
+      ApplicationHomeSubClusterProto sc) {
+    return new ApplicationHomeSubClusterPBImpl(sc);
+  }
+
+  private ApplicationHomeSubClusterProto convertToProtoFormat(
+      ApplicationHomeSubCluster sc) {
+    return ((ApplicationHomeSubClusterPBImpl) sc).getProto();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/048df39b/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/GetApplicationsHomeSubClusterRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/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/GetApplicationsHomeSubClusterRequestPBImpl.java b/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/GetApplicationsHomeSubClusterRequestPBImpl.java
new file mode 100644
index 0000000..3ce8d74
--- /dev/null
+++ b/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/GetApplicationsHomeSubClusterRequestPBImpl.java
@@ -0,0 +1,78 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records.impl.pb;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.GetApplicationsHomeSubClusterRequestProto;
+import org.apache.hadoop.yarn.server.federation.store.records.GetApplicationsHomeSubClusterRequest;
+
+import com.google.protobuf.TextFormat;
+
+/**
+ * Protocol buffer based implementation of
+ * {@link GetApplicationsHomeSubClusterRequest}.
+ */
+@Private
+@Unstable
+public class GetApplicationsHomeSubClusterRequestPBImpl
+    extends GetApplicationsHomeSubClusterRequest {
+
+  private GetApplicationsHomeSubClusterRequestProto proto =
+      GetApplicationsHomeSubClusterRequestProto.getDefaultInstance();
+  private GetApplicationsHomeSubClusterRequestProto.Builder builder = null;
+  private boolean viaProto = false;
+
+  public GetApplicationsHomeSubClusterRequestPBImpl() {
+    builder = GetApplicationsHomeSubClusterRequestProto.newBuilder();
+  }
+
+  public GetApplicationsHomeSubClusterRequestPBImpl(
+      GetApplicationsHomeSubClusterRequestProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public GetApplicationsHomeSubClusterRequestProto getProto() {
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null) {
+      return false;
+    }
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
+  }
+
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[24/42] hadoop git commit: YARN-5612. Return SubClusterId in FederationStateStoreFacade#addApplicationHomeSubCluster for Router Failover. (Giovanni Matteo Fumarola via Subru).

Posted by su...@apache.org.
YARN-5612. Return SubClusterId in FederationStateStoreFacade#addApplicationHomeSubCluster for Router Failover. (Giovanni Matteo Fumarola via Subru).


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/e5067ecd
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/e5067ecd
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/e5067ecd

Branch: refs/heads/YARN-2915
Commit: e5067ecdcfab7715ddcf18f6c76da0b47d22d94c
Parents: e3d508f
Author: Subru Krishnan <su...@apache.org>
Authored: Thu Sep 1 13:55:54 2016 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon May 8 16:59:12 2017 -0700

----------------------------------------------------------------------
 .../utils/FederationStateStoreFacade.java       | 11 ++++---
 .../utils/TestFederationStateStoreFacade.java   | 30 ++++++++++++++++++++
 2 files changed, 37 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e5067ecd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java
index f1c8218..66a0b60 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java
@@ -48,6 +48,7 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.server.federation.store.FederationStateStore;
 import org.apache.hadoop.yarn.server.federation.store.records.AddApplicationHomeSubClusterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.AddApplicationHomeSubClusterResponse;
 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;
@@ -298,13 +299,15 @@ public final class FederationStateStoreFacade {
    *
    * @param appHomeSubCluster the mapping of the application to it's home
    *          sub-cluster
+   * @return the stored Subcluster from StateStore
    * @throws YarnException if the call to the state store is unsuccessful
    */
-  public void addApplicationHomeSubCluster(
+  public SubClusterId addApplicationHomeSubCluster(
       ApplicationHomeSubCluster appHomeSubCluster) throws YarnException {
-    stateStore.addApplicationHomeSubCluster(
-        AddApplicationHomeSubClusterRequest.newInstance(appHomeSubCluster));
-    return;
+    AddApplicationHomeSubClusterResponse response =
+        stateStore.addApplicationHomeSubCluster(
+            AddApplicationHomeSubClusterRequest.newInstance(appHomeSubCluster));
+    return response.getHomeSubCluster();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e5067ecd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/TestFederationStateStoreFacade.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/TestFederationStateStoreFacade.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/TestFederationStateStoreFacade.java
index 53f4f84..d46bef0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/TestFederationStateStoreFacade.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/TestFederationStateStoreFacade.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.federation.store.FederationStateStore;
 import org.apache.hadoop.yarn.server.federation.store.impl.MemoryFederationStateStore;
+import org.apache.hadoop.yarn.server.federation.store.records.ApplicationHomeSubCluster;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyConfiguration;
@@ -145,4 +146,33 @@ public class TestFederationStateStoreFacade {
     }
   }
 
+  @Test
+  public void testAddApplicationHomeSubCluster() throws YarnException {
+
+    // Inserting <AppId, Home1> into FederationStateStore
+    ApplicationId appId = ApplicationId.newInstance(clusterTs, numApps + 1);
+    SubClusterId subClusterId1 = SubClusterId.newInstance("Home1");
+
+    ApplicationHomeSubCluster appHomeSubCluster =
+        ApplicationHomeSubCluster.newInstance(appId, subClusterId1);
+
+    SubClusterId result =
+        facade.addApplicationHomeSubCluster(appHomeSubCluster);
+
+    Assert.assertEquals(facade.getApplicationHomeSubCluster(appId), result);
+    Assert.assertEquals(subClusterId1, result);
+
+    // Inserting <AppId, Home2> into FederationStateStore.
+    // The application is already present.
+    // FederationFacade will return Home1 as SubClusterId.
+    SubClusterId subClusterId2 = SubClusterId.newInstance("Home2");
+    appHomeSubCluster =
+        ApplicationHomeSubCluster.newInstance(appId, subClusterId2);
+
+    result = facade.addApplicationHomeSubCluster(appHomeSubCluster);
+
+    Assert.assertEquals(facade.getApplicationHomeSubCluster(appId), result);
+    Assert.assertEquals(subClusterId1, result);
+  }
+
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[02/42] hadoop git commit: YARN-5676. Add a HashBasedRouterPolicy, and small policies and test refactoring. (Carlo Curino via Subru).

Posted by su...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/b370ac69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestRouterPolicyFacade.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestRouterPolicyFacade.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestRouterPolicyFacade.java
index 4975a9f..5fa02d6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestRouterPolicyFacade.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestRouterPolicyFacade.java
@@ -29,6 +29,8 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.federation.policies.dao.WeightedPolicyInfo;
 import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
+import org.apache.hadoop.yarn.server.federation.policies.manager.PriorityBroadcastPolicyManager;
+import org.apache.hadoop.yarn.server.federation.policies.manager.UniformBroadcastPolicyManager;
 import org.apache.hadoop.yarn.server.federation.policies.router.PriorityRouterPolicy;
 import org.apache.hadoop.yarn.server.federation.policies.router.UniformRandomRouterPolicy;
 import org.apache.hadoop.yarn.server.federation.resolver.SubClusterResolver;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b370ac69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestUniformBroadcastPolicyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestUniformBroadcastPolicyManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestUniformBroadcastPolicyManager.java
deleted file mode 100644
index 542a5ae..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestUniformBroadcastPolicyManager.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership.  The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations under
- * the License.
- */
-
-package org.apache.hadoop.yarn.server.federation.policies;
-
-import org.apache.hadoop.yarn.server.federation.policies.amrmproxy.BroadcastAMRMProxyPolicy;
-import org.apache.hadoop.yarn.server.federation.policies.router.UniformRandomRouterPolicy;
-import org.junit.Before;
-
-/**
- * Simple test of {@link UniformBroadcastPolicyManager}.
- */
-public class TestUniformBroadcastPolicyManager extends BasePolicyManagerTest {
-
-  @Before
-  public void setup() {
-    //config policy
-    wfp = new UniformBroadcastPolicyManager();
-    wfp.setQueue("queue1");
-
-    //set expected params that the base test class will use for tests
-    expectedPolicyManager = UniformBroadcastPolicyManager.class;
-    expectedAMRMProxyPolicy = BroadcastAMRMProxyPolicy.class;
-    expectedRouterPolicy = UniformRandomRouterPolicy.class;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b370ac69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestWeightedLocalityPolicyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestWeightedLocalityPolicyManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestWeightedLocalityPolicyManager.java
deleted file mode 100644
index ab9cec4..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestWeightedLocalityPolicyManager.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership.  The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations under
- * the License.
- */
-
-package org.apache.hadoop.yarn.server.federation.policies;
-
-import org.apache.hadoop.yarn.server.federation.policies.amrmproxy.LocalityMulticastAMRMProxyPolicy;
-import org.apache.hadoop.yarn.server.federation.policies.dao.WeightedPolicyInfo;
-import org.apache.hadoop.yarn.server.federation.policies.router.WeightedRandomRouterPolicy;
-import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
-import org.apache.hadoop.yarn.server.federation.store.records.SubClusterIdInfo;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.util.HashMap;
-import java.util.Map;
-
-/**
- * Simple test of {@link WeightedLocalityPolicyManager}.
- */
-public class TestWeightedLocalityPolicyManager extends
-    BasePolicyManagerTest {
-
-  private WeightedPolicyInfo policyInfo;
-
-  @Before
-  public void setup() {
-    // configure a policy
-
-    wfp = new WeightedLocalityPolicyManager();
-    wfp.setQueue("queue1");
-    SubClusterId sc1 = SubClusterId.newInstance("sc1");
-    SubClusterId sc2 = SubClusterId.newInstance("sc2");
-    policyInfo = new WeightedPolicyInfo();
-
-    Map<SubClusterIdInfo, Float> routerWeights = new HashMap<>();
-    routerWeights.put(new SubClusterIdInfo(sc1), 0.2f);
-    routerWeights.put(new SubClusterIdInfo(sc2), 0.8f);
-    policyInfo.setRouterPolicyWeights(routerWeights);
-
-    Map<SubClusterIdInfo, Float> amrmWeights = new HashMap<>();
-    amrmWeights.put(new SubClusterIdInfo(sc1), 0.2f);
-    amrmWeights.put(new SubClusterIdInfo(sc2), 0.8f);
-    policyInfo.setAMRMPolicyWeights(amrmWeights);
-
-    ((WeightedLocalityPolicyManager) wfp).setWeightedPolicyInfo(
-        policyInfo);
-
-    //set expected params that the base test class will use for tests
-    expectedPolicyManager = WeightedLocalityPolicyManager.class;
-    expectedAMRMProxyPolicy = LocalityMulticastAMRMProxyPolicy.class;
-    expectedRouterPolicy = WeightedRandomRouterPolicy.class;
-  }
-
-  @Test
-  public void testPolicyInfoSetCorrectly() throws Exception {
-    serializeAndDeserializePolicyManager(wfp, expectedPolicyManager,
-                                         expectedAMRMProxyPolicy,
-                                         expectedRouterPolicy);
-
-    //check the policyInfo propagates through ser/der correctly
-    Assert.assertEquals(((WeightedLocalityPolicyManager) wfp)
-                            .getWeightedPolicyInfo(), policyInfo);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b370ac69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/manager/BasePolicyManagerTest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/manager/BasePolicyManagerTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/manager/BasePolicyManagerTest.java
new file mode 100644
index 0000000..3cf73b6
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/manager/BasePolicyManagerTest.java
@@ -0,0 +1,104 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies.manager;
+
+import org.apache.hadoop.yarn.server.federation.policies.FederationPolicyInitializationContext;
+import org.apache.hadoop.yarn.server.federation.policies.amrmproxy.FederationAMRMProxyPolicy;
+import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
+import org.apache.hadoop.yarn.server.federation.policies.router.FederationRouterPolicy;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyConfiguration;
+import org.apache.hadoop.yarn.server.federation.utils.FederationPoliciesTestUtil;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * This class provides common test methods for testing {@code
+ * FederationPolicyManager}s.
+ */
+public abstract class BasePolicyManagerTest {
+
+  @SuppressWarnings("checkstyle:visibilitymodifier")
+  protected FederationPolicyManager wfp = null;
+  @SuppressWarnings("checkstyle:visibilitymodifier")
+  protected Class expectedPolicyManager;
+  @SuppressWarnings("checkstyle:visibilitymodifier")
+  protected Class expectedAMRMProxyPolicy;
+  @SuppressWarnings("checkstyle:visibilitymodifier")
+  protected Class expectedRouterPolicy;
+
+  @Test
+  public void testSerializeAndInstantiate() throws Exception {
+    serializeAndDeserializePolicyManager(wfp, expectedPolicyManager,
+        expectedAMRMProxyPolicy, expectedRouterPolicy);
+  }
+
+  @Test(expected = FederationPolicyInitializationException.class)
+  public void testSerializeAndInstantiateBad1() throws Exception {
+    serializeAndDeserializePolicyManager(wfp, String.class,
+        expectedAMRMProxyPolicy, expectedRouterPolicy);
+  }
+
+  @Test(expected = AssertionError.class)
+  public void testSerializeAndInstantiateBad2() throws Exception {
+    serializeAndDeserializePolicyManager(wfp, expectedPolicyManager,
+        String.class, expectedRouterPolicy);
+  }
+
+  @Test(expected = AssertionError.class)
+  public void testSerializeAndInstantiateBad3() throws Exception {
+    serializeAndDeserializePolicyManager(wfp, expectedPolicyManager,
+        expectedAMRMProxyPolicy, String.class);
+  }
+
+  protected static void serializeAndDeserializePolicyManager(
+      FederationPolicyManager wfp, Class policyManagerType,
+      Class expAMRMProxyPolicy, Class expRouterPolicy) throws Exception {
+
+    // serializeConf it in a context
+    SubClusterPolicyConfiguration fpc = wfp.serializeConf();
+    fpc.setType(policyManagerType.getCanonicalName());
+    FederationPolicyInitializationContext context =
+        new FederationPolicyInitializationContext();
+    context.setSubClusterPolicyConfiguration(fpc);
+    context
+        .setFederationStateStoreFacade(FederationPoliciesTestUtil.initFacade());
+    context.setFederationSubclusterResolver(
+        FederationPoliciesTestUtil.initResolver());
+    context.setHomeSubcluster(SubClusterId.newInstance("homesubcluster"));
+
+    // based on the "context" created instantiate new class and use it
+    Class c = Class.forName(wfp.getClass().getCanonicalName());
+    FederationPolicyManager wfp2 = (FederationPolicyManager) c.newInstance();
+
+    FederationAMRMProxyPolicy federationAMRMProxyPolicy =
+        wfp2.getAMRMPolicy(context, null);
+
+    // needed only for tests (getARMRMPolicy change the "type" in conf)
+    fpc.setType(wfp.getClass().getCanonicalName());
+
+    FederationRouterPolicy federationRouterPolicy =
+        wfp2.getRouterPolicy(context, null);
+
+    Assert.assertEquals(federationAMRMProxyPolicy.getClass(),
+        expAMRMProxyPolicy);
+
+    Assert.assertEquals(federationRouterPolicy.getClass(), expRouterPolicy);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b370ac69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/manager/TestHashBasedBroadcastPolicyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/manager/TestHashBasedBroadcastPolicyManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/manager/TestHashBasedBroadcastPolicyManager.java
new file mode 100644
index 0000000..5fc4a56
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/manager/TestHashBasedBroadcastPolicyManager.java
@@ -0,0 +1,40 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies.manager;
+
+import org.apache.hadoop.yarn.server.federation.policies.amrmproxy.BroadcastAMRMProxyPolicy;
+import org.apache.hadoop.yarn.server.federation.policies.router.HashBasedRouterPolicy;
+import org.junit.Before;
+
+/**
+ * Simple test of {@link HashBroadcastPolicyManager}.
+ */
+public class TestHashBasedBroadcastPolicyManager extends BasePolicyManagerTest {
+
+  @Before
+  public void setup() {
+    // config policy
+    wfp = new HashBroadcastPolicyManager();
+    wfp.setQueue("queue1");
+
+    // set expected params that the base test class will use for tests
+    expectedPolicyManager = HashBroadcastPolicyManager.class;
+    expectedAMRMProxyPolicy = BroadcastAMRMProxyPolicy.class;
+    expectedRouterPolicy = HashBasedRouterPolicy.class;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b370ac69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/manager/TestPriorityBroadcastPolicyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/manager/TestPriorityBroadcastPolicyManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/manager/TestPriorityBroadcastPolicyManager.java
new file mode 100644
index 0000000..21b39e9
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/manager/TestPriorityBroadcastPolicyManager.java
@@ -0,0 +1,72 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies.manager;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.yarn.server.federation.policies.amrmproxy.BroadcastAMRMProxyPolicy;
+import org.apache.hadoop.yarn.server.federation.policies.dao.WeightedPolicyInfo;
+import org.apache.hadoop.yarn.server.federation.policies.router.PriorityRouterPolicy;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterIdInfo;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Simple test of {@link PriorityBroadcastPolicyManager}.
+ */
+public class TestPriorityBroadcastPolicyManager extends BasePolicyManagerTest {
+
+  private WeightedPolicyInfo policyInfo;
+
+  @Before
+  public void setup() {
+    // configure a policy
+
+    wfp = new PriorityBroadcastPolicyManager();
+    wfp.setQueue("queue1");
+    SubClusterId sc1 = SubClusterId.newInstance("sc1");
+    SubClusterId sc2 = SubClusterId.newInstance("sc2");
+    policyInfo = new WeightedPolicyInfo();
+
+    Map<SubClusterIdInfo, Float> routerWeights = new HashMap<>();
+    routerWeights.put(new SubClusterIdInfo(sc1), 0.2f);
+    routerWeights.put(new SubClusterIdInfo(sc2), 0.8f);
+    policyInfo.setRouterPolicyWeights(routerWeights);
+
+    ((PriorityBroadcastPolicyManager) wfp).setWeightedPolicyInfo(policyInfo);
+
+    // set expected params that the base test class will use for tests
+    expectedPolicyManager = PriorityBroadcastPolicyManager.class;
+    expectedAMRMProxyPolicy = BroadcastAMRMProxyPolicy.class;
+    expectedRouterPolicy = PriorityRouterPolicy.class;
+  }
+
+  @Test
+  public void testPolicyInfoSetCorrectly() throws Exception {
+    serializeAndDeserializePolicyManager(wfp, expectedPolicyManager,
+        expectedAMRMProxyPolicy, expectedRouterPolicy);
+
+    // check the policyInfo propagates through ser/der correctly
+    Assert.assertEquals(
+        ((PriorityBroadcastPolicyManager) wfp).getWeightedPolicyInfo(),
+        policyInfo);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b370ac69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/manager/TestUniformBroadcastPolicyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/manager/TestUniformBroadcastPolicyManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/manager/TestUniformBroadcastPolicyManager.java
new file mode 100644
index 0000000..57fafdc
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/manager/TestUniformBroadcastPolicyManager.java
@@ -0,0 +1,40 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies.manager;
+
+import org.apache.hadoop.yarn.server.federation.policies.amrmproxy.BroadcastAMRMProxyPolicy;
+import org.apache.hadoop.yarn.server.federation.policies.router.UniformRandomRouterPolicy;
+import org.junit.Before;
+
+/**
+ * Simple test of {@link UniformBroadcastPolicyManager}.
+ */
+public class TestUniformBroadcastPolicyManager extends BasePolicyManagerTest {
+
+  @Before
+  public void setup() {
+    //config policy
+    wfp = new UniformBroadcastPolicyManager();
+    wfp.setQueue("queue1");
+
+    //set expected params that the base test class will use for tests
+    expectedPolicyManager = UniformBroadcastPolicyManager.class;
+    expectedAMRMProxyPolicy = BroadcastAMRMProxyPolicy.class;
+    expectedRouterPolicy = UniformRandomRouterPolicy.class;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b370ac69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/manager/TestWeightedLocalityPolicyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/manager/TestWeightedLocalityPolicyManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/manager/TestWeightedLocalityPolicyManager.java
new file mode 100644
index 0000000..51661473
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/manager/TestWeightedLocalityPolicyManager.java
@@ -0,0 +1,79 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies.manager;
+
+import org.apache.hadoop.yarn.server.federation.policies.amrmproxy.LocalityMulticastAMRMProxyPolicy;
+import org.apache.hadoop.yarn.server.federation.policies.dao.WeightedPolicyInfo;
+import org.apache.hadoop.yarn.server.federation.policies.router.WeightedRandomRouterPolicy;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterIdInfo;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Simple test of {@link WeightedLocalityPolicyManager}.
+ */
+public class TestWeightedLocalityPolicyManager extends
+    BasePolicyManagerTest {
+
+  private WeightedPolicyInfo policyInfo;
+
+  @Before
+  public void setup() {
+    // configure a policy
+
+    wfp = new WeightedLocalityPolicyManager();
+    wfp.setQueue("queue1");
+    SubClusterId sc1 = SubClusterId.newInstance("sc1");
+    SubClusterId sc2 = SubClusterId.newInstance("sc2");
+    policyInfo = new WeightedPolicyInfo();
+
+    Map<SubClusterIdInfo, Float> routerWeights = new HashMap<>();
+    routerWeights.put(new SubClusterIdInfo(sc1), 0.2f);
+    routerWeights.put(new SubClusterIdInfo(sc2), 0.8f);
+    policyInfo.setRouterPolicyWeights(routerWeights);
+
+    Map<SubClusterIdInfo, Float> amrmWeights = new HashMap<>();
+    amrmWeights.put(new SubClusterIdInfo(sc1), 0.2f);
+    amrmWeights.put(new SubClusterIdInfo(sc2), 0.8f);
+    policyInfo.setAMRMPolicyWeights(amrmWeights);
+
+    ((WeightedLocalityPolicyManager) wfp).setWeightedPolicyInfo(
+        policyInfo);
+
+    //set expected params that the base test class will use for tests
+    expectedPolicyManager = WeightedLocalityPolicyManager.class;
+    expectedAMRMProxyPolicy = LocalityMulticastAMRMProxyPolicy.class;
+    expectedRouterPolicy = WeightedRandomRouterPolicy.class;
+  }
+
+  @Test
+  public void testPolicyInfoSetCorrectly() throws Exception {
+    serializeAndDeserializePolicyManager(wfp, expectedPolicyManager,
+                                         expectedAMRMProxyPolicy,
+                                         expectedRouterPolicy);
+
+    //check the policyInfo propagates through ser/der correctly
+    Assert.assertEquals(((WeightedLocalityPolicyManager) wfp)
+                            .getWeightedPolicyInfo(), policyInfo);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b370ac69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/BaseRouterPoliciesTest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/BaseRouterPoliciesTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/BaseRouterPoliciesTest.java
new file mode 100644
index 0000000..2e7a0af
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/BaseRouterPoliciesTest.java
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies.router;
+
+import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.federation.policies.BaseFederationPoliciesTest;
+import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyException;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.util.resource.Resources;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Base class for router policies tests, tests for null input cases.
+ */
+public abstract class BaseRouterPoliciesTest
+    extends BaseFederationPoliciesTest {
+
+  @Test
+  public void testNullQueueRouting() throws YarnException {
+    FederationRouterPolicy localPolicy = (FederationRouterPolicy) getPolicy();
+    ApplicationSubmissionContext applicationSubmissionContext =
+        ApplicationSubmissionContext.newInstance(null, null, null, null, null,
+            false, false, 0, Resources.none(), null, false, null, null);
+    SubClusterId chosen =
+        localPolicy.getHomeSubcluster(applicationSubmissionContext);
+    Assert.assertNotNull(chosen);
+  }
+
+  @Test(expected = FederationPolicyException.class)
+  public void testNullAppContext() throws YarnException {
+    ((FederationRouterPolicy) getPolicy()).getHomeSubcluster(null);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b370ac69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestHashBasedRouterPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestHashBasedRouterPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestHashBasedRouterPolicy.java
new file mode 100644
index 0000000..af7fe43
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestHashBasedRouterPolicy.java
@@ -0,0 +1,83 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies.router;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.federation.utils.FederationPoliciesTestUtil;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Simple test class for the {@link HashBasedRouterPolicy}. Tests that one of
+ * the active sub-cluster is chosen.
+ */
+public class TestHashBasedRouterPolicy extends BaseRouterPoliciesTest {
+
+  private int numSubclusters = 10;
+
+  @Before
+  public void setUp() throws Exception {
+
+    // set policy in base class
+    setPolicy(new HashBasedRouterPolicy());
+
+    // setting up the active sub-clusters for this test
+    setMockActiveSubclusters(numSubclusters);
+
+    // initialize policy with context
+    FederationPoliciesTestUtil.initializePolicyContext(getPolicy(),
+        getPolicyInfo(), getActiveSubclusters());
+  }
+
+  @Test
+  public void testHashSpreadUniformlyAmongSubclusters() throws YarnException {
+    SubClusterId chosen;
+
+    Map<SubClusterId, AtomicLong> counter = new HashMap<>();
+    for (SubClusterId id : getActiveSubclusters().keySet()) {
+      counter.put(id, new AtomicLong(0));
+    }
+
+    long jobPerSub = 100;
+
+    ApplicationSubmissionContext applicationSubmissionContext =
+        mock(ApplicationSubmissionContext.class);
+    for (int i = 0; i < jobPerSub * numSubclusters; i++) {
+      when(applicationSubmissionContext.getQueue()).thenReturn("queue" + i);
+      chosen = ((FederationRouterPolicy) getPolicy())
+          .getHomeSubcluster(applicationSubmissionContext);
+      counter.get(chosen).addAndGet(1);
+    }
+
+    // hash spread the jobs equally among the subclusters
+    for (AtomicLong a : counter.values()) {
+      Assert.assertEquals(a.get(), jobPerSub);
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b370ac69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestLoadBasedRouterPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestLoadBasedRouterPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestLoadBasedRouterPolicy.java
index 906e35f..b70b4aa 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestLoadBasedRouterPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestLoadBasedRouterPolicy.java
@@ -21,7 +21,6 @@ import java.util.HashMap;
 import java.util.Map;
 
 import org.apache.hadoop.yarn.exceptions.YarnException;
-import org.apache.hadoop.yarn.server.federation.policies.BaseFederationPoliciesTest;
 import org.apache.hadoop.yarn.server.federation.policies.dao.WeightedPolicyInfo;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterIdInfo;
@@ -36,7 +35,7 @@ import org.junit.Test;
  * Simple test class for the {@link LoadBasedRouterPolicy}. Test that the load
  * is properly considered for allocation.
  */
-public class TestLoadBasedRouterPolicy extends BaseFederationPoliciesTest {
+public class TestLoadBasedRouterPolicy extends BaseRouterPoliciesTest {
 
   @Before
   public void setUp() throws Exception {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b370ac69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestPriorityRouterPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestPriorityRouterPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestPriorityRouterPolicy.java
index eefcfd9..42d919d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestPriorityRouterPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestPriorityRouterPolicy.java
@@ -23,7 +23,6 @@ import java.util.HashMap;
 import java.util.Map;
 
 import org.apache.hadoop.yarn.exceptions.YarnException;
-import org.apache.hadoop.yarn.server.federation.policies.BaseFederationPoliciesTest;
 import org.apache.hadoop.yarn.server.federation.policies.dao.WeightedPolicyInfo;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterIdInfo;
@@ -38,7 +37,7 @@ import org.junit.Test;
  * Simple test class for the {@link PriorityRouterPolicy}. Tests that the
  * weights are correctly used for ordering the choice of sub-clusters.
  */
-public class TestPriorityRouterPolicy extends BaseFederationPoliciesTest {
+public class TestPriorityRouterPolicy extends BaseRouterPoliciesTest {
 
   @Before
   public void setUp() throws Exception {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b370ac69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestUniformRandomRouterPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestUniformRandomRouterPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestUniformRandomRouterPolicy.java
index ac41ab5..b45aa2a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestUniformRandomRouterPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestUniformRandomRouterPolicy.java
@@ -21,7 +21,6 @@ import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
 import org.apache.hadoop.yarn.exceptions.YarnException;
-import org.apache.hadoop.yarn.server.federation.policies.BaseFederationPoliciesTest;
 import org.apache.hadoop.yarn.server.federation.policies.dao.WeightedPolicyInfo;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterIdInfo;
@@ -36,7 +35,7 @@ import org.junit.Test;
  * Simple test class for the {@link UniformRandomRouterPolicy}. Tests that one
  * of the active subcluster is chosen.
  */
-public class TestUniformRandomRouterPolicy extends BaseFederationPoliciesTest {
+public class TestUniformRandomRouterPolicy extends BaseRouterPoliciesTest {
 
   @Before
   public void setUp() throws Exception {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b370ac69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestWeightedRandomRouterPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestWeightedRandomRouterPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestWeightedRandomRouterPolicy.java
index 78967d0..34cc5f8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestWeightedRandomRouterPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestWeightedRandomRouterPolicy.java
@@ -24,8 +24,8 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.concurrent.atomic.AtomicLong;
 
+import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.exceptions.YarnException;
-import org.apache.hadoop.yarn.server.federation.policies.BaseFederationPoliciesTest;
 import org.apache.hadoop.yarn.server.federation.policies.dao.WeightedPolicyInfo;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterIdInfo;
@@ -41,7 +41,7 @@ import org.junit.Test;
  * number of randomized tests to check we are weighiting correctly even if
  * clusters go inactive.
  */
-public class TestWeightedRandomRouterPolicy extends BaseFederationPoliciesTest {
+public class TestWeightedRandomRouterPolicy extends BaseRouterPoliciesTest {
 
   @Before
   public void setUp() throws Exception {
@@ -78,13 +78,18 @@ public class TestWeightedRandomRouterPolicy extends BaseFederationPoliciesTest {
   @Test
   public void testClusterChosenWithRightProbability() throws YarnException {
 
+    ApplicationSubmissionContext context =
+        mock(ApplicationSubmissionContext.class);
+    when(context.getQueue()).thenReturn("queue1");
+    setApplicationSubmissionContext(context);
+
     Map<SubClusterId, AtomicLong> counter = new HashMap<>();
     for (SubClusterIdInfo id : getPolicyInfo().getRouterPolicyWeights()
         .keySet()) {
       counter.put(id.toId(), new AtomicLong(0));
     }
 
-    float numberOfDraws = 1000000;
+    float numberOfDraws = 100000;
 
     for (float i = 0; i < numberOfDraws; i++) {
       SubClusterId chosenId = ((FederationRouterPolicy) getPolicy())
@@ -113,8 +118,8 @@ public class TestWeightedRandomRouterPolicy extends BaseFederationPoliciesTest {
         Assert.assertTrue(
             "Id " + counterEntry.getKey() + " Actual weight: " + actualWeight
                 + " expected weight: " + expectedWeight,
-            expectedWeight == 0 || (actualWeight / expectedWeight) < 1.1
-                && (actualWeight / expectedWeight) > 0.9);
+            expectedWeight == 0 || (actualWeight / expectedWeight) < 1.2
+                && (actualWeight / expectedWeight) > 0.8);
       } else {
         Assert
             .assertTrue(


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[25/42] hadoop git commit: YARN-5324. Stateless Federation router policies implementation. (Carlo Curino via Subru).

Posted by su...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d38f164/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/FederationPoliciesTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/FederationPoliciesTestUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/FederationPoliciesTestUtil.java
index 8c2115b..f901329 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/FederationPoliciesTestUtil.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/FederationPoliciesTestUtil.java
@@ -19,13 +19,20 @@ package org.apache.hadoop.yarn.server.federation.utils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.federation.policies.ConfigurableFederationPolicy;
+import org.apache.hadoop.yarn.server.federation.policies.FederationPolicyInitializationContext;
+import org.apache.hadoop.yarn.server.federation.policies.dao.WeightedPolicyInfo;
 import org.apache.hadoop.yarn.server.federation.resolver.DefaultSubClusterResolverImpl;
 import org.apache.hadoop.yarn.server.federation.resolver.SubClusterResolver;
 import org.apache.hadoop.yarn.server.federation.store.FederationStateStore;
-import org.apache.hadoop.yarn.server.federation.store.records.GetSubClustersInfoResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.*;
 
 import java.net.URL;
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 
 import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.mock;
@@ -41,6 +48,41 @@ public final class FederationPoliciesTestUtil {
     // disabled.
   }
 
+
+  public static void initializePolicyContext(
+      FederationPolicyInitializationContext fpc, ConfigurableFederationPolicy
+      policy, WeightedPolicyInfo policyInfo,
+      Map<SubClusterId, SubClusterInfo> activeSubclusters)
+      throws YarnException {
+    ByteBuffer buf = policyInfo.toByteBuffer();
+    fpc.setSubClusterPolicyConfiguration(SubClusterPolicyConfiguration
+        .newInstance("queue1", policy.getClass().getCanonicalName(), buf));
+    FederationStateStoreFacade facade = FederationStateStoreFacade
+        .getInstance();
+    FederationStateStore fss = mock(FederationStateStore.class);
+
+    if (activeSubclusters == null) {
+      activeSubclusters = new HashMap<SubClusterId, SubClusterInfo>();
+    }
+    GetSubClustersInfoResponse response = GetSubClustersInfoResponse
+        .newInstance(new ArrayList<SubClusterInfo>(activeSubclusters.values()));
+
+    when(fss.getSubClusters(any())).thenReturn(response);
+    facade.reinitialize(fss, new Configuration());
+    fpc.setFederationStateStoreFacade(facade);
+    policy.reinitialize(fpc);
+  }
+
+  public static void initializePolicyContext(
+      ConfigurableFederationPolicy policy,
+      WeightedPolicyInfo policyInfo, Map<SubClusterId,
+      SubClusterInfo> activeSubclusters) throws YarnException {
+    FederationPolicyInitializationContext context =
+        new FederationPolicyInitializationContext(null, initResolver(),
+            initFacade());
+    initializePolicyContext(context, policy, policyInfo, activeSubclusters);
+  }
+
   /**
    * Initialize a {@link SubClusterResolver}.
    *
@@ -66,18 +108,52 @@ public final class FederationPoliciesTestUtil {
    * Initialiaze a main-memory {@link FederationStateStoreFacade} used for
    * testing, wiht a mock resolver.
    *
+   * @param subClusterInfos the list of subclusters to be served on
+   *                        getSubClusters invocations.
+   *
    * @return the facade.
    *
    * @throws YarnException in case the initialization is not successful.
    */
-  public static FederationStateStoreFacade initFacade() throws YarnException {
+
+  public static FederationStateStoreFacade initFacade(
+      List<SubClusterInfo> subClusterInfos, SubClusterPolicyConfiguration
+      policyConfiguration) throws YarnException {
     FederationStateStoreFacade goodFacade = FederationStateStoreFacade
         .getInstance();
     FederationStateStore fss = mock(FederationStateStore.class);
     GetSubClustersInfoResponse response = GetSubClustersInfoResponse
-        .newInstance(new ArrayList<>());
+        .newInstance(subClusterInfos);
     when(fss.getSubClusters(any())).thenReturn(response);
+
+    List<SubClusterPolicyConfiguration> configurations = new ArrayList<>();
+    configurations.add(policyConfiguration);
+
+    GetSubClusterPoliciesConfigurationsResponse policiesResponse =
+        GetSubClusterPoliciesConfigurationsResponse
+            .newInstance(configurations);
+    when(fss.getPoliciesConfigurations(any())).thenReturn(policiesResponse);
+
+    GetSubClusterPolicyConfigurationResponse policyResponse =
+        GetSubClusterPolicyConfigurationResponse
+            .newInstance(policyConfiguration);
+    when(fss.getPolicyConfiguration(any())).thenReturn(policyResponse);
+
     goodFacade.reinitialize(fss, new Configuration());
     return goodFacade;
   }
+
+  /**
+   * Initialiaze a main-memory {@link FederationStateStoreFacade} used for
+   * testing, wiht a mock resolver.
+   *
+   * @return the facade.
+   *
+   * @throws YarnException in case the initialization is not successful.
+   */
+  public static FederationStateStoreFacade initFacade() throws YarnException {
+    return initFacade(new ArrayList<>(), mock(SubClusterPolicyConfiguration
+        .class));
+  }
+
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[03/42] hadoop git commit: YARN-5676. Add a HashBasedRouterPolicy, and small policies and test refactoring. (Carlo Curino via Subru).

Posted by su...@apache.org.
YARN-5676. Add a HashBasedRouterPolicy, and small policies and test refactoring. (Carlo Curino via Subru).


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/b370ac69
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/b370ac69
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/b370ac69

Branch: refs/heads/YARN-2915
Commit: b370ac693bd759b1f2f9bb498bbe2c8861abaffe
Parents: 03801af
Author: Subru Krishnan <su...@apache.org>
Authored: Tue Nov 22 15:02:22 2016 -0800
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon May 8 16:59:12 2017 -0700

----------------------------------------------------------------------
 .../hadoop/yarn/conf/YarnConfiguration.java     |   3 +-
 .../policies/AbstractPolicyManager.java         | 175 -----------------
 .../policies/FederationPolicyManager.java       | 117 ------------
 .../PriorityBroadcastPolicyManager.java         |  66 -------
 .../federation/policies/RouterPolicyFacade.java |   1 +
 .../policies/UniformBroadcastPolicyManager.java |  56 ------
 .../policies/WeightedLocalityPolicyManager.java |  67 -------
 .../policies/manager/AbstractPolicyManager.java | 190 +++++++++++++++++++
 .../manager/FederationPolicyManager.java        | 118 ++++++++++++
 .../manager/HashBroadcastPolicyManager.java     |  38 ++++
 .../manager/PriorityBroadcastPolicyManager.java |  66 +++++++
 .../manager/UniformBroadcastPolicyManager.java  |  44 +++++
 .../manager/WeightedLocalityPolicyManager.java  |  67 +++++++
 .../policies/manager/package-info.java          |  19 ++
 .../policies/router/AbstractRouterPolicy.java   |  19 ++
 .../policies/router/HashBasedRouterPolicy.java  |  81 ++++++++
 .../policies/router/LoadBasedRouterPolicy.java  |   3 +
 .../policies/router/PriorityRouterPolicy.java   |   3 +
 .../router/UniformRandomRouterPolicy.java       |  10 +-
 .../router/WeightedRandomRouterPolicy.java      |   3 +
 .../policies/BaseFederationPoliciesTest.java    |  17 +-
 .../policies/BasePolicyManagerTest.java         | 108 -----------
 ...ionPolicyInitializationContextValidator.java |   1 +
 .../TestPriorityBroadcastPolicyManager.java     |  72 -------
 .../policies/TestRouterPolicyFacade.java        |   2 +
 .../TestUniformBroadcastPolicyManager.java      |  40 ----
 .../TestWeightedLocalityPolicyManager.java      |  79 --------
 .../policies/manager/BasePolicyManagerTest.java | 104 ++++++++++
 .../TestHashBasedBroadcastPolicyManager.java    |  40 ++++
 .../TestPriorityBroadcastPolicyManager.java     |  72 +++++++
 .../TestUniformBroadcastPolicyManager.java      |  40 ++++
 .../TestWeightedLocalityPolicyManager.java      |  79 ++++++++
 .../policies/router/BaseRouterPoliciesTest.java |  51 +++++
 .../router/TestHashBasedRouterPolicy.java       |  83 ++++++++
 .../router/TestLoadBasedRouterPolicy.java       |   3 +-
 .../router/TestPriorityRouterPolicy.java        |   3 +-
 .../router/TestUniformRandomRouterPolicy.java   |   3 +-
 .../router/TestWeightedRandomRouterPolicy.java  |  15 +-
 38 files changed, 1160 insertions(+), 798 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b370ac69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
index a593a2c..dd05411 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
@@ -2578,7 +2578,8 @@ public class YarnConfiguration extends Configuration {
       + "policy-manager";
 
   public static final String DEFAULT_FEDERATION_POLICY_MANAGER = "org.apache"
-      + ".hadoop.yarn.server.federation.policies.UniformBroadcastPolicyManager";
+      + ".hadoop.yarn.server.federation.policies"
+      + ".manager.UniformBroadcastPolicyManager";
 
   public static final String FEDERATION_POLICY_MANAGER_PARAMS =
       FEDERATION_PREFIX + "policy-manager-params";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b370ac69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/AbstractPolicyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/AbstractPolicyManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/AbstractPolicyManager.java
deleted file mode 100644
index e77f2e3..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/AbstractPolicyManager.java
+++ /dev/null
@@ -1,175 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership.  The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations under
- * the License.
- */
-
-package org.apache.hadoop.yarn.server.federation.policies;
-
-import org.apache.hadoop.yarn.server.federation.policies.amrmproxy.FederationAMRMProxyPolicy;
-import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
-import org.apache.hadoop.yarn.server.federation.policies.router.FederationRouterPolicy;
-import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyConfiguration;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * This class provides basic implementation for common methods that multiple
- * policies will need to implement.
- */
-public abstract class AbstractPolicyManager implements
-    FederationPolicyManager {
-
-  private String queue;
-  @SuppressWarnings("checkstyle:visibilitymodifier")
-  protected Class routerFederationPolicy;
-  @SuppressWarnings("checkstyle:visibilitymodifier")
-  protected Class amrmProxyFederationPolicy;
-
-  public static final Logger LOG =
-      LoggerFactory.getLogger(AbstractPolicyManager.class);
-  /**
-   * This default implementation validates the
-   * {@link FederationPolicyInitializationContext},
-   * then checks whether it needs to reinstantiate the class (null or
-   * mismatching type), and reinitialize the policy.
-   *
-   * @param federationPolicyContext the current context
-   * @param oldInstance             the existing (possibly null) instance.
-   *
-   * @return a valid and fully reinitalized {@link FederationAMRMProxyPolicy}
-   * instance
-   *
-   * @throws FederationPolicyInitializationException if the reinitalization is
-   *                                                 not valid, and ensure
-   *                                                 previous state is preserved
-   */
-  public FederationAMRMProxyPolicy getAMRMPolicy(
-      FederationPolicyInitializationContext federationPolicyContext,
-      FederationAMRMProxyPolicy oldInstance)
-      throws FederationPolicyInitializationException {
-
-    if (amrmProxyFederationPolicy == null) {
-      throw new FederationPolicyInitializationException("The parameter "
-          + "amrmProxyFederationPolicy should be initialized in "
-          + this.getClass().getSimpleName() + " constructor.");
-    }
-
-    try {
-      return (FederationAMRMProxyPolicy) internalPolicyGetter(
-          federationPolicyContext, oldInstance, amrmProxyFederationPolicy);
-    } catch (ClassCastException e) {
-      throw new FederationPolicyInitializationException(e);
-    }
-
-  }
-
-  /**
-   * This default implementation validates the
-   * {@link FederationPolicyInitializationContext},
-   * then checks whether it needs to reinstantiate the class (null or
-   * mismatching type), and reinitialize the policy.
-   *
-   * @param federationPolicyContext the current context
-   * @param oldInstance             the existing (possibly null) instance.
-   *
-   * @return a valid and fully reinitalized {@link FederationRouterPolicy}
-   * instance
-   *
-   * @throws FederationPolicyInitializationException if the reinitalization is
-   *                                                 not valid, and ensure
-   *                                                 previous state is preserved
-   */
-
-  public FederationRouterPolicy getRouterPolicy(
-      FederationPolicyInitializationContext federationPolicyContext,
-      FederationRouterPolicy oldInstance)
-      throws FederationPolicyInitializationException {
-
-    //checks that sub-types properly initialize the types of policies
-    if (routerFederationPolicy == null) {
-      throw new FederationPolicyInitializationException("The policy "
-          + "type should be initialized in " + this.getClass().getSimpleName()
-          + " constructor.");
-    }
-
-    try {
-      return (FederationRouterPolicy) internalPolicyGetter(
-          federationPolicyContext, oldInstance, routerFederationPolicy);
-    } catch (ClassCastException e) {
-      throw new FederationPolicyInitializationException(e);
-    }
-  }
-
-  @Override
-  public String getQueue() {
-    return queue;
-  }
-
-  @Override
-  public void setQueue(String queue) {
-    this.queue = queue;
-  }
-
-  /**
-   * Common functionality to instantiate a reinitialize a {@link
-   * ConfigurableFederationPolicy}.
-   */
-  private ConfigurableFederationPolicy internalPolicyGetter(
-      final FederationPolicyInitializationContext federationPolicyContext,
-      ConfigurableFederationPolicy oldInstance, Class policy)
-      throws FederationPolicyInitializationException {
-
-    FederationPolicyInitializationContextValidator
-        .validate(federationPolicyContext, this.getClass().getCanonicalName());
-
-    if (oldInstance == null || !oldInstance.getClass().equals(policy)) {
-      try {
-        oldInstance = (ConfigurableFederationPolicy) policy.newInstance();
-      } catch (InstantiationException e) {
-        throw new FederationPolicyInitializationException(e);
-      } catch (IllegalAccessException e) {
-        throw new FederationPolicyInitializationException(e);
-      }
-    }
-
-    //copying the context to avoid side-effects
-    FederationPolicyInitializationContext modifiedContext =
-        updateContext(federationPolicyContext,
-            oldInstance.getClass().getCanonicalName());
-
-    oldInstance.reinitialize(modifiedContext);
-    return oldInstance;
-  }
-
-  /**
-   * This method is used to copy-on-write the context, that will be passed
-   * downstream to the router/amrmproxy policies.
-   */
-  private FederationPolicyInitializationContext updateContext(
-      FederationPolicyInitializationContext federationPolicyContext,
-      String type) {
-    // copying configuration and context to avoid modification of original
-    SubClusterPolicyConfiguration newConf = SubClusterPolicyConfiguration
-        .newInstance(federationPolicyContext
-            .getSubClusterPolicyConfiguration());
-    newConf.setType(type);
-
-    return new FederationPolicyInitializationContext(newConf,
-                  federationPolicyContext.getFederationSubclusterResolver(),
-                  federationPolicyContext.getFederationStateStoreFacade(),
-                  federationPolicyContext.getHomeSubcluster());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b370ac69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyManager.java
deleted file mode 100644
index 39fdba3..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyManager.java
+++ /dev/null
@@ -1,117 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership.  The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations under
- * the License.
- */
-
-package org.apache.hadoop.yarn.server.federation.policies;
-
-import org.apache.hadoop.yarn.server.federation.policies.amrmproxy.FederationAMRMProxyPolicy;
-import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
-import org.apache.hadoop.yarn.server.federation.policies.router.FederationRouterPolicy;
-import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyConfiguration;
-
-/**
- *
- * Implementors need to provide the ability to serliaze a policy and its
- * configuration as a {@link SubClusterPolicyConfiguration}, as well as provide
- * (re)initialization mechanics for the underlying
- * {@link FederationAMRMProxyPolicy} and {@link FederationRouterPolicy}.
- *
- * The serialization aspects are used by admin APIs or a policy engine to store
- * a serialized configuration in the {@code FederationStateStore}, while the
- * getters methods are used to obtain a propertly inizialized policy in the
- * {@code Router} and {@code AMRMProxy} respectively.
- *
- * This interface by design binds together {@link FederationAMRMProxyPolicy} and
- * {@link FederationRouterPolicy} and provide lifecycle support for
- * serialization and deserialization, to reduce configuration mistakes
- * (combining incompatible policies).
- *
- */
-public interface FederationPolicyManager {
-
-  /**
-   * If the current instance is compatible, this method returns the same
-   * instance of {@link FederationAMRMProxyPolicy} reinitialized with the
-   * current context, otherwise a new instance initialized with the current
-   * context is provided. If the instance is compatible with the current class
-   * the implementors should attempt to reinitalize (retaining state). To affect
-   * a complete policy reset oldInstance should be null.
-   *
-   * @param policyContext the current context
-   * @param oldInstance the existing (possibly null) instance.
-   *
-   * @return an updated {@link FederationAMRMProxyPolicy }.
-   *
-   * @throws FederationPolicyInitializationException if the initialization
-   *           cannot be completed properly. The oldInstance should be still
-   *           valid in case of failed initialization.
-   */
-  FederationAMRMProxyPolicy getAMRMPolicy(
-      FederationPolicyInitializationContext policyContext,
-      FederationAMRMProxyPolicy oldInstance)
-      throws FederationPolicyInitializationException;
-
-  /**
-   * If the current instance is compatible, this method returns the same
-   * instance of {@link FederationRouterPolicy} reinitialized with the current
-   * context, otherwise a new instance initialized with the current context is
-   * provided. If the instance is compatible with the current class the
-   * implementors should attempt to reinitalize (retaining state). To affect a
-   * complete policy reset oldInstance shoulb be set to null.
-   *
-   * @param policyContext the current context
-   * @param oldInstance the existing (possibly null) instance.
-   *
-   * @return an updated {@link FederationRouterPolicy}.
-   *
-   * @throws FederationPolicyInitializationException if the initalization cannot
-   *           be completed properly. The oldInstance should be still valid in
-   *           case of failed initialization.
-   */
-  FederationRouterPolicy getRouterPolicy(
-      FederationPolicyInitializationContext policyContext,
-      FederationRouterPolicy oldInstance)
-      throws FederationPolicyInitializationException;
-
-  /**
-   * This method is invoked to derive a {@link SubClusterPolicyConfiguration}.
-   * This is to be used when writing a policy object in the federation policy
-   * store.
-   *
-   * @return a valid policy configuration representing this object
-   *         parametrization.
-   *
-   * @throws FederationPolicyInitializationException if the current state cannot
-   *           be serialized properly
-   */
-  SubClusterPolicyConfiguration serializeConf()
-      throws FederationPolicyInitializationException;
-
-  /**
-   * This method returns the queue this policy is configured for.
-   *
-   * @return the name of the queue.
-   */
-  String getQueue();
-
-  /**
-   * This methods provides a setter for the queue this policy is specified for.
-   *
-   * @param queue the name of the queue.
-   */
-  void setQueue(String queue);
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b370ac69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/PriorityBroadcastPolicyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/PriorityBroadcastPolicyManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/PriorityBroadcastPolicyManager.java
deleted file mode 100644
index ebdcf42..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/PriorityBroadcastPolicyManager.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership.  The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations under
- * the License.
- */
-
-package org.apache.hadoop.yarn.server.federation.policies;
-
-import java.nio.ByteBuffer;
-
-import org.apache.hadoop.yarn.server.federation.policies.amrmproxy.BroadcastAMRMProxyPolicy;
-import org.apache.hadoop.yarn.server.federation.policies.dao.WeightedPolicyInfo;
-import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
-import org.apache.hadoop.yarn.server.federation.policies.router.PriorityRouterPolicy;
-import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyConfiguration;
-
-import com.google.common.annotations.VisibleForTesting;
-
-/**
- * Policy that allows operator to configure "weights" for routing. This picks a
- * {@link PriorityRouterPolicy} for the router and a
- * {@link BroadcastAMRMProxyPolicy} for the amrmproxy as they are designed to
- * work together.
- */
-public class PriorityBroadcastPolicyManager extends AbstractPolicyManager {
-
-  private WeightedPolicyInfo weightedPolicyInfo;
-
-  public PriorityBroadcastPolicyManager() {
-    // this structurally hard-codes two compatible policies for Router and
-    // AMRMProxy.
-    routerFederationPolicy = PriorityRouterPolicy.class;
-    amrmProxyFederationPolicy = BroadcastAMRMProxyPolicy.class;
-    weightedPolicyInfo = new WeightedPolicyInfo();
-  }
-
-  @Override
-  public SubClusterPolicyConfiguration serializeConf()
-      throws FederationPolicyInitializationException {
-    ByteBuffer buf = weightedPolicyInfo.toByteBuffer();
-    return SubClusterPolicyConfiguration.newInstance(getQueue(),
-        this.getClass().getCanonicalName(), buf);
-  }
-
-  @VisibleForTesting
-  public WeightedPolicyInfo getWeightedPolicyInfo() {
-    return weightedPolicyInfo;
-  }
-
-  @VisibleForTesting
-  public void setWeightedPolicyInfo(WeightedPolicyInfo weightedPolicyInfo) {
-    this.weightedPolicyInfo = weightedPolicyInfo;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b370ac69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/RouterPolicyFacade.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/RouterPolicyFacade.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/RouterPolicyFacade.java
index a3fd15a..8c22623 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/RouterPolicyFacade.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/RouterPolicyFacade.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.federation.policies.amrmproxy.LocalityMulticastAMRMProxyPolicy;
 import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyException;
 import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
+import org.apache.hadoop.yarn.server.federation.policies.manager.FederationPolicyManager;
 import org.apache.hadoop.yarn.server.federation.policies.router.FederationRouterPolicy;
 import org.apache.hadoop.yarn.server.federation.resolver.SubClusterResolver;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b370ac69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/UniformBroadcastPolicyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/UniformBroadcastPolicyManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/UniformBroadcastPolicyManager.java
deleted file mode 100644
index a01f8fa..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/UniformBroadcastPolicyManager.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership.  The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations under
- * the License.
- */
-
-package org.apache.hadoop.yarn.server.federation.policies;
-
-import org.apache.hadoop.yarn.server.federation.policies.amrmproxy.BroadcastAMRMProxyPolicy;
-import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
-import org.apache.hadoop.yarn.server.federation.policies.router.UniformRandomRouterPolicy;
-import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyConfiguration;
-
-import java.nio.ByteBuffer;
-
-/**
- * This class represents a simple implementation of a {@code
- * FederationPolicyManager}.
- *
- * It combines the basic policies: {@link UniformRandomRouterPolicy} and
- * {@link BroadcastAMRMProxyPolicy}, which are designed to work together and
- * "spread" the load among sub-clusters uniformly.
- *
- * This simple policy might impose heavy load on the RMs and return more
- * containers than a job requested as all requests are (replicated and)
- * broadcasted.
- */
-public class UniformBroadcastPolicyManager
-    extends AbstractPolicyManager {
-
-  public UniformBroadcastPolicyManager() {
-    //this structurally hard-codes two compatible policies for Router and
-    // AMRMProxy.
-    routerFederationPolicy = UniformRandomRouterPolicy.class;
-    amrmProxyFederationPolicy = BroadcastAMRMProxyPolicy.class;
-  }
-
-  @Override
-  public SubClusterPolicyConfiguration serializeConf()
-      throws FederationPolicyInitializationException {
-    ByteBuffer buf = ByteBuffer.allocate(0);
-    return SubClusterPolicyConfiguration
-        .newInstance(getQueue(), this.getClass().getCanonicalName(), buf);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b370ac69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/WeightedLocalityPolicyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/WeightedLocalityPolicyManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/WeightedLocalityPolicyManager.java
deleted file mode 100644
index f3c6673..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/WeightedLocalityPolicyManager.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership.  The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations under
- * the License.
- */
-
-package org.apache.hadoop.yarn.server.federation.policies;
-
-import com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.yarn.server.federation.policies.amrmproxy.LocalityMulticastAMRMProxyPolicy;
-import org.apache.hadoop.yarn.server.federation.policies.dao.WeightedPolicyInfo;
-import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
-import org.apache.hadoop.yarn.server.federation.policies.router.WeightedRandomRouterPolicy;
-import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyConfiguration;
-
-import java.nio.ByteBuffer;
-
-/**
- * Policy that allows operator to configure "weights" for routing. This picks a
- * {@link WeightedRandomRouterPolicy} for the router and a {@link
- * LocalityMulticastAMRMProxyPolicy} for the amrmproxy as they are designed to
- * work together.
- */
-public class WeightedLocalityPolicyManager
-    extends AbstractPolicyManager {
-
-  private WeightedPolicyInfo weightedPolicyInfo;
-
-  public WeightedLocalityPolicyManager() {
-    //this structurally hard-codes two compatible policies for Router and
-    // AMRMProxy.
-    routerFederationPolicy =  WeightedRandomRouterPolicy.class;
-    amrmProxyFederationPolicy = LocalityMulticastAMRMProxyPolicy.class;
-    weightedPolicyInfo = new WeightedPolicyInfo();
-  }
-
-  @Override
-  public SubClusterPolicyConfiguration serializeConf()
-      throws FederationPolicyInitializationException {
-    ByteBuffer buf = weightedPolicyInfo.toByteBuffer();
-    return SubClusterPolicyConfiguration
-        .newInstance(getQueue(), this.getClass().getCanonicalName(), buf);
-  }
-
-  @VisibleForTesting
-  public WeightedPolicyInfo getWeightedPolicyInfo() {
-    return weightedPolicyInfo;
-  }
-
-  @VisibleForTesting
-  public void setWeightedPolicyInfo(
-      WeightedPolicyInfo weightedPolicyInfo) {
-    this.weightedPolicyInfo = weightedPolicyInfo;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b370ac69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/manager/AbstractPolicyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/manager/AbstractPolicyManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/manager/AbstractPolicyManager.java
new file mode 100644
index 0000000..f7a89c6
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/manager/AbstractPolicyManager.java
@@ -0,0 +1,190 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies.manager;
+
+import org.apache.hadoop.yarn.server.federation.policies.ConfigurableFederationPolicy;
+import org.apache.hadoop.yarn.server.federation.policies.FederationPolicyInitializationContext;
+import org.apache.hadoop.yarn.server.federation.policies.FederationPolicyInitializationContextValidator;
+import org.apache.hadoop.yarn.server.federation.policies.amrmproxy.FederationAMRMProxyPolicy;
+import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
+import org.apache.hadoop.yarn.server.federation.policies.router.FederationRouterPolicy;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyConfiguration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.ByteBuffer;
+
+/**
+ * This class provides basic implementation for common methods that multiple
+ * policies will need to implement.
+ */
+public abstract class AbstractPolicyManager implements
+    FederationPolicyManager {
+
+  private String queue;
+  @SuppressWarnings("checkstyle:visibilitymodifier")
+  protected Class routerFederationPolicy;
+  @SuppressWarnings("checkstyle:visibilitymodifier")
+  protected Class amrmProxyFederationPolicy;
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(AbstractPolicyManager.class);
+  /**
+   * This default implementation validates the
+   * {@link FederationPolicyInitializationContext},
+   * then checks whether it needs to reinstantiate the class (null or
+   * mismatching type), and reinitialize the policy.
+   *
+   * @param federationPolicyContext the current context
+   * @param oldInstance             the existing (possibly null) instance.
+   *
+   * @return a valid and fully reinitalized {@link FederationAMRMProxyPolicy}
+   * instance
+   *
+   * @throws FederationPolicyInitializationException if the reinitalization is
+   *                                                 not valid, and ensure
+   *                                                 previous state is preserved
+   */
+  public FederationAMRMProxyPolicy getAMRMPolicy(
+      FederationPolicyInitializationContext federationPolicyContext,
+      FederationAMRMProxyPolicy oldInstance)
+      throws FederationPolicyInitializationException {
+
+    if (amrmProxyFederationPolicy == null) {
+      throw new FederationPolicyInitializationException("The parameter "
+          + "amrmProxyFederationPolicy should be initialized in "
+          + this.getClass().getSimpleName() + " constructor.");
+    }
+
+    try {
+      return (FederationAMRMProxyPolicy) internalPolicyGetter(
+          federationPolicyContext, oldInstance, amrmProxyFederationPolicy);
+    } catch (ClassCastException e) {
+      throw new FederationPolicyInitializationException(e);
+    }
+
+  }
+
+  /**
+   * This default implementation validates the
+   * {@link FederationPolicyInitializationContext},
+   * then checks whether it needs to reinstantiate the class (null or
+   * mismatching type), and reinitialize the policy.
+   *
+   * @param federationPolicyContext the current context
+   * @param oldInstance             the existing (possibly null) instance.
+   *
+   * @return a valid and fully reinitalized {@link FederationRouterPolicy}
+   * instance
+   *
+   * @throws FederationPolicyInitializationException if the reinitalization is
+   *                                                 not valid, and ensure
+   *                                                 previous state is preserved
+   */
+
+  public FederationRouterPolicy getRouterPolicy(
+      FederationPolicyInitializationContext federationPolicyContext,
+      FederationRouterPolicy oldInstance)
+      throws FederationPolicyInitializationException {
+
+    //checks that sub-types properly initialize the types of policies
+    if (routerFederationPolicy == null) {
+      throw new FederationPolicyInitializationException("The policy "
+          + "type should be initialized in " + this.getClass().getSimpleName()
+          + " constructor.");
+    }
+
+    try {
+      return (FederationRouterPolicy) internalPolicyGetter(
+          federationPolicyContext, oldInstance, routerFederationPolicy);
+    } catch (ClassCastException e) {
+      throw new FederationPolicyInitializationException(e);
+    }
+  }
+
+  @Override
+  public SubClusterPolicyConfiguration serializeConf()
+      throws FederationPolicyInitializationException {
+    // default implementation works only for sub-classes which do not require
+    // any parameters
+    ByteBuffer buf = ByteBuffer.allocate(0);
+    return SubClusterPolicyConfiguration
+        .newInstance(getQueue(), this.getClass().getCanonicalName(), buf);
+  }
+
+  @Override
+  public String getQueue() {
+    return queue;
+  }
+
+  @Override
+  public void setQueue(String queue) {
+    this.queue = queue;
+  }
+
+  /**
+   * Common functionality to instantiate a reinitialize a {@link
+   * ConfigurableFederationPolicy}.
+   */
+  private ConfigurableFederationPolicy internalPolicyGetter(
+      final FederationPolicyInitializationContext federationPolicyContext,
+      ConfigurableFederationPolicy oldInstance, Class policy)
+      throws FederationPolicyInitializationException {
+
+    FederationPolicyInitializationContextValidator
+        .validate(federationPolicyContext, this.getClass().getCanonicalName());
+
+    if (oldInstance == null || !oldInstance.getClass().equals(policy)) {
+      try {
+        oldInstance = (ConfigurableFederationPolicy) policy.newInstance();
+      } catch (InstantiationException e) {
+        throw new FederationPolicyInitializationException(e);
+      } catch (IllegalAccessException e) {
+        throw new FederationPolicyInitializationException(e);
+      }
+    }
+
+    //copying the context to avoid side-effects
+    FederationPolicyInitializationContext modifiedContext =
+        updateContext(federationPolicyContext,
+            oldInstance.getClass().getCanonicalName());
+
+    oldInstance.reinitialize(modifiedContext);
+    return oldInstance;
+  }
+
+  /**
+   * This method is used to copy-on-write the context, that will be passed
+   * downstream to the router/amrmproxy policies.
+   */
+  private FederationPolicyInitializationContext updateContext(
+      FederationPolicyInitializationContext federationPolicyContext,
+      String type) {
+    // copying configuration and context to avoid modification of original
+    SubClusterPolicyConfiguration newConf = SubClusterPolicyConfiguration
+        .newInstance(federationPolicyContext
+            .getSubClusterPolicyConfiguration());
+    newConf.setType(type);
+
+    return new FederationPolicyInitializationContext(newConf,
+                  federationPolicyContext.getFederationSubclusterResolver(),
+                  federationPolicyContext.getFederationStateStoreFacade(),
+                  federationPolicyContext.getHomeSubcluster());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b370ac69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/manager/FederationPolicyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/manager/FederationPolicyManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/manager/FederationPolicyManager.java
new file mode 100644
index 0000000..1434c80
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/manager/FederationPolicyManager.java
@@ -0,0 +1,118 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies.manager;
+
+import org.apache.hadoop.yarn.server.federation.policies.FederationPolicyInitializationContext;
+import org.apache.hadoop.yarn.server.federation.policies.amrmproxy.FederationAMRMProxyPolicy;
+import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
+import org.apache.hadoop.yarn.server.federation.policies.router.FederationRouterPolicy;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyConfiguration;
+
+/**
+ *
+ * Implementors need to provide the ability to serliaze a policy and its
+ * configuration as a {@link SubClusterPolicyConfiguration}, as well as provide
+ * (re)initialization mechanics for the underlying
+ * {@link FederationAMRMProxyPolicy} and {@link FederationRouterPolicy}.
+ *
+ * The serialization aspects are used by admin APIs or a policy engine to store
+ * a serialized configuration in the {@code FederationStateStore}, while the
+ * getters methods are used to obtain a propertly inizialized policy in the
+ * {@code Router} and {@code AMRMProxy} respectively.
+ *
+ * This interface by design binds together {@link FederationAMRMProxyPolicy} and
+ * {@link FederationRouterPolicy} and provide lifecycle support for
+ * serialization and deserialization, to reduce configuration mistakes
+ * (combining incompatible policies).
+ *
+ */
+public interface FederationPolicyManager {
+
+  /**
+   * If the current instance is compatible, this method returns the same
+   * instance of {@link FederationAMRMProxyPolicy} reinitialized with the
+   * current context, otherwise a new instance initialized with the current
+   * context is provided. If the instance is compatible with the current class
+   * the implementors should attempt to reinitalize (retaining state). To affect
+   * a complete policy reset oldInstance should be null.
+   *
+   * @param policyContext the current context
+   * @param oldInstance the existing (possibly null) instance.
+   *
+   * @return an updated {@link FederationAMRMProxyPolicy }.
+   *
+   * @throws FederationPolicyInitializationException if the initialization
+   *           cannot be completed properly. The oldInstance should be still
+   *           valid in case of failed initialization.
+   */
+  FederationAMRMProxyPolicy getAMRMPolicy(
+      FederationPolicyInitializationContext policyContext,
+      FederationAMRMProxyPolicy oldInstance)
+      throws FederationPolicyInitializationException;
+
+  /**
+   * If the current instance is compatible, this method returns the same
+   * instance of {@link FederationRouterPolicy} reinitialized with the current
+   * context, otherwise a new instance initialized with the current context is
+   * provided. If the instance is compatible with the current class the
+   * implementors should attempt to reinitalize (retaining state). To affect a
+   * complete policy reset oldInstance shoulb be set to null.
+   *
+   * @param policyContext the current context
+   * @param oldInstance the existing (possibly null) instance.
+   *
+   * @return an updated {@link FederationRouterPolicy}.
+   *
+   * @throws FederationPolicyInitializationException if the initalization cannot
+   *           be completed properly. The oldInstance should be still valid in
+   *           case of failed initialization.
+   */
+  FederationRouterPolicy getRouterPolicy(
+      FederationPolicyInitializationContext policyContext,
+      FederationRouterPolicy oldInstance)
+      throws FederationPolicyInitializationException;
+
+  /**
+   * This method is invoked to derive a {@link SubClusterPolicyConfiguration}.
+   * This is to be used when writing a policy object in the federation policy
+   * store.
+   *
+   * @return a valid policy configuration representing this object
+   *         parametrization.
+   *
+   * @throws FederationPolicyInitializationException if the current state cannot
+   *           be serialized properly
+   */
+  SubClusterPolicyConfiguration serializeConf()
+      throws FederationPolicyInitializationException;
+
+  /**
+   * This method returns the queue this policy is configured for.
+   *
+   * @return the name of the queue.
+   */
+  String getQueue();
+
+  /**
+   * This methods provides a setter for the queue this policy is specified for.
+   *
+   * @param queue the name of the queue.
+   */
+  void setQueue(String queue);
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b370ac69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/manager/HashBroadcastPolicyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/manager/HashBroadcastPolicyManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/manager/HashBroadcastPolicyManager.java
new file mode 100644
index 0000000..08ab08f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/manager/HashBroadcastPolicyManager.java
@@ -0,0 +1,38 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies.manager;
+
+import org.apache.hadoop.yarn.server.federation.policies.amrmproxy.BroadcastAMRMProxyPolicy;
+import org.apache.hadoop.yarn.server.federation.policies.router.HashBasedRouterPolicy;
+
+/**
+ * Policy that routes applications via hashing of their queuename, and broadcast
+ * resource requests. This picks a {@link HashBasedRouterPolicy} for the router
+ * and a {@link BroadcastAMRMProxyPolicy} for the amrmproxy as they are designed
+ * to work together.
+ */
+public class HashBroadcastPolicyManager extends AbstractPolicyManager {
+
+  public HashBroadcastPolicyManager() {
+    // this structurally hard-codes two compatible policies for Router and
+    // AMRMProxy.
+    routerFederationPolicy = HashBasedRouterPolicy.class;
+    amrmProxyFederationPolicy = BroadcastAMRMProxyPolicy.class;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b370ac69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/manager/PriorityBroadcastPolicyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/manager/PriorityBroadcastPolicyManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/manager/PriorityBroadcastPolicyManager.java
new file mode 100644
index 0000000..8139e12
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/manager/PriorityBroadcastPolicyManager.java
@@ -0,0 +1,66 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies.manager;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.yarn.server.federation.policies.amrmproxy.BroadcastAMRMProxyPolicy;
+import org.apache.hadoop.yarn.server.federation.policies.dao.WeightedPolicyInfo;
+import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
+import org.apache.hadoop.yarn.server.federation.policies.router.PriorityRouterPolicy;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyConfiguration;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Policy that allows operator to configure "weights" for routing. This picks a
+ * {@link PriorityRouterPolicy} for the router and a
+ * {@link BroadcastAMRMProxyPolicy} for the amrmproxy as they are designed to
+ * work together.
+ */
+public class PriorityBroadcastPolicyManager extends AbstractPolicyManager {
+
+  private WeightedPolicyInfo weightedPolicyInfo;
+
+  public PriorityBroadcastPolicyManager() {
+    // this structurally hard-codes two compatible policies for Router and
+    // AMRMProxy.
+    routerFederationPolicy = PriorityRouterPolicy.class;
+    amrmProxyFederationPolicy = BroadcastAMRMProxyPolicy.class;
+    weightedPolicyInfo = new WeightedPolicyInfo();
+  }
+
+  @Override
+  public SubClusterPolicyConfiguration serializeConf()
+      throws FederationPolicyInitializationException {
+    ByteBuffer buf = weightedPolicyInfo.toByteBuffer();
+    return SubClusterPolicyConfiguration.newInstance(getQueue(),
+        this.getClass().getCanonicalName(), buf);
+  }
+
+  @VisibleForTesting
+  public WeightedPolicyInfo getWeightedPolicyInfo() {
+    return weightedPolicyInfo;
+  }
+
+  @VisibleForTesting
+  public void setWeightedPolicyInfo(WeightedPolicyInfo weightedPolicyInfo) {
+    this.weightedPolicyInfo = weightedPolicyInfo;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b370ac69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/manager/UniformBroadcastPolicyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/manager/UniformBroadcastPolicyManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/manager/UniformBroadcastPolicyManager.java
new file mode 100644
index 0000000..5db0466
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/manager/UniformBroadcastPolicyManager.java
@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies.manager;
+
+import org.apache.hadoop.yarn.server.federation.policies.amrmproxy.BroadcastAMRMProxyPolicy;
+import org.apache.hadoop.yarn.server.federation.policies.router.UniformRandomRouterPolicy;
+
+/**
+ * This class represents a simple implementation of a {@code
+ * FederationPolicyManager}.
+ *
+ * It combines the basic policies: {@link UniformRandomRouterPolicy} and
+ * {@link BroadcastAMRMProxyPolicy}, which are designed to work together and
+ * "spread" the load among sub-clusters uniformly.
+ *
+ * This simple policy might impose heavy load on the RMs and return more
+ * containers than a job requested as all requests are (replicated and)
+ * broadcasted.
+ */
+public class UniformBroadcastPolicyManager extends AbstractPolicyManager {
+
+  public UniformBroadcastPolicyManager() {
+    // this structurally hard-codes two compatible policies for Router and
+    // AMRMProxy.
+    routerFederationPolicy = UniformRandomRouterPolicy.class;
+    amrmProxyFederationPolicy = BroadcastAMRMProxyPolicy.class;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b370ac69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/manager/WeightedLocalityPolicyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/manager/WeightedLocalityPolicyManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/manager/WeightedLocalityPolicyManager.java
new file mode 100644
index 0000000..109b534
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/manager/WeightedLocalityPolicyManager.java
@@ -0,0 +1,67 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies.manager;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.yarn.server.federation.policies.amrmproxy.LocalityMulticastAMRMProxyPolicy;
+import org.apache.hadoop.yarn.server.federation.policies.dao.WeightedPolicyInfo;
+import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
+import org.apache.hadoop.yarn.server.federation.policies.router.WeightedRandomRouterPolicy;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyConfiguration;
+
+import java.nio.ByteBuffer;
+
+/**
+ * Policy that allows operator to configure "weights" for routing. This picks a
+ * {@link WeightedRandomRouterPolicy} for the router and a {@link
+ * LocalityMulticastAMRMProxyPolicy} for the amrmproxy as they are designed to
+ * work together.
+ */
+public class WeightedLocalityPolicyManager
+    extends AbstractPolicyManager {
+
+  private WeightedPolicyInfo weightedPolicyInfo;
+
+  public WeightedLocalityPolicyManager() {
+    //this structurally hard-codes two compatible policies for Router and
+    // AMRMProxy.
+    routerFederationPolicy =  WeightedRandomRouterPolicy.class;
+    amrmProxyFederationPolicy = LocalityMulticastAMRMProxyPolicy.class;
+    weightedPolicyInfo = new WeightedPolicyInfo();
+  }
+
+  @Override
+  public SubClusterPolicyConfiguration serializeConf()
+      throws FederationPolicyInitializationException {
+    ByteBuffer buf = weightedPolicyInfo.toByteBuffer();
+    return SubClusterPolicyConfiguration
+        .newInstance(getQueue(), this.getClass().getCanonicalName(), buf);
+  }
+
+  @VisibleForTesting
+  public WeightedPolicyInfo getWeightedPolicyInfo() {
+    return weightedPolicyInfo;
+  }
+
+  @VisibleForTesting
+  public void setWeightedPolicyInfo(
+      WeightedPolicyInfo weightedPolicyInfo) {
+    this.weightedPolicyInfo = weightedPolicyInfo;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b370ac69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/manager/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/manager/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/manager/package-info.java
new file mode 100644
index 0000000..9515c01
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/manager/package-info.java
@@ -0,0 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/** Various implementation of FederationPolicyManager. **/
+package org.apache.hadoop.yarn.server.federation.policies.manager;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b370ac69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/AbstractRouterPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/AbstractRouterPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/AbstractRouterPolicy.java
index f49af1d..730fb41 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/AbstractRouterPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/AbstractRouterPolicy.java
@@ -20,8 +20,11 @@ package org.apache.hadoop.yarn.server.federation.policies.router;
 
 import java.util.Map;
 
+import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.federation.policies.AbstractConfigurableFederationPolicy;
 import org.apache.hadoop.yarn.server.federation.policies.dao.WeightedPolicyInfo;
+import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyException;
 import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterIdInfo;
 
@@ -44,4 +47,20 @@ public abstract class AbstractRouterPolicy extends
     }
   }
 
+  public void validate(ApplicationSubmissionContext appSubmissionContext)
+      throws FederationPolicyException {
+
+    if (appSubmissionContext == null) {
+      throw new FederationPolicyException(
+          "Cannot route an application with null context.");
+    }
+
+    // if the queue is not specified we set it to default value, to be
+    // compatible with YARN behavior.
+    String queue = appSubmissionContext.getQueue();
+    if (queue == null) {
+      appSubmissionContext.setQueue(YarnConfiguration.DEFAULT_QUEUE_NAME);
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b370ac69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/HashBasedRouterPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/HashBasedRouterPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/HashBasedRouterPolicy.java
new file mode 100644
index 0000000..e40e87e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/HashBasedRouterPolicy.java
@@ -0,0 +1,81 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies.router;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.federation.policies.FederationPolicyInitializationContext;
+import org.apache.hadoop.yarn.server.federation.policies.FederationPolicyInitializationContextValidator;
+import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
+
+/**
+ * This {@link FederationRouterPolicy} pick a subcluster based on the hash of
+ * the job's queue name. Useful to provide a default behavior when too many
+ * queues exist in a system. This also ensures that all jobs belonging to a
+ * queue are mapped to the same sub-cluster (likely help with locality).
+ */
+public class HashBasedRouterPolicy extends AbstractRouterPolicy {
+
+  @Override
+  public void reinitialize(
+      FederationPolicyInitializationContext federationPolicyContext)
+      throws FederationPolicyInitializationException {
+    FederationPolicyInitializationContextValidator
+        .validate(federationPolicyContext, this.getClass().getCanonicalName());
+
+    // note: this overrides BaseRouterPolicy and ignores the weights
+    setPolicyContext(federationPolicyContext);
+  }
+
+  /**
+   * Simply picks from alphabetically-sorted active subclusters based on the
+   * hash of quey name. Jobs of the same queue will all be routed to the same
+   * sub-cluster, as far as the number of active sub-cluster and their names
+   * remain the same.
+   *
+   * @param appSubmissionContext the context for the app being submitted.
+   *
+   * @return a hash-based chosen subcluster.
+   *
+   * @throws YarnException if there are no active subclusters.
+   */
+  public SubClusterId getHomeSubcluster(
+      ApplicationSubmissionContext appSubmissionContext) throws YarnException {
+
+    // throws if no active subclusters available
+    Map<SubClusterId, SubClusterInfo> activeSubclusters =
+        getActiveSubclusters();
+
+    validate(appSubmissionContext);
+
+    int chosenPosition = Math.abs(
+        appSubmissionContext.getQueue().hashCode() % activeSubclusters.size());
+
+    List<SubClusterId> list = new ArrayList<>(activeSubclusters.keySet());
+    Collections.sort(list);
+    return list.get(chosenPosition);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b370ac69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/LoadBasedRouterPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/LoadBasedRouterPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/LoadBasedRouterPolicy.java
index 5de749f..2ca15bf 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/LoadBasedRouterPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/LoadBasedRouterPolicy.java
@@ -64,6 +64,9 @@ public class LoadBasedRouterPolicy extends AbstractRouterPolicy {
   public SubClusterId getHomeSubcluster(
       ApplicationSubmissionContext appSubmissionContext) throws YarnException {
 
+    // null checks and default-queue behavior
+    validate(appSubmissionContext);
+
     Map<SubClusterId, SubClusterInfo> activeSubclusters =
         getActiveSubclusters();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b370ac69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/PriorityRouterPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/PriorityRouterPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/PriorityRouterPolicy.java
index bc3a1f7..13d9140 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/PriorityRouterPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/PriorityRouterPolicy.java
@@ -36,6 +36,9 @@ public class PriorityRouterPolicy extends AbstractRouterPolicy {
   public SubClusterId getHomeSubcluster(
       ApplicationSubmissionContext appSubmissionContext) throws YarnException {
 
+    // null checks and default-queue behavior
+    validate(appSubmissionContext);
+
     Map<SubClusterId, SubClusterInfo> activeSubclusters =
         getActiveSubclusters();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b370ac69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/UniformRandomRouterPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/UniformRandomRouterPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/UniformRandomRouterPolicy.java
index b8f9cc3..d820449 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/UniformRandomRouterPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/UniformRandomRouterPolicy.java
@@ -48,11 +48,10 @@ public class UniformRandomRouterPolicy extends AbstractRouterPolicy {
   }
 
   @Override
-  public void reinitialize(
-      FederationPolicyInitializationContext policyContext)
+  public void reinitialize(FederationPolicyInitializationContext policyContext)
       throws FederationPolicyInitializationException {
-    FederationPolicyInitializationContextValidator
-        .validate(policyContext, this.getClass().getCanonicalName());
+    FederationPolicyInitializationContextValidator.validate(policyContext,
+        this.getClass().getCanonicalName());
 
     // note: this overrides AbstractRouterPolicy and ignores the weights
 
@@ -73,6 +72,9 @@ public class UniformRandomRouterPolicy extends AbstractRouterPolicy {
   public SubClusterId getHomeSubcluster(
       ApplicationSubmissionContext appSubmissionContext) throws YarnException {
 
+    // null checks and default-queue behavior
+    validate(appSubmissionContext);
+
     Map<SubClusterId, SubClusterInfo> activeSubclusters =
         getActiveSubclusters();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b370ac69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/WeightedRandomRouterPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/WeightedRandomRouterPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/WeightedRandomRouterPolicy.java
index ac75ae9..5727134 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/WeightedRandomRouterPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/WeightedRandomRouterPolicy.java
@@ -43,6 +43,9 @@ public class WeightedRandomRouterPolicy extends AbstractRouterPolicy {
   public SubClusterId getHomeSubcluster(
       ApplicationSubmissionContext appSubmissionContext) throws YarnException {
 
+    // null checks and default-queue behavior
+    validate(appSubmissionContext);
+
     Map<SubClusterId, SubClusterInfo> activeSubclusters =
         getActiveSubclusters();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b370ac69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/BaseFederationPoliciesTest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/BaseFederationPoliciesTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/BaseFederationPoliciesTest.java
index ba897da..6bd8bf0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/BaseFederationPoliciesTest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/BaseFederationPoliciesTest.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.yarn.server.federation.policies;
 
 import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
 
 import java.nio.ByteBuffer;
 import java.util.HashMap;
@@ -35,8 +36,10 @@ import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPo
 import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
 import org.apache.hadoop.yarn.server.federation.policies.router.FederationRouterPolicy;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterIdInfo;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyConfiguration;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterState;
 import org.apache.hadoop.yarn.server.federation.utils.FederationPoliciesTestUtil;
 import org.junit.Test;
 
@@ -46,7 +49,7 @@ import org.junit.Test;
 public abstract class BaseFederationPoliciesTest {
 
   private ConfigurableFederationPolicy policy;
-  private WeightedPolicyInfo policyInfo;
+  private WeightedPolicyInfo policyInfo = mock(WeightedPolicyInfo.class);
   private Map<SubClusterId, SubClusterInfo> activeSubclusters = new HashMap<>();
   private FederationPolicyInitializationContext federationPolicyContext;
   private ApplicationSubmissionContext applicationSubmissionContext =
@@ -103,7 +106,7 @@ public abstract class BaseFederationPoliciesTest {
       ((FederationRouterPolicy) localPolicy)
           .getHomeSubcluster(getApplicationSubmissionContext());
     } else {
-      String[] hosts = new String[] {"host1", "host2" };
+      String[] hosts = new String[] {"host1", "host2"};
       List<ResourceRequest> resourceRequests = FederationPoliciesTestUtil
           .createResourceRequests(hosts, 2 * 1024, 2, 1, 3, null, false);
       ((FederationAMRMProxyPolicy) localPolicy)
@@ -170,4 +173,14 @@ public abstract class BaseFederationPoliciesTest {
     this.homeSubCluster = homeSubCluster;
   }
 
+  public void setMockActiveSubclusters(int numSubclusters) {
+    for (int i = 1; i <= numSubclusters; i++) {
+      SubClusterIdInfo sc = new SubClusterIdInfo("sc" + i);
+      SubClusterInfo sci = mock(SubClusterInfo.class);
+      when(sci.getState()).thenReturn(SubClusterState.SC_RUNNING);
+      when(sci.getSubClusterId()).thenReturn(sc.toId());
+      getActiveSubclusters().put(sc.toId(), sci);
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b370ac69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/BasePolicyManagerTest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/BasePolicyManagerTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/BasePolicyManagerTest.java
deleted file mode 100644
index c609886..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/BasePolicyManagerTest.java
+++ /dev/null
@@ -1,108 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership.  The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations under
- * the License.
- */
-
-package org.apache.hadoop.yarn.server.federation.policies;
-
-import org.apache.hadoop.yarn.server.federation.policies.amrmproxy.FederationAMRMProxyPolicy;
-import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
-import org.apache.hadoop.yarn.server.federation.policies.router.FederationRouterPolicy;
-import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
-import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyConfiguration;
-import org.apache.hadoop.yarn.server.federation.utils.FederationPoliciesTestUtil;
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * This class provides common test methods for testing {@code
- * FederationPolicyManager}s.
- */
-public abstract class BasePolicyManagerTest {
-
-
-  @SuppressWarnings("checkstyle:visibilitymodifier")
-  protected FederationPolicyManager wfp = null;
-  @SuppressWarnings("checkstyle:visibilitymodifier")
-  protected Class expectedPolicyManager;
-  @SuppressWarnings("checkstyle:visibilitymodifier")
-  protected Class expectedAMRMProxyPolicy;
-  @SuppressWarnings("checkstyle:visibilitymodifier")
-  protected Class expectedRouterPolicy;
-
-
-  @Test
-  public void testSerializeAndInstantiate() throws Exception {
-    serializeAndDeserializePolicyManager(wfp, expectedPolicyManager,
-        expectedAMRMProxyPolicy,
-        expectedRouterPolicy);
-  }
-
-  @Test(expected = FederationPolicyInitializationException.class)
-  public void testSerializeAndInstantiateBad1() throws Exception {
-    serializeAndDeserializePolicyManager(wfp, String.class,
-        expectedAMRMProxyPolicy, expectedRouterPolicy);
-  }
-
-  @Test(expected = AssertionError.class)
-  public void testSerializeAndInstantiateBad2() throws Exception {
-    serializeAndDeserializePolicyManager(wfp, expectedPolicyManager,
-        String.class, expectedRouterPolicy);
-  }
-
-  @Test(expected = AssertionError.class)
-  public void testSerializeAndInstantiateBad3() throws Exception {
-    serializeAndDeserializePolicyManager(wfp, expectedPolicyManager,
-        expectedAMRMProxyPolicy, String.class);
-  }
-
-  protected static void serializeAndDeserializePolicyManager(
-      FederationPolicyManager wfp, Class policyManagerType,
-      Class expAMRMProxyPolicy, Class expRouterPolicy) throws Exception {
-
-    // serializeConf it in a context
-    SubClusterPolicyConfiguration fpc =
-        wfp.serializeConf();
-    fpc.setType(policyManagerType.getCanonicalName());
-    FederationPolicyInitializationContext context = new
-        FederationPolicyInitializationContext();
-    context.setSubClusterPolicyConfiguration(fpc);
-    context
-        .setFederationStateStoreFacade(FederationPoliciesTestUtil.initFacade());
-    context.setFederationSubclusterResolver(
-        FederationPoliciesTestUtil.initResolver());
-    context.setHomeSubcluster(SubClusterId.newInstance("homesubcluster"));
-
-    // based on the "context" created instantiate new class and use it
-    Class c = Class.forName(wfp.getClass().getCanonicalName());
-    FederationPolicyManager wfp2 = (FederationPolicyManager) c.newInstance();
-
-    FederationAMRMProxyPolicy federationAMRMProxyPolicy =
-        wfp2.getAMRMPolicy(context, null);
-
-    //needed only for tests (getARMRMPolicy change the "type" in conf)
-    fpc.setType(wfp.getClass().getCanonicalName());
-
-    FederationRouterPolicy federationRouterPolicy =
-        wfp2.getRouterPolicy(context, null);
-
-    Assert.assertEquals(federationAMRMProxyPolicy.getClass(),
-        expAMRMProxyPolicy);
-
-    Assert.assertEquals(federationRouterPolicy.getClass(),
-        expRouterPolicy);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b370ac69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestFederationPolicyInitializationContextValidator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestFederationPolicyInitializationContextValidator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestFederationPolicyInitializationContextValidator.java
index d906b92..611a486 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestFederationPolicyInitializationContextValidator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestFederationPolicyInitializationContextValidator.java
@@ -22,6 +22,7 @@ import java.nio.ByteBuffer;
 
 import org.apache.hadoop.yarn.server.federation.policies.amrmproxy.FederationAMRMProxyPolicy;
 import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
+import org.apache.hadoop.yarn.server.federation.policies.manager.FederationPolicyManager;
 import org.apache.hadoop.yarn.server.federation.policies.router.FederationRouterPolicy;
 import org.apache.hadoop.yarn.server.federation.resolver.SubClusterResolver;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b370ac69/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestPriorityBroadcastPolicyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestPriorityBroadcastPolicyManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestPriorityBroadcastPolicyManager.java
deleted file mode 100644
index 5e5bc83..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestPriorityBroadcastPolicyManager.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership.  The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations under
- * the License.
- */
-
-package org.apache.hadoop.yarn.server.federation.policies;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.hadoop.yarn.server.federation.policies.amrmproxy.BroadcastAMRMProxyPolicy;
-import org.apache.hadoop.yarn.server.federation.policies.dao.WeightedPolicyInfo;
-import org.apache.hadoop.yarn.server.federation.policies.router.PriorityRouterPolicy;
-import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
-import org.apache.hadoop.yarn.server.federation.store.records.SubClusterIdInfo;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-/**
- * Simple test of {@link PriorityBroadcastPolicyManager}.
- */
-public class TestPriorityBroadcastPolicyManager extends BasePolicyManagerTest {
-
-  private WeightedPolicyInfo policyInfo;
-
-  @Before
-  public void setup() {
-    // configure a policy
-
-    wfp = new PriorityBroadcastPolicyManager();
-    wfp.setQueue("queue1");
-    SubClusterId sc1 = SubClusterId.newInstance("sc1");
-    SubClusterId sc2 = SubClusterId.newInstance("sc2");
-    policyInfo = new WeightedPolicyInfo();
-
-    Map<SubClusterIdInfo, Float> routerWeights = new HashMap<>();
-    routerWeights.put(new SubClusterIdInfo(sc1), 0.2f);
-    routerWeights.put(new SubClusterIdInfo(sc2), 0.8f);
-    policyInfo.setRouterPolicyWeights(routerWeights);
-
-    ((PriorityBroadcastPolicyManager) wfp).setWeightedPolicyInfo(policyInfo);
-
-    // set expected params that the base test class will use for tests
-    expectedPolicyManager = PriorityBroadcastPolicyManager.class;
-    expectedAMRMProxyPolicy = BroadcastAMRMProxyPolicy.class;
-    expectedRouterPolicy = PriorityRouterPolicy.class;
-  }
-
-  @Test
-  public void testPolicyInfoSetCorrectly() throws Exception {
-    serializeAndDeserializePolicyManager(wfp, expectedPolicyManager,
-        expectedAMRMProxyPolicy, expectedRouterPolicy);
-
-    // check the policyInfo propagates through ser/der correctly
-    Assert.assertEquals(
-        ((PriorityBroadcastPolicyManager) wfp).getWeightedPolicyInfo(),
-        policyInfo);
-  }
-}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[34/42] hadoop git commit: YARN-6247. Share a single instance of SubClusterResolver instead of instantiating one per AM. (Botong Huang via Subru)

Posted by su...@apache.org.
YARN-6247. Share a single instance of SubClusterResolver instead of instantiating one per AM. (Botong Huang via Subru)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/a3450faf
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/a3450faf
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/a3450faf

Branch: refs/heads/YARN-2915
Commit: a3450faf11ac0fbdc00d8d0241af8835545ba02c
Parents: 8221ffa
Author: Subru Krishnan <su...@apache.org>
Authored: Thu Mar 2 18:54:53 2017 -0800
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon May 8 16:59:13 2017 -0700

----------------------------------------------------------------------
 .../hadoop/yarn/conf/YarnConfiguration.java     |  6 +++
 .../src/main/resources/yarn-default.xml         |  7 +++
 .../resolver/AbstractSubClusterResolver.java    |  6 +--
 .../federation/resolver/SubClusterResolver.java |  4 +-
 .../utils/FederationStateStoreFacade.java       | 48 +++++++++++++++++---
 5 files changed, 59 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a3450faf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
index dd05411..4ba8c85 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
@@ -2572,6 +2572,12 @@ public class YarnConfiguration extends Configuration {
   public static final String FEDERATION_MACHINE_LIST =
       FEDERATION_PREFIX + "machine-list";
 
+  public static final String FEDERATION_CLUSTER_RESOLVER_CLASS =
+      FEDERATION_PREFIX + "subcluster-resolver.class";
+  public static final String DEFAULT_FEDERATION_CLUSTER_RESOLVER_CLASS =
+      "org.apache.hadoop.yarn.server.federation.resolver."
+          + "DefaultSubClusterResolverImpl";
+
   public static final String DEFAULT_FEDERATION_POLICY_KEY = "*";
 
   public static final String FEDERATION_POLICY_MANAGER = FEDERATION_PREFIX

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a3450faf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
index 0d40482..75b32ce 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
@@ -2683,6 +2683,13 @@
     </description>
     <name>yarn.federation.machine-list</name>
   </property>
+  <property>
+    <description>
+      Class name for SubClusterResolver
+    </description>
+    <name>yarn.federation.subcluster-resolver.class</name>
+    <value>org.apache.hadoop.yarn.server.federation.resolver.DefaultSubClusterResolverImpl</value>
+  </property>
 
   <property>
     <description>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a3450faf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/resolver/AbstractSubClusterResolver.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/resolver/AbstractSubClusterResolver.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/resolver/AbstractSubClusterResolver.java
index 6b4f60c..bccff2d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/resolver/AbstractSubClusterResolver.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/resolver/AbstractSubClusterResolver.java
@@ -21,8 +21,8 @@ package org.apache.hadoop.yarn.server.federation.resolver;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
 
-import java.util.HashMap;
 import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.Map;
 
 /**
@@ -31,9 +31,9 @@ import java.util.Map;
  */
 public abstract class AbstractSubClusterResolver implements SubClusterResolver {
   private Map<String, SubClusterId> nodeToSubCluster =
-      new HashMap<String, SubClusterId>();
+      new ConcurrentHashMap<String, SubClusterId>();
   private Map<String, Set<SubClusterId>> rackToSubClusters =
-      new HashMap<String, Set<SubClusterId>>();
+      new ConcurrentHashMap<String, Set<SubClusterId>>();
 
   @Override
   public SubClusterId getSubClusterForNode(String nodename)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a3450faf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/resolver/SubClusterResolver.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/resolver/SubClusterResolver.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/resolver/SubClusterResolver.java
index c6adfa6..612d396 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/resolver/SubClusterResolver.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/resolver/SubClusterResolver.java
@@ -25,8 +25,8 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
 
 /**
- * An utility that helps to determine the sub-cluster that a specified node
- * belongs to.
+ * An utility that helps to determine the sub-cluster that a specified node or
+ * rack belongs to. All implementing classes should be thread-safe.
  */
 public interface SubClusterResolver extends Configurable {
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a3450faf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java
index 9b794de..e8f245e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java
@@ -46,6 +46,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.hadoop.yarn.server.federation.resolver.SubClusterResolver;
 import org.apache.hadoop.yarn.server.federation.store.FederationStateStore;
 import org.apache.hadoop.yarn.server.federation.store.records.AddApplicationHomeSubClusterRequest;
 import org.apache.hadoop.yarn.server.federation.store.records.AddApplicationHomeSubClusterResponse;
@@ -90,6 +91,7 @@ public final class FederationStateStoreFacade {
   private int cacheTimeToLive;
   private Configuration conf;
   private Cache<Object, Object> cache;
+  private SubClusterResolver subclusterResolver;
 
   private FederationStateStoreFacade() {
     initializeFacadeInternal(new Configuration());
@@ -104,6 +106,12 @@ public final class FederationStateStoreFacade {
           FederationStateStore.class, createRetryPolicy(conf));
       this.stateStore.init(conf);
 
+      this.subclusterResolver = createInstance(conf,
+          YarnConfiguration.FEDERATION_CLUSTER_RESOLVER_CLASS,
+          YarnConfiguration.DEFAULT_FEDERATION_CLUSTER_RESOLVER_CLASS,
+          SubClusterResolver.class);
+      this.subclusterResolver.load();
+
       initCache();
 
     } catch (YarnException ex) {
@@ -348,6 +356,15 @@ public final class FederationStateStoreFacade {
   }
 
   /**
+   * Get the singleton instance of SubClusterResolver.
+   *
+   * @return SubClusterResolver instance
+   */
+  public SubClusterResolver getSubClusterResolver() {
+    return this.subclusterResolver;
+  }
+
+  /**
    * Helper method to create instances of Object using the class name defined in
    * the configuration object. The instances creates {@link RetryProxy} using
    * the specific {@link RetryPolicy}.
@@ -359,23 +376,40 @@ public final class FederationStateStoreFacade {
    * @param retryPolicy the policy for retrying method call failures
    * @return a retry proxy for the specified interface
    */
-  @SuppressWarnings("unchecked")
   public static <T> Object createRetryInstance(Configuration conf,
       String configuredClassName, String defaultValue, Class<T> type,
       RetryPolicy retryPolicy) {
 
+    return RetryProxy.create(type,
+        createInstance(conf, configuredClassName, defaultValue, type),
+        retryPolicy);
+  }
+
+  /**
+   * Helper method to create instances of Object using the class name specified
+   * in the configuration object.
+   *
+   * @param conf the yarn configuration
+   * @param configuredClassName the configuration provider key
+   * @param defaultValue the default implementation class
+   * @param type the required interface/base class
+   * @param <T> The type of the instance to create
+   * @return the instances created
+   */
+  @SuppressWarnings("unchecked")
+  public static <T> T createInstance(Configuration conf,
+      String configuredClassName, String defaultValue, Class<T> type) {
+
     String className = conf.get(configuredClassName, defaultValue);
     try {
       Class<?> clusterResolverClass = conf.getClassByName(className);
       if (type.isAssignableFrom(clusterResolverClass)) {
-        return RetryProxy.create(type,
-            (T) ReflectionUtils.newInstance(clusterResolverClass, conf),
-            retryPolicy);
+        return (T) ReflectionUtils.newInstance(clusterResolverClass, conf);
       } else {
-        throw new YarnRuntimeException(
-            "Class: " + className + " not instance of " + type.getSimpleName());
+        throw new YarnRuntimeException("Class: " + className
+            + " not instance of " + type.getCanonicalName());
       }
-    } catch (Exception e) {
+    } catch (ClassNotFoundException e) {
       throw new YarnRuntimeException("Could not instantiate : " + className, e);
     }
   }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[04/42] hadoop git commit: YARN-5391. PolicyManager to tie together Router/AMRM Federation policies. (Carlo Curino via Subru).

Posted by su...@apache.org.
YARN-5391. PolicyManager to tie together Router/AMRM Federation policies. (Carlo Curino via Subru).


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/c8ab3967
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/c8ab3967
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/c8ab3967

Branch: refs/heads/YARN-2915
Commit: c8ab396741a6614d13f5a29dd17df9483991e683
Parents: 53ca351
Author: Subru Krishnan <su...@apache.org>
Authored: Tue Nov 1 19:54:18 2016 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon May 8 16:59:12 2017 -0700

----------------------------------------------------------------------
 .../policies/AbstractPolicyManager.java         | 175 +++++++++++++++++++
 .../FederationPolicyInitializationContext.java  |   3 +-
 .../policies/UniformBroadcastPolicyManager.java |  56 ++++++
 .../policies/WeightedLocalityPolicyManager.java |  67 +++++++
 .../records/SubClusterPolicyConfiguration.java  |  13 ++
 .../policies/BasePolicyManagerTest.java         | 108 ++++++++++++
 ...ionPolicyInitializationContextValidator.java |   5 +-
 .../TestUniformBroadcastPolicyManager.java      |  40 +++++
 .../TestWeightedLocalityPolicyManager.java      |  79 +++++++++
 .../utils/FederationPoliciesTestUtil.java       |   2 +-
 10 files changed, 545 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c8ab3967/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/AbstractPolicyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/AbstractPolicyManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/AbstractPolicyManager.java
new file mode 100644
index 0000000..e77f2e3
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/AbstractPolicyManager.java
@@ -0,0 +1,175 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies;
+
+import org.apache.hadoop.yarn.server.federation.policies.amrmproxy.FederationAMRMProxyPolicy;
+import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
+import org.apache.hadoop.yarn.server.federation.policies.router.FederationRouterPolicy;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyConfiguration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class provides basic implementation for common methods that multiple
+ * policies will need to implement.
+ */
+public abstract class AbstractPolicyManager implements
+    FederationPolicyManager {
+
+  private String queue;
+  @SuppressWarnings("checkstyle:visibilitymodifier")
+  protected Class routerFederationPolicy;
+  @SuppressWarnings("checkstyle:visibilitymodifier")
+  protected Class amrmProxyFederationPolicy;
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(AbstractPolicyManager.class);
+  /**
+   * This default implementation validates the
+   * {@link FederationPolicyInitializationContext},
+   * then checks whether it needs to reinstantiate the class (null or
+   * mismatching type), and reinitialize the policy.
+   *
+   * @param federationPolicyContext the current context
+   * @param oldInstance             the existing (possibly null) instance.
+   *
+   * @return a valid and fully reinitalized {@link FederationAMRMProxyPolicy}
+   * instance
+   *
+   * @throws FederationPolicyInitializationException if the reinitalization is
+   *                                                 not valid, and ensure
+   *                                                 previous state is preserved
+   */
+  public FederationAMRMProxyPolicy getAMRMPolicy(
+      FederationPolicyInitializationContext federationPolicyContext,
+      FederationAMRMProxyPolicy oldInstance)
+      throws FederationPolicyInitializationException {
+
+    if (amrmProxyFederationPolicy == null) {
+      throw new FederationPolicyInitializationException("The parameter "
+          + "amrmProxyFederationPolicy should be initialized in "
+          + this.getClass().getSimpleName() + " constructor.");
+    }
+
+    try {
+      return (FederationAMRMProxyPolicy) internalPolicyGetter(
+          federationPolicyContext, oldInstance, amrmProxyFederationPolicy);
+    } catch (ClassCastException e) {
+      throw new FederationPolicyInitializationException(e);
+    }
+
+  }
+
+  /**
+   * This default implementation validates the
+   * {@link FederationPolicyInitializationContext},
+   * then checks whether it needs to reinstantiate the class (null or
+   * mismatching type), and reinitialize the policy.
+   *
+   * @param federationPolicyContext the current context
+   * @param oldInstance             the existing (possibly null) instance.
+   *
+   * @return a valid and fully reinitalized {@link FederationRouterPolicy}
+   * instance
+   *
+   * @throws FederationPolicyInitializationException if the reinitalization is
+   *                                                 not valid, and ensure
+   *                                                 previous state is preserved
+   */
+
+  public FederationRouterPolicy getRouterPolicy(
+      FederationPolicyInitializationContext federationPolicyContext,
+      FederationRouterPolicy oldInstance)
+      throws FederationPolicyInitializationException {
+
+    //checks that sub-types properly initialize the types of policies
+    if (routerFederationPolicy == null) {
+      throw new FederationPolicyInitializationException("The policy "
+          + "type should be initialized in " + this.getClass().getSimpleName()
+          + " constructor.");
+    }
+
+    try {
+      return (FederationRouterPolicy) internalPolicyGetter(
+          federationPolicyContext, oldInstance, routerFederationPolicy);
+    } catch (ClassCastException e) {
+      throw new FederationPolicyInitializationException(e);
+    }
+  }
+
+  @Override
+  public String getQueue() {
+    return queue;
+  }
+
+  @Override
+  public void setQueue(String queue) {
+    this.queue = queue;
+  }
+
+  /**
+   * Common functionality to instantiate a reinitialize a {@link
+   * ConfigurableFederationPolicy}.
+   */
+  private ConfigurableFederationPolicy internalPolicyGetter(
+      final FederationPolicyInitializationContext federationPolicyContext,
+      ConfigurableFederationPolicy oldInstance, Class policy)
+      throws FederationPolicyInitializationException {
+
+    FederationPolicyInitializationContextValidator
+        .validate(federationPolicyContext, this.getClass().getCanonicalName());
+
+    if (oldInstance == null || !oldInstance.getClass().equals(policy)) {
+      try {
+        oldInstance = (ConfigurableFederationPolicy) policy.newInstance();
+      } catch (InstantiationException e) {
+        throw new FederationPolicyInitializationException(e);
+      } catch (IllegalAccessException e) {
+        throw new FederationPolicyInitializationException(e);
+      }
+    }
+
+    //copying the context to avoid side-effects
+    FederationPolicyInitializationContext modifiedContext =
+        updateContext(federationPolicyContext,
+            oldInstance.getClass().getCanonicalName());
+
+    oldInstance.reinitialize(modifiedContext);
+    return oldInstance;
+  }
+
+  /**
+   * This method is used to copy-on-write the context, that will be passed
+   * downstream to the router/amrmproxy policies.
+   */
+  private FederationPolicyInitializationContext updateContext(
+      FederationPolicyInitializationContext federationPolicyContext,
+      String type) {
+    // copying configuration and context to avoid modification of original
+    SubClusterPolicyConfiguration newConf = SubClusterPolicyConfiguration
+        .newInstance(federationPolicyContext
+            .getSubClusterPolicyConfiguration());
+    newConf.setType(type);
+
+    return new FederationPolicyInitializationContext(newConf,
+                  federationPolicyContext.getFederationSubclusterResolver(),
+                  federationPolicyContext.getFederationStateStoreFacade(),
+                  federationPolicyContext.getHomeSubcluster());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c8ab3967/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyInitializationContext.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyInitializationContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyInitializationContext.java
index 46dd6eb..4d29a41 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyInitializationContext.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyInitializationContext.java
@@ -41,10 +41,11 @@ public class FederationPolicyInitializationContext {
 
   public FederationPolicyInitializationContext(
       SubClusterPolicyConfiguration policy, SubClusterResolver resolver,
-      FederationStateStoreFacade storeFacade) {
+      FederationStateStoreFacade storeFacade, SubClusterId home) {
     this.federationPolicyConfiguration = policy;
     this.federationSubclusterResolver = resolver;
     this.federationStateStoreFacade = storeFacade;
+    this.homeSubcluster = home;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c8ab3967/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/UniformBroadcastPolicyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/UniformBroadcastPolicyManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/UniformBroadcastPolicyManager.java
new file mode 100644
index 0000000..a01f8fa
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/UniformBroadcastPolicyManager.java
@@ -0,0 +1,56 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies;
+
+import org.apache.hadoop.yarn.server.federation.policies.amrmproxy.BroadcastAMRMProxyPolicy;
+import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
+import org.apache.hadoop.yarn.server.federation.policies.router.UniformRandomRouterPolicy;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyConfiguration;
+
+import java.nio.ByteBuffer;
+
+/**
+ * This class represents a simple implementation of a {@code
+ * FederationPolicyManager}.
+ *
+ * It combines the basic policies: {@link UniformRandomRouterPolicy} and
+ * {@link BroadcastAMRMProxyPolicy}, which are designed to work together and
+ * "spread" the load among sub-clusters uniformly.
+ *
+ * This simple policy might impose heavy load on the RMs and return more
+ * containers than a job requested as all requests are (replicated and)
+ * broadcasted.
+ */
+public class UniformBroadcastPolicyManager
+    extends AbstractPolicyManager {
+
+  public UniformBroadcastPolicyManager() {
+    //this structurally hard-codes two compatible policies for Router and
+    // AMRMProxy.
+    routerFederationPolicy = UniformRandomRouterPolicy.class;
+    amrmProxyFederationPolicy = BroadcastAMRMProxyPolicy.class;
+  }
+
+  @Override
+  public SubClusterPolicyConfiguration serializeConf()
+      throws FederationPolicyInitializationException {
+    ByteBuffer buf = ByteBuffer.allocate(0);
+    return SubClusterPolicyConfiguration
+        .newInstance(getQueue(), this.getClass().getCanonicalName(), buf);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c8ab3967/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/WeightedLocalityPolicyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/WeightedLocalityPolicyManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/WeightedLocalityPolicyManager.java
new file mode 100644
index 0000000..f3c6673
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/WeightedLocalityPolicyManager.java
@@ -0,0 +1,67 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.yarn.server.federation.policies.amrmproxy.LocalityMulticastAMRMProxyPolicy;
+import org.apache.hadoop.yarn.server.federation.policies.dao.WeightedPolicyInfo;
+import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
+import org.apache.hadoop.yarn.server.federation.policies.router.WeightedRandomRouterPolicy;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyConfiguration;
+
+import java.nio.ByteBuffer;
+
+/**
+ * Policy that allows operator to configure "weights" for routing. This picks a
+ * {@link WeightedRandomRouterPolicy} for the router and a {@link
+ * LocalityMulticastAMRMProxyPolicy} for the amrmproxy as they are designed to
+ * work together.
+ */
+public class WeightedLocalityPolicyManager
+    extends AbstractPolicyManager {
+
+  private WeightedPolicyInfo weightedPolicyInfo;
+
+  public WeightedLocalityPolicyManager() {
+    //this structurally hard-codes two compatible policies for Router and
+    // AMRMProxy.
+    routerFederationPolicy =  WeightedRandomRouterPolicy.class;
+    amrmProxyFederationPolicy = LocalityMulticastAMRMProxyPolicy.class;
+    weightedPolicyInfo = new WeightedPolicyInfo();
+  }
+
+  @Override
+  public SubClusterPolicyConfiguration serializeConf()
+      throws FederationPolicyInitializationException {
+    ByteBuffer buf = weightedPolicyInfo.toByteBuffer();
+    return SubClusterPolicyConfiguration
+        .newInstance(getQueue(), this.getClass().getCanonicalName(), buf);
+  }
+
+  @VisibleForTesting
+  public WeightedPolicyInfo getWeightedPolicyInfo() {
+    return weightedPolicyInfo;
+  }
+
+  @VisibleForTesting
+  public void setWeightedPolicyInfo(
+      WeightedPolicyInfo weightedPolicyInfo) {
+    this.weightedPolicyInfo = weightedPolicyInfo;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c8ab3967/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterPolicyConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterPolicyConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterPolicyConfiguration.java
index 2839139..52807d9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterPolicyConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterPolicyConfiguration.java
@@ -40,6 +40,7 @@ import java.nio.ByteBuffer;
 @Unstable
 public abstract class SubClusterPolicyConfiguration {
 
+
   @Private
   @Unstable
   public static SubClusterPolicyConfiguration newInstance(String queue,
@@ -52,6 +53,18 @@ public abstract class SubClusterPolicyConfiguration {
     return policy;
   }
 
+  @Private
+  @Unstable
+  public static SubClusterPolicyConfiguration newInstance(
+      SubClusterPolicyConfiguration conf) {
+    SubClusterPolicyConfiguration policy =
+        Records.newRecord(SubClusterPolicyConfiguration.class);
+    policy.setQueue(conf.getQueue());
+    policy.setType(conf.getType());
+    policy.setParams(conf.getParams());
+    return policy;
+  }
+
   /**
    * Get the name of the queue for which we are configuring a policy.
    *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c8ab3967/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/BasePolicyManagerTest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/BasePolicyManagerTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/BasePolicyManagerTest.java
new file mode 100644
index 0000000..c609886
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/BasePolicyManagerTest.java
@@ -0,0 +1,108 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies;
+
+import org.apache.hadoop.yarn.server.federation.policies.amrmproxy.FederationAMRMProxyPolicy;
+import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
+import org.apache.hadoop.yarn.server.federation.policies.router.FederationRouterPolicy;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyConfiguration;
+import org.apache.hadoop.yarn.server.federation.utils.FederationPoliciesTestUtil;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * This class provides common test methods for testing {@code
+ * FederationPolicyManager}s.
+ */
+public abstract class BasePolicyManagerTest {
+
+
+  @SuppressWarnings("checkstyle:visibilitymodifier")
+  protected FederationPolicyManager wfp = null;
+  @SuppressWarnings("checkstyle:visibilitymodifier")
+  protected Class expectedPolicyManager;
+  @SuppressWarnings("checkstyle:visibilitymodifier")
+  protected Class expectedAMRMProxyPolicy;
+  @SuppressWarnings("checkstyle:visibilitymodifier")
+  protected Class expectedRouterPolicy;
+
+
+  @Test
+  public void testSerializeAndInstantiate() throws Exception {
+    serializeAndDeserializePolicyManager(wfp, expectedPolicyManager,
+        expectedAMRMProxyPolicy,
+        expectedRouterPolicy);
+  }
+
+  @Test(expected = FederationPolicyInitializationException.class)
+  public void testSerializeAndInstantiateBad1() throws Exception {
+    serializeAndDeserializePolicyManager(wfp, String.class,
+        expectedAMRMProxyPolicy, expectedRouterPolicy);
+  }
+
+  @Test(expected = AssertionError.class)
+  public void testSerializeAndInstantiateBad2() throws Exception {
+    serializeAndDeserializePolicyManager(wfp, expectedPolicyManager,
+        String.class, expectedRouterPolicy);
+  }
+
+  @Test(expected = AssertionError.class)
+  public void testSerializeAndInstantiateBad3() throws Exception {
+    serializeAndDeserializePolicyManager(wfp, expectedPolicyManager,
+        expectedAMRMProxyPolicy, String.class);
+  }
+
+  protected static void serializeAndDeserializePolicyManager(
+      FederationPolicyManager wfp, Class policyManagerType,
+      Class expAMRMProxyPolicy, Class expRouterPolicy) throws Exception {
+
+    // serializeConf it in a context
+    SubClusterPolicyConfiguration fpc =
+        wfp.serializeConf();
+    fpc.setType(policyManagerType.getCanonicalName());
+    FederationPolicyInitializationContext context = new
+        FederationPolicyInitializationContext();
+    context.setSubClusterPolicyConfiguration(fpc);
+    context
+        .setFederationStateStoreFacade(FederationPoliciesTestUtil.initFacade());
+    context.setFederationSubclusterResolver(
+        FederationPoliciesTestUtil.initResolver());
+    context.setHomeSubcluster(SubClusterId.newInstance("homesubcluster"));
+
+    // based on the "context" created instantiate new class and use it
+    Class c = Class.forName(wfp.getClass().getCanonicalName());
+    FederationPolicyManager wfp2 = (FederationPolicyManager) c.newInstance();
+
+    FederationAMRMProxyPolicy federationAMRMProxyPolicy =
+        wfp2.getAMRMPolicy(context, null);
+
+    //needed only for tests (getARMRMPolicy change the "type" in conf)
+    fpc.setType(wfp.getClass().getCanonicalName());
+
+    FederationRouterPolicy federationRouterPolicy =
+        wfp2.getRouterPolicy(context, null);
+
+    Assert.assertEquals(federationAMRMProxyPolicy.getClass(),
+        expAMRMProxyPolicy);
+
+    Assert.assertEquals(federationRouterPolicy.getClass(),
+        expRouterPolicy);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c8ab3967/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestFederationPolicyInitializationContextValidator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestFederationPolicyInitializationContextValidator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestFederationPolicyInitializationContextValidator.java
index c79fd2a..d906b92 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestFederationPolicyInitializationContextValidator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestFederationPolicyInitializationContextValidator.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.yarn.server.federation.policies.amrmproxy.FederationAMR
 import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
 import org.apache.hadoop.yarn.server.federation.policies.router.FederationRouterPolicy;
 import org.apache.hadoop.yarn.server.federation.resolver.SubClusterResolver;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyConfiguration;
 import org.apache.hadoop.yarn.server.federation.utils.FederationPoliciesTestUtil;
 import org.apache.hadoop.yarn.server.federation.utils.FederationStateStoreFacade;
@@ -38,6 +39,7 @@ public class TestFederationPolicyInitializationContextValidator {
   private SubClusterPolicyConfiguration goodConfig;
   private SubClusterResolver goodSR;
   private FederationStateStoreFacade goodFacade;
+  private SubClusterId goodHome;
   private FederationPolicyInitializationContext context;
 
   @Before
@@ -45,8 +47,9 @@ public class TestFederationPolicyInitializationContextValidator {
     goodFacade = FederationPoliciesTestUtil.initFacade();
     goodConfig = new MockPolicyManager().serializeConf();
     goodSR = FederationPoliciesTestUtil.initResolver();
+    goodHome = SubClusterId.newInstance("homesubcluster");
     context = new FederationPolicyInitializationContext(goodConfig, goodSR,
-        goodFacade);
+        goodFacade, goodHome);
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c8ab3967/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestUniformBroadcastPolicyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestUniformBroadcastPolicyManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestUniformBroadcastPolicyManager.java
new file mode 100644
index 0000000..542a5ae
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestUniformBroadcastPolicyManager.java
@@ -0,0 +1,40 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies;
+
+import org.apache.hadoop.yarn.server.federation.policies.amrmproxy.BroadcastAMRMProxyPolicy;
+import org.apache.hadoop.yarn.server.federation.policies.router.UniformRandomRouterPolicy;
+import org.junit.Before;
+
+/**
+ * Simple test of {@link UniformBroadcastPolicyManager}.
+ */
+public class TestUniformBroadcastPolicyManager extends BasePolicyManagerTest {
+
+  @Before
+  public void setup() {
+    //config policy
+    wfp = new UniformBroadcastPolicyManager();
+    wfp.setQueue("queue1");
+
+    //set expected params that the base test class will use for tests
+    expectedPolicyManager = UniformBroadcastPolicyManager.class;
+    expectedAMRMProxyPolicy = BroadcastAMRMProxyPolicy.class;
+    expectedRouterPolicy = UniformRandomRouterPolicy.class;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c8ab3967/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestWeightedLocalityPolicyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestWeightedLocalityPolicyManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestWeightedLocalityPolicyManager.java
new file mode 100644
index 0000000..ab9cec4
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestWeightedLocalityPolicyManager.java
@@ -0,0 +1,79 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies;
+
+import org.apache.hadoop.yarn.server.federation.policies.amrmproxy.LocalityMulticastAMRMProxyPolicy;
+import org.apache.hadoop.yarn.server.federation.policies.dao.WeightedPolicyInfo;
+import org.apache.hadoop.yarn.server.federation.policies.router.WeightedRandomRouterPolicy;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterIdInfo;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Simple test of {@link WeightedLocalityPolicyManager}.
+ */
+public class TestWeightedLocalityPolicyManager extends
+    BasePolicyManagerTest {
+
+  private WeightedPolicyInfo policyInfo;
+
+  @Before
+  public void setup() {
+    // configure a policy
+
+    wfp = new WeightedLocalityPolicyManager();
+    wfp.setQueue("queue1");
+    SubClusterId sc1 = SubClusterId.newInstance("sc1");
+    SubClusterId sc2 = SubClusterId.newInstance("sc2");
+    policyInfo = new WeightedPolicyInfo();
+
+    Map<SubClusterIdInfo, Float> routerWeights = new HashMap<>();
+    routerWeights.put(new SubClusterIdInfo(sc1), 0.2f);
+    routerWeights.put(new SubClusterIdInfo(sc2), 0.8f);
+    policyInfo.setRouterPolicyWeights(routerWeights);
+
+    Map<SubClusterIdInfo, Float> amrmWeights = new HashMap<>();
+    amrmWeights.put(new SubClusterIdInfo(sc1), 0.2f);
+    amrmWeights.put(new SubClusterIdInfo(sc2), 0.8f);
+    policyInfo.setAMRMPolicyWeights(amrmWeights);
+
+    ((WeightedLocalityPolicyManager) wfp).setWeightedPolicyInfo(
+        policyInfo);
+
+    //set expected params that the base test class will use for tests
+    expectedPolicyManager = WeightedLocalityPolicyManager.class;
+    expectedAMRMProxyPolicy = LocalityMulticastAMRMProxyPolicy.class;
+    expectedRouterPolicy = WeightedRandomRouterPolicy.class;
+  }
+
+  @Test
+  public void testPolicyInfoSetCorrectly() throws Exception {
+    serializeAndDeserializePolicyManager(wfp, expectedPolicyManager,
+                                         expectedAMRMProxyPolicy,
+                                         expectedRouterPolicy);
+
+    //check the policyInfo propagates through ser/der correctly
+    Assert.assertEquals(((WeightedLocalityPolicyManager) wfp)
+                            .getWeightedPolicyInfo(), policyInfo);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c8ab3967/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/FederationPoliciesTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/FederationPoliciesTestUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/FederationPoliciesTestUtil.java
index 87ed8d1..85fdc96 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/FederationPoliciesTestUtil.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/FederationPoliciesTestUtil.java
@@ -143,7 +143,7 @@ public final class FederationPoliciesTestUtil {
       SubClusterInfo> activeSubclusters) throws YarnException {
     FederationPolicyInitializationContext context =
         new FederationPolicyInitializationContext(null, initResolver(),
-            initFacade());
+            initFacade(), SubClusterId.newInstance("homesubcluster"));
     initializePolicyContext(context, policy, policyInfo, activeSubclusters);
   }
 


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[07/42] hadoop git commit: YARN-3662. Federation Membership State Store internal APIs.

Posted by su...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/abe79cb6/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/SubClusterDeregisterRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/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/SubClusterDeregisterRequestPBImpl.java b/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/SubClusterDeregisterRequestPBImpl.java
new file mode 100644
index 0000000..d4c5451
--- /dev/null
+++ b/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/SubClusterDeregisterRequestPBImpl.java
@@ -0,0 +1,156 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records.impl.pb;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SubClusterDeregisterRequestProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SubClusterDeregisterRequestProtoOrBuilder;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SubClusterIdProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SubClusterStateProto;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterDeregisterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterState;
+
+import com.google.protobuf.TextFormat;
+
+/**
+ * Protocol buffer based implementation of {@link SubClusterDeregisterRequest}.
+ */
+@Private
+@Unstable
+public class SubClusterDeregisterRequestPBImpl
+    extends SubClusterDeregisterRequest {
+
+  private SubClusterDeregisterRequestProto proto =
+      SubClusterDeregisterRequestProto.getDefaultInstance();
+  private SubClusterDeregisterRequestProto.Builder builder = null;
+  private boolean viaProto = false;
+
+  public SubClusterDeregisterRequestPBImpl() {
+    builder = SubClusterDeregisterRequestProto.newBuilder();
+  }
+
+  public SubClusterDeregisterRequestPBImpl(
+      SubClusterDeregisterRequestProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public SubClusterDeregisterRequestProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = SubClusterDeregisterRequestProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  private void mergeLocalToBuilder() {
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null) {
+      return false;
+    }
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
+  }
+
+  @Override
+  public SubClusterId getSubClusterId() {
+    SubClusterDeregisterRequestProtoOrBuilder p = viaProto ? proto : builder;
+    if (!p.hasSubClusterId()) {
+      return null;
+    }
+    return convertFromProtoFormat(p.getSubClusterId());
+  }
+
+  @Override
+  public void setSubClusterId(SubClusterId subClusterId) {
+    maybeInitBuilder();
+    if (subClusterId == null) {
+      builder.clearSubClusterId();
+      return;
+    }
+    builder.setSubClusterId(convertToProtoFormat(subClusterId));
+  }
+
+  @Override
+  public SubClusterState getState() {
+    SubClusterDeregisterRequestProtoOrBuilder p = viaProto ? proto : builder;
+    if (!p.hasState()) {
+      return null;
+    }
+    return convertFromProtoFormat(p.getState());
+  }
+
+  @Override
+  public void setState(SubClusterState state) {
+    maybeInitBuilder();
+    if (state == null) {
+      builder.clearState();
+      return;
+    }
+    builder.setState(convertToProtoFormat(state));
+  }
+
+  private SubClusterId convertFromProtoFormat(SubClusterIdProto sc) {
+    return new SubClusterIdPBImpl(sc);
+  }
+
+  private SubClusterIdProto convertToProtoFormat(SubClusterId sc) {
+    return ((SubClusterIdPBImpl) sc).getProto();
+  }
+
+  private SubClusterState convertFromProtoFormat(SubClusterStateProto state) {
+    return SubClusterState.valueOf(state.name());
+  }
+
+  private SubClusterStateProto convertToProtoFormat(SubClusterState state) {
+    return SubClusterStateProto.valueOf(state.name());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/abe79cb6/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/SubClusterDeregisterResponsePBImpl.java
----------------------------------------------------------------------
diff --git a/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/SubClusterDeregisterResponsePBImpl.java b/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/SubClusterDeregisterResponsePBImpl.java
new file mode 100644
index 0000000..9e00796
--- /dev/null
+++ b/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/SubClusterDeregisterResponsePBImpl.java
@@ -0,0 +1,77 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records.impl.pb;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SubClusterDeregisterResponseProto;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterDeregisterResponse;
+
+import com.google.protobuf.TextFormat;
+
+/**
+ * Protocol buffer based implementation of {@link SubClusterDeregisterResponse}.
+ */
+@Private
+@Unstable
+public class SubClusterDeregisterResponsePBImpl
+    extends SubClusterDeregisterResponse {
+
+  private SubClusterDeregisterResponseProto proto =
+      SubClusterDeregisterResponseProto.getDefaultInstance();
+  private SubClusterDeregisterResponseProto.Builder builder = null;
+  private boolean viaProto = false;
+
+  public SubClusterDeregisterResponsePBImpl() {
+    builder = SubClusterDeregisterResponseProto.newBuilder();
+  }
+
+  public SubClusterDeregisterResponsePBImpl(
+      SubClusterDeregisterResponseProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public SubClusterDeregisterResponseProto getProto() {
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null) {
+      return false;
+    }
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/abe79cb6/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/SubClusterHeartbeatRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/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/SubClusterHeartbeatRequestPBImpl.java b/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/SubClusterHeartbeatRequestPBImpl.java
new file mode 100644
index 0000000..ca6b154
--- /dev/null
+++ b/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/SubClusterHeartbeatRequestPBImpl.java
@@ -0,0 +1,192 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records.impl.pb;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SubClusterHeartbeatRequestProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SubClusterHeartbeatRequestProtoOrBuilder;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SubClusterIdProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SubClusterStateProto;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterHeartbeatRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterState;
+
+import com.google.protobuf.TextFormat;
+
+/**
+ * Protocol buffer based implementation of {@link SubClusterHeartbeatRequest}.
+ */
+@Private
+@Unstable
+public class SubClusterHeartbeatRequestPBImpl
+    extends SubClusterHeartbeatRequest {
+
+  private SubClusterHeartbeatRequestProto proto =
+      SubClusterHeartbeatRequestProto.getDefaultInstance();
+  private SubClusterHeartbeatRequestProto.Builder builder = null;
+  private boolean viaProto = false;
+
+  private SubClusterId subClusterId = null;
+
+  public SubClusterHeartbeatRequestPBImpl() {
+    builder = SubClusterHeartbeatRequestProto.newBuilder();
+  }
+
+  public SubClusterHeartbeatRequestPBImpl(
+      SubClusterHeartbeatRequestProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public SubClusterHeartbeatRequestProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = SubClusterHeartbeatRequestProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.subClusterId != null) {
+      builder.setSubClusterId(convertToProtoFormat(this.subClusterId));
+    }
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null) {
+      return false;
+    }
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
+  }
+
+  @Override
+  public SubClusterId getSubClusterId() {
+    SubClusterHeartbeatRequestProtoOrBuilder p = viaProto ? proto : builder;
+    if (this.subClusterId != null) {
+      return this.subClusterId;
+    }
+    if (!p.hasSubClusterId()) {
+      return null;
+    }
+    this.subClusterId = convertFromProtoFormat(p.getSubClusterId());
+    return this.subClusterId;
+  }
+
+  @Override
+  public void setSubClusterId(SubClusterId subClusterId) {
+    maybeInitBuilder();
+    if (subClusterId == null) {
+      builder.clearSubClusterId();
+    }
+    this.subClusterId = subClusterId;
+  }
+
+  @Override
+  public long getLastHeartBeat() {
+    SubClusterHeartbeatRequestProtoOrBuilder p = viaProto ? proto : builder;
+    return p.getLastHeartBeat();
+  }
+
+  @Override
+  public void setLastHeartBeat(long time) {
+    maybeInitBuilder();
+    builder.setLastHeartBeat(time);
+  }
+
+  @Override
+  public SubClusterState getState() {
+    SubClusterHeartbeatRequestProtoOrBuilder p = viaProto ? proto : builder;
+    if (!p.hasState()) {
+      return null;
+    }
+    return convertFromProtoFormat(p.getState());
+  }
+
+  @Override
+  public void setState(SubClusterState state) {
+    maybeInitBuilder();
+    if (state == null) {
+      builder.clearState();
+      return;
+    }
+    builder.setState(convertToProtoFormat(state));
+  }
+
+  @Override
+  public String getCapability() {
+    SubClusterHeartbeatRequestProtoOrBuilder p = viaProto ? proto : builder;
+    return (p.hasCapability()) ? p.getCapability() : null;
+  }
+
+  @Override
+  public void setCapability(String capability) {
+    maybeInitBuilder();
+    if (capability == null) {
+      builder.clearCapability();
+      return;
+    }
+    builder.setCapability(capability);
+  }
+
+  private SubClusterId convertFromProtoFormat(SubClusterIdProto clusterId) {
+    return new SubClusterIdPBImpl(clusterId);
+  }
+
+  private SubClusterIdProto convertToProtoFormat(SubClusterId clusterId) {
+    return ((SubClusterIdPBImpl) clusterId).getProto();
+  }
+
+  private SubClusterState convertFromProtoFormat(SubClusterStateProto state) {
+    return SubClusterState.valueOf(state.name());
+  }
+
+  private SubClusterStateProto convertToProtoFormat(SubClusterState state) {
+    return SubClusterStateProto.valueOf(state.name());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/abe79cb6/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/SubClusterHeartbeatResponsePBImpl.java
----------------------------------------------------------------------
diff --git a/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/SubClusterHeartbeatResponsePBImpl.java b/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/SubClusterHeartbeatResponsePBImpl.java
new file mode 100644
index 0000000..2020c1a
--- /dev/null
+++ b/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/SubClusterHeartbeatResponsePBImpl.java
@@ -0,0 +1,77 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records.impl.pb;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SubClusterHeartbeatResponseProto;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterHeartbeatResponse;
+
+import com.google.protobuf.TextFormat;
+
+/**
+ * Protocol buffer based implementation of {@link SubClusterHeartbeatResponse}.
+ */
+@Private
+@Unstable
+public class SubClusterHeartbeatResponsePBImpl
+    extends SubClusterHeartbeatResponse {
+
+  private SubClusterHeartbeatResponseProto proto =
+      SubClusterHeartbeatResponseProto.getDefaultInstance();
+  private SubClusterHeartbeatResponseProto.Builder builder = null;
+  private boolean viaProto = false;
+
+  public SubClusterHeartbeatResponsePBImpl() {
+    builder = SubClusterHeartbeatResponseProto.newBuilder();
+  }
+
+  public SubClusterHeartbeatResponsePBImpl(
+      SubClusterHeartbeatResponseProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public SubClusterHeartbeatResponseProto getProto() {
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null) {
+      return false;
+    }
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/abe79cb6/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/SubClusterIdPBImpl.java
----------------------------------------------------------------------
diff --git a/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/SubClusterIdPBImpl.java b/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/SubClusterIdPBImpl.java
new file mode 100644
index 0000000..1bf96bf
--- /dev/null
+++ b/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/SubClusterIdPBImpl.java
@@ -0,0 +1,75 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records.impl.pb;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SubClusterIdProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SubClusterIdProtoOrBuilder;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+
+/**
+ * Protocol buffer based implementation of {@link SubClusterId}.
+ */
+@Private
+@Unstable
+public class SubClusterIdPBImpl extends SubClusterId {
+
+  private SubClusterIdProto proto = SubClusterIdProto.getDefaultInstance();
+  private SubClusterIdProto.Builder builder = null;
+  private boolean viaProto = false;
+
+  public SubClusterIdPBImpl() {
+    builder = SubClusterIdProto.newBuilder();
+  }
+
+  public SubClusterIdPBImpl(SubClusterIdProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public SubClusterIdProto getProto() {
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = SubClusterIdProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  @Override
+  public String getId() {
+    SubClusterIdProtoOrBuilder p = viaProto ? proto : builder;
+    return p.getId();
+  }
+
+  @Override
+  protected void setId(String subClusterId) {
+    maybeInitBuilder();
+    if (subClusterId == null) {
+      builder.clearId();
+      return;
+    }
+    builder.setId(subClusterId);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/abe79cb6/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/SubClusterInfoPBImpl.java
----------------------------------------------------------------------
diff --git a/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/SubClusterInfoPBImpl.java b/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/SubClusterInfoPBImpl.java
new file mode 100644
index 0000000..b650b5f
--- /dev/null
+++ b/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/SubClusterInfoPBImpl.java
@@ -0,0 +1,267 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records.impl.pb;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SubClusterIdProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SubClusterInfoProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SubClusterInfoProtoOrBuilder;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SubClusterStateProto;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterState;
+
+import com.google.common.base.Preconditions;
+import com.google.protobuf.TextFormat;
+
+/**
+ * Protocol buffer based implementation of {@link SubClusterInfo}.
+ */
+@Private
+@Unstable
+public class SubClusterInfoPBImpl extends SubClusterInfo {
+
+  private SubClusterInfoProto proto = SubClusterInfoProto.getDefaultInstance();
+  private SubClusterInfoProto.Builder builder = null;
+  private boolean viaProto = false;
+
+  private SubClusterId subClusterId = null;
+
+  public SubClusterInfoPBImpl() {
+    builder = SubClusterInfoProto.newBuilder();
+  }
+
+  public SubClusterInfoPBImpl(SubClusterInfoProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public SubClusterInfoProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = SubClusterInfoProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.subClusterId != null) {
+      builder.setSubClusterId(convertToProtoFormat(this.subClusterId));
+    }
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null) {
+      return false;
+    }
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
+  }
+
+  @Override
+  public SubClusterId getSubClusterId() {
+    SubClusterInfoProtoOrBuilder p = viaProto ? proto : builder;
+    if (this.subClusterId != null) {
+      return this.subClusterId;
+    }
+    if (!p.hasSubClusterId()) {
+      return null;
+    }
+    this.subClusterId = convertFromProtoFormat(p.getSubClusterId());
+    return this.subClusterId;
+  }
+
+  @Override
+  public void setSubClusterId(SubClusterId subClusterId) {
+    maybeInitBuilder();
+    if (subClusterId == null) {
+      builder.clearSubClusterId();
+    }
+    this.subClusterId = subClusterId;
+  }
+
+  @Override
+  public String getAMRMServiceAddress() {
+    SubClusterInfoProtoOrBuilder p = viaProto ? proto : builder;
+    return (p.hasAMRMServiceAddress()) ? p.getAMRMServiceAddress() : null;
+  }
+
+  @Override
+  public void setAMRMServiceAddress(String amRMServiceAddress) {
+    maybeInitBuilder();
+    if (amRMServiceAddress == null) {
+      builder.clearAMRMServiceAddress();
+      return;
+    }
+    builder.setAMRMServiceAddress(amRMServiceAddress);
+  }
+
+  @Override
+  public String getClientRMServiceAddress() {
+    SubClusterInfoProtoOrBuilder p = viaProto ? proto : builder;
+    return (p.hasClientRMServiceAddress()) ? p.getClientRMServiceAddress()
+        : null;
+  }
+
+  @Override
+  public void setClientRMServiceAddress(String clientRMServiceAddress) {
+    maybeInitBuilder();
+    if (clientRMServiceAddress == null) {
+      builder.clearClientRMServiceAddress();
+      return;
+    }
+    builder.setClientRMServiceAddress(clientRMServiceAddress);
+  }
+
+  @Override
+  public String getRMAdminServiceAddress() {
+    SubClusterInfoProtoOrBuilder p = viaProto ? proto : builder;
+    return (p.hasRMAdminServiceAddress()) ? p.getRMAdminServiceAddress() : null;
+  }
+
+  @Override
+  public void setRMAdminServiceAddress(String rmAdminServiceAddress) {
+    maybeInitBuilder();
+    if (rmAdminServiceAddress == null) {
+      builder.clearRMAdminServiceAddress();
+      return;
+    }
+    builder.setRMAdminServiceAddress(rmAdminServiceAddress);
+  }
+
+  @Override
+  public String getRMWebServiceAddress() {
+    SubClusterInfoProtoOrBuilder p = viaProto ? proto : builder;
+    return (p.hasRMWebServiceAddress()) ? p.getRMWebServiceAddress() : null;
+  }
+
+  @Override
+  public void setRMWebServiceAddress(String rmWebServiceAddress) {
+    maybeInitBuilder();
+    if (rmWebServiceAddress == null) {
+      builder.clearRMWebServiceAddress();
+      return;
+    }
+    builder.setRMWebServiceAddress(rmWebServiceAddress);
+  }
+
+  @Override
+  public long getLastHeartBeat() {
+    SubClusterInfoProtoOrBuilder p = viaProto ? proto : builder;
+    return p.getLastHeartBeat();
+  }
+
+  @Override
+  public void setLastHeartBeat(long time) {
+    maybeInitBuilder();
+    builder.setLastHeartBeat(time);
+  }
+
+  @Override
+  public SubClusterState getState() {
+    SubClusterInfoProtoOrBuilder p = viaProto ? proto : builder;
+    if (!p.hasState()) {
+      return null;
+    }
+    return convertFromProtoFormat(p.getState());
+  }
+
+  @Override
+  public void setState(SubClusterState state) {
+    maybeInitBuilder();
+    if (state == null) {
+      builder.clearState();
+      return;
+    }
+    builder.setState(convertToProtoFormat(state));
+  }
+
+  @Override
+  public long getLastStartTime() {
+    SubClusterInfoProtoOrBuilder p = viaProto ? proto : builder;
+    return (p.hasLastStartTime()) ? p.getLastStartTime() : 0;
+  }
+
+  @Override
+  public void setLastStartTime(long lastStartTime) {
+    Preconditions.checkNotNull(builder);
+    builder.setLastStartTime(lastStartTime);
+  }
+
+  @Override
+  public String getCapability() {
+    SubClusterInfoProtoOrBuilder p = viaProto ? proto : builder;
+    return (p.hasCapability()) ? p.getCapability() : null;
+  }
+
+  @Override
+  public void setCapability(String capability) {
+    maybeInitBuilder();
+    if (capability == null) {
+      builder.clearCapability();
+      return;
+    }
+    builder.setCapability(capability);
+  }
+
+  private SubClusterId convertFromProtoFormat(SubClusterIdProto clusterId) {
+    return new SubClusterIdPBImpl(clusterId);
+  }
+
+  private SubClusterIdProto convertToProtoFormat(SubClusterId clusterId) {
+    return ((SubClusterIdPBImpl) clusterId).getProto();
+  }
+
+  private SubClusterState convertFromProtoFormat(SubClusterStateProto state) {
+    return SubClusterState.valueOf(state.name());
+  }
+
+  private SubClusterStateProto convertToProtoFormat(SubClusterState state) {
+    return SubClusterStateProto.valueOf(state.name());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/abe79cb6/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/SubClusterRegisterRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/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/SubClusterRegisterRequestPBImpl.java b/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/SubClusterRegisterRequestPBImpl.java
new file mode 100644
index 0000000..3429cc9
--- /dev/null
+++ b/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/SubClusterRegisterRequestPBImpl.java
@@ -0,0 +1,134 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records.impl.pb;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SubClusterInfoProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SubClusterRegisterRequestProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SubClusterRegisterRequestProtoOrBuilder;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterRegisterRequest;
+
+import com.google.protobuf.TextFormat;
+
+/**
+ * Protocol buffer based implementation of {@link SubClusterRegisterRequest}.
+ */
+@Private
+@Unstable
+public class SubClusterRegisterRequestPBImpl extends SubClusterRegisterRequest {
+
+  private SubClusterRegisterRequestProto proto =
+      SubClusterRegisterRequestProto.getDefaultInstance();
+  private SubClusterRegisterRequestProto.Builder builder = null;
+  private boolean viaProto = false;
+
+  private SubClusterInfo subClusterInfo = null;
+
+  public SubClusterRegisterRequestPBImpl() {
+    builder = SubClusterRegisterRequestProto.newBuilder();
+  }
+
+  public SubClusterRegisterRequestPBImpl(SubClusterRegisterRequestProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public SubClusterRegisterRequestProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = SubClusterRegisterRequestProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.subClusterInfo != null) {
+      builder.setSubClusterInfo(convertToProtoFormat(this.subClusterInfo));
+    }
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null) {
+      return false;
+    }
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
+  }
+
+  @Override
+  public SubClusterInfo getSubClusterInfo() {
+    SubClusterRegisterRequestProtoOrBuilder p = viaProto ? proto : builder;
+    if (this.subClusterInfo != null) {
+      return this.subClusterInfo;
+    }
+    if (!p.hasSubClusterInfo()) {
+      return null;
+    }
+    this.subClusterInfo = convertFromProtoFormat(p.getSubClusterInfo());
+    return this.subClusterInfo;
+  }
+
+  @Override
+  public void setSubClusterInfo(SubClusterInfo subClusterInfo) {
+    maybeInitBuilder();
+    if (subClusterInfo == null) {
+      builder.clearSubClusterInfo();
+    }
+    this.subClusterInfo = subClusterInfo;
+  }
+
+  private SubClusterInfo convertFromProtoFormat(
+      SubClusterInfoProto clusterInfo) {
+    return new SubClusterInfoPBImpl(clusterInfo);
+  }
+
+  private SubClusterInfoProto convertToProtoFormat(SubClusterInfo clusterInfo) {
+    return ((SubClusterInfoPBImpl) clusterInfo).getProto();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/abe79cb6/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/SubClusterRegisterResponsePBImpl.java
----------------------------------------------------------------------
diff --git a/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/SubClusterRegisterResponsePBImpl.java b/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/SubClusterRegisterResponsePBImpl.java
new file mode 100644
index 0000000..68930e3
--- /dev/null
+++ b/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/SubClusterRegisterResponsePBImpl.java
@@ -0,0 +1,77 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records.impl.pb;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SubClusterRegisterResponseProto;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterRegisterResponse;
+
+import com.google.protobuf.TextFormat;
+
+/**
+ * Protocol buffer based implementation of {@link SubClusterRegisterResponse}.
+ */
+@Private
+@Unstable
+public class SubClusterRegisterResponsePBImpl
+    extends SubClusterRegisterResponse {
+
+  private SubClusterRegisterResponseProto proto =
+      SubClusterRegisterResponseProto.getDefaultInstance();
+  private SubClusterRegisterResponseProto.Builder builder = null;
+  private boolean viaProto = false;
+
+  public SubClusterRegisterResponsePBImpl() {
+    builder = SubClusterRegisterResponseProto.newBuilder();
+  }
+
+  public SubClusterRegisterResponsePBImpl(
+      SubClusterRegisterResponseProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public SubClusterRegisterResponseProto getProto() {
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null) {
+      return false;
+    }
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/abe79cb6/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/package-info.java
----------------------------------------------------------------------
diff --git a/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/package-info.java b/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/package-info.java
new file mode 100644
index 0000000..2f85c48
--- /dev/null
+++ b/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/package-info.java
@@ -0,0 +1,17 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.yarn.server.federation.store.records.impl.pb;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/abe79cb6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/package-info.java
new file mode 100644
index 0000000..9a9b282
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/package-info.java
@@ -0,0 +1,17 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.yarn.server.federation.store.records;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/abe79cb6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_federation_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_federation_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_federation_protos.proto
new file mode 100644
index 0000000..1b2e53e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_federation_protos.proto
@@ -0,0 +1,93 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+option java_package = "org.apache.hadoop.yarn.federation.proto";
+option java_outer_classname = "YarnServerFederationProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+package hadoop.yarn;
+
+import "yarn_protos.proto";
+import "yarn_server_common_protos.proto";
+
+message SubClusterIdProto {
+  optional string id = 1;
+}
+
+enum SubClusterStateProto {
+  SC_NEW = 1;
+  SC_RUNNING = 2;
+  SC_UNHEALTHY = 3;
+  SC_DECOMMISSIONING = 4;
+  SC_LOST = 5;
+  SC_UNREGISTERED = 6;
+  SC_DECOMMISSIONED = 7;
+}
+
+message SubClusterInfoProto {
+  optional SubClusterIdProto sub_cluster_id = 1;
+  optional string aMRM_service_address = 2;
+  optional string client_rM_service_address = 3;
+  optional string rM_admin_service_address = 4;
+  optional string rM_web_service_address = 5;
+  optional int64 lastHeartBeat = 6;
+  optional SubClusterStateProto state = 7;
+  optional int64 lastStartTime = 8;
+  optional string capability = 9;
+}
+
+message SubClusterRegisterRequestProto {
+  optional SubClusterInfoProto sub_cluster_info = 1;
+}
+
+message SubClusterRegisterResponseProto {
+}
+
+message SubClusterHeartbeatRequestProto {
+  optional SubClusterIdProto sub_cluster_id = 1;
+  optional int64 lastHeartBeat = 2;
+  optional SubClusterStateProto state = 3;
+  optional string capability = 4;
+}
+
+message SubClusterHeartbeatResponseProto {
+}
+
+message SubClusterDeregisterRequestProto {
+  optional SubClusterIdProto sub_cluster_id = 1;
+  optional SubClusterStateProto state = 2;
+}
+
+message SubClusterDeregisterResponseProto {
+}
+
+message GetSubClusterInfoRequestProto {
+  optional SubClusterIdProto sub_cluster_id = 1;
+}
+
+message GetSubClusterInfoResponseProto {
+  optional SubClusterInfoProto sub_cluster_info = 1;
+}
+
+message GetSubClustersInfoRequestProto {
+  optional bool filter_inactive_subclusters = 1 [default = true];
+}
+
+message GetSubClustersInfoResponseProto {
+  repeated SubClusterInfoProto sub_cluster_infos = 1;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/abe79cb6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/records/TestFederationProtocolRecords.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/records/TestFederationProtocolRecords.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/records/TestFederationProtocolRecords.java
new file mode 100644
index 0000000..681edb1
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/records/TestFederationProtocolRecords.java
@@ -0,0 +1,133 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.records;
+
+import org.apache.hadoop.yarn.api.BasePBImplRecordsTest;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.GetSubClusterInfoRequestProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.GetSubClusterInfoResponseProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.GetSubClustersInfoRequestProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.GetSubClustersInfoResponseProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SubClusterDeregisterRequestProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SubClusterDeregisterResponseProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SubClusterHeartbeatRequestProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SubClusterHeartbeatResponseProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SubClusterIdProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SubClusterInfoProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SubClusterRegisterRequestProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SubClusterRegisterResponseProto;
+import org.apache.hadoop.yarn.server.federation.store.records.impl.pb.GetSubClusterInfoRequestPBImpl;
+import org.apache.hadoop.yarn.server.federation.store.records.impl.pb.GetSubClusterInfoResponsePBImpl;
+import org.apache.hadoop.yarn.server.federation.store.records.impl.pb.GetSubClustersInfoRequestPBImpl;
+import org.apache.hadoop.yarn.server.federation.store.records.impl.pb.GetSubClustersInfoResponsePBImpl;
+import org.apache.hadoop.yarn.server.federation.store.records.impl.pb.SubClusterDeregisterRequestPBImpl;
+import org.apache.hadoop.yarn.server.federation.store.records.impl.pb.SubClusterDeregisterResponsePBImpl;
+import org.apache.hadoop.yarn.server.federation.store.records.impl.pb.SubClusterHeartbeatRequestPBImpl;
+import org.apache.hadoop.yarn.server.federation.store.records.impl.pb.SubClusterHeartbeatResponsePBImpl;
+import org.apache.hadoop.yarn.server.federation.store.records.impl.pb.SubClusterIdPBImpl;
+import org.apache.hadoop.yarn.server.federation.store.records.impl.pb.SubClusterInfoPBImpl;
+import org.apache.hadoop.yarn.server.federation.store.records.impl.pb.SubClusterRegisterRequestPBImpl;
+import org.apache.hadoop.yarn.server.federation.store.records.impl.pb.SubClusterRegisterResponsePBImpl;
+import org.apache.hadoop.yarn.server.records.Version;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Test class for federation protocol records.
+ */
+public class TestFederationProtocolRecords extends BasePBImplRecordsTest {
+
+  @BeforeClass
+  public static void setup() throws Exception {
+    generateByNewInstance(ApplicationId.class);
+    generateByNewInstance(Version.class);
+    generateByNewInstance(SubClusterId.class);
+    generateByNewInstance(SubClusterInfo.class);
+  }
+
+  @Test
+  public void testSubClusterId() throws Exception {
+    validatePBImplRecord(SubClusterIdPBImpl.class, SubClusterIdProto.class);
+  }
+
+  @Test
+  public void testSubClusterInfo() throws Exception {
+    validatePBImplRecord(SubClusterInfoPBImpl.class, SubClusterInfoProto.class);
+  }
+
+  @Test
+  public void testSubClusterRegisterRequest() throws Exception {
+    validatePBImplRecord(SubClusterRegisterRequestPBImpl.class,
+        SubClusterRegisterRequestProto.class);
+  }
+
+  @Test
+  public void testSubClusterRegisterResponse() throws Exception {
+    validatePBImplRecord(SubClusterRegisterResponsePBImpl.class,
+        SubClusterRegisterResponseProto.class);
+  }
+
+  @Test
+  public void testSubClusterDeregisterRequest() throws Exception {
+    validatePBImplRecord(SubClusterDeregisterRequestPBImpl.class,
+        SubClusterDeregisterRequestProto.class);
+  }
+
+  @Test
+  public void testSubClusterDeregisterResponse() throws Exception {
+    validatePBImplRecord(SubClusterDeregisterResponsePBImpl.class,
+        SubClusterDeregisterResponseProto.class);
+  }
+
+  @Test
+  public void testSubClusterHeartbeatRequest() throws Exception {
+    validatePBImplRecord(SubClusterHeartbeatRequestPBImpl.class,
+        SubClusterHeartbeatRequestProto.class);
+  }
+
+  @Test
+  public void testSubClusterHeartbeatResponse() throws Exception {
+    validatePBImplRecord(SubClusterHeartbeatResponsePBImpl.class,
+        SubClusterHeartbeatResponseProto.class);
+  }
+
+  @Test
+  public void testGetSubClusterRequest() throws Exception {
+    validatePBImplRecord(GetSubClusterInfoRequestPBImpl.class,
+        GetSubClusterInfoRequestProto.class);
+  }
+
+  @Test
+  public void testGetSubClusterResponse() throws Exception {
+    validatePBImplRecord(GetSubClusterInfoResponsePBImpl.class,
+        GetSubClusterInfoResponseProto.class);
+  }
+
+  @Test
+  public void testGetSubClustersInfoRequest() throws Exception {
+    validatePBImplRecord(GetSubClustersInfoRequestPBImpl.class,
+        GetSubClustersInfoRequestProto.class);
+  }
+
+  @Test
+  public void testGetSubClustersInfoResponse() throws Exception {
+    validatePBImplRecord(GetSubClustersInfoResponsePBImpl.class,
+        GetSubClustersInfoResponseProto.class);
+  }
+
+}
\ No newline at end of file


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[05/42] hadoop git commit: YARN-5325. Stateless ARMRMProxy policies implementation. (Carlo Curino via Subru).

Posted by su...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/53ca351b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/LoadBasedRouterPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/LoadBasedRouterPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/LoadBasedRouterPolicy.java
index e57709f..5de749f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/LoadBasedRouterPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/LoadBasedRouterPolicy.java
@@ -17,8 +17,8 @@
 
 package org.apache.hadoop.yarn.server.federation.policies.router;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import java.util.Map;
+
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.federation.policies.FederationPolicyInitializationContext;
@@ -30,34 +30,27 @@ import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
 import org.codehaus.jettison.json.JSONException;
 import org.codehaus.jettison.json.JSONObject;
 
-import java.util.Map;
-
 /**
  * This implements a simple load-balancing policy. The policy "weights" are
  * binary 0/1 values that enable/disable each sub-cluster, and the policy peaks
  * the sub-cluster with the least load to forward this application.
  */
-public class LoadBasedRouterPolicy
-    extends BaseWeightedRouterPolicy {
-
-  private static final Log LOG =
-      LogFactory.getLog(LoadBasedRouterPolicy.class);
+public class LoadBasedRouterPolicy extends AbstractRouterPolicy {
 
   @Override
-  public void reinitialize(FederationPolicyInitializationContext
-      federationPolicyContext)
+  public void reinitialize(FederationPolicyInitializationContext policyContext)
       throws FederationPolicyInitializationException {
 
     // remember old policyInfo
     WeightedPolicyInfo tempPolicy = getPolicyInfo();
 
-    //attempt new initialization
-    super.reinitialize(federationPolicyContext);
+    // attempt new initialization
+    super.reinitialize(policyContext);
 
-    //check extra constraints
+    // check extra constraints
     for (Float weight : getPolicyInfo().getRouterPolicyWeights().values()) {
       if (weight != 0 && weight != 1) {
-        //reset to old policyInfo if check fails
+        // reset to old policyInfo if check fails
         setPolicyInfo(tempPolicy);
         throw new FederationPolicyInitializationException(
             this.getClass().getCanonicalName()
@@ -69,18 +62,16 @@ public class LoadBasedRouterPolicy
 
   @Override
   public SubClusterId getHomeSubcluster(
-      ApplicationSubmissionContext appSubmissionContext)
-      throws YarnException {
+      ApplicationSubmissionContext appSubmissionContext) throws YarnException {
 
     Map<SubClusterId, SubClusterInfo> activeSubclusters =
         getActiveSubclusters();
 
-    Map<SubClusterIdInfo, Float> weights = getPolicyInfo()
-        .getRouterPolicyWeights();
+    Map<SubClusterIdInfo, Float> weights =
+        getPolicyInfo().getRouterPolicyWeights();
     SubClusterIdInfo chosen = null;
     long currBestMem = -1;
-    for (Map.Entry<SubClusterId, SubClusterInfo> entry :
-        activeSubclusters
+    for (Map.Entry<SubClusterId, SubClusterInfo> entry : activeSubclusters
         .entrySet()) {
       SubClusterIdInfo id = new SubClusterIdInfo(entry.getKey());
       if (weights.containsKey(id) && weights.get(id) > 0) {
@@ -95,8 +86,7 @@ public class LoadBasedRouterPolicy
     return chosen.toId();
   }
 
-  private long getAvailableMemory(SubClusterInfo value)
-      throws YarnException {
+  private long getAvailableMemory(SubClusterInfo value) throws YarnException {
     try {
       long mem = -1;
       JSONObject obj = new JSONObject(value.getCapability());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53ca351b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/PriorityRouterPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/PriorityRouterPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/PriorityRouterPolicy.java
index a8ac5f7..bc3a1f7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/PriorityRouterPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/PriorityRouterPolicy.java
@@ -17,39 +17,32 @@
 
 package org.apache.hadoop.yarn.server.federation.policies.router;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import java.util.Map;
+
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterIdInfo;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
 
-import java.util.Map;
-
 /**
  * This implements a policy that interprets "weights" as a ordered list of
  * preferences among sub-clusters. Highest weight among active subclusters is
  * chosen.
  */
-public class PriorityRouterPolicy
-    extends BaseWeightedRouterPolicy {
-
-  private static final Log LOG =
-      LogFactory.getLog(PriorityRouterPolicy.class);
+public class PriorityRouterPolicy extends AbstractRouterPolicy {
 
   @Override
   public SubClusterId getHomeSubcluster(
-      ApplicationSubmissionContext appSubmissionContext)
-      throws YarnException {
+      ApplicationSubmissionContext appSubmissionContext) throws YarnException {
 
     Map<SubClusterId, SubClusterInfo> activeSubclusters =
         getActiveSubclusters();
 
     // This finds the sub-cluster with the highest weight among the
     // currently active ones.
-    Map<SubClusterIdInfo, Float> weights = getPolicyInfo()
-        .getRouterPolicyWeights();
+    Map<SubClusterIdInfo, Float> weights =
+        getPolicyInfo().getRouterPolicyWeights();
     SubClusterId chosen = null;
     Float currentBest = Float.MIN_VALUE;
     for (SubClusterId id : activeSubclusters.keySet()) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53ca351b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/UniformRandomRouterPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/UniformRandomRouterPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/UniformRandomRouterPolicy.java
index 1774961..b8f9cc3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/UniformRandomRouterPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/UniformRandomRouterPolicy.java
@@ -17,6 +17,11 @@
 
 package org.apache.hadoop.yarn.server.federation.policies.router;
 
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.federation.policies.FederationPolicyInitializationContext;
@@ -25,11 +30,6 @@ import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPo
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
 
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
-
 /**
  * This simple policy picks at uniform random among any of the currently active
  * subclusters. This policy is easy to use and good for testing.
@@ -39,7 +39,7 @@ import java.util.Random;
  * of the "weights", in which case the {@link UniformRandomRouterPolicy} send
  * load to them, while {@code WeightedRandomRouterPolicy} does not.
  */
-public class UniformRandomRouterPolicy extends BaseWeightedRouterPolicy {
+public class UniformRandomRouterPolicy extends AbstractRouterPolicy {
 
   private Random rand;
 
@@ -49,14 +49,14 @@ public class UniformRandomRouterPolicy extends BaseWeightedRouterPolicy {
 
   @Override
   public void reinitialize(
-      FederationPolicyInitializationContext federationPolicyContext)
+      FederationPolicyInitializationContext policyContext)
       throws FederationPolicyInitializationException {
     FederationPolicyInitializationContextValidator
-        .validate(federationPolicyContext, this.getClass().getCanonicalName());
+        .validate(policyContext, this.getClass().getCanonicalName());
 
-    //note: this overrides BaseWeighterRouterPolicy and ignores the weights
+    // note: this overrides AbstractRouterPolicy and ignores the weights
 
-    setPolicyContext(federationPolicyContext);
+    setPolicyContext(policyContext);
   }
 
   /**
@@ -64,21 +64,19 @@ public class UniformRandomRouterPolicy extends BaseWeightedRouterPolicy {
    * depend on the weights in the policy).
    *
    * @param appSubmissionContext the context for the app being submitted
-   *                             (ignored).
+   *          (ignored).
    *
    * @return a randomly chosen subcluster.
    *
    * @throws YarnException if there are no active subclusters.
    */
   public SubClusterId getHomeSubcluster(
-      ApplicationSubmissionContext appSubmissionContext)
-      throws YarnException {
+      ApplicationSubmissionContext appSubmissionContext) throws YarnException {
 
     Map<SubClusterId, SubClusterInfo> activeSubclusters =
         getActiveSubclusters();
 
-    List<SubClusterId> list =
-        new ArrayList<>(activeSubclusters.keySet());
+    List<SubClusterId> list = new ArrayList<>(activeSubclusters.keySet());
     return list.get(rand.nextInt(list.size()));
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53ca351b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/WeightedRandomRouterPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/WeightedRandomRouterPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/WeightedRandomRouterPolicy.java
index 0777677..ac75ae9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/WeightedRandomRouterPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/WeightedRandomRouterPolicy.java
@@ -18,32 +18,30 @@
 
 package org.apache.hadoop.yarn.server.federation.policies.router;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import java.util.Map;
+import java.util.Random;
+
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterIdInfo;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
-
-import java.util.Map;
-import java.util.Random;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * This policy implements a weighted random sample among currently active
  * sub-clusters.
  */
-public class WeightedRandomRouterPolicy
-    extends BaseWeightedRouterPolicy {
+public class WeightedRandomRouterPolicy extends AbstractRouterPolicy {
 
-  private static final Log LOG =
-      LogFactory.getLog(WeightedRandomRouterPolicy.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(WeightedRandomRouterPolicy.class);
   private Random rand = new Random(System.currentTimeMillis());
 
   @Override
   public SubClusterId getHomeSubcluster(
-      ApplicationSubmissionContext appSubmissionContext)
-      throws YarnException {
+      ApplicationSubmissionContext appSubmissionContext) throws YarnException {
 
     Map<SubClusterId, SubClusterInfo> activeSubclusters =
         getActiveSubclusters();
@@ -52,13 +50,13 @@ public class WeightedRandomRouterPolicy
     // changes dynamically (and this would unfairly spread the load to
     // sub-clusters adjacent to an inactive one), hence we need to count/scan
     // the list and based on weight pick the next sub-cluster.
-    Map<SubClusterIdInfo, Float> weights = getPolicyInfo()
-        .getRouterPolicyWeights();
+    Map<SubClusterIdInfo, Float> weights =
+        getPolicyInfo().getRouterPolicyWeights();
 
     float totActiveWeight = 0;
-    for(Map.Entry<SubClusterIdInfo, Float> entry : weights.entrySet()){
-      if(entry.getKey()!=null && activeSubclusters.containsKey(entry.getKey()
-          .toId())){
+    for (Map.Entry<SubClusterIdInfo, Float> entry : weights.entrySet()) {
+      if (entry.getKey() != null
+          && activeSubclusters.containsKey(entry.getKey().toId())) {
         totActiveWeight += entry.getValue();
       }
     }
@@ -73,7 +71,7 @@ public class WeightedRandomRouterPolicy
         return id;
       }
     }
-    //should never happen
+    // should never happen
     return null;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53ca351b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/package-info.java
index 5d0fcb6..e445ac3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/package-info.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/package-info.java
@@ -17,4 +17,3 @@
  */
 /** Router policies. **/
 package org.apache.hadoop.yarn.server.federation.policies.router;
-

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53ca351b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/resolver/AbstractSubClusterResolver.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/resolver/AbstractSubClusterResolver.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/resolver/AbstractSubClusterResolver.java
index 8238633..6b4f60c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/resolver/AbstractSubClusterResolver.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/resolver/AbstractSubClusterResolver.java
@@ -57,11 +57,11 @@ public abstract class AbstractSubClusterResolver implements SubClusterResolver {
     return rackToSubClusters.get(rackname);
   }
 
-  protected Map<String, SubClusterId> getNodeToSubCluster() {
+  public Map<String, SubClusterId> getNodeToSubCluster() {
     return nodeToSubCluster;
   }
 
-  protected Map<String, Set<SubClusterId>> getRackToSubClusters() {
+  public Map<String, Set<SubClusterId>> getRackToSubClusters() {
     return rackToSubClusters;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53ca351b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/BaseFederationPoliciesTest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/BaseFederationPoliciesTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/BaseFederationPoliciesTest.java
index 8da92b9..ba897da 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/BaseFederationPoliciesTest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/BaseFederationPoliciesTest.java
@@ -22,14 +22,17 @@ import static org.mockito.Mockito.mock;
 
 import java.nio.ByteBuffer;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.Random;
 
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.federation.policies.amrmproxy.FederationAMRMProxyPolicy;
 import org.apache.hadoop.yarn.server.federation.policies.dao.WeightedPolicyInfo;
+import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyException;
 import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
-import org.apache.hadoop.yarn.server.federation.policies.exceptions.NoActiveSubclustersException;
 import org.apache.hadoop.yarn.server.federation.policies.router.FederationRouterPolicy;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
@@ -49,6 +52,7 @@ public abstract class BaseFederationPoliciesTest {
   private ApplicationSubmissionContext applicationSubmissionContext =
       mock(ApplicationSubmissionContext.class);
   private Random rand = new Random();
+  private SubClusterId homeSubCluster;
 
   @Test
   public void testReinitilialize() throws YarnException {
@@ -88,16 +92,22 @@ public abstract class BaseFederationPoliciesTest {
     getPolicy().reinitialize(fpc);
   }
 
-  @Test(expected = NoActiveSubclustersException.class)
+  @Test(expected = FederationPolicyException.class)
   public void testNoSubclusters() throws YarnException {
     // empty the activeSubclusters map
     FederationPoliciesTestUtil.initializePolicyContext(getPolicy(),
         getPolicyInfo(), new HashMap<>());
 
-    ConfigurableFederationPolicy currentPolicy = getPolicy();
-    if (currentPolicy instanceof FederationRouterPolicy) {
-      ((FederationRouterPolicy) currentPolicy)
+    ConfigurableFederationPolicy localPolicy = getPolicy();
+    if (localPolicy instanceof FederationRouterPolicy) {
+      ((FederationRouterPolicy) localPolicy)
           .getHomeSubcluster(getApplicationSubmissionContext());
+    } else {
+      String[] hosts = new String[] {"host1", "host2" };
+      List<ResourceRequest> resourceRequests = FederationPoliciesTestUtil
+          .createResourceRequests(hosts, 2 * 1024, 2, 1, 3, null, false);
+      ((FederationAMRMProxyPolicy) localPolicy)
+          .splitResourceRequests(resourceRequests);
     }
   }
 
@@ -152,4 +162,12 @@ public abstract class BaseFederationPoliciesTest {
     this.rand = rand;
   }
 
+  public SubClusterId getHomeSubCluster() {
+    return homeSubCluster;
+  }
+
+  public void setHomeSubCluster(SubClusterId homeSubCluster) {
+    this.homeSubCluster = homeSubCluster;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53ca351b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestFederationPolicyInitializationContextValidator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestFederationPolicyInitializationContextValidator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestFederationPolicyInitializationContextValidator.java
index e840b3f..c79fd2a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestFederationPolicyInitializationContextValidator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestFederationPolicyInitializationContextValidator.java
@@ -16,22 +16,20 @@
  * limitations under the License.
  */
 
-
 package org.apache.hadoop.yarn.server.federation.policies;
 
+import java.nio.ByteBuffer;
+
 import org.apache.hadoop.yarn.server.federation.policies.amrmproxy.FederationAMRMProxyPolicy;
 import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
 import org.apache.hadoop.yarn.server.federation.policies.router.FederationRouterPolicy;
 import org.apache.hadoop.yarn.server.federation.resolver.SubClusterResolver;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyConfiguration;
-
 import org.apache.hadoop.yarn.server.federation.utils.FederationPoliciesTestUtil;
 import org.apache.hadoop.yarn.server.federation.utils.FederationStateStoreFacade;
 import org.junit.Before;
 import org.junit.Test;
 
-import java.nio.ByteBuffer;
-
 /**
  * Test class for {@link FederationPolicyInitializationContextValidator}.
  */
@@ -45,11 +43,10 @@ public class TestFederationPolicyInitializationContextValidator {
   @Before
   public void setUp() throws Exception {
     goodFacade = FederationPoliciesTestUtil.initFacade();
-    goodConfig =
-        new MockPolicyManager().serializeConf();
-    goodSR =FederationPoliciesTestUtil.initResolver();
-    context = new
-        FederationPolicyInitializationContext(goodConfig, goodSR, goodFacade);
+    goodConfig = new MockPolicyManager().serializeConf();
+    goodSR = FederationPoliciesTestUtil.initResolver();
+    context = new FederationPolicyInitializationContext(goodConfig, goodSR,
+        goodFacade);
   }
 
   @Test
@@ -100,8 +97,7 @@ public class TestFederationPolicyInitializationContextValidator {
 
     @Override
     public FederationAMRMProxyPolicy getAMRMPolicy(
-        FederationPolicyInitializationContext
-            federationPolicyInitializationContext,
+        FederationPolicyInitializationContext policyContext,
         FederationAMRMProxyPolicy oldInstance)
         throws FederationPolicyInitializationException {
       return null;
@@ -109,8 +105,7 @@ public class TestFederationPolicyInitializationContextValidator {
 
     @Override
     public FederationRouterPolicy getRouterPolicy(
-        FederationPolicyInitializationContext
-            federationPolicyInitializationContext,
+        FederationPolicyInitializationContext policyContext,
         FederationRouterPolicy oldInstance)
         throws FederationPolicyInitializationException {
       return null;
@@ -120,8 +115,8 @@ public class TestFederationPolicyInitializationContextValidator {
     public SubClusterPolicyConfiguration serializeConf()
         throws FederationPolicyInitializationException {
       ByteBuffer buf = ByteBuffer.allocate(0);
-      return SubClusterPolicyConfiguration
-          .newInstance("queue1", this.getClass().getCanonicalName(), buf);
+      return SubClusterPolicyConfiguration.newInstance("queue1",
+          this.getClass().getCanonicalName(), buf);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53ca351b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/TestBroadcastAMRMProxyFederationPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/TestBroadcastAMRMProxyFederationPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/TestBroadcastAMRMProxyFederationPolicy.java
new file mode 100644
index 0000000..a21f53d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/TestBroadcastAMRMProxyFederationPolicy.java
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies.amrmproxy;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
+import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.server.federation.policies.BaseFederationPoliciesTest;
+import org.apache.hadoop.yarn.server.federation.policies.dao.WeightedPolicyInfo;
+import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyException;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterIdInfo;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterState;
+import org.apache.hadoop.yarn.server.federation.utils.FederationPoliciesTestUtil;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Simple test class for the {@link BroadcastAMRMProxyPolicy}.
+ */
+public class TestBroadcastAMRMProxyFederationPolicy
+    extends BaseFederationPoliciesTest {
+
+  @Before
+  public void setUp() throws Exception {
+    setPolicy(new BroadcastAMRMProxyPolicy());
+    // needed for base test to work
+    setPolicyInfo(mock(WeightedPolicyInfo.class));
+
+    for (int i = 1; i <= 2; i++) {
+      SubClusterIdInfo sc = new SubClusterIdInfo("sc" + i);
+      SubClusterInfo sci = mock(SubClusterInfo.class);
+      when(sci.getState()).thenReturn(SubClusterState.SC_RUNNING);
+      when(sci.getSubClusterId()).thenReturn(sc.toId());
+      getActiveSubclusters().put(sc.toId(), sci);
+    }
+
+    FederationPoliciesTestUtil.initializePolicyContext(getPolicy(),
+        mock(WeightedPolicyInfo.class), getActiveSubclusters());
+
+  }
+
+  @Test
+  public void testSplitAllocateRequest() throws Exception {
+    // verify the request is broadcasted to all subclusters
+    String[] hosts = new String[] {"host1", "host2" };
+    List<ResourceRequest> resourceRequests = FederationPoliciesTestUtil
+        .createResourceRequests(hosts, 2 * 1024, 2, 1, 3, null, false);
+
+    Map<SubClusterId, List<ResourceRequest>> response =
+        ((FederationAMRMProxyPolicy) getPolicy())
+            .splitResourceRequests(resourceRequests);
+    Assert.assertTrue(response.size() == 2);
+    for (Map.Entry<SubClusterId, List<ResourceRequest>> entry : response
+        .entrySet()) {
+      Assert.assertTrue(getActiveSubclusters().get(entry.getKey()) != null);
+      for (ResourceRequest r : entry.getValue()) {
+        Assert.assertTrue(resourceRequests.contains(r));
+      }
+    }
+    for (SubClusterId subClusterId : getActiveSubclusters().keySet()) {
+      for (ResourceRequest r : response.get(subClusterId)) {
+        Assert.assertTrue(resourceRequests.contains(r));
+      }
+    }
+  }
+
+  @Test
+  public void testNotifyOfResponse() throws Exception {
+    String[] hosts = new String[] {"host1", "host2" };
+    List<ResourceRequest> resourceRequests = FederationPoliciesTestUtil
+        .createResourceRequests(hosts, 2 * 1024, 2, 1, 3, null, false);
+    Map<SubClusterId, List<ResourceRequest>> response =
+        ((FederationAMRMProxyPolicy) getPolicy())
+            .splitResourceRequests(resourceRequests);
+
+    try {
+      ((FederationAMRMProxyPolicy) getPolicy()).notifyOfResponse(
+          SubClusterId.newInstance("sc3"), mock(AllocateResponse.class));
+      Assert.fail();
+    } catch (FederationPolicyException f) {
+      System.out.println("Expected: " + f.getMessage());
+    }
+
+    ((FederationAMRMProxyPolicy) getPolicy()).notifyOfResponse(
+        SubClusterId.newInstance("sc1"), mock(AllocateResponse.class));
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53ca351b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/TestLocalityMulticastAMRMProxyPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/TestLocalityMulticastAMRMProxyPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/TestLocalityMulticastAMRMProxyPolicy.java
new file mode 100644
index 0000000..2654a06
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/TestLocalityMulticastAMRMProxyPolicy.java
@@ -0,0 +1,566 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies.amrmproxy;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
+import org.apache.hadoop.yarn.api.records.NMToken;
+import org.apache.hadoop.yarn.api.records.NodeReport;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.federation.policies.BaseFederationPoliciesTest;
+import org.apache.hadoop.yarn.server.federation.policies.FederationPolicyInitializationContext;
+import org.apache.hadoop.yarn.server.federation.policies.dao.WeightedPolicyInfo;
+import org.apache.hadoop.yarn.server.federation.resolver.DefaultSubClusterResolverImpl;
+import org.apache.hadoop.yarn.server.federation.resolver.SubClusterResolver;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterIdInfo;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyConfiguration;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterState;
+import org.apache.hadoop.yarn.server.federation.utils.FederationPoliciesTestUtil;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Simple test class for the {@link LocalityMulticastAMRMProxyPolicy}.
+ */
+public class TestLocalityMulticastAMRMProxyPolicy
+    extends BaseFederationPoliciesTest {
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(TestLocalityMulticastAMRMProxyPolicy.class);
+
+  @Before
+  public void setUp() throws Exception {
+    setPolicy(new LocalityMulticastAMRMProxyPolicy());
+    setPolicyInfo(new WeightedPolicyInfo());
+    Map<SubClusterIdInfo, Float> routerWeights = new HashMap<>();
+    Map<SubClusterIdInfo, Float> amrmWeights = new HashMap<>();
+
+    // simulate 20 subclusters with a 5% chance of being inactive
+    for (int i = 0; i < 6; i++) {
+      SubClusterIdInfo sc = new SubClusterIdInfo("subcluster" + i);
+      // sub-cluster 3 is not active
+      if (i != 3) {
+        SubClusterInfo sci = mock(SubClusterInfo.class);
+        when(sci.getState()).thenReturn(SubClusterState.SC_RUNNING);
+        when(sci.getSubClusterId()).thenReturn(sc.toId());
+        getActiveSubclusters().put(sc.toId(), sci);
+      }
+
+      float weight = 1 / 10f;
+      routerWeights.put(sc, weight);
+      amrmWeights.put(sc, weight);
+      // sub-cluster 4 is "disabled" in the weights
+      if (i == 4) {
+        routerWeights.put(sc, 0f);
+        amrmWeights.put(sc, 0f);
+      }
+    }
+
+    getPolicyInfo().setRouterPolicyWeights(routerWeights);
+    getPolicyInfo().setAMRMPolicyWeights(amrmWeights);
+    getPolicyInfo().setHeadroomAlpha(0.5f);
+    setHomeSubCluster(SubClusterId.newInstance("homesubcluster"));
+
+  }
+
+  @Test
+  public void testReinitilialize() throws YarnException {
+    initializePolicy();
+  }
+
+  private void initializePolicy() throws YarnException {
+    setFederationPolicyContext(new FederationPolicyInitializationContext());
+    SubClusterResolver resolver = FederationPoliciesTestUtil.initResolver();
+    getFederationPolicyContext().setFederationSubclusterResolver(resolver);
+    ByteBuffer buf = getPolicyInfo().toByteBuffer();
+    getFederationPolicyContext().setSubClusterPolicyConfiguration(
+        SubClusterPolicyConfiguration.newInstance("queue1",
+            getPolicy().getClass().getCanonicalName(), buf));
+    getFederationPolicyContext().setHomeSubcluster(getHomeSubCluster());
+    FederationPoliciesTestUtil.initializePolicyContext(
+        getFederationPolicyContext(), getPolicy(), getPolicyInfo(),
+        getActiveSubclusters());
+  }
+
+  @Test
+  public void testSplitBasedOnHeadroom() throws Exception {
+
+    // Tests how the headroom info are used to split based on the capacity
+    // each RM claims to give us.
+    // Configure policy to be 100% headroom based
+    getPolicyInfo().setHeadroomAlpha(1.0f);
+
+    initializePolicy();
+    List<ResourceRequest> resourceRequests = createSimpleRequest();
+
+    prepPolicyWithHeadroom();
+
+    Map<SubClusterId, List<ResourceRequest>> response =
+        ((FederationAMRMProxyPolicy) getPolicy())
+            .splitResourceRequests(resourceRequests);
+
+    // pretty print requests
+    LOG.info("Initial headroom");
+    prettyPrintRequests(response);
+
+    validateSplit(response, resourceRequests);
+
+    // based on headroom, we expect 75 containers to got to subcluster0,
+    // as it advertise lots of headroom (100), no containers for sublcuster1
+    // as it advertise zero headroom, 1 to subcluster 2 (as it advertise little
+    // headroom (1), and 25 to subcluster5 which has unknown headroom, and so
+    // it gets 1/4th of the load
+    checkExpectedAllocation(response, "subcluster0", 1, 75);
+    checkExpectedAllocation(response, "subcluster1", 1, -1);
+    checkExpectedAllocation(response, "subcluster2", 1, 1);
+    checkExpectedAllocation(response, "subcluster5", 1, 25);
+
+    // notify a change in headroom and try again
+    AllocateResponse ar = getAllocateResponseWithTargetHeadroom(100);
+    ((FederationAMRMProxyPolicy) getPolicy())
+        .notifyOfResponse(SubClusterId.newInstance("subcluster2"), ar);
+    ((FederationAMRMProxyPolicy) getPolicy())
+        .splitResourceRequests(resourceRequests);
+
+    LOG.info("After headroom update");
+    prettyPrintRequests(response);
+    validateSplit(response, resourceRequests);
+
+    // we simulated a change in headroom for subcluster2, which will now
+    // have the same headroom of subcluster0 and so it splits the requests
+    // note that the total is still less or equal to (userAsk + numSubClusters)
+    checkExpectedAllocation(response, "subcluster0", 1, 38);
+    checkExpectedAllocation(response, "subcluster1", 1, -1);
+    checkExpectedAllocation(response, "subcluster2", 1, 38);
+    checkExpectedAllocation(response, "subcluster5", 1, 25);
+
+  }
+
+  @Test(timeout = 5000)
+  public void testStressPolicy() throws Exception {
+
+    // Tests how the headroom info are used to split based on the capacity
+    // each RM claims to give us.
+    // Configure policy to be 100% headroom based
+    getPolicyInfo().setHeadroomAlpha(1.0f);
+
+    initializePolicy();
+
+    int numRR = 1000;
+    List<ResourceRequest> resourceRequests = createLargeRandomList(numRR);
+
+    prepPolicyWithHeadroom();
+
+    int numIterations = 1000;
+    long tstart = System.currentTimeMillis();
+    for (int i = 0; i < numIterations; i++) {
+      Map<SubClusterId, List<ResourceRequest>> response =
+          ((FederationAMRMProxyPolicy) getPolicy())
+              .splitResourceRequests(resourceRequests);
+      validateSplit(response, resourceRequests);
+    }
+    long tend = System.currentTimeMillis();
+
+    LOG.info("Performed " + numIterations + " policy invocations (and "
+        + "validations) in " + (tend - tstart) + "ms");
+  }
+
+  @Test
+  public void testFWDAllZeroANY() throws Exception {
+
+    // Tests how the headroom info are used to split based on the capacity
+    // each RM claims to give us.
+    // Configure policy to be 100% headroom based
+    getPolicyInfo().setHeadroomAlpha(0.5f);
+
+    initializePolicy();
+    List<ResourceRequest> resourceRequests = createZeroSizedANYRequest();
+
+    // this receives responses from sc0,sc1,sc2
+    prepPolicyWithHeadroom();
+
+    Map<SubClusterId, List<ResourceRequest>> response =
+        ((FederationAMRMProxyPolicy) getPolicy())
+            .splitResourceRequests(resourceRequests);
+
+    // we expect all three to appear for a zero-sized ANY
+
+    // pretty print requests
+    prettyPrintRequests(response);
+
+    validateSplit(response, resourceRequests);
+
+    // we expect the zero size request to be sent to the first 3 rm (due to
+    // the fact that we received responses only from these 3 sublcusters)
+    checkExpectedAllocation(response, "subcluster0", 1, 0);
+    checkExpectedAllocation(response, "subcluster1", 1, 0);
+    checkExpectedAllocation(response, "subcluster2", 1, 0);
+    checkExpectedAllocation(response, "subcluster3", -1, -1);
+    checkExpectedAllocation(response, "subcluster4", -1, -1);
+    checkExpectedAllocation(response, "subcluster5", -1, -1);
+  }
+
+  @Test
+  public void testSplitBasedOnHeadroomAndWeights() throws Exception {
+
+    // Tests how the headroom info are used to split based on the capacity
+    // each RM claims to give us.
+
+    // Configure policy to be 50% headroom based and 50% weight based
+    getPolicyInfo().setHeadroomAlpha(0.5f);
+
+    initializePolicy();
+    List<ResourceRequest> resourceRequests = createSimpleRequest();
+
+    prepPolicyWithHeadroom();
+
+    Map<SubClusterId, List<ResourceRequest>> response =
+        ((FederationAMRMProxyPolicy) getPolicy())
+            .splitResourceRequests(resourceRequests);
+
+    // pretty print requests
+    prettyPrintRequests(response);
+
+    validateSplit(response, resourceRequests);
+
+    // in this case the headroom allocates 50 containers, while weights allocate
+    // the rest. due to weights we have 12.5 (round to 13) containers for each
+    // sublcuster, the rest is due to headroom.
+    checkExpectedAllocation(response, "subcluster0", 1, 50);
+    checkExpectedAllocation(response, "subcluster1", 1, 13);
+    checkExpectedAllocation(response, "subcluster2", 1, 13);
+    checkExpectedAllocation(response, "subcluster3", -1, -1);
+    checkExpectedAllocation(response, "subcluster4", -1, -1);
+    checkExpectedAllocation(response, "subcluster5", 1, 25);
+
+  }
+
+  private void prepPolicyWithHeadroom() throws YarnException {
+    AllocateResponse ar = getAllocateResponseWithTargetHeadroom(100);
+    ((FederationAMRMProxyPolicy) getPolicy())
+        .notifyOfResponse(SubClusterId.newInstance("subcluster0"), ar);
+
+    ar = getAllocateResponseWithTargetHeadroom(0);
+    ((FederationAMRMProxyPolicy) getPolicy())
+        .notifyOfResponse(SubClusterId.newInstance("subcluster1"), ar);
+
+    ar = getAllocateResponseWithTargetHeadroom(1);
+    ((FederationAMRMProxyPolicy) getPolicy())
+        .notifyOfResponse(SubClusterId.newInstance("subcluster2"), ar);
+  }
+
+  private AllocateResponse getAllocateResponseWithTargetHeadroom(
+      int numContainers) {
+    return AllocateResponse.newInstance(0, null, null,
+        Collections.<NodeReport> emptyList(),
+        Resource.newInstance(numContainers * 1024, numContainers), null, 10,
+        null, Collections.<NMToken> emptyList());
+  }
+
+  @Test
+  public void testSplitAllocateRequest() throws Exception {
+
+    // Test a complex List<ResourceRequest> is split correctly
+    initializePolicy();
+
+    // modify default initialization to include a "homesubcluster"
+    // which we will use as the default for when nodes or racks are unknown
+    SubClusterInfo sci = mock(SubClusterInfo.class);
+    when(sci.getState()).thenReturn(SubClusterState.SC_RUNNING);
+    when(sci.getSubClusterId()).thenReturn(getHomeSubCluster());
+    getActiveSubclusters().put(getHomeSubCluster(), sci);
+    SubClusterIdInfo sc = new SubClusterIdInfo(getHomeSubCluster().getId());
+
+    getPolicyInfo().getRouterPolicyWeights().put(sc, 0.1f);
+    getPolicyInfo().getAMRMPolicyWeights().put(sc, 0.1f);
+
+    FederationPoliciesTestUtil.initializePolicyContext(
+        getFederationPolicyContext(), getPolicy(), getPolicyInfo(),
+        getActiveSubclusters());
+
+    List<ResourceRequest> resourceRequests = createComplexRequest();
+
+    Map<SubClusterId, List<ResourceRequest>> response =
+        ((FederationAMRMProxyPolicy) getPolicy())
+            .splitResourceRequests(resourceRequests);
+
+    validateSplit(response, resourceRequests);
+    prettyPrintRequests(response);
+
+    // we expect 4 entry for home subcluster (3 for request-id 4, and a part
+    // of the broadcast of request-id 2
+    checkExpectedAllocation(response, getHomeSubCluster().getId(), 4, 23);
+
+    // for subcluster0 we expect 3 entry from request-id 0, and 3 from
+    // request-id 3, as well as part of the request-id 2 broadast
+    checkExpectedAllocation(response, "subcluster0", 7, 26);
+
+    // we expect 5 entry for subcluster1 (4 from request-id 1, and part
+    // of the broadcast of request-id 2
+    checkExpectedAllocation(response, "subcluster1", 5, 25);
+
+    // sub-cluster 2 should contain 3 entry from request-id 1 and 1 from the
+    // broadcast of request-id 2, and no request-id 0
+    checkExpectedAllocation(response, "subcluster2", 4, 23);
+
+    // subcluster id 3, 4 should not appear (due to weights or active/inactive)
+    checkExpectedAllocation(response, "subcluster3", -1, -1);
+    checkExpectedAllocation(response, "subcluster4", -1, -1);
+
+    // subcluster5 should get only part of the request-id 2 broadcast
+    checkExpectedAllocation(response, "subcluster5", 1, 20);
+
+    // check that the allocations that show up are what expected
+    for (ResourceRequest rr : response.get(getHomeSubCluster())) {
+      Assert.assertTrue(rr.getAllocationRequestId() == 4L
+          || rr.getAllocationRequestId() == 2L);
+    }
+
+    for (ResourceRequest rr : response.get(getHomeSubCluster())) {
+      Assert.assertTrue(rr.getAllocationRequestId() != 1L);
+    }
+
+    List<ResourceRequest> rrs =
+        response.get(SubClusterId.newInstance("subcluster0"));
+    for (ResourceRequest rr : rrs) {
+      Assert.assertTrue(rr.getAllocationRequestId() != 1L);
+    }
+
+    for (ResourceRequest rr : response
+        .get(SubClusterId.newInstance("subcluster2"))) {
+      Assert.assertTrue(rr.getAllocationRequestId() != 0L);
+    }
+
+    for (ResourceRequest rr : response
+        .get(SubClusterId.newInstance("subcluster5"))) {
+      Assert.assertTrue(rr.getAllocationRequestId() >= 2);
+      Assert.assertTrue(rr.getRelaxLocality());
+    }
+  }
+
+  // check that the number of containers in the first ResourceRequest in
+  // response for this sub-cluster matches expectations. -1 indicate the
+  // response should be null
+  private void checkExpectedAllocation(
+      Map<SubClusterId, List<ResourceRequest>> response, String subCluster,
+      long totResourceRequests, long totContainers) {
+    if (totContainers == -1) {
+      Assert.assertNull(response.get(SubClusterId.newInstance(subCluster)));
+    } else {
+      SubClusterId sc = SubClusterId.newInstance(subCluster);
+      Assert.assertEquals(totResourceRequests, response.get(sc).size());
+
+      long actualContCount = 0;
+      for (ResourceRequest rr : response.get(sc)) {
+        actualContCount += rr.getNumContainers();
+      }
+      Assert.assertEquals(totContainers, actualContCount);
+    }
+  }
+
+  private void validateSplit(Map<SubClusterId, List<ResourceRequest>> split,
+      List<ResourceRequest> original) throws YarnException {
+
+    SubClusterResolver resolver =
+        getFederationPolicyContext().getFederationSubclusterResolver();
+
+    // Apply general validation rules
+    int numUsedSubclusters = split.size();
+
+    Set<Long> originalIds = new HashSet<>();
+    Set<Long> splitIds = new HashSet<>();
+
+    int originalContainers = 0;
+    for (ResourceRequest rr : original) {
+      originalContainers += rr.getNumContainers();
+      originalIds.add(rr.getAllocationRequestId());
+    }
+
+    int splitContainers = 0;
+    for (Map.Entry<SubClusterId, List<ResourceRequest>> rrs : split
+        .entrySet()) {
+      for (ResourceRequest rr : rrs.getValue()) {
+        splitContainers += rr.getNumContainers();
+        splitIds.add(rr.getAllocationRequestId());
+        // check node-local asks are sent to right RM (only)
+        SubClusterId fid = null;
+        try {
+          fid = resolver.getSubClusterForNode(rr.getResourceName());
+        } catch (YarnException e) {
+          // ignore code will handle
+        }
+        if (!rrs.getKey().equals(getHomeSubCluster()) && fid != null
+            && !fid.equals(rrs.getKey())) {
+          Assert.fail("A node-local (or resolvable rack-local) RR should not "
+              + "be send to an RM other than what it resolves to.");
+        }
+      }
+    }
+
+    // check we are not inventing Allocation Ids
+    Assert.assertEquals(originalIds, splitIds);
+
+    // check we are not exceedingly replicating the container asks among
+    // RMs (a little is allowed due to rounding of fractional splits)
+    Assert.assertTrue(
+        " Containers requested (" + splitContainers + ") should "
+            + "not exceed the original count of containers ("
+            + originalContainers + ") by more than the number of subclusters ("
+            + numUsedSubclusters + ")",
+        originalContainers + numUsedSubclusters >= splitContainers);
+
+    // Test target Ids
+    for (SubClusterId targetId : split.keySet()) {
+      Assert.assertTrue("Target subclusters should be in the active set",
+          getActiveSubclusters().containsKey(targetId));
+      Assert.assertTrue(
+          "Target subclusters (" + targetId + ") should have weight >0 in "
+              + "the policy ",
+          getPolicyInfo().getRouterPolicyWeights()
+              .get(new SubClusterIdInfo(targetId)) > 0);
+    }
+  }
+
+  private void prettyPrintRequests(
+      Map<SubClusterId, List<ResourceRequest>> response) {
+    for (Map.Entry<SubClusterId, List<ResourceRequest>> entry : response
+        .entrySet()) {
+      String str = "";
+      for (ResourceRequest rr : entry.getValue()) {
+        str += " [id:" + rr.getAllocationRequestId() + " loc:"
+            + rr.getResourceName() + " numCont:" + rr.getNumContainers()
+            + "], ";
+      }
+      LOG.info(entry.getKey() + " --> " + str);
+    }
+  }
+
+  private List<ResourceRequest> createLargeRandomList(int numRR)
+      throws Exception {
+
+    List<ResourceRequest> out = new ArrayList<>();
+    Random rand = new Random(1);
+    DefaultSubClusterResolverImpl resolver =
+        (DefaultSubClusterResolverImpl) getFederationPolicyContext()
+            .getFederationSubclusterResolver();
+
+    List<String> nodes =
+        new ArrayList<>(resolver.getNodeToSubCluster().keySet());
+
+    for (int i = 0; i < numRR; i++) {
+      String nodeName = nodes.get(rand.nextInt(nodes.size()));
+      long allocationId = (long) rand.nextInt(20);
+
+      // create a single container request in sc0
+      out.add(FederationPoliciesTestUtil.createResourceRequest(allocationId,
+          nodeName, 1024, 1, 1, rand.nextInt(100), null, rand.nextBoolean()));
+    }
+    return out;
+  }
+
+  private List<ResourceRequest> createSimpleRequest() throws Exception {
+
+    List<ResourceRequest> out = new ArrayList<>();
+
+    // create a single container request in sc0
+    out.add(FederationPoliciesTestUtil.createResourceRequest(0L,
+        ResourceRequest.ANY, 1024, 1, 1, 100, null, true));
+    return out;
+  }
+
+  private List<ResourceRequest> createZeroSizedANYRequest() throws Exception {
+
+    List<ResourceRequest> out = new ArrayList<>();
+
+    // create a single container request in sc0
+    out.add(FederationPoliciesTestUtil.createResourceRequest(0L,
+        ResourceRequest.ANY, 1024, 1, 1, 0, null, true));
+    return out;
+  }
+
+  private List<ResourceRequest> createComplexRequest() throws Exception {
+
+    List<ResourceRequest> out = new ArrayList<>();
+
+    // create a single container request in sc0
+    out.add(FederationPoliciesTestUtil.createResourceRequest(0L,
+        "subcluster0-rack0-host0", 1024, 1, 1, 1, null, false));
+    out.add(FederationPoliciesTestUtil.createResourceRequest(0L,
+        "subcluster0-rack0", 1024, 1, 1, 1, null, false));
+    out.add(FederationPoliciesTestUtil.createResourceRequest(0L,
+        ResourceRequest.ANY, 1024, 1, 1, 1, null, false));
+
+    // create a single container request with 3 alternative hosts across sc1,sc2
+    // where we want 2 containers in sc1 and 1 in sc2
+    out.add(FederationPoliciesTestUtil.createResourceRequest(1L,
+        "subcluster1-rack1-host1", 1024, 1, 1, 1, null, false));
+    out.add(FederationPoliciesTestUtil.createResourceRequest(1L,
+        "subcluster1-rack1-host2", 1024, 1, 1, 1, null, false));
+    out.add(FederationPoliciesTestUtil.createResourceRequest(1L,
+        "subcluster2-rack3-host3", 1024, 1, 1, 1, null, false));
+    out.add(FederationPoliciesTestUtil.createResourceRequest(1L,
+        "subcluster1-rack1", 1024, 1, 1, 2, null, false));
+    out.add(FederationPoliciesTestUtil.createResourceRequest(1L,
+        "subcluster2-rack3", 1024, 1, 1, 1, null, false));
+    out.add(FederationPoliciesTestUtil.createResourceRequest(1L,
+        ResourceRequest.ANY, 1024, 1, 1, 2, null, false));
+
+    // create a non-local ANY request that can span anything
+    out.add(FederationPoliciesTestUtil.createResourceRequest(2L,
+        ResourceRequest.ANY, 1024, 1, 1, 100, null, true));
+
+    // create a single container request in sc0 with relaxed locality
+    out.add(FederationPoliciesTestUtil.createResourceRequest(3L,
+        "subcluster0-rack0-host0", 1024, 1, 1, 1, null, true));
+    out.add(FederationPoliciesTestUtil.createResourceRequest(3L,
+        "subcluster0-rack0", 1024, 1, 1, 1, null, true));
+    out.add(FederationPoliciesTestUtil.createResourceRequest(3L,
+        ResourceRequest.ANY, 1024, 1, 1, 1, null, true));
+
+    // create a request of an unknown node/rack and expect this to show up
+    // in homesubcluster
+    out.add(FederationPoliciesTestUtil.createResourceRequest(4L, "unknownNode",
+        1024, 1, 1, 1, null, false));
+    out.add(FederationPoliciesTestUtil.createResourceRequest(4L, "unknownRack",
+        1024, 1, 1, 1, null, false));
+    out.add(FederationPoliciesTestUtil.createResourceRequest(4L,
+        ResourceRequest.ANY, 1024, 1, 1, 1, null, false));
+
+    return out;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53ca351b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestLoadBasedRouterPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestLoadBasedRouterPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestLoadBasedRouterPolicy.java
index 9e94f72..906e35f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestLoadBasedRouterPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestLoadBasedRouterPolicy.java
@@ -17,6 +17,9 @@
 
 package org.apache.hadoop.yarn.server.federation.policies.router;
 
+import java.util.HashMap;
+import java.util.Map;
+
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.federation.policies.BaseFederationPoliciesTest;
 import org.apache.hadoop.yarn.server.federation.policies.dao.WeightedPolicyInfo;
@@ -29,12 +32,9 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
-import java.util.HashMap;
-import java.util.Map;
-
 /**
- * Simple test class for the {@link LoadBasedRouterPolicy}. Test that the
- * load is properly considered for allocation.
+ * Simple test class for the {@link LoadBasedRouterPolicy}. Test that the load
+ * is properly considered for allocation.
  */
 public class TestLoadBasedRouterPolicy extends BaseFederationPoliciesTest {
 
@@ -47,12 +47,10 @@ public class TestLoadBasedRouterPolicy extends BaseFederationPoliciesTest {
 
     // simulate 20 active subclusters
     for (int i = 0; i < 20; i++) {
-      SubClusterIdInfo sc =
-          new SubClusterIdInfo(String.format("sc%02d", i));
+      SubClusterIdInfo sc = new SubClusterIdInfo(String.format("sc%02d", i));
       SubClusterInfo federationSubClusterInfo =
           SubClusterInfo.newInstance(sc.toId(), null, null, null, null, -1,
-              SubClusterState.SC_RUNNING, -1,
-              generateClusterMetricsInfo(i));
+              SubClusterState.SC_RUNNING, -1, generateClusterMetricsInfo(i));
       getActiveSubclusters().put(sc.toId(), federationSubClusterInfo);
       float weight = getRand().nextInt(2);
       if (i == 5) {
@@ -76,7 +74,7 @@ public class TestLoadBasedRouterPolicy extends BaseFederationPoliciesTest {
   private String generateClusterMetricsInfo(int id) {
 
     long mem = 1024 * getRand().nextInt(277 * 100 - 1);
-    //plant a best cluster
+    // plant a best cluster
     if (id == 5) {
       mem = 1024 * 277 * 100;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53ca351b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestPriorityRouterPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestPriorityRouterPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestPriorityRouterPolicy.java
index ff5175d..eefcfd9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestPriorityRouterPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestPriorityRouterPolicy.java
@@ -16,6 +16,12 @@
  */
 package org.apache.hadoop.yarn.server.federation.policies.router;
 
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.HashMap;
+import java.util.Map;
+
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.federation.policies.BaseFederationPoliciesTest;
 import org.apache.hadoop.yarn.server.federation.policies.dao.WeightedPolicyInfo;
@@ -28,12 +34,6 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
-import java.util.HashMap;
-import java.util.Map;
-
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
 /**
  * Simple test class for the {@link PriorityRouterPolicy}. Tests that the
  * weights are correctly used for ordering the choice of sub-clusters.
@@ -72,8 +72,7 @@ public class TestPriorityRouterPolicy extends BaseFederationPoliciesTest {
     getPolicyInfo().setRouterPolicyWeights(routerWeights);
     getPolicyInfo().setAMRMPolicyWeights(amrmWeights);
     FederationPoliciesTestUtil.initializePolicyContext(getPolicy(),
-        getPolicyInfo(),
-        getActiveSubclusters());
+        getPolicyInfo(), getActiveSubclusters());
 
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53ca351b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestWeightedRandomRouterPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestWeightedRandomRouterPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestWeightedRandomRouterPolicy.java
index a612685..78967d0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestWeightedRandomRouterPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestWeightedRandomRouterPolicy.java
@@ -17,6 +17,13 @@
 
 package org.apache.hadoop.yarn.server.federation.policies.router;
 
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicLong;
+
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.federation.policies.BaseFederationPoliciesTest;
 import org.apache.hadoop.yarn.server.federation.policies.dao.WeightedPolicyInfo;
@@ -29,13 +36,6 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
-import java.util.HashMap;
-import java.util.Map;
-import java.util.concurrent.atomic.AtomicLong;
-
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
 /**
  * Simple test class for the {@link WeightedRandomRouterPolicy}. Generate large
  * number of randomized tests to check we are weighiting correctly even if
@@ -71,8 +71,7 @@ public class TestWeightedRandomRouterPolicy extends BaseFederationPoliciesTest {
     getPolicyInfo().setAMRMPolicyWeights(amrmWeights);
 
     FederationPoliciesTestUtil.initializePolicyContext(getPolicy(),
-        getPolicyInfo(),
-        getActiveSubclusters());
+        getPolicyInfo(), getActiveSubclusters());
 
   }
 
@@ -88,8 +87,8 @@ public class TestWeightedRandomRouterPolicy extends BaseFederationPoliciesTest {
     float numberOfDraws = 1000000;
 
     for (float i = 0; i < numberOfDraws; i++) {
-      SubClusterId chosenId = ((FederationRouterPolicy) getPolicy()).
-          getHomeSubcluster(getApplicationSubmissionContext());
+      SubClusterId chosenId = ((FederationRouterPolicy) getPolicy())
+          .getHomeSubcluster(getApplicationSubmissionContext());
       counter.get(chosenId).incrementAndGet();
     }
 
@@ -113,13 +112,15 @@ public class TestWeightedRandomRouterPolicy extends BaseFederationPoliciesTest {
       if (getActiveSubclusters().containsKey(counterEntry.getKey())) {
         Assert.assertTrue(
             "Id " + counterEntry.getKey() + " Actual weight: " + actualWeight
-                + " expected weight: " + expectedWeight, expectedWeight == 0 ||
-                (actualWeight / expectedWeight) < 1.1
-                    && (actualWeight / expectedWeight) > 0.9);
+                + " expected weight: " + expectedWeight,
+            expectedWeight == 0 || (actualWeight / expectedWeight) < 1.1
+                && (actualWeight / expectedWeight) > 0.9);
       } else {
-        Assert.assertTrue(
-            "Id " + counterEntry.getKey() + " Actual weight: " + actualWeight
-                + " expected weight: " + expectedWeight, actualWeight == 0);
+        Assert
+            .assertTrue(
+                "Id " + counterEntry.getKey() + " Actual weight: "
+                    + actualWeight + " expected weight: " + expectedWeight,
+                actualWeight == 0);
 
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53ca351b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/FederationPoliciesTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/FederationPoliciesTestUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/FederationPoliciesTestUtil.java
index f901329..87ed8d1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/FederationPoliciesTestUtil.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/FederationPoliciesTestUtil.java
@@ -17,6 +17,7 @@
 package org.apache.hadoop.yarn.server.federation.utils;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.*;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.federation.policies.ConfigurableFederationPolicy;
@@ -26,6 +27,7 @@ import org.apache.hadoop.yarn.server.federation.resolver.DefaultSubClusterResolv
 import org.apache.hadoop.yarn.server.federation.resolver.SubClusterResolver;
 import org.apache.hadoop.yarn.server.federation.store.FederationStateStore;
 import org.apache.hadoop.yarn.server.federation.store.records.*;
+import org.apache.hadoop.yarn.util.Records;
 
 import java.net.URL;
 import java.nio.ByteBuffer;
@@ -48,6 +50,68 @@ public final class FederationPoliciesTestUtil {
     // disabled.
   }
 
+  private static final String FEDR_NODE_PREFIX = "fedr-test-node-";
+
+
+  public static List<ResourceRequest> createResourceRequests(String[] hosts,
+      int memory, int vCores, int priority, int containers,
+      String labelExpression, boolean relaxLocality) throws YarnException {
+    List<ResourceRequest> reqs = new ArrayList<ResourceRequest>();
+    for (String host : hosts) {
+      ResourceRequest hostReq =
+          createResourceRequest(host, memory, vCores, priority, containers,
+              labelExpression, relaxLocality);
+      reqs.add(hostReq);
+      ResourceRequest rackReq =
+          createResourceRequest("/default-rack", memory, vCores, priority,
+              containers, labelExpression, relaxLocality);
+      reqs.add(rackReq);
+    }
+
+    ResourceRequest offRackReq =
+        createResourceRequest(ResourceRequest.ANY, memory, vCores, priority,
+            containers, labelExpression, relaxLocality);
+    reqs.add(offRackReq);
+    return reqs;
+  }
+
+  protected static ResourceRequest createResourceRequest(String resource,
+      int memory, int vCores, int priority, int containers,
+      boolean relaxLocality) throws YarnException {
+    return createResourceRequest(resource, memory, vCores, priority, containers,
+        null, relaxLocality);
+  }
+
+  @SuppressWarnings("checkstyle:parameternumber")
+  public static ResourceRequest createResourceRequest(long id, String resource,
+      int memory, int vCores, int priority, int containers,
+      String labelExpression, boolean relaxLocality) throws YarnException {
+    ResourceRequest out =
+        createResourceRequest(resource, memory, vCores, priority, containers,
+            labelExpression, relaxLocality);
+    out.setAllocationRequestId(id);
+    return out;
+  }
+
+  public static ResourceRequest createResourceRequest(String resource,
+      int memory, int vCores, int priority, int containers,
+      String labelExpression, boolean relaxLocality) throws YarnException {
+    ResourceRequest req = Records.newRecord(ResourceRequest.class);
+    req.setResourceName(resource);
+    req.setNumContainers(containers);
+    Priority pri = Records.newRecord(Priority.class);
+    pri.setPriority(priority);
+    req.setPriority(pri);
+    Resource capability = Records.newRecord(Resource.class);
+    capability.setMemorySize(memory);
+    capability.setVirtualCores(vCores);
+    req.setCapability(capability);
+    if (labelExpression != null) {
+      req.setNodeLabelExpression(labelExpression);
+    }
+    req.setRelaxLocality(relaxLocality);
+    return req;
+  }
 
   public static void initializePolicyContext(
       FederationPolicyInitializationContext fpc, ConfigurableFederationPolicy

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53ca351b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/resources/nodes
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/resources/nodes b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/resources/nodes
index e4d6112..2b7e237 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/resources/nodes
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/resources/nodes
@@ -1,4 +1,8 @@
 node1,subcluster1,rack1
  node2 , subcluster2, RACK1
 noDE3,subcluster3, rack2
-node4, subcluster3, rack2
\ No newline at end of file
+node4, subcluster3, rack2
+subcluster0-rack0-host0,subcluster0, subcluster0-rack0
+Subcluster1-RACK1-HOST1,subcluster1, subCluster1-RACK1
+SUBCLUSTER1-RACK1-HOST2,subcluster1, subCluster1-RACK1
+SubCluster2-RACK3-HOST3,subcluster2, subcluster2-rack3


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[37/42] hadoop git commit: YARN-6203: Occasional test failure in TestWeightedRandomRouterPolicy (curino)

Posted by su...@apache.org.
YARN-6203: Occasional test failure in TestWeightedRandomRouterPolicy (curino)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/8c175272
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/8c175272
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/8c175272

Branch: refs/heads/YARN-2915
Commit: 8c175272230a8c484f38cf346cd37eb90575bc34
Parents: fa7e31e
Author: Carlo Curino <cu...@apache.org>
Authored: Thu Apr 13 12:09:39 2017 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon May 8 16:59:13 2017 -0700

----------------------------------------------------------------------
 .../router/TestWeightedRandomRouterPolicy.java    | 18 +++++++++++-------
 1 file changed, 11 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8c175272/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestWeightedRandomRouterPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestWeightedRandomRouterPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestWeightedRandomRouterPolicy.java
index 34cc5f8..09173e6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestWeightedRandomRouterPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestWeightedRandomRouterPolicy.java
@@ -50,8 +50,9 @@ public class TestWeightedRandomRouterPolicy extends BaseRouterPoliciesTest {
     Map<SubClusterIdInfo, Float> routerWeights = new HashMap<>();
     Map<SubClusterIdInfo, Float> amrmWeights = new HashMap<>();
 
-    // simulate 20 subclusters with a 5% chance of being inactive
-    for (int i = 0; i < 20; i++) {
+    float numSubClusters = 20;
+    // simulate N subclusters each with a 5% chance of being inactive
+    for (int i = 0; i < numSubClusters; i++) {
       SubClusterIdInfo sc = new SubClusterIdInfo("sc" + i);
       // with 5% omit a subcluster
       if (getRand().nextFloat() < 0.95f) {
@@ -60,8 +61,12 @@ public class TestWeightedRandomRouterPolicy extends BaseRouterPoliciesTest {
         when(sci.getSubClusterId()).thenReturn(sc.toId());
         getActiveSubclusters().put(sc.toId(), sci);
       }
-      // 5% chance we omit one of the weights
-      float weight = getRand().nextFloat();
+
+      // 80% of the weight is evenly spread, 20% is randomly generated
+      float weight =
+          (0.8f * 1f / numSubClusters) + (0.2f * getRand().nextFloat());
+
+      // also 5% chance we omit one of the weights
       if (i <= 5 || getRand().nextFloat() > 0.05f) {
         routerWeights.put(sc, weight);
         amrmWeights.put(sc, weight);
@@ -89,7 +94,7 @@ public class TestWeightedRandomRouterPolicy extends BaseRouterPoliciesTest {
       counter.put(id.toId(), new AtomicLong(0));
     }
 
-    float numberOfDraws = 100000;
+    float numberOfDraws = 10000;
 
     for (float i = 0; i < numberOfDraws; i++) {
       SubClusterId chosenId = ((FederationRouterPolicy) getPolicy())
@@ -118,8 +123,7 @@ public class TestWeightedRandomRouterPolicy extends BaseRouterPoliciesTest {
         Assert.assertTrue(
             "Id " + counterEntry.getKey() + " Actual weight: " + actualWeight
                 + " expected weight: " + expectedWeight,
-            expectedWeight == 0 || (actualWeight / expectedWeight) < 1.2
-                && (actualWeight / expectedWeight) > 0.8);
+            Math.abs(actualWeight - expectedWeight) < 0.01);
       } else {
         Assert
             .assertTrue(


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[42/42] hadoop git commit: YARN-3663. Federation State and Policy Store (DBMS implementation). (Giovanni Matteo Fumarola via curino).

Posted by su...@apache.org.
YARN-3663. Federation State and Policy Store (DBMS implementation). (Giovanni Matteo Fumarola via curino).


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/5599b916
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/5599b916
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/5599b916

Branch: refs/heads/YARN-2915
Commit: 5599b916ff653c6d9c8105148989b1f6eae4fc75
Parents: 8c17527
Author: Carlo Curino <cu...@apache.org>
Authored: Tue Apr 25 15:14:02 2017 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon May 8 16:59:13 2017 -0700

----------------------------------------------------------------------
 LICENSE.txt                                     |   1 +
 hadoop-project/pom.xml                          |  12 +
 .../hadoop/yarn/conf/YarnConfiguration.java     |  23 +
 .../yarn/conf/TestYarnConfigurationFields.java  |  14 +
 .../hadoop-yarn-server-common/pom.xml           |  20 +
 .../FederationStateStoreErrorCode.java          | 105 ---
 .../FederationStateStoreException.java          |  17 +-
 .../store/impl/MemoryFederationStateStore.java  |  81 +-
 .../store/impl/SQLFederationStateStore.java     | 937 +++++++++++++++++++
 .../store/records/SubClusterState.java          |  21 +
 ...cationHomeSubClusterStoreInputValidator.java |  12 +-
 ...ationMembershipStateStoreInputValidator.java |  14 +-
 .../FederationPolicyStoreInputValidator.java    |   6 +-
 .../store/utils/FederationStateStoreUtils.java  | 109 ++-
 .../impl/FederationStateStoreBaseTest.java      |  74 +-
 .../store/impl/HSQLDBFederationStateStore.java  | 252 +++++
 .../impl/TestMemoryFederationStateStore.java    |   3 +-
 .../store/impl/TestSQLFederationStateStore.java |  49 +
 .../TestFederationStateStoreInputValidator.java | 146 +--
 .../TestFederationStateStoreFacadeRetry.java    |   7 +-
 .../FederationStateStoreStoreProcs.sql          | 511 ++++++++++
 .../SQLServer/FederationStateStoreTables.sql    | 122 +++
 22 files changed, 2228 insertions(+), 308 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5599b916/LICENSE.txt
----------------------------------------------------------------------
diff --git a/LICENSE.txt b/LICENSE.txt
index 969708f..1582f6c 100644
--- a/LICENSE.txt
+++ b/LICENSE.txt
@@ -661,6 +661,7 @@ hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/jquery-1.10.2.min.js
 hadoop-tools/hadoop-sls/src/main/html/js/thirdparty/jquery.js
 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/jquery
 Apache HBase - Server which contains JQuery minified javascript library version 1.8.3
+Microsoft SQLServer - JDBC version 6.1.0.jre7
 --------------------------------------------------------------------------------
 
 Copyright 2005, 2012, 2013 jQuery Foundation and other contributors, https://jquery.org/

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5599b916/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index b07c510..61112b2 100755
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -100,6 +100,8 @@
 
     <jcache.version>1.0.0</jcache.version>
     <ehcache.version>3.0.3</ehcache.version>
+    <hikari.version>2.4.11</hikari.version>
+    <mssql.version>6.1.0.jre7</mssql.version>
 
     <!-- define the Java language version used by the compiler -->
     <javac.version>1.8</javac.version>
@@ -1281,6 +1283,16 @@
           <artifactId>ehcache</artifactId>
           <version>${ehcache.version}</version>
         </dependency>
+        <dependency>
+          <groupId>com.zaxxer</groupId>
+          <artifactId>HikariCP-java7</artifactId>
+          <version>${hikari.version}</version>
+        </dependency>
+        <dependency>
+          <groupId>com.microsoft.sqlserver</groupId>
+          <artifactId>mssql-jdbc</artifactId>
+          <version>${mssql.version}</version>
+        </dependency>
     </dependencies>
   </dependencyManagement>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5599b916/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
index 4ba8c85..7139394 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
@@ -2592,6 +2592,29 @@ public class YarnConfiguration extends Configuration {
 
   public static final String DEFAULT_FEDERATION_POLICY_MANAGER_PARAMS = "";
 
+  private static final String FEDERATION_STATESTORE_SQL_PREFIX =
+      FEDERATION_PREFIX + "state-store.sql.";
+
+  public static final String FEDERATION_STATESTORE_SQL_USERNAME =
+      FEDERATION_STATESTORE_SQL_PREFIX + "username";
+
+  public static final String FEDERATION_STATESTORE_SQL_PASSWORD =
+      FEDERATION_STATESTORE_SQL_PREFIX + "password";
+
+  public static final String FEDERATION_STATESTORE_SQL_URL =
+      FEDERATION_STATESTORE_SQL_PREFIX + "url";
+
+  public static final String FEDERATION_STATESTORE_SQL_JDBC_CLASS =
+      FEDERATION_STATESTORE_SQL_PREFIX + "jdbc-class";
+
+  public static final String DEFAULT_FEDERATION_STATESTORE_SQL_JDBC_CLASS =
+      "org.hsqldb.jdbc.JDBCDataSource";
+
+  public static final String FEDERATION_STATESTORE_SQL_MAXCONNECTIONS =
+      FEDERATION_STATESTORE_SQL_PREFIX + "max-connections";
+
+  public static final int DEFAULT_FEDERATION_STATESTORE_SQL_MAXCONNECTIONS = 1;
+
   ////////////////////////////////
   // Other Configs
   ////////////////////////////////

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5599b916/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
index 6e33c0a..c3cb78d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
@@ -90,6 +90,20 @@ public class TestYarnConfigurationFields extends TestConfigurationFieldsBase {
     configurationPropsToSkipCompare
         .add(YarnConfiguration.DEFAULT_FEDERATION_POLICY_MANAGER_PARAMS);
 
+    // Federation StateStore SQL implementation configs to be ignored
+    configurationPropsToSkipCompare
+        .add(YarnConfiguration.FEDERATION_STATESTORE_SQL_JDBC_CLASS);
+    configurationPropsToSkipCompare
+        .add(YarnConfiguration.DEFAULT_FEDERATION_STATESTORE_SQL_JDBC_CLASS);
+    configurationPropsToSkipCompare
+        .add(YarnConfiguration.FEDERATION_STATESTORE_SQL_USERNAME);
+    configurationPropsToSkipCompare
+        .add(YarnConfiguration.FEDERATION_STATESTORE_SQL_PASSWORD);
+    configurationPropsToSkipCompare
+        .add(YarnConfiguration.FEDERATION_STATESTORE_SQL_URL);
+    configurationPropsToSkipCompare
+        .add(YarnConfiguration.FEDERATION_STATESTORE_SQL_MAXCONNECTIONS);
+
     // Ignore blacklisting nodes for AM failures feature since it is still a
     // "work in progress"
     configurationPropsToSkipCompare.add(YarnConfiguration.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5599b916/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml
index 5ae8889..3bf1b88 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml
@@ -116,6 +116,26 @@
       <groupId>org.ehcache</groupId>
       <artifactId>ehcache</artifactId>
     </dependency>
+    <dependency>
+      <groupId>com.zaxxer</groupId>
+      <artifactId>HikariCP-java7</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.hsqldb</groupId>
+      <artifactId>hsqldb</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.microsoft.sqlserver</groupId>
+      <artifactId>mssql-jdbc</artifactId>
+      <scope>runtime</scope>
+       <exclusions>
+        <exclusion>
+          <groupId>com.microsoft.azure</groupId>
+          <artifactId>azure-keyvault</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
   </dependencies>
 
   <build>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5599b916/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/exception/FederationStateStoreErrorCode.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/exception/FederationStateStoreErrorCode.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/exception/FederationStateStoreErrorCode.java
deleted file mode 100644
index 88e2d3a..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/exception/FederationStateStoreErrorCode.java
+++ /dev/null
@@ -1,105 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.server.federation.store.exception;
-
-import org.apache.hadoop.classification.InterfaceAudience.Public;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
-
-/**
- * <p>
- * Logical error codes from <code>FederationStateStore</code>.
- * </p>
- */
-@Public
-@Unstable
-public enum FederationStateStoreErrorCode {
-
-  MEMBERSHIP_INSERT_FAIL(1101, "Fail to insert a tuple into Membership table."),
-
-  MEMBERSHIP_DELETE_FAIL(1102, "Fail to delete a tuple from Membership table."),
-
-  MEMBERSHIP_SINGLE_SELECT_FAIL(1103,
-      "Fail to select a tuple from Membership table."),
-
-  MEMBERSHIP_MULTIPLE_SELECT_FAIL(1104,
-      "Fail to select multiple tuples from Membership table."),
-
-  MEMBERSHIP_UPDATE_DEREGISTER_FAIL(1105,
-      "Fail to update/deregister a tuple in Membership table."),
-
-  MEMBERSHIP_UPDATE_HEARTBEAT_FAIL(1106,
-      "Fail to update/heartbeat a tuple in Membership table."),
-
-  APPLICATIONS_INSERT_FAIL(1201,
-      "Fail to insert a tuple into ApplicationsHomeSubCluster table."),
-
-  APPLICATIONS_DELETE_FAIL(1202,
-      "Fail to delete a tuple from ApplicationsHomeSubCluster table"),
-
-  APPLICATIONS_SINGLE_SELECT_FAIL(1203,
-      "Fail to select a tuple from ApplicationsHomeSubCluster table."),
-
-  APPLICATIONS_MULTIPLE_SELECT_FAIL(1204,
-      "Fail to select multiple tuple from ApplicationsHomeSubCluster table."),
-
-  APPLICATIONS_UPDATE_FAIL(1205,
-      "Fail to update a tuple in ApplicationsHomeSubCluster table."),
-
-  POLICY_INSERT_FAIL(1301, "Fail to insert a tuple into Policy table."),
-
-  POLICY_DELETE_FAIL(1302, "Fail to delete a tuple from Membership table."),
-
-  POLICY_SINGLE_SELECT_FAIL(1303, "Fail to select a tuple from Policy table."),
-
-  POLICY_MULTIPLE_SELECT_FAIL(1304,
-      "Fail to select multiple tuples from Policy table."),
-
-  POLICY_UPDATE_FAIL(1305, "Fail to update a tuple in Policy table.");
-
-  private final int id;
-  private final String msg;
-
-  FederationStateStoreErrorCode(int id, String msg) {
-    this.id = id;
-    this.msg = msg;
-  }
-
-  /**
-   * Get the error code related to the FederationStateStore failure.
-   *
-   * @return the error code related to the FederationStateStore failure.
-   */
-  public int getId() {
-    return this.id;
-  }
-
-  /**
-   * Get the error message related to the FederationStateStore failure.
-   *
-   * @return the error message related to the FederationStateStore failure.
-   */
-  public String getMsg() {
-    return this.msg;
-  }
-
-  @Override
-  public String toString() {
-    return "\nError Code: " + this.id + "\nError Message: " + this.msg;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5599b916/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/exception/FederationStateStoreException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/exception/FederationStateStoreException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/exception/FederationStateStoreException.java
index 81a9e99..1013ec6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/exception/FederationStateStoreException.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/exception/FederationStateStoreException.java
@@ -31,15 +31,20 @@ public class FederationStateStoreException extends YarnException {
    */
   private static final long serialVersionUID = -6453353714832159296L;
 
-  private FederationStateStoreErrorCode code;
-
-  public FederationStateStoreException(FederationStateStoreErrorCode code) {
+  public FederationStateStoreException() {
     super();
-    this.code = code;
   }
 
-  public FederationStateStoreErrorCode getCode() {
-    return code;
+  public FederationStateStoreException(String message) {
+    super(message);
+  }
+
+  public FederationStateStoreException(Throwable cause) {
+    super(cause);
+  }
+
+  public FederationStateStoreException(String message, Throwable cause) {
+    super(message, cause);
   }
 
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5599b916/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/MemoryFederationStateStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/MemoryFederationStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/MemoryFederationStateStore.java
index 127bf82..fbdb7bf 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/MemoryFederationStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/MemoryFederationStateStore.java
@@ -18,21 +18,17 @@
 package org.apache.hadoop.yarn.server.federation.store.impl;
 
 import java.util.ArrayList;
+import java.util.Calendar;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.TimeZone;
 import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.exceptions.YarnException;
-import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
-import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
-import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyConfiguration;
 import org.apache.hadoop.yarn.server.federation.store.FederationStateStore;
-import org.apache.hadoop.yarn.server.federation.store.exception.FederationStateStoreErrorCode;
-import org.apache.hadoop.yarn.server.federation.store.records.SubClusterDeregisterRequest;
-import org.apache.hadoop.yarn.server.federation.store.records.SubClusterDeregisterResponse;
 import org.apache.hadoop.yarn.server.federation.store.records.AddApplicationHomeSubClusterRequest;
 import org.apache.hadoop.yarn.server.federation.store.records.AddApplicationHomeSubClusterResponse;
 import org.apache.hadoop.yarn.server.federation.store.records.ApplicationHomeSubCluster;
@@ -52,8 +48,13 @@ import org.apache.hadoop.yarn.server.federation.store.records.GetSubClustersInfo
 import org.apache.hadoop.yarn.server.federation.store.records.GetSubClustersInfoResponse;
 import org.apache.hadoop.yarn.server.federation.store.records.SetSubClusterPolicyConfigurationRequest;
 import org.apache.hadoop.yarn.server.federation.store.records.SetSubClusterPolicyConfigurationResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterDeregisterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterDeregisterResponse;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterHeartbeatRequest;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterHeartbeatResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyConfiguration;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterRegisterRequest;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterRegisterResponse;
 import org.apache.hadoop.yarn.server.federation.store.records.UpdateApplicationHomeSubClusterRequest;
@@ -98,16 +99,18 @@ public class MemoryFederationStateStore implements FederationStateStore {
   @Override
   public SubClusterRegisterResponse registerSubCluster(
       SubClusterRegisterRequest request) throws YarnException {
-    FederationMembershipStateStoreInputValidator
-        .validateSubClusterRegisterRequest(request);
+    FederationMembershipStateStoreInputValidator.validate(request);
     SubClusterInfo subClusterInfo = request.getSubClusterInfo();
 
+    long currentTime =
+        Calendar.getInstance(TimeZone.getTimeZone("UTC")).getTimeInMillis();
+
     SubClusterInfo subClusterInfoToSave =
         SubClusterInfo.newInstance(subClusterInfo.getSubClusterId(),
             subClusterInfo.getAMRMServiceAddress(),
             subClusterInfo.getClientRMServiceAddress(),
             subClusterInfo.getRMAdminServiceAddress(),
-            subClusterInfo.getRMWebServiceAddress(), clock.getTime(),
+            subClusterInfo.getRMWebServiceAddress(), currentTime,
             subClusterInfo.getState(), subClusterInfo.getLastStartTime(),
             subClusterInfo.getCapability());
 
@@ -118,15 +121,12 @@ public class MemoryFederationStateStore implements FederationStateStore {
   @Override
   public SubClusterDeregisterResponse deregisterSubCluster(
       SubClusterDeregisterRequest request) throws YarnException {
-    FederationMembershipStateStoreInputValidator
-        .validateSubClusterDeregisterRequest(request);
+    FederationMembershipStateStoreInputValidator.validate(request);
     SubClusterInfo subClusterInfo = membership.get(request.getSubClusterId());
     if (subClusterInfo == null) {
       String errMsg =
           "SubCluster " + request.getSubClusterId().toString() + " not found";
-      FederationStateStoreUtils.logAndThrowStoreException(LOG,
-          FederationStateStoreErrorCode.MEMBERSHIP_UPDATE_DEREGISTER_FAIL,
-          errMsg);
+      FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
     } else {
       subClusterInfo.setState(request.getState());
     }
@@ -138,20 +138,20 @@ public class MemoryFederationStateStore implements FederationStateStore {
   public SubClusterHeartbeatResponse subClusterHeartbeat(
       SubClusterHeartbeatRequest request) throws YarnException {
 
-    FederationMembershipStateStoreInputValidator
-        .validateSubClusterHeartbeatRequest(request);
+    FederationMembershipStateStoreInputValidator.validate(request);
     SubClusterId subClusterId = request.getSubClusterId();
     SubClusterInfo subClusterInfo = membership.get(subClusterId);
 
     if (subClusterInfo == null) {
-      String errMsg = "Subcluster " + subClusterId.toString()
+      String errMsg = "SubCluster " + subClusterId.toString()
           + " does not exist; cannot heartbeat";
-      FederationStateStoreUtils.logAndThrowStoreException(LOG,
-          FederationStateStoreErrorCode.MEMBERSHIP_UPDATE_HEARTBEAT_FAIL,
-          errMsg);
+      FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
     }
 
-    subClusterInfo.setLastHeartBeat(clock.getTime());
+    long currentTime =
+        Calendar.getInstance(TimeZone.getTimeZone("UTC")).getTimeInMillis();
+
+    subClusterInfo.setLastHeartBeat(currentTime);
     subClusterInfo.setState(request.getState());
     subClusterInfo.setCapability(request.getCapability());
 
@@ -162,14 +162,12 @@ public class MemoryFederationStateStore implements FederationStateStore {
   public GetSubClusterInfoResponse getSubCluster(
       GetSubClusterInfoRequest request) throws YarnException {
 
-    FederationMembershipStateStoreInputValidator
-        .validateGetSubClusterInfoRequest(request);
+    FederationMembershipStateStoreInputValidator.validate(request);
     SubClusterId subClusterId = request.getSubClusterId();
     if (!membership.containsKey(subClusterId)) {
       String errMsg =
-          "Subcluster " + subClusterId.toString() + " does not exist";
-      FederationStateStoreUtils.logAndThrowStoreException(LOG,
-          FederationStateStoreErrorCode.MEMBERSHIP_SINGLE_SELECT_FAIL, errMsg);
+          "SubCluster " + subClusterId.toString() + " does not exist";
+      FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
     }
 
     return GetSubClusterInfoResponse.newInstance(membership.get(subClusterId));
@@ -195,8 +193,7 @@ public class MemoryFederationStateStore implements FederationStateStore {
   public AddApplicationHomeSubClusterResponse addApplicationHomeSubCluster(
       AddApplicationHomeSubClusterRequest request) throws YarnException {
 
-    FederationApplicationHomeSubClusterStoreInputValidator
-        .validateAddApplicationHomeSubClusterRequest(request);
+    FederationApplicationHomeSubClusterStoreInputValidator.validate(request);
     ApplicationId appId =
         request.getApplicationHomeSubCluster().getApplicationId();
 
@@ -213,14 +210,12 @@ public class MemoryFederationStateStore implements FederationStateStore {
   public UpdateApplicationHomeSubClusterResponse updateApplicationHomeSubCluster(
       UpdateApplicationHomeSubClusterRequest request) throws YarnException {
 
-    FederationApplicationHomeSubClusterStoreInputValidator
-        .validateUpdateApplicationHomeSubClusterRequest(request);
+    FederationApplicationHomeSubClusterStoreInputValidator.validate(request);
     ApplicationId appId =
         request.getApplicationHomeSubCluster().getApplicationId();
     if (!applications.containsKey(appId)) {
       String errMsg = "Application " + appId + " does not exist";
-      FederationStateStoreUtils.logAndThrowStoreException(LOG,
-          FederationStateStoreErrorCode.APPLICATIONS_UPDATE_FAIL, errMsg);
+      FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
     }
 
     applications.put(appId,
@@ -232,14 +227,11 @@ public class MemoryFederationStateStore implements FederationStateStore {
   public GetApplicationHomeSubClusterResponse getApplicationHomeSubCluster(
       GetApplicationHomeSubClusterRequest request) throws YarnException {
 
-    FederationApplicationHomeSubClusterStoreInputValidator
-        .validateGetApplicationHomeSubClusterRequest(request);
+    FederationApplicationHomeSubClusterStoreInputValidator.validate(request);
     ApplicationId appId = request.getApplicationId();
     if (!applications.containsKey(appId)) {
       String errMsg = "Application " + appId + " does not exist";
-      FederationStateStoreUtils.logAndThrowStoreException(LOG,
-          FederationStateStoreErrorCode.APPLICATIONS_SINGLE_SELECT_FAIL,
-          errMsg);
+      FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
     }
 
     return GetApplicationHomeSubClusterResponse.newInstance(
@@ -264,13 +256,11 @@ public class MemoryFederationStateStore implements FederationStateStore {
   public DeleteApplicationHomeSubClusterResponse deleteApplicationHomeSubCluster(
       DeleteApplicationHomeSubClusterRequest request) throws YarnException {
 
-    FederationApplicationHomeSubClusterStoreInputValidator
-        .validateDeleteApplicationHomeSubClusterRequest(request);
+    FederationApplicationHomeSubClusterStoreInputValidator.validate(request);
     ApplicationId appId = request.getApplicationId();
     if (!applications.containsKey(appId)) {
       String errMsg = "Application " + appId + " does not exist";
-      FederationStateStoreUtils.logAndThrowStoreException(LOG,
-          FederationStateStoreErrorCode.APPLICATIONS_DELETE_FAIL, errMsg);
+      FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
     }
 
     applications.remove(appId);
@@ -281,13 +271,11 @@ public class MemoryFederationStateStore implements FederationStateStore {
   public GetSubClusterPolicyConfigurationResponse getPolicyConfiguration(
       GetSubClusterPolicyConfigurationRequest request) throws YarnException {
 
-    FederationPolicyStoreInputValidator
-        .validateGetSubClusterPolicyConfigurationRequest(request);
+    FederationPolicyStoreInputValidator.validate(request);
     String queue = request.getQueue();
     if (!policies.containsKey(queue)) {
       String errMsg = "Policy for queue " + queue + " does not exist";
-      FederationStateStoreUtils.logAndThrowStoreException(LOG,
-          FederationStateStoreErrorCode.POLICY_SINGLE_SELECT_FAIL, errMsg);
+      FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
     }
 
     return GetSubClusterPolicyConfigurationResponse
@@ -298,8 +286,7 @@ public class MemoryFederationStateStore implements FederationStateStore {
   public SetSubClusterPolicyConfigurationResponse setPolicyConfiguration(
       SetSubClusterPolicyConfigurationRequest request) throws YarnException {
 
-    FederationPolicyStoreInputValidator
-        .validateSetSubClusterPolicyConfigurationRequest(request);
+    FederationPolicyStoreInputValidator.validate(request);
     policies.put(request.getPolicyConfiguration().getQueue(),
         request.getPolicyConfiguration());
     return SetSubClusterPolicyConfigurationResponse.newInstance();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5599b916/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
----------------------------------------------------------------------
diff --git a/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 b/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
new file mode 100644
index 0000000..a849c6a
--- /dev/null
+++ b/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
@@ -0,0 +1,937 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.impl;
+
+import java.nio.ByteBuffer;
+import java.sql.CallableStatement;
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Timestamp;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.List;
+import java.util.TimeZone;
+
+import org.apache.commons.lang.NotImplementedException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.federation.store.FederationStateStore;
+import org.apache.hadoop.yarn.server.federation.store.exception.FederationStateStoreInvalidInputException;
+import org.apache.hadoop.yarn.server.federation.store.records.AddApplicationHomeSubClusterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.AddApplicationHomeSubClusterResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.ApplicationHomeSubCluster;
+import org.apache.hadoop.yarn.server.federation.store.records.DeleteApplicationHomeSubClusterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.DeleteApplicationHomeSubClusterResponse;
+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.server.federation.store.records.GetApplicationsHomeSubClusterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.GetApplicationsHomeSubClusterResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterInfoRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterInfoResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterPoliciesConfigurationsRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterPoliciesConfigurationsResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterPolicyConfigurationRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterPolicyConfigurationResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClustersInfoRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClustersInfoResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.SetSubClusterPolicyConfigurationRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.SetSubClusterPolicyConfigurationResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterDeregisterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterDeregisterResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterHeartbeatRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterHeartbeatResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyConfiguration;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterRegisterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterRegisterResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterState;
+import org.apache.hadoop.yarn.server.federation.store.records.UpdateApplicationHomeSubClusterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.UpdateApplicationHomeSubClusterResponse;
+import org.apache.hadoop.yarn.server.federation.store.utils.FederationApplicationHomeSubClusterStoreInputValidator;
+import org.apache.hadoop.yarn.server.federation.store.utils.FederationMembershipStateStoreInputValidator;
+import org.apache.hadoop.yarn.server.federation.store.utils.FederationPolicyStoreInputValidator;
+import org.apache.hadoop.yarn.server.federation.store.utils.FederationStateStoreUtils;
+import org.apache.hadoop.yarn.server.records.Version;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.zaxxer.hikari.HikariDataSource;
+
+/**
+ * SQL implementation of {@link FederationStateStore}.
+ */
+public class SQLFederationStateStore implements FederationStateStore {
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(SQLFederationStateStore.class);
+
+  // Stored procedures patterns
+
+  private static final String CALL_SP_REGISTER_SUBCLUSTER =
+      "{call sp_registerSubCluster(?, ?, ?, ?, ?, ?, ?, ?, ?)}";
+
+  private static final String CALL_SP_DEREGISTER_SUBCLUSTER =
+      "{call sp_deregisterSubCluster(?, ?, ?)}";
+
+  private static final String CALL_SP_GET_SUBCLUSTER =
+      "{call sp_getSubCluster(?, ?, ?, ?, ?, ?, ?, ?, ?)}";
+
+  private static final String CALL_SP_GET_SUBCLUSTERS =
+      "{call sp_getSubClusters()}";
+
+  private static final String CALL_SP_SUBCLUSTER_HEARTBEAT =
+      "{call sp_subClusterHeartbeat(?, ?, ?, ?)}";
+
+  private static final String CALL_SP_ADD_APPLICATION_HOME_SUBCLUSTER =
+      "{call sp_addApplicationHomeSubCluster(?, ?, ?, ?)}";
+
+  private static final String CALL_SP_UPDATE_APPLICATION_HOME_SUBCLUSTER =
+      "{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(?, ?)}";
+
+  private static final String CALL_SP_GET_APPLICATIONS_HOME_SUBCLUSTER =
+      "{call sp_getApplicationsHomeSubCluster()}";
+
+  private static final String CALL_SP_SET_POLICY_CONFIGURATION =
+      "{call sp_setPolicyConfiguration(?, ?, ?, ?)}";
+
+  private static final String CALL_SP_GET_POLICY_CONFIGURATION =
+      "{call sp_getPolicyConfiguration(?, ?, ?)}";
+
+  private static final String CALL_SP_GET_POLICIES_CONFIGURATIONS =
+      "{call sp_getPoliciesConfigurations()}";
+
+  private Calendar utcCalendar =
+      Calendar.getInstance(TimeZone.getTimeZone("UTC"));
+
+  // SQL database configurations
+
+  private String userName;
+  private String password;
+  private String driverClass;
+  private String url;
+  private int maximumPoolSize;
+  private HikariDataSource dataSource = null;
+
+  @Override
+  public void init(Configuration conf) throws YarnException {
+    driverClass =
+        conf.get(YarnConfiguration.FEDERATION_STATESTORE_SQL_JDBC_CLASS,
+            YarnConfiguration.DEFAULT_FEDERATION_STATESTORE_SQL_JDBC_CLASS);
+    maximumPoolSize =
+        conf.getInt(YarnConfiguration.FEDERATION_STATESTORE_SQL_MAXCONNECTIONS,
+            YarnConfiguration.DEFAULT_FEDERATION_STATESTORE_SQL_MAXCONNECTIONS);
+
+    // An helper method avoids to assign a null value to these property
+    userName = conf.get(YarnConfiguration.FEDERATION_STATESTORE_SQL_USERNAME);
+    password = conf.get(YarnConfiguration.FEDERATION_STATESTORE_SQL_PASSWORD);
+    url = conf.get(YarnConfiguration.FEDERATION_STATESTORE_SQL_URL);
+
+    try {
+      Class.forName(driverClass);
+    } catch (ClassNotFoundException e) {
+      FederationStateStoreUtils.logAndThrowException(LOG,
+          "Driver class not found.", e);
+    }
+
+    // Create the data source to pool connections in a thread-safe manner
+    dataSource = new HikariDataSource();
+    dataSource.setDataSourceClassName(driverClass);
+    FederationStateStoreUtils.setUsername(dataSource, userName);
+    FederationStateStoreUtils.setPassword(dataSource, password);
+    FederationStateStoreUtils.setProperty(dataSource,
+        FederationStateStoreUtils.FEDERATION_STORE_URL, url);
+    dataSource.setMaximumPoolSize(maximumPoolSize);
+    LOG.info("Initialized connection pool to the Federation StateStore "
+        + "database at address: " + url);
+  }
+
+  @Override
+  public SubClusterRegisterResponse registerSubCluster(
+      SubClusterRegisterRequest registerSubClusterRequest)
+      throws YarnException {
+
+    // Input validator
+    FederationMembershipStateStoreInputValidator
+        .validate(registerSubClusterRequest);
+
+    CallableStatement cstmt = null;
+    Connection conn = null;
+
+    SubClusterInfo subClusterInfo =
+        registerSubClusterRequest.getSubClusterInfo();
+    SubClusterId subClusterId = subClusterInfo.getSubClusterId();
+
+    try {
+      conn = getConnection();
+      cstmt = conn.prepareCall(CALL_SP_REGISTER_SUBCLUSTER);
+
+      // Set the parameters for the stored procedure
+      cstmt.setString(1, subClusterId.getId());
+      cstmt.setString(2, subClusterInfo.getAMRMServiceAddress());
+      cstmt.setString(3, subClusterInfo.getClientRMServiceAddress());
+      cstmt.setString(4, subClusterInfo.getRMAdminServiceAddress());
+      cstmt.setString(5, subClusterInfo.getRMWebServiceAddress());
+      cstmt.setString(6, subClusterInfo.getState().toString());
+      cstmt.setLong(7, subClusterInfo.getLastStartTime());
+      cstmt.setString(8, subClusterInfo.getCapability());
+      cstmt.registerOutParameter(9, java.sql.Types.INTEGER);
+
+      // Execute the query
+      cstmt.executeUpdate();
+
+      // Check the ROWCOUNT value, if it is equal to 0 it means the call
+      // did not add a new subcluster into FederationStateStore
+      if (cstmt.getInt(9) == 0) {
+        String errMsg = "SubCluster " + subClusterId
+            + " was not registered into the StateStore";
+        FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+      }
+      // Check the ROWCOUNT value, if it is different from 1 it means the call
+      // had a wrong behavior. Maybe the database is not set correctly.
+      if (cstmt.getInt(9) != 1) {
+        String errMsg = "Wrong behavior during registration of SubCluster "
+            + subClusterId + " into the StateStore";
+        FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+      }
+
+      LOG.info(
+          "Registered the SubCluster " + subClusterId + " into the StateStore");
+
+    } catch (SQLException e) {
+      FederationStateStoreUtils.logAndThrowRetriableException(LOG,
+          "Unable to register the SubCluster " + subClusterId
+              + " into the StateStore",
+          e);
+    } finally {
+      // Return to the pool the CallableStatement and the Connection
+      FederationStateStoreUtils.returnToPool(LOG, cstmt, conn);
+    }
+    return SubClusterRegisterResponse.newInstance();
+  }
+
+  @Override
+  public SubClusterDeregisterResponse deregisterSubCluster(
+      SubClusterDeregisterRequest subClusterDeregisterRequest)
+      throws YarnException {
+
+    // Input validator
+    FederationMembershipStateStoreInputValidator
+        .validate(subClusterDeregisterRequest);
+
+    CallableStatement cstmt = null;
+    Connection conn = null;
+
+    SubClusterId subClusterId = subClusterDeregisterRequest.getSubClusterId();
+    SubClusterState state = subClusterDeregisterRequest.getState();
+
+    try {
+      conn = getConnection();
+      cstmt = conn.prepareCall(CALL_SP_DEREGISTER_SUBCLUSTER);
+
+      // Set the parameters for the stored procedure
+      cstmt.setString(1, subClusterId.getId());
+      cstmt.setString(2, state.toString());
+      cstmt.registerOutParameter(3, java.sql.Types.INTEGER);
+
+      // Execute the query
+      cstmt.executeUpdate();
+
+      // Check the ROWCOUNT value, if it is equal to 0 it means the call
+      // did not deregister the subcluster into FederationStateStore
+      if (cstmt.getInt(3) == 0) {
+        String errMsg = "SubCluster " + subClusterId + " not found";
+        FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+      }
+      // Check the ROWCOUNT value, if it is different from 1 it means the call
+      // had a wrong behavior. Maybe the database is not set correctly.
+      if (cstmt.getInt(3) != 1) {
+        String errMsg = "Wrong behavior during deregistration of SubCluster "
+            + subClusterId + " from the StateStore";
+        FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+      }
+
+      LOG.info("Deregistered the SubCluster " + subClusterId + " state to "
+          + state.toString());
+
+    } catch (SQLException e) {
+      FederationStateStoreUtils.logAndThrowRetriableException(LOG,
+          "Unable to deregister the sub-cluster " + subClusterId + " state to "
+              + state.toString(),
+          e);
+    } finally {
+      // Return to the pool the CallableStatement and the Connection
+      FederationStateStoreUtils.returnToPool(LOG, cstmt, conn);
+    }
+    return SubClusterDeregisterResponse.newInstance();
+  }
+
+  @Override
+  public SubClusterHeartbeatResponse subClusterHeartbeat(
+      SubClusterHeartbeatRequest subClusterHeartbeatRequest)
+      throws YarnException {
+
+    // Input validator
+    FederationMembershipStateStoreInputValidator
+        .validate(subClusterHeartbeatRequest);
+
+    CallableStatement cstmt = null;
+    Connection conn = null;
+
+    SubClusterId subClusterId = subClusterHeartbeatRequest.getSubClusterId();
+    SubClusterState state = subClusterHeartbeatRequest.getState();
+
+    try {
+      conn = getConnection();
+      cstmt = conn.prepareCall(CALL_SP_SUBCLUSTER_HEARTBEAT);
+
+      // Set the parameters for the stored procedure
+      cstmt.setString(1, subClusterId.getId());
+      cstmt.setString(2, state.toString());
+      cstmt.setString(3, subClusterHeartbeatRequest.getCapability());
+      cstmt.registerOutParameter(4, java.sql.Types.INTEGER);
+
+      // Execute the query
+      cstmt.executeUpdate();
+
+      // Check the ROWCOUNT value, if it is equal to 0 it means the call
+      // did not update the subcluster into FederationStateStore
+      if (cstmt.getInt(4) == 0) {
+        String errMsg = "SubCluster " + subClusterId.toString()
+            + " does not exist; cannot heartbeat";
+        FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+      }
+      // Check the ROWCOUNT value, if it is different from 1 it means the call
+      // had a wrong behavior. Maybe the database is not set correctly.
+      if (cstmt.getInt(4) != 1) {
+        String errMsg =
+            "Wrong behavior during the heartbeat of SubCluster " + subClusterId;
+        FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+      }
+
+      LOG.info("Heartbeated the StateStore for the specified SubCluster "
+          + subClusterId);
+
+    } catch (SQLException e) {
+      FederationStateStoreUtils.logAndThrowRetriableException(LOG,
+          "Unable to heartbeat the StateStore for the specified SubCluster "
+              + subClusterId,
+          e);
+    } finally {
+      // Return to the pool the CallableStatement and the Connection
+      FederationStateStoreUtils.returnToPool(LOG, cstmt, conn);
+    }
+    return SubClusterHeartbeatResponse.newInstance();
+  }
+
+  @Override
+  public GetSubClusterInfoResponse getSubCluster(
+      GetSubClusterInfoRequest subClusterRequest) throws YarnException {
+
+    // Input validator
+    FederationMembershipStateStoreInputValidator.validate(subClusterRequest);
+
+    CallableStatement cstmt = null;
+    Connection conn = null;
+
+    SubClusterInfo subClusterInfo = null;
+    SubClusterId subClusterId = subClusterRequest.getSubClusterId();
+
+    try {
+      conn = getConnection();
+      cstmt = conn.prepareCall(CALL_SP_GET_SUBCLUSTER);
+      cstmt.setString(1, subClusterId.getId());
+
+      // Set the parameters for the stored procedure
+      cstmt.registerOutParameter(2, java.sql.Types.VARCHAR);
+      cstmt.registerOutParameter(3, java.sql.Types.VARCHAR);
+      cstmt.registerOutParameter(4, java.sql.Types.VARCHAR);
+      cstmt.registerOutParameter(5, java.sql.Types.VARCHAR);
+      cstmt.registerOutParameter(6, java.sql.Types.TIMESTAMP);
+      cstmt.registerOutParameter(7, java.sql.Types.VARCHAR);
+      cstmt.registerOutParameter(8, java.sql.Types.BIGINT);
+      cstmt.registerOutParameter(9, java.sql.Types.VARCHAR);
+
+      // Execute the query
+      cstmt.execute();
+
+      String amRMAddress = cstmt.getString(2);
+      String clientRMAddress = cstmt.getString(3);
+      String rmAdminAddress = cstmt.getString(4);
+      String webAppAddress = cstmt.getString(5);
+
+      Timestamp heartBeatTimeStamp = cstmt.getTimestamp(6, utcCalendar);
+      long lastHeartBeat =
+          heartBeatTimeStamp != null ? heartBeatTimeStamp.getTime() : 0;
+
+      SubClusterState state = SubClusterState.fromString(cstmt.getString(7));
+      long lastStartTime = cstmt.getLong(8);
+      String capability = cstmt.getString(9);
+
+      subClusterInfo = SubClusterInfo.newInstance(subClusterId, amRMAddress,
+          clientRMAddress, rmAdminAddress, webAppAddress, lastHeartBeat, state,
+          lastStartTime, capability);
+
+      // Check if the output it is a valid subcluster
+      try {
+        FederationMembershipStateStoreInputValidator
+            .checkSubClusterInfo(subClusterInfo);
+      } catch (FederationStateStoreInvalidInputException e) {
+        String errMsg =
+            "SubCluster " + subClusterId.toString() + " does not exist";
+        FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+      }
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Got the information about the specified SubCluster "
+            + subClusterInfo.toString());
+      }
+    } catch (SQLException e) {
+      FederationStateStoreUtils.logAndThrowRetriableException(LOG,
+          "Unable to obtain the SubCluster information for " + subClusterId, e);
+    } finally {
+      // Return to the pool the CallableStatement and the Connection
+      FederationStateStoreUtils.returnToPool(LOG, cstmt, conn);
+    }
+    return GetSubClusterInfoResponse.newInstance(subClusterInfo);
+  }
+
+  @Override
+  public GetSubClustersInfoResponse getSubClusters(
+      GetSubClustersInfoRequest subClustersRequest) throws YarnException {
+    CallableStatement cstmt = null;
+    Connection conn = null;
+    ResultSet rs = null;
+    List<SubClusterInfo> subClusters = new ArrayList<SubClusterInfo>();
+
+    try {
+      conn = getConnection();
+      cstmt = conn.prepareCall(CALL_SP_GET_SUBCLUSTERS);
+
+      // Execute the query
+      rs = cstmt.executeQuery();
+
+      while (rs.next()) {
+
+        // Extract the output for each tuple
+        String subClusterName = rs.getString(1);
+        String amRMAddress = rs.getString(2);
+        String clientRMAddress = rs.getString(3);
+        String rmAdminAddress = rs.getString(4);
+        String webAppAddress = rs.getString(5);
+        long lastHeartBeat = rs.getTimestamp(6, utcCalendar).getTime();
+        SubClusterState state = SubClusterState.fromString(rs.getString(7));
+        long lastStartTime = rs.getLong(8);
+        String capability = rs.getString(9);
+
+        SubClusterId subClusterId = SubClusterId.newInstance(subClusterName);
+        SubClusterInfo subClusterInfo = SubClusterInfo.newInstance(subClusterId,
+            amRMAddress, clientRMAddress, rmAdminAddress, webAppAddress,
+            lastHeartBeat, state, lastStartTime, capability);
+
+        // Check if the output it is a valid subcluster
+        try {
+          FederationMembershipStateStoreInputValidator
+              .checkSubClusterInfo(subClusterInfo);
+        } catch (FederationStateStoreInvalidInputException e) {
+          String errMsg =
+              "SubCluster " + subClusterId.toString() + " is not valid";
+          FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+        }
+
+        // Filter the inactive
+        if (!subClustersRequest.getFilterInactiveSubClusters()
+            || subClusterInfo.getState().isActive()) {
+          subClusters.add(subClusterInfo);
+        }
+      }
+
+    } catch (SQLException e) {
+      FederationStateStoreUtils.logAndThrowRetriableException(LOG,
+          "Unable to obtain the information for all the SubClusters ", e);
+    } finally {
+      // Return to the pool the CallableStatement and the Connection
+      FederationStateStoreUtils.returnToPool(LOG, cstmt, conn, rs);
+    }
+    return GetSubClustersInfoResponse.newInstance(subClusters);
+  }
+
+  @Override
+  public AddApplicationHomeSubClusterResponse addApplicationHomeSubCluster(
+      AddApplicationHomeSubClusterRequest request) throws YarnException {
+
+    // Input validator
+    FederationApplicationHomeSubClusterStoreInputValidator.validate(request);
+
+    CallableStatement cstmt = null;
+    Connection conn = null;
+
+    String subClusterHome = null;
+    ApplicationId appId =
+        request.getApplicationHomeSubCluster().getApplicationId();
+    SubClusterId subClusterId =
+        request.getApplicationHomeSubCluster().getHomeSubCluster();
+
+    try {
+      conn = getConnection();
+      cstmt = conn.prepareCall(CALL_SP_ADD_APPLICATION_HOME_SUBCLUSTER);
+
+      // Set the parameters for the stored procedure
+      cstmt.setString(1, appId.toString());
+      cstmt.setString(2, subClusterId.getId());
+      cstmt.registerOutParameter(3, java.sql.Types.VARCHAR);
+      cstmt.registerOutParameter(4, java.sql.Types.INTEGER);
+
+      // Execute the query
+      cstmt.executeUpdate();
+
+      subClusterHome = cstmt.getString(3);
+      SubClusterId subClusterIdHome = SubClusterId.newInstance(subClusterHome);
+
+      // For failover reason, we check the returned SubClusterId.
+      // If it is equal to the subclusterId we sent, the call added the new
+      // application into FederationStateStore. If the call returns a different
+      // SubClusterId it means we already tried to insert this application but a
+      // component (Router/StateStore/RM) failed during the submission.
+      if (subClusterId.equals(subClusterIdHome)) {
+        // Check the ROWCOUNT value, if it is equal to 0 it means the call
+        // did not add a new application into FederationStateStore
+        if (cstmt.getInt(4) == 0) {
+          String errMsg = "The application " + appId
+              + " was not insert into the StateStore";
+          FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+        }
+        // Check the ROWCOUNT value, if it is different from 1 it means the call
+        // had a wrong behavior. Maybe the database is not set correctly.
+        if (cstmt.getInt(4) != 1) {
+          String errMsg = "Wrong behavior during the insertion of SubCluster "
+              + subClusterId;
+          FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+        }
+
+        LOG.info("Insert into the StateStore the application: " + appId
+            + " in SubCluster:  " + subClusterHome);
+      } else {
+        // Check the ROWCOUNT value, if it is different from 0 it means the call
+        // did edited the table
+        if (cstmt.getInt(4) != 0) {
+          String errMsg =
+              "The application " + appId + " does exist but was overwritten";
+          FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+        }
+        LOG.info("Application: " + appId + " already present with SubCluster:  "
+            + subClusterHome);
+      }
+
+    } catch (SQLException e) {
+      FederationStateStoreUtils
+          .logAndThrowRetriableException(LOG,
+              "Unable to insert the newly generated application "
+                  + request.getApplicationHomeSubCluster().getApplicationId(),
+              e);
+    } finally {
+      // Return to the pool the CallableStatement and the Connection
+      FederationStateStoreUtils.returnToPool(LOG, cstmt, conn);
+    }
+    return AddApplicationHomeSubClusterResponse
+        .newInstance(SubClusterId.newInstance(subClusterHome));
+  }
+
+  @Override
+  public UpdateApplicationHomeSubClusterResponse updateApplicationHomeSubCluster(
+      UpdateApplicationHomeSubClusterRequest request) throws YarnException {
+
+    // Input validator
+    FederationApplicationHomeSubClusterStoreInputValidator.validate(request);
+
+    CallableStatement cstmt = null;
+    Connection conn = null;
+
+    ApplicationId appId =
+        request.getApplicationHomeSubCluster().getApplicationId();
+    SubClusterId subClusterId =
+        request.getApplicationHomeSubCluster().getHomeSubCluster();
+
+    try {
+      conn = getConnection();
+      cstmt = conn.prepareCall(CALL_SP_UPDATE_APPLICATION_HOME_SUBCLUSTER);
+
+      // Set the parameters for the stored procedure
+      cstmt.setString(1, appId.toString());
+      cstmt.setString(2, subClusterId.getId());
+      cstmt.registerOutParameter(3, java.sql.Types.INTEGER);
+
+      // Execute the query
+      cstmt.executeUpdate();
+
+      // Check the ROWCOUNT value, if it is equal to 0 it means the call
+      // did not update the application into FederationStateStore
+      if (cstmt.getInt(3) == 0) {
+        String errMsg = "Application " + appId + " does not exist";
+        FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+      }
+      // Check the ROWCOUNT value, if it is different from 1 it means the call
+      // had a wrong behavior. Maybe the database is not set correctly.
+      if (cstmt.getInt(3) != 1) {
+        String errMsg =
+            "Wrong behavior during the update of SubCluster " + subClusterId;
+        FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+      }
+
+      LOG.info(
+          "Update the SubCluster to {} for application {} in the StateStore",
+          subClusterId, appId);
+
+    } catch (SQLException e) {
+      FederationStateStoreUtils
+          .logAndThrowRetriableException(LOG,
+              "Unable to update the application "
+                  + request.getApplicationHomeSubCluster().getApplicationId(),
+              e);
+    } finally {
+      // Return to the pool the CallableStatement and the Connection
+      FederationStateStoreUtils.returnToPool(LOG, cstmt, conn);
+    }
+    return UpdateApplicationHomeSubClusterResponse.newInstance();
+  }
+
+  @Override
+  public GetApplicationHomeSubClusterResponse getApplicationHomeSubCluster(
+      GetApplicationHomeSubClusterRequest request) throws YarnException {
+    // Input validator
+    FederationApplicationHomeSubClusterStoreInputValidator.validate(request);
+
+    CallableStatement cstmt = null;
+    Connection conn = null;
+
+    SubClusterId homeRM = null;
+
+    try {
+      conn = getConnection();
+      cstmt = conn.prepareCall(CALL_SP_GET_APPLICATION_HOME_SUBCLUSTER);
+
+      // Set the parameters for the stored procedure
+      cstmt.setString(1, request.getApplicationId().toString());
+      cstmt.registerOutParameter(2, java.sql.Types.VARCHAR);
+
+      // Execute the query
+      cstmt.execute();
+
+      if (cstmt.getString(2) != null) {
+        homeRM = SubClusterId.newInstance(cstmt.getString(2));
+      } else {
+        String errMsg =
+            "Application " + request.getApplicationId() + " does not exist";
+        FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+      }
+
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Got the information about the specified application  "
+            + request.getApplicationId() + ". The AM is running in " + homeRM);
+      }
+    } catch (SQLException e) {
+      FederationStateStoreUtils.logAndThrowRetriableException(LOG,
+          "Unable to obtain the application information "
+              + "for the specified application " + request.getApplicationId(),
+          e);
+    } finally {
+
+      // Return to the pool the CallableStatement and the Connection
+      FederationStateStoreUtils.returnToPool(LOG, cstmt, conn);
+    }
+    return GetApplicationHomeSubClusterResponse
+        .newInstance(ApplicationHomeSubCluster
+            .newInstance(request.getApplicationId(), homeRM));
+  }
+
+  @Override
+  public GetApplicationsHomeSubClusterResponse getApplicationsHomeSubCluster(
+      GetApplicationsHomeSubClusterRequest request) throws YarnException {
+    CallableStatement cstmt = null;
+    Connection conn = null;
+    ResultSet rs = null;
+    List<ApplicationHomeSubCluster> appsHomeSubClusters =
+        new ArrayList<ApplicationHomeSubCluster>();
+
+    try {
+      conn = getConnection();
+      cstmt = conn.prepareCall(CALL_SP_GET_APPLICATIONS_HOME_SUBCLUSTER);
+
+      // Execute the query
+      rs = cstmt.executeQuery();
+
+      while (rs.next()) {
+
+        // Extract the output for each tuple
+        String applicationId = rs.getString(1);
+        String homeSubCluster = rs.getString(2);
+
+        appsHomeSubClusters.add(ApplicationHomeSubCluster.newInstance(
+            ApplicationId.fromString(applicationId),
+            SubClusterId.newInstance(homeSubCluster)));
+      }
+
+    } catch (SQLException e) {
+      FederationStateStoreUtils.logAndThrowRetriableException(LOG,
+          "Unable to obtain the information for all the applications ", e);
+    } finally {
+      // Return to the pool the CallableStatement and the Connection
+      FederationStateStoreUtils.returnToPool(LOG, cstmt, conn, rs);
+    }
+    return GetApplicationsHomeSubClusterResponse
+        .newInstance(appsHomeSubClusters);
+  }
+
+  @Override
+  public DeleteApplicationHomeSubClusterResponse deleteApplicationHomeSubCluster(
+      DeleteApplicationHomeSubClusterRequest request) throws YarnException {
+
+    // Input validator
+    FederationApplicationHomeSubClusterStoreInputValidator.validate(request);
+
+    CallableStatement cstmt = null;
+    Connection conn = null;
+
+    try {
+      conn = getConnection();
+      cstmt = conn.prepareCall(CALL_SP_DELETE_APPLICATION_HOME_SUBCLUSTER);
+
+      // Set the parameters for the stored procedure
+      cstmt.setString(1, request.getApplicationId().toString());
+      cstmt.registerOutParameter(2, java.sql.Types.INTEGER);
+
+      // Execute the query
+      cstmt.executeUpdate();
+
+      // Check the ROWCOUNT value, if it is equal to 0 it means the call
+      // did not delete the application from FederationStateStore
+      if (cstmt.getInt(2) == 0) {
+        String errMsg =
+            "Application " + request.getApplicationId() + " does not exist";
+        FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+      }
+      // Check the ROWCOUNT value, if it is different from 1 it means the call
+      // had a wrong behavior. Maybe the database is not set correctly.
+      if (cstmt.getInt(2) != 1) {
+        String errMsg = "Wrong behavior during deleting the application "
+            + request.getApplicationId();
+        FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+      }
+
+      LOG.info("Delete from the StateStore the application: {}",
+          request.getApplicationId());
+
+    } catch (SQLException e) {
+      FederationStateStoreUtils.logAndThrowRetriableException(LOG,
+          "Unable to delete the application " + request.getApplicationId(), e);
+    } finally {
+      // Return to the pool the CallableStatement and the Connection
+      FederationStateStoreUtils.returnToPool(LOG, cstmt, conn);
+    }
+    return DeleteApplicationHomeSubClusterResponse.newInstance();
+  }
+
+  @Override
+  public GetSubClusterPolicyConfigurationResponse getPolicyConfiguration(
+      GetSubClusterPolicyConfigurationRequest request) throws YarnException {
+
+    // Input validator
+    FederationPolicyStoreInputValidator.validate(request);
+
+    CallableStatement cstmt = null;
+    Connection conn = null;
+    SubClusterPolicyConfiguration subClusterPolicyConfiguration = null;
+
+    try {
+      conn = getConnection();
+      cstmt = conn.prepareCall(CALL_SP_GET_POLICY_CONFIGURATION);
+
+      // Set the parameters for the stored procedure
+      cstmt.setString(1, request.getQueue());
+      cstmt.registerOutParameter(2, java.sql.Types.VARCHAR);
+      cstmt.registerOutParameter(3, java.sql.Types.VARBINARY);
+
+      // Execute the query
+      cstmt.executeUpdate();
+
+      // Check if the output it is a valid policy
+      if (cstmt.getString(2) != null && cstmt.getBytes(3) != null) {
+        subClusterPolicyConfiguration =
+            SubClusterPolicyConfiguration.newInstance(request.getQueue(),
+                cstmt.getString(2), ByteBuffer.wrap(cstmt.getBytes(3)));
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Selected from StateStore the policy for the queue: "
+              + subClusterPolicyConfiguration.toString());
+        }
+      } else {
+        String errMsg =
+            "Policy for queue " + request.getQueue() + " does not exist";
+        FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+      }
+
+    } catch (SQLException e) {
+      FederationStateStoreUtils.logAndThrowRetriableException(LOG,
+          "Unable to select the policy for the queue :" + request.getQueue(),
+          e);
+    } finally {
+      // Return to the pool the CallableStatement and the Connection
+      FederationStateStoreUtils.returnToPool(LOG, cstmt, conn);
+    }
+    return GetSubClusterPolicyConfigurationResponse
+        .newInstance(subClusterPolicyConfiguration);
+  }
+
+  @Override
+  public SetSubClusterPolicyConfigurationResponse setPolicyConfiguration(
+      SetSubClusterPolicyConfigurationRequest request) throws YarnException {
+
+    // Input validator
+    FederationPolicyStoreInputValidator.validate(request);
+
+    CallableStatement cstmt = null;
+    Connection conn = null;
+
+    SubClusterPolicyConfiguration policyConf = request.getPolicyConfiguration();
+
+    try {
+      conn = getConnection();
+      cstmt = conn.prepareCall(CALL_SP_SET_POLICY_CONFIGURATION);
+
+      // Set the parameters for the stored procedure
+      cstmt.setString(1, policyConf.getQueue());
+      cstmt.setString(2, policyConf.getType());
+      cstmt.setBytes(3, getByteArray(policyConf.getParams()));
+      cstmt.registerOutParameter(4, java.sql.Types.INTEGER);
+
+      // Execute the query
+      cstmt.executeUpdate();
+
+      // Check the ROWCOUNT value, if it is equal to 0 it means the call
+      // did not add a new policy into FederationStateStore
+      if (cstmt.getInt(4) == 0) {
+        String errMsg = "The policy " + policyConf.getQueue()
+            + " was not insert into the StateStore";
+        FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+      }
+      // Check the ROWCOUNT value, if it is different from 1 it means the call
+      // had a wrong behavior. Maybe the database is not set correctly.
+      if (cstmt.getInt(4) != 1) {
+        String errMsg =
+            "Wrong behavior during insert the policy " + policyConf.getQueue();
+        FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+      }
+
+      LOG.info("Insert into the state store the policy for the queue: "
+          + policyConf.getQueue());
+
+    } catch (SQLException e) {
+      FederationStateStoreUtils.logAndThrowRetriableException(LOG,
+          "Unable to insert the newly generated policy for the queue :"
+              + policyConf.getQueue(),
+          e);
+    } finally {
+      // Return to the pool the CallableStatement and the Connection
+      FederationStateStoreUtils.returnToPool(LOG, cstmt, conn);
+    }
+    return SetSubClusterPolicyConfigurationResponse.newInstance();
+  }
+
+  @Override
+  public GetSubClusterPoliciesConfigurationsResponse getPoliciesConfigurations(
+      GetSubClusterPoliciesConfigurationsRequest request) throws YarnException {
+
+    CallableStatement cstmt = null;
+    Connection conn = null;
+    ResultSet rs = null;
+    List<SubClusterPolicyConfiguration> policyConfigurations =
+        new ArrayList<SubClusterPolicyConfiguration>();
+
+    try {
+      conn = getConnection();
+      cstmt = conn.prepareCall(CALL_SP_GET_POLICIES_CONFIGURATIONS);
+
+      // Execute the query
+      rs = cstmt.executeQuery();
+
+      while (rs.next()) {
+
+        // Extract the output for each tuple
+        String queue = rs.getString(1);
+        String type = rs.getString(2);
+        byte[] policyInfo = rs.getBytes(3);
+
+        SubClusterPolicyConfiguration subClusterPolicyConfiguration =
+            SubClusterPolicyConfiguration.newInstance(queue, type,
+                ByteBuffer.wrap(policyInfo));
+        policyConfigurations.add(subClusterPolicyConfiguration);
+      }
+    } catch (SQLException e) {
+      FederationStateStoreUtils.logAndThrowRetriableException(LOG,
+          "Unable to obtain the policy information for all the queues.", e);
+    } finally {
+      // Return to the pool the CallableStatement and the Connection
+      FederationStateStoreUtils.returnToPool(LOG, cstmt, conn, rs);
+    }
+
+    return GetSubClusterPoliciesConfigurationsResponse
+        .newInstance(policyConfigurations);
+  }
+
+  @Override
+  public Version getCurrentVersion() {
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public Version loadVersion() {
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public void close() throws Exception {
+    if (dataSource != null) {
+      dataSource.close();
+    }
+  }
+
+  /**
+   * Get a connection from the DataSource pool.
+   *
+   * @return a connection from the DataSource pool.
+   * @throws SQLException on failure
+   */
+  public Connection getConnection() throws SQLException {
+    return dataSource.getConnection();
+  }
+
+  private static byte[] getByteArray(ByteBuffer bb) {
+    byte[] ba = new byte[bb.limit()];
+    bb.get(ba);
+    return ba;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5599b916/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterState.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterState.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterState.java
index ff49aaa..b30bd32 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterState.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterState.java
@@ -19,6 +19,8 @@ package org.apache.hadoop.yarn.server.federation.store.records;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * <p>
@@ -61,4 +63,23 @@ public enum SubClusterState {
     return (this == SC_UNREGISTERED || this == SC_DECOMMISSIONED
         || this == SC_LOST);
   }
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(SubClusterState.class);
+
+  /**
+   * Convert a string into {@code SubClusterState}.
+   *
+   * @param x the string to convert in SubClusterState
+   * @return the respective {@code SubClusterState}
+   */
+  public static SubClusterState fromString(String x) {
+    try {
+      return SubClusterState.valueOf(x);
+    } catch (Exception e) {
+      LOG.error("Invalid SubCluster State value in the StateStore does not"
+          + " match with the YARN Federation standard.");
+      return null;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5599b916/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationApplicationHomeSubClusterStoreInputValidator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationApplicationHomeSubClusterStoreInputValidator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationApplicationHomeSubClusterStoreInputValidator.java
index d920144..0184c9f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationApplicationHomeSubClusterStoreInputValidator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationApplicationHomeSubClusterStoreInputValidator.java
@@ -51,8 +51,7 @@ public final class FederationApplicationHomeSubClusterStoreInputValidator {
    *          against
    * @throws FederationStateStoreInvalidInputException if the request is invalid
    */
-  public static void validateAddApplicationHomeSubClusterRequest(
-      AddApplicationHomeSubClusterRequest request)
+  public static void validate(AddApplicationHomeSubClusterRequest request)
       throws FederationStateStoreInvalidInputException {
     if (request == null) {
       String message = "Missing AddApplicationHomeSubCluster Request."
@@ -75,8 +74,7 @@ public final class FederationApplicationHomeSubClusterStoreInputValidator {
    *          validate against
    * @throws FederationStateStoreInvalidInputException if the request is invalid
    */
-  public static void validateUpdateApplicationHomeSubClusterRequest(
-      UpdateApplicationHomeSubClusterRequest request)
+  public static void validate(UpdateApplicationHomeSubClusterRequest request)
       throws FederationStateStoreInvalidInputException {
     if (request == null) {
       String message = "Missing UpdateApplicationHomeSubCluster Request."
@@ -99,8 +97,7 @@ public final class FederationApplicationHomeSubClusterStoreInputValidator {
    *          against
    * @throws FederationStateStoreInvalidInputException if the request is invalid
    */
-  public static void validateGetApplicationHomeSubClusterRequest(
-      GetApplicationHomeSubClusterRequest request)
+  public static void validate(GetApplicationHomeSubClusterRequest request)
       throws FederationStateStoreInvalidInputException {
     if (request == null) {
       String message = "Missing GetApplicationHomeSubCluster Request."
@@ -122,8 +119,7 @@ public final class FederationApplicationHomeSubClusterStoreInputValidator {
    *          validate against
    * @throws FederationStateStoreInvalidInputException if the request is invalid
    */
-  public static void validateDeleteApplicationHomeSubClusterRequest(
-      DeleteApplicationHomeSubClusterRequest request)
+  public static void validate(DeleteApplicationHomeSubClusterRequest request)
       throws FederationStateStoreInvalidInputException {
     if (request == null) {
       String message = "Missing DeleteApplicationHomeSubCluster Request."

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5599b916/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationMembershipStateStoreInputValidator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationMembershipStateStoreInputValidator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationMembershipStateStoreInputValidator.java
index ebe622b..0ec8e5d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationMembershipStateStoreInputValidator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationMembershipStateStoreInputValidator.java
@@ -53,8 +53,7 @@ public final class FederationMembershipStateStoreInputValidator {
    * @param request the {@link SubClusterRegisterRequest} to validate against
    * @throws FederationStateStoreInvalidInputException if the request is invalid
    */
-  public static void validateSubClusterRegisterRequest(
-      SubClusterRegisterRequest request)
+  public static void validate(SubClusterRegisterRequest request)
       throws FederationStateStoreInvalidInputException {
 
     // check if the request is present
@@ -79,8 +78,7 @@ public final class FederationMembershipStateStoreInputValidator {
    * @param request the {@link SubClusterDeregisterRequest} to validate against
    * @throws FederationStateStoreInvalidInputException if the request is invalid
    */
-  public static void validateSubClusterDeregisterRequest(
-      SubClusterDeregisterRequest request)
+  public static void validate(SubClusterDeregisterRequest request)
       throws FederationStateStoreInvalidInputException {
 
     // check if the request is present
@@ -111,8 +109,7 @@ public final class FederationMembershipStateStoreInputValidator {
    * @param request the {@link SubClusterHeartbeatRequest} to validate against
    * @throws FederationStateStoreInvalidInputException if the request is invalid
    */
-  public static void validateSubClusterHeartbeatRequest(
-      SubClusterHeartbeatRequest request)
+  public static void validate(SubClusterHeartbeatRequest request)
       throws FederationStateStoreInvalidInputException {
 
     // check if the request is present
@@ -143,8 +140,7 @@ public final class FederationMembershipStateStoreInputValidator {
    * @param request the {@link GetSubClusterInfoRequest} to validate against
    * @throws FederationStateStoreInvalidInputException if the request is invalid
    */
-  public static void validateGetSubClusterInfoRequest(
-      GetSubClusterInfoRequest request)
+  public static void validate(GetSubClusterInfoRequest request)
       throws FederationStateStoreInvalidInputException {
 
     // check if the request is present
@@ -169,7 +165,7 @@ public final class FederationMembershipStateStoreInputValidator {
    * @throws FederationStateStoreInvalidInputException if the SubCluster Info
    *           are invalid
    */
-  private static void checkSubClusterInfo(SubClusterInfo subClusterInfo)
+  public static void checkSubClusterInfo(SubClusterInfo subClusterInfo)
       throws FederationStateStoreInvalidInputException {
     if (subClusterInfo == null) {
       String message = "Missing SubCluster Information."

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5599b916/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationPolicyStoreInputValidator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationPolicyStoreInputValidator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationPolicyStoreInputValidator.java
index 0df2d85..3c68bfd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationPolicyStoreInputValidator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationPolicyStoreInputValidator.java
@@ -48,8 +48,7 @@ public final class FederationPolicyStoreInputValidator {
    *          validate against
    * @throws FederationStateStoreInvalidInputException if the request is invalid
    */
-  public static void validateGetSubClusterPolicyConfigurationRequest(
-      GetSubClusterPolicyConfigurationRequest request)
+  public static void validate(GetSubClusterPolicyConfigurationRequest request)
       throws FederationStateStoreInvalidInputException {
     if (request == null) {
       String message = "Missing GetSubClusterPolicyConfiguration Request."
@@ -72,8 +71,7 @@ public final class FederationPolicyStoreInputValidator {
    *          validate against
    * @throws FederationStateStoreInvalidInputException if the request is invalid
    */
-  public static void validateSetSubClusterPolicyConfigurationRequest(
-      SetSubClusterPolicyConfigurationRequest request)
+  public static void validate(SetSubClusterPolicyConfigurationRequest request)
       throws FederationStateStoreInvalidInputException {
     if (request == null) {
       String message = "Missing SetSubClusterPolicyConfiguration Request."

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5599b916/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationStateStoreUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationStateStoreUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationStateStoreUtils.java
index 7dbb20a..3b870de 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationStateStoreUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationStateStoreUtils.java
@@ -20,16 +20,18 @@ package org.apache.hadoop.yarn.server.federation.store.utils;
 
 import java.sql.CallableStatement;
 import java.sql.Connection;
+import java.sql.ResultSet;
 import java.sql.SQLException;
 
 import org.apache.hadoop.yarn.exceptions.YarnException;
-import org.apache.hadoop.yarn.server.federation.store.exception.FederationStateStoreErrorCode;
 import org.apache.hadoop.yarn.server.federation.store.exception.FederationStateStoreException;
 import org.apache.hadoop.yarn.server.federation.store.exception.FederationStateStoreInvalidInputException;
 import org.apache.hadoop.yarn.server.federation.store.exception.FederationStateStoreRetriableException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.zaxxer.hikari.HikariDataSource;
+
 /**
  * Common utility methods used by the store implementations.
  *
@@ -39,19 +41,22 @@ public final class FederationStateStoreUtils {
   public static final Logger LOG =
       LoggerFactory.getLogger(FederationStateStoreUtils.class);
 
+  public final static String FEDERATION_STORE_URL = "url";
+
   private FederationStateStoreUtils() {
   }
 
   /**
-   * Returns the SQL <code>FederationStateStore</code> connection to the pool.
+   * Returns the SQL <code>FederationStateStore</code> connections to the pool.
    *
    * @param log the logger interface
    * @param cstmt the interface used to execute SQL stored procedures
    * @param conn the SQL connection
+   * @param rs the ResultSet interface used to execute SQL stored procedures
    * @throws YarnException on failure
    */
   public static void returnToPool(Logger log, CallableStatement cstmt,
-      Connection conn) throws YarnException {
+      Connection conn, ResultSet rs) throws YarnException {
     if (cstmt != null) {
       try {
         cstmt.close();
@@ -69,6 +74,28 @@ public final class FederationStateStoreUtils {
             e);
       }
     }
+
+    if (rs != null) {
+      try {
+        rs.close();
+      } catch (SQLException e) {
+        logAndThrowException(log, "Exception while trying to close ResultSet",
+            e);
+      }
+    }
+  }
+
+  /**
+   * Returns the SQL <code>FederationStateStore</code> connections to the pool.
+   *
+   * @param log the logger interface
+   * @param cstmt the interface used to execute SQL stored procedures
+   * @param conn the SQL connection
+   * @throws YarnException on failure
+   */
+  public static void returnToPool(Logger log, CallableStatement cstmt,
+      Connection conn) throws YarnException {
+    returnToPool(log, cstmt, conn, null);
   }
 
   /**
@@ -95,28 +122,13 @@ public final class FederationStateStoreUtils {
    * <code>FederationStateStore</code>.
    *
    * @param log the logger interface
-   * @param code FederationStateStoreErrorCode of the error
    * @param errMsg the error message
    * @throws YarnException on failure
    */
-  public static void logAndThrowStoreException(Logger log,
-      FederationStateStoreErrorCode code, String errMsg) throws YarnException {
-    log.error(errMsg + " " + code.toString());
-    throw new FederationStateStoreException(code);
-  }
-
-  /**
-   * Throws an <code>FederationStateStoreException</code> due to an error in
-   * <code>FederationStateStore</code>.
-   *
-   * @param log the logger interface
-   * @param code FederationStateStoreErrorCode of the error
-   * @throws YarnException on failure
-   */
-  public static void logAndThrowStoreException(Logger log,
-      FederationStateStoreErrorCode code) throws YarnException {
-    log.error(code.toString());
-    throw new FederationStateStoreException(code);
+  public static void logAndThrowStoreException(Logger log, String errMsg)
+      throws YarnException {
+    log.error(errMsg);
+    throw new FederationStateStoreException(errMsg);
   }
 
   /**
@@ -129,7 +141,7 @@ public final class FederationStateStoreUtils {
    */
   public static void logAndThrowInvalidInputException(Logger log, String errMsg)
       throws YarnException {
-    LOG.error(errMsg);
+    log.error(errMsg);
     throw new FederationStateStoreInvalidInputException(errMsg);
   }
 
@@ -145,11 +157,58 @@ public final class FederationStateStoreUtils {
   public static void logAndThrowRetriableException(Logger log, String errMsg,
       Throwable t) throws YarnException {
     if (t != null) {
-      LOG.error(errMsg, t);
+      log.error(errMsg, t);
       throw new FederationStateStoreRetriableException(errMsg, t);
     } else {
-      LOG.error(errMsg);
+      log.error(errMsg);
       throw new FederationStateStoreRetriableException(errMsg);
     }
   }
+
+  /**
+   * Sets a specific value for a specific property of
+   * <code>HikariDataSource</code> SQL connections.
+   *
+   * @param dataSource the <code>HikariDataSource</code> connections
+   * @param property the property to set
+   * @param value the value to set
+   */
+  public static void setProperty(HikariDataSource dataSource, String property,
+      String value) {
+    LOG.debug("Setting property {} with value {}", property, value);
+    if (property != null && !property.isEmpty() && value != null) {
+      dataSource.addDataSourceProperty(property, value);
+    }
+  }
+
+  /**
+   * Sets a specific username for <code>HikariDataSource</code> SQL connections.
+   *
+   * @param dataSource the <code>HikariDataSource</code> connections
+   * @param userNameDB the value to set
+   */
+  public static void setUsername(HikariDataSource dataSource,
+      String userNameDB) {
+    if (userNameDB != null) {
+      dataSource.setUsername(userNameDB);
+      LOG.debug("Setting non NULL Username for Store connection");
+    } else {
+      LOG.debug("NULL Username specified for Store connection, so ignoring");
+    }
+  }
+
+  /**
+   * Sets a specific password for <code>HikariDataSource</code> SQL connections.
+   *
+   * @param dataSource the <code>HikariDataSource</code> connections
+   * @param password the value to set
+   */
+  public static void setPassword(HikariDataSource dataSource, String password) {
+    if (password != null) {
+      dataSource.setPassword(password);
+      LOG.debug("Setting non NULL Credentials for Store connection");
+    } else {
+      LOG.debug("NULL Credentials specified for Store connection, so ignoring");
+    }
+  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[06/42] hadoop git commit: YARN-5325. Stateless ARMRMProxy policies implementation. (Carlo Curino via Subru).

Posted by su...@apache.org.
YARN-5325. Stateless ARMRMProxy policies implementation. (Carlo Curino via Subru).


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/53ca351b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/53ca351b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/53ca351b

Branch: refs/heads/YARN-2915
Commit: 53ca351b307dccd2c6ac691283ab473dc5bed855
Parents: 5d38f16
Author: Subru Krishnan <su...@apache.org>
Authored: Thu Oct 13 17:59:13 2016 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon May 8 16:59:12 2017 -0700

----------------------------------------------------------------------
 .../AbstractConfigurableFederationPolicy.java   | 155 +++++
 .../policies/ConfigurableFederationPolicy.java  |   9 +-
 .../FederationPolicyInitializationContext.java  |  37 +-
 ...ionPolicyInitializationContextValidator.java |  28 +-
 .../policies/FederationPolicyManager.java       |  59 +-
 .../amrmproxy/AbstractAMRMProxyPolicy.java      |  47 ++
 .../amrmproxy/BroadcastAMRMProxyPolicy.java     |  85 +++
 .../amrmproxy/FederationAMRMProxyPolicy.java    |  25 +-
 .../LocalityMulticastAMRMProxyPolicy.java       | 583 +++++++++++++++++++
 .../policies/amrmproxy/package-info.java        |   1 -
 .../policies/dao/WeightedPolicyInfo.java        | 180 +++---
 .../federation/policies/dao/package-info.java   |   1 -
 .../policies/exceptions/package-info.java       |   1 -
 .../federation/policies/package-info.java       |   1 -
 .../policies/router/AbstractRouterPolicy.java   |  47 ++
 .../router/BaseWeightedRouterPolicy.java        | 150 -----
 .../policies/router/FederationRouterPolicy.java |   5 +-
 .../policies/router/LoadBasedRouterPolicy.java  |  36 +-
 .../policies/router/PriorityRouterPolicy.java   |  19 +-
 .../router/UniformRandomRouterPolicy.java       |  28 +-
 .../router/WeightedRandomRouterPolicy.java      |  32 +-
 .../policies/router/package-info.java           |   1 -
 .../resolver/AbstractSubClusterResolver.java    |   4 +-
 .../policies/BaseFederationPoliciesTest.java    |  28 +-
 ...ionPolicyInitializationContextValidator.java |  25 +-
 .../TestBroadcastAMRMProxyFederationPolicy.java | 112 ++++
 .../TestLocalityMulticastAMRMProxyPolicy.java   | 566 ++++++++++++++++++
 .../router/TestLoadBasedRouterPolicy.java       |  18 +-
 .../router/TestPriorityRouterPolicy.java        |  15 +-
 .../router/TestWeightedRandomRouterPolicy.java  |  35 +-
 .../utils/FederationPoliciesTestUtil.java       |  64 ++
 .../src/test/resources/nodes                    |   6 +-
 32 files changed, 1950 insertions(+), 453 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/53ca351b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/AbstractConfigurableFederationPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/AbstractConfigurableFederationPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/AbstractConfigurableFederationPolicy.java
new file mode 100644
index 0000000..4cb9bbe
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/AbstractConfigurableFederationPolicy.java
@@ -0,0 +1,155 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies;
+
+import java.util.Map;
+
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.federation.policies.dao.WeightedPolicyInfo;
+import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
+import org.apache.hadoop.yarn.server.federation.policies.exceptions.NoActiveSubclustersException;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
+
+/**
+ * Base abstract class for a weighted {@link ConfigurableFederationPolicy}.
+ */
+public abstract class AbstractConfigurableFederationPolicy
+    implements ConfigurableFederationPolicy {
+
+  private WeightedPolicyInfo policyInfo = null;
+  private FederationPolicyInitializationContext policyContext;
+  private boolean isDirty;
+
+  public AbstractConfigurableFederationPolicy() {
+  }
+
+  @Override
+  public void reinitialize(
+      FederationPolicyInitializationContext initializationContext)
+      throws FederationPolicyInitializationException {
+    isDirty = true;
+    FederationPolicyInitializationContextValidator
+        .validate(initializationContext, this.getClass().getCanonicalName());
+
+    // perform consistency checks
+    WeightedPolicyInfo newPolicyInfo = WeightedPolicyInfo.fromByteBuffer(
+        initializationContext.getSubClusterPolicyConfiguration().getParams());
+
+    // if nothing has changed skip the rest of initialization
+    // and signal to childs that the reinit is free via isDirty var.
+    if (policyInfo != null && policyInfo.equals(newPolicyInfo)) {
+      isDirty = false;
+      return;
+    }
+
+    validate(newPolicyInfo);
+    setPolicyInfo(newPolicyInfo);
+    this.policyContext = initializationContext;
+  }
+
+  /**
+   * Overridable validation step for the policy configuration.
+   *
+   * @param newPolicyInfo the configuration to test.
+   *
+   * @throws FederationPolicyInitializationException if the configuration is not
+   *           valid.
+   */
+  public void validate(WeightedPolicyInfo newPolicyInfo)
+      throws FederationPolicyInitializationException {
+    if (newPolicyInfo == null) {
+      throw new FederationPolicyInitializationException(
+          "The policy to " + "validate should not be null.");
+    }
+  }
+
+  /**
+   * Returns true whether the last reinitialization requires actual changes, or
+   * was "free" as the weights have not changed. This is used by subclasses
+   * overriding reinitialize and calling super.reinitialize() to know wheter to
+   * quit early.
+   *
+   * @return whether more work is needed to initialize.
+   */
+  public boolean getIsDirty() {
+    return isDirty;
+  }
+
+  /**
+   * Getter method for the configuration weights.
+   *
+   * @return the {@link WeightedPolicyInfo} representing the policy
+   *         configuration.
+   */
+  public WeightedPolicyInfo getPolicyInfo() {
+    return policyInfo;
+  }
+
+  /**
+   * Setter method for the configuration weights.
+   *
+   * @param policyInfo the {@link WeightedPolicyInfo} representing the policy
+   *          configuration.
+   */
+  public void setPolicyInfo(WeightedPolicyInfo policyInfo) {
+    this.policyInfo = policyInfo;
+  }
+
+  /**
+   * Getter method for the {@link FederationPolicyInitializationContext}.
+   *
+   * @return the context for this policy.
+   */
+  public FederationPolicyInitializationContext getPolicyContext() {
+    return policyContext;
+  }
+
+  /**
+   * Setter method for the {@link FederationPolicyInitializationContext}.
+   *
+   * @param policyContext the context to assign to this policy.
+   */
+  public void setPolicyContext(
+      FederationPolicyInitializationContext policyContext) {
+    this.policyContext = policyContext;
+  }
+
+  /**
+   * This methods gets active subclusters map from the {@code
+   * FederationStateStoreFacade} and validate it not being null/empty.
+   *
+   * @return the map of ids to info for all active subclusters.
+   *
+   * @throws YarnException if we can't get the list.
+   */
+  protected Map<SubClusterId, SubClusterInfo> getActiveSubclusters()
+      throws YarnException {
+
+    Map<SubClusterId, SubClusterInfo> activeSubclusters =
+        getPolicyContext().getFederationStateStoreFacade().getSubClusters(true);
+
+    if (activeSubclusters == null || activeSubclusters.size() < 1) {
+      throw new NoActiveSubclustersException(
+          "Zero active subclusters, cannot pick where to send job.");
+    }
+    return activeSubclusters;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53ca351b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/ConfigurableFederationPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/ConfigurableFederationPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/ConfigurableFederationPolicy.java
index fd6ceea..5245772 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/ConfigurableFederationPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/ConfigurableFederationPolicy.java
@@ -31,14 +31,11 @@ public interface ConfigurableFederationPolicy {
    * policies. The implementor should provide try-n-swap semantics, and retain
    * state if possible.
    *
-   * @param federationPolicyInitializationContext the new context to provide to
-   *                                              implementor.
+   * @param policyContext the new context to provide to implementor.
    *
    * @throws FederationPolicyInitializationException in case the initialization
-   *                                                 fails.
+   *           fails.
    */
-  void reinitialize(
-      FederationPolicyInitializationContext
-          federationPolicyInitializationContext)
+  void reinitialize(FederationPolicyInitializationContext policyContext)
       throws FederationPolicyInitializationException;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53ca351b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyInitializationContext.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyInitializationContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyInitializationContext.java
index 9347fd0..46dd6eb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyInitializationContext.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyInitializationContext.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.yarn.server.federation.policies;
 
 import org.apache.hadoop.yarn.server.federation.resolver.SubClusterResolver;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyConfiguration;
 import org.apache.hadoop.yarn.server.federation.utils.FederationStateStoreFacade;
 
@@ -30,6 +31,7 @@ public class FederationPolicyInitializationContext {
   private SubClusterPolicyConfiguration federationPolicyConfiguration;
   private SubClusterResolver federationSubclusterResolver;
   private FederationStateStoreFacade federationStateStoreFacade;
+  private SubClusterId homeSubcluster;
 
   public FederationPolicyInitializationContext() {
     federationPolicyConfiguration = null;
@@ -37,20 +39,19 @@ public class FederationPolicyInitializationContext {
     federationStateStoreFacade = null;
   }
 
-  public FederationPolicyInitializationContext(SubClusterPolicyConfiguration
-      policy, SubClusterResolver resolver, FederationStateStoreFacade
-      storeFacade) {
+  public FederationPolicyInitializationContext(
+      SubClusterPolicyConfiguration policy, SubClusterResolver resolver,
+      FederationStateStoreFacade storeFacade) {
     this.federationPolicyConfiguration = policy;
     this.federationSubclusterResolver = resolver;
     this.federationStateStoreFacade = storeFacade;
   }
 
-
   /**
    * Getter for the {@link SubClusterPolicyConfiguration}.
    *
    * @return the {@link SubClusterPolicyConfiguration} to be used for
-   * initialization.
+   *         initialization.
    */
   public SubClusterPolicyConfiguration getSubClusterPolicyConfiguration() {
     return federationPolicyConfiguration;
@@ -59,8 +60,8 @@ public class FederationPolicyInitializationContext {
   /**
    * Setter for the {@link SubClusterPolicyConfiguration}.
    *
-   * @param fedPolicyConfiguration the {@link SubClusterPolicyConfiguration}
-   *                               to be used for initialization.
+   * @param fedPolicyConfiguration the {@link SubClusterPolicyConfiguration} to
+   *          be used for initialization.
    */
   public void setSubClusterPolicyConfiguration(
       SubClusterPolicyConfiguration fedPolicyConfiguration) {
@@ -80,7 +81,7 @@ public class FederationPolicyInitializationContext {
    * Setter for the {@link SubClusterResolver}.
    *
    * @param federationSubclusterResolver the {@link SubClusterResolver} to be
-   *                                     used for initialization.
+   *          used for initialization.
    */
   public void setFederationSubclusterResolver(
       SubClusterResolver federationSubclusterResolver) {
@@ -105,4 +106,24 @@ public class FederationPolicyInitializationContext {
       FederationStateStoreFacade federationStateStoreFacade) {
     this.federationStateStoreFacade = federationStateStoreFacade;
   }
+
+  /**
+   * Returns the current home sub-cluster. Useful for default policy behaviors.
+   *
+   * @return the home sub-cluster.
+   */
+  public SubClusterId getHomeSubcluster() {
+    return homeSubcluster;
+  }
+
+  /**
+   * Sets in the context the home sub-cluster. Useful for default policy
+   * behaviors.
+   *
+   * @param homeSubcluster value to set.
+   */
+  public void setHomeSubcluster(SubClusterId homeSubcluster) {
+    this.homeSubcluster = homeSubcluster;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53ca351b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyInitializationContextValidator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyInitializationContextValidator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyInitializationContextValidator.java
index 31f83d4..1b83bbc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyInitializationContextValidator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyInitializationContextValidator.java
@@ -25,50 +25,44 @@ import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPo
 public final class FederationPolicyInitializationContextValidator {
 
   private FederationPolicyInitializationContextValidator() {
-    //disable constructor per checkstyle
+    // disable constructor per checkstyle
   }
 
   public static void validate(
-      FederationPolicyInitializationContext
-          federationPolicyInitializationContext,
-      String myType) throws FederationPolicyInitializationException {
+      FederationPolicyInitializationContext policyContext, String myType)
+      throws FederationPolicyInitializationException {
 
     if (myType == null) {
-      throw new FederationPolicyInitializationException("The myType parameter"
-          + " should not be null.");
+      throw new FederationPolicyInitializationException(
+          "The myType parameter" + " should not be null.");
     }
 
-    if (federationPolicyInitializationContext == null) {
+    if (policyContext == null) {
       throw new FederationPolicyInitializationException(
           "The FederationPolicyInitializationContext provided is null. Cannot"
-              + " reinitalize "
-              + "successfully.");
+              + " reinitalize " + "successfully.");
     }
 
-    if (federationPolicyInitializationContext.getFederationStateStoreFacade()
-        == null) {
+    if (policyContext.getFederationStateStoreFacade() == null) {
       throw new FederationPolicyInitializationException(
           "The FederationStateStoreFacade provided is null. Cannot"
               + " reinitalize successfully.");
     }
 
-    if (federationPolicyInitializationContext.getFederationSubclusterResolver()
-        == null) {
+    if (policyContext.getFederationSubclusterResolver() == null) {
       throw new FederationPolicyInitializationException(
           "The FederationStateStoreFacase provided is null. Cannot"
               + " reinitalize successfully.");
     }
 
-    if (federationPolicyInitializationContext.getSubClusterPolicyConfiguration()
-        == null) {
+    if (policyContext.getSubClusterPolicyConfiguration() == null) {
       throw new FederationPolicyInitializationException(
           "The FederationSubclusterResolver provided is null. Cannot "
               + "reinitalize successfully.");
     }
 
     String intendedType =
-        federationPolicyInitializationContext.getSubClusterPolicyConfiguration()
-            .getType();
+        policyContext.getSubClusterPolicyConfiguration().getType();
 
     if (!myType.equals(intendedType)) {
       throw new FederationPolicyInitializationException(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53ca351b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyManager.java
index e5dba63..39fdba3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyManager.java
@@ -25,19 +25,19 @@ import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyCo
 /**
  *
  * Implementors need to provide the ability to serliaze a policy and its
- * configuration as a {@link SubClusterPolicyConfiguration}, as well as
- * provide (re)initialization mechanics for the underlying
+ * configuration as a {@link SubClusterPolicyConfiguration}, as well as provide
+ * (re)initialization mechanics for the underlying
  * {@link FederationAMRMProxyPolicy} and {@link FederationRouterPolicy}.
  *
- * The serialization aspects are used by admin APIs or a policy engine to
- * store a serialized configuration in the {@code FederationStateStore},
- * while the getters methods are used to obtain a propertly inizialized
- * policy in the {@code Router} and {@code AMRMProxy} respectively.
+ * The serialization aspects are used by admin APIs or a policy engine to store
+ * a serialized configuration in the {@code FederationStateStore}, while the
+ * getters methods are used to obtain a propertly inizialized policy in the
+ * {@code Router} and {@code AMRMProxy} respectively.
  *
- * This interface by design binds together
- * {@link FederationAMRMProxyPolicy} and {@link FederationRouterPolicy} and
- * provide lifecycle support for serialization and deserialization, to reduce
- * configuration mistakes (combining incompatible policies).
+ * This interface by design binds together {@link FederationAMRMProxyPolicy} and
+ * {@link FederationRouterPolicy} and provide lifecycle support for
+ * serialization and deserialization, to reduce configuration mistakes
+ * (combining incompatible policies).
  *
  */
 public interface FederationPolicyManager {
@@ -50,23 +50,17 @@ public interface FederationPolicyManager {
    * the implementors should attempt to reinitalize (retaining state). To affect
    * a complete policy reset oldInstance should be null.
    *
-   * @param federationPolicyInitializationContext the current context
-   * @param oldInstance                           the existing (possibly null)
-   *                                              instance.
+   * @param policyContext the current context
+   * @param oldInstance the existing (possibly null) instance.
    *
-   * @return an updated {@link FederationAMRMProxyPolicy
-  }.
+   * @return an updated {@link FederationAMRMProxyPolicy }.
    *
    * @throws FederationPolicyInitializationException if the initialization
-   *                                                 cannot be completed
-   *                                                 properly. The oldInstance
-   *                                                 should be still valid in
-   *                                                 case of failed
-   *                                                 initialization.
+   *           cannot be completed properly. The oldInstance should be still
+   *           valid in case of failed initialization.
    */
   FederationAMRMProxyPolicy getAMRMPolicy(
-      FederationPolicyInitializationContext
-          federationPolicyInitializationContext,
+      FederationPolicyInitializationContext policyContext,
       FederationAMRMProxyPolicy oldInstance)
       throws FederationPolicyInitializationException;
 
@@ -78,21 +72,17 @@ public interface FederationPolicyManager {
    * implementors should attempt to reinitalize (retaining state). To affect a
    * complete policy reset oldInstance shoulb be set to null.
    *
-   * @param federationPolicyInitializationContext the current context
-   * @param oldInstance                           the existing (possibly null)
-   *                                              instance.
+   * @param policyContext the current context
+   * @param oldInstance the existing (possibly null) instance.
    *
    * @return an updated {@link FederationRouterPolicy}.
    *
    * @throws FederationPolicyInitializationException if the initalization cannot
-   *                                                 be completed properly. The
-   *                                                 oldInstance should be still
-   *                                                 valid in case of failed
-   *                                                 initialization.
+   *           be completed properly. The oldInstance should be still valid in
+   *           case of failed initialization.
    */
   FederationRouterPolicy getRouterPolicy(
-      FederationPolicyInitializationContext
-          federationPolicyInitializationContext,
+      FederationPolicyInitializationContext policyContext,
       FederationRouterPolicy oldInstance)
       throws FederationPolicyInitializationException;
 
@@ -102,23 +92,24 @@ public interface FederationPolicyManager {
    * store.
    *
    * @return a valid policy configuration representing this object
-   * parametrization.
+   *         parametrization.
    *
    * @throws FederationPolicyInitializationException if the current state cannot
-   *                                                 be serialized properly
+   *           be serialized properly
    */
   SubClusterPolicyConfiguration serializeConf()
       throws FederationPolicyInitializationException;
 
-
   /**
    * This method returns the queue this policy is configured for.
+   *
    * @return the name of the queue.
    */
   String getQueue();
 
   /**
    * This methods provides a setter for the queue this policy is specified for.
+   *
    * @param queue the name of the queue.
    */
   void setQueue(String queue);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53ca351b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/AbstractAMRMProxyPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/AbstractAMRMProxyPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/AbstractAMRMProxyPolicy.java
new file mode 100644
index 0000000..e853744
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/AbstractAMRMProxyPolicy.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies.amrmproxy;
+
+import java.util.Map;
+
+import org.apache.hadoop.yarn.server.federation.policies.AbstractConfigurableFederationPolicy;
+import org.apache.hadoop.yarn.server.federation.policies.dao.WeightedPolicyInfo;
+import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterIdInfo;
+
+/**
+ * Base abstract class for {@link FederationAMRMProxyPolicy} implementations,
+ * that provides common validation for reinitialization.
+ */
+public abstract class AbstractAMRMProxyPolicy extends
+    AbstractConfigurableFederationPolicy implements FederationAMRMProxyPolicy {
+
+  @Override
+  public void validate(WeightedPolicyInfo newPolicyInfo)
+      throws FederationPolicyInitializationException {
+    super.validate(newPolicyInfo);
+    Map<SubClusterIdInfo, Float> newWeights =
+        newPolicyInfo.getAMRMPolicyWeights();
+    if (newWeights == null || newWeights.size() < 1) {
+      throw new FederationPolicyInitializationException(
+          "Weight vector cannot be null/empty.");
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53ca351b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/BroadcastAMRMProxyPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/BroadcastAMRMProxyPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/BroadcastAMRMProxyPolicy.java
new file mode 100644
index 0000000..679f4d5
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/BroadcastAMRMProxyPolicy.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies.amrmproxy;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
+import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.federation.policies.FederationPolicyInitializationContext;
+import org.apache.hadoop.yarn.server.federation.policies.FederationPolicyInitializationContextValidator;
+import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
+import org.apache.hadoop.yarn.server.federation.policies.exceptions.UnknownSubclusterException;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
+
+/**
+ * An implementation of the {@link FederationAMRMProxyPolicy} that simply
+ * broadcasts each {@link ResourceRequest} to all the available sub-clusters.
+ */
+public class BroadcastAMRMProxyPolicy extends AbstractAMRMProxyPolicy {
+
+  private Set<SubClusterId> knownClusterIds = new HashSet<>();
+
+  @Override
+  public void reinitialize(
+      FederationPolicyInitializationContext policyContext)
+      throws FederationPolicyInitializationException {
+    // overrides initialize to avoid weight checks that do no apply for
+    // this policy.
+    FederationPolicyInitializationContextValidator
+        .validate(policyContext, this.getClass().getCanonicalName());
+    setPolicyContext(policyContext);
+  }
+
+  @Override
+  public Map<SubClusterId, List<ResourceRequest>> splitResourceRequests(
+      List<ResourceRequest> resourceRequests) throws YarnException {
+
+    Map<SubClusterId, SubClusterInfo> activeSubclusters =
+        getActiveSubclusters();
+
+    Map<SubClusterId, List<ResourceRequest>> answer = new HashMap<>();
+
+    // simply broadcast the resource request to all sub-clusters
+    for (SubClusterId subClusterId : activeSubclusters.keySet()) {
+      answer.put(subClusterId, resourceRequests);
+      knownClusterIds.add(subClusterId);
+    }
+
+    return answer;
+  }
+
+  @Override
+  public void notifyOfResponse(SubClusterId subClusterId,
+      AllocateResponse response) throws YarnException {
+    if (!knownClusterIds.contains(subClusterId)) {
+      throw new UnknownSubclusterException(
+          "The response is received from a subcluster that is unknown to this "
+              + "policy.");
+    }
+    // stateless policy does not care about responses
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53ca351b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/FederationAMRMProxyPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/FederationAMRMProxyPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/FederationAMRMProxyPolicy.java
index 4a3305c..0541df4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/FederationAMRMProxyPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/FederationAMRMProxyPolicy.java
@@ -17,18 +17,18 @@
 
 package org.apache.hadoop.yarn.server.federation.policies.amrmproxy;
 
+import java.util.List;
+import java.util.Map;
+
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.federation.policies.ConfigurableFederationPolicy;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
 
-import java.util.List;
-import java.util.Map;
-
 /**
- * Implementors of this interface provide logic to split the list of {@link
- * ResourceRequest}s received by the AM among various RMs.
+ * Implementors of this interface provide logic to split the list of
+ * {@link ResourceRequest}s received by the AM among various RMs.
  */
 public interface FederationAMRMProxyPolicy
     extends ConfigurableFederationPolicy {
@@ -37,18 +37,17 @@ public interface FederationAMRMProxyPolicy
    * Splits the {@link ResourceRequest}s from the client across one or more
    * sub-clusters based on the policy semantics (e.g., broadcast, load-based).
    *
-   * @param resourceRequests the list of {@link ResourceRequest}s from the
-   *                         AM to be split
+   * @param resourceRequests the list of {@link ResourceRequest}s from the AM to
+   *          be split
    *
    * @return map of sub-cluster as identified by {@link SubClusterId} to the
-   * list of {@link ResourceRequest}s that should be forwarded to it
+   *         list of {@link ResourceRequest}s that should be forwarded to it
    *
    * @throws YarnException in case the request is malformed or no viable
-   *                       sub-clusters can be found.
+   *           sub-clusters can be found.
    */
   Map<SubClusterId, List<ResourceRequest>> splitResourceRequests(
-      List<ResourceRequest> resourceRequests)
-      throws YarnException;
+      List<ResourceRequest> resourceRequests) throws YarnException;
 
   /**
    * This method should be invoked to notify the policy about responses being
@@ -60,7 +59,7 @@ public interface FederationAMRMProxyPolicy
    *
    * @throws YarnException in case the response is not valid
    */
-  void notifyOfResponse(SubClusterId subClusterId,
-      AllocateResponse response) throws YarnException;
+  void notifyOfResponse(SubClusterId subClusterId, AllocateResponse response)
+      throws YarnException;
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53ca351b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/LocalityMulticastAMRMProxyPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/LocalityMulticastAMRMProxyPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/LocalityMulticastAMRMProxyPolicy.java
new file mode 100644
index 0000000..283f89e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/LocalityMulticastAMRMProxyPolicy.java
@@ -0,0 +1,583 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies.amrmproxy;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.federation.policies.FederationPolicyInitializationContext;
+import org.apache.hadoop.yarn.server.federation.policies.dao.WeightedPolicyInfo;
+import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
+import org.apache.hadoop.yarn.server.federation.policies.exceptions.NoActiveSubclustersException;
+import org.apache.hadoop.yarn.server.federation.resolver.SubClusterResolver;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterIdInfo;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
+import org.apache.hadoop.yarn.server.federation.utils.FederationStateStoreFacade;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * An implementation of the {@link FederationAMRMProxyPolicy} interface that
+ * carefully multicasts the requests with the following behavior:
+ *
+ * <p>
+ * Host localized {@link ResourceRequest}s are always forwarded to the RM that
+ * owns the corresponding node, based on the feedback of a
+ * {@link SubClusterResolver}. If the {@link SubClusterResolver} cannot resolve
+ * this node we default to forwarding the {@link ResourceRequest} to the home
+ * sub-cluster.
+ * </p>
+ *
+ * <p>
+ * Rack localized {@link ResourceRequest}s are forwarded to the RMs that owns
+ * the corresponding rack. Note that in some deployments each rack could be
+ * striped across multiple RMs. Thsi policy respects that. If the
+ * {@link SubClusterResolver} cannot resolve this rack we default to forwarding
+ * the {@link ResourceRequest} to the home sub-cluster.
+ * </p>
+ *
+ * <p>
+ * ANY requests corresponding to node/rack local requests are forwarded only to
+ * the set of RMs that owns the corresponding localized requests. The number of
+ * containers listed in each ANY is proportional to the number of localized
+ * container requests (associated to this ANY via the same allocateRequestId).
+ * </p>
+ *
+ * <p>
+ * ANY that are not associated to node/rack local requests are split among RMs
+ * based on the "weights" in the {@link WeightedPolicyInfo} configuration *and*
+ * headroom information. The {@code headroomAlpha} parameter of the policy
+ * configuration indicates how much headroom contributes to the splitting
+ * choice. Value of 1.0f indicates the weights are interpreted only as 0/1
+ * boolean but all splitting is based on the advertised headroom (fallback to
+ * 1/N for RMs that we don't have headroom info from). An {@code headroomAlpha}
+ * value of 0.0f means headroom is ignored and all splitting decisions are
+ * proportional to the "weights" in the configuration of the policy.
+ * </p>
+ *
+ * <p>
+ * ANY of zero size are forwarded to all known subclusters (i.e., subclusters
+ * where we scheduled containers before), as they may represent a user attempt
+ * to cancel a previous request (and we are mostly stateless now, so should
+ * forward to all known RMs).
+ * </p>
+ *
+ * <p>
+ * Invariants:
+ * </p>
+ *
+ * <p>
+ * The policy always excludes non-active RMs.
+ * </p>
+ *
+ * <p>
+ * The policy always excludes RMs that do not appear in the policy configuration
+ * weights, or have a weight of 0 (even if localized resources explicit refer to
+ * it).
+ * </p>
+ *
+ * <p>
+ * (Bar rounding to closest ceiling of fractional containers) The sum of
+ * requests made to multiple RMs at the ANY level "adds-up" to the user request.
+ * The maximum possible excess in a given request is a number of containers less
+ * or equal to number of sub-clusters in the federation.
+ * </p>
+ */
+public class LocalityMulticastAMRMProxyPolicy extends AbstractAMRMProxyPolicy {
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(LocalityMulticastAMRMProxyPolicy.class);
+
+  private Map<SubClusterId, Float> weights;
+  private SubClusterResolver resolver;
+
+  private Map<SubClusterId, Resource> headroom;
+  private float hrAlpha;
+  private FederationStateStoreFacade federationFacade;
+  private AllocationBookkeeper bookkeeper;
+  private SubClusterId homeSubcluster;
+
+  @Override
+  public void reinitialize(
+      FederationPolicyInitializationContext policyContext)
+      throws FederationPolicyInitializationException {
+
+    // save reference to old weights
+    WeightedPolicyInfo tempPolicy = getPolicyInfo();
+
+    super.reinitialize(policyContext);
+    if (!getIsDirty()) {
+      return;
+    }
+
+    Map<SubClusterId, Float> newWeightsConverted = new HashMap<>();
+    boolean allInactive = true;
+    WeightedPolicyInfo policy = getPolicyInfo();
+    if (policy.getAMRMPolicyWeights() == null
+        || policy.getAMRMPolicyWeights().size() == 0) {
+      allInactive = false;
+    } else {
+      for (Map.Entry<SubClusterIdInfo, Float> e : policy.getAMRMPolicyWeights()
+          .entrySet()) {
+        if (e.getValue() > 0) {
+          allInactive = false;
+        }
+        newWeightsConverted.put(e.getKey().toId(), e.getValue());
+      }
+    }
+    if (allInactive) {
+      // reset the policyInfo and throw
+      setPolicyInfo(tempPolicy);
+      throw new FederationPolicyInitializationException(
+          "The weights used to configure "
+              + "this policy are all set to zero! (no ResourceRequest could be "
+              + "forwarded with this setting.)");
+    }
+
+    if (policyContext.getHomeSubcluster() == null) {
+      setPolicyInfo(tempPolicy);
+      throw new FederationPolicyInitializationException("The homeSubcluster "
+          + "filed in the context must be initialized to use this policy");
+    }
+
+    weights = newWeightsConverted;
+    resolver = policyContext.getFederationSubclusterResolver();
+
+    if (headroom == null) {
+      headroom = new ConcurrentHashMap<>();
+    }
+    hrAlpha = policy.getHeadroomAlpha();
+
+    this.federationFacade =
+        policyContext.getFederationStateStoreFacade();
+    this.bookkeeper = new AllocationBookkeeper();
+    this.homeSubcluster = policyContext.getHomeSubcluster();
+
+  }
+
+  @Override
+  public void notifyOfResponse(SubClusterId subClusterId,
+      AllocateResponse response) throws YarnException {
+    // stateless policy does not care about responses except tracking headroom
+    headroom.put(subClusterId, response.getAvailableResources());
+  }
+
+  @Override
+  public Map<SubClusterId, List<ResourceRequest>> splitResourceRequests(
+      List<ResourceRequest> resourceRequests) throws YarnException {
+
+    // object used to accumulate statistics about the answer, initialize with
+    // active subclusters.
+    bookkeeper.reinitialize(federationFacade.getSubClusters(true));
+
+    List<ResourceRequest> nonLocalizedRequests =
+        new ArrayList<ResourceRequest>();
+
+    SubClusterId targetId = null;
+    Set<SubClusterId> targetIds = null;
+
+    // if the RR is resolved to a local subcluster add it directly (node and
+    // resolvable racks)
+    for (ResourceRequest rr : resourceRequests) {
+      targetId = null;
+      targetIds = null;
+
+      // Handle: ANY (accumulated for later)
+      if (ResourceRequest.isAnyLocation(rr.getResourceName())) {
+        nonLocalizedRequests.add(rr);
+        continue;
+      }
+
+      // Handle "node" requests
+      try {
+        targetId = resolver.getSubClusterForNode(rr.getResourceName());
+      } catch (YarnException e) {
+        // this might happen as we can't differentiate node from rack names
+        // we log altogether later
+      }
+      if (bookkeeper.isActiveAndEnabled(targetId)) {
+        bookkeeper.addLocalizedNodeRR(targetId, rr);
+        continue;
+      }
+
+      // Handle "rack" requests
+      try {
+        targetIds = resolver.getSubClustersForRack(rr.getResourceName());
+      } catch (YarnException e) {
+        // this might happen as we can't differentiate node from rack names
+        // we log altogether later
+      }
+      if (targetIds != null && targetIds.size() > 0) {
+        for (SubClusterId tid : targetIds) {
+          if (bookkeeper.isActiveAndEnabled(tid)) {
+            bookkeeper.addRackRR(tid, rr);
+          }
+        }
+        continue;
+      }
+
+      // Handle node/rack requests that the SubClusterResolver cannot map to
+      // any cluster. Defaulting to home subcluster.
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("ERROR resolving sub-cluster for resourceName: "
+            + rr.getResourceName() + " we are falling back to homeSubCluster:"
+            + homeSubcluster);
+      }
+
+      // If home-subcluster is not active, ignore node/rack request
+      if (bookkeeper.isActiveAndEnabled(homeSubcluster)) {
+        bookkeeper.addLocalizedNodeRR(homeSubcluster, rr);
+      } else {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("The homeSubCluster (" + homeSubcluster + ") we are "
+              + "defaulting to is not active, the ResourceRequest "
+              + "will be ignored.");
+        }
+      }
+    }
+
+    // handle all non-localized requests (ANY)
+    splitAnyRequests(nonLocalizedRequests, bookkeeper);
+
+    return bookkeeper.getAnswer();
+  }
+
+  /**
+   * It splits a list of non-localized resource requests among sub-clusters.
+   */
+  private void splitAnyRequests(List<ResourceRequest> originalResourceRequests,
+      AllocationBookkeeper allocationBookkeeper) throws YarnException {
+
+    for (ResourceRequest resourceRequest : originalResourceRequests) {
+
+      // FIRST: pick the target set of subclusters (based on whether this RR
+      // is associated with other localized requests via an allocationId)
+      Long allocationId = resourceRequest.getAllocationRequestId();
+      Set<SubClusterId> targetSubclusters;
+      if (allocationBookkeeper.getSubClustersForId(allocationId) != null) {
+        targetSubclusters =
+            allocationBookkeeper.getSubClustersForId(allocationId);
+      } else {
+        targetSubclusters = allocationBookkeeper.getActiveAndEnabledSC();
+      }
+
+      // SECOND: pick how much to ask to each RM for each request
+      splitIndividualAny(resourceRequest, targetSubclusters,
+          allocationBookkeeper);
+    }
+  }
+
+  /**
+   * Return a projection of this ANY {@link ResourceRequest} that belongs to
+   * this sub-cluster. This is done based on the "count" of the containers that
+   * require locality in each sublcuster (if any) or based on the "weights" and
+   * headroom.
+   */
+  private void splitIndividualAny(ResourceRequest originalResourceRequest,
+      Set<SubClusterId> targetSubclusters,
+      AllocationBookkeeper allocationBookkeeper) {
+
+    long allocationId = originalResourceRequest.getAllocationRequestId();
+
+    for (SubClusterId targetId : targetSubclusters) {
+      float numContainer = originalResourceRequest.getNumContainers();
+
+      // If the ANY request has 0 containers to begin with we must forward it to
+      // any RM we have previously contacted (this might be the user way
+      // to cancel a previous request).
+      if (numContainer == 0 && headroom.containsKey(targetId)) {
+        allocationBookkeeper.addAnyRR(targetId, originalResourceRequest);
+      }
+
+      // If ANY is associated with localized asks, split based on their ratio
+      if (allocationBookkeeper.getSubClustersForId(allocationId) != null) {
+        float localityBasedWeight = getLocalityBasedWeighting(allocationId,
+            targetId, allocationBookkeeper);
+        numContainer = numContainer * localityBasedWeight;
+      } else {
+        // split ANY based on load and policy configuration
+        float headroomWeighting =
+            getHeadroomWeighting(targetId, allocationBookkeeper);
+        float policyWeighting =
+            getPolicyConfigWeighting(targetId, allocationBookkeeper);
+        // hrAlpha controls how much headroom influencing decision
+        numContainer = numContainer
+            * (hrAlpha * headroomWeighting + (1 - hrAlpha) * policyWeighting);
+      }
+
+      // if the calculated request is non-empty add it to the answer
+      if (numContainer > 0) {
+        ResourceRequest out =
+            ResourceRequest.newInstance(originalResourceRequest.getPriority(),
+                originalResourceRequest.getResourceName(),
+                originalResourceRequest.getCapability(),
+                originalResourceRequest.getNumContainers(),
+                originalResourceRequest.getRelaxLocality(),
+                originalResourceRequest.getNodeLabelExpression(),
+                originalResourceRequest.getExecutionTypeRequest());
+        out.setAllocationRequestId(allocationId);
+        out.setNumContainers((int) Math.ceil(numContainer));
+        if (out.isAnyLocation(out.getResourceName())) {
+          allocationBookkeeper.addAnyRR(targetId, out);
+        } else {
+          allocationBookkeeper.addRackRR(targetId, out);
+        }
+      }
+    }
+  }
+
+  /**
+   * Compute the weight to assign to a subcluster based on how many local
+   * requests a subcluster is target of.
+   */
+  private float getLocalityBasedWeighting(long reqId, SubClusterId targetId,
+      AllocationBookkeeper allocationBookkeeper) {
+    float totWeight = allocationBookkeeper.getTotNumLocalizedContainers();
+    float localWeight =
+        allocationBookkeeper.getNumLocalizedContainers(reqId, targetId);
+    return totWeight > 0 ? localWeight / totWeight : 0;
+  }
+
+  /**
+   * Compute the "weighting" to give to a sublcuster based on the configured
+   * policy weights (for the active subclusters).
+   */
+  private float getPolicyConfigWeighting(SubClusterId targetId,
+      AllocationBookkeeper allocationBookkeeper) {
+    float totWeight = allocationBookkeeper.totPolicyWeight;
+    Float localWeight = weights.get(targetId);
+    return (localWeight != null && totWeight > 0) ? localWeight / totWeight : 0;
+  }
+
+  /**
+   * Compute the weighting based on available headroom. This is proportional to
+   * the available headroom memory announced by RM, or to 1/N for RMs we have
+   * not seen yet. If all RMs report zero headroom, we fallback to 1/N again.
+   */
+  private float getHeadroomWeighting(SubClusterId targetId,
+      AllocationBookkeeper allocationBookkeeper) {
+
+    // baseline weight for all RMs
+    float headroomWeighting =
+        1 / (float) allocationBookkeeper.getActiveAndEnabledSC().size();
+
+    // if we have headroom infomration for this sub-cluster (and we are safe
+    // from /0 issues)
+    if (headroom.containsKey(targetId)
+        && allocationBookkeeper.totHeadroomMemory > 0) {
+      // compute which portion of the RMs that are active/enabled have reported
+      // their headroom (needed as adjustment factor)
+      // (note: getActiveAndEnabledSC should never be null/zero)
+      float ratioHeadroomKnown = allocationBookkeeper.totHeadRoomEnabledRMs
+          / (float) allocationBookkeeper.getActiveAndEnabledSC().size();
+
+      // headroomWeighting is the ratio of headroom memory in the targetId
+      // cluster / total memory. The ratioHeadroomKnown factor is applied to
+      // adjust for missing information and ensure sum of allocated containers
+      // closely approximate what the user asked (small excess).
+      headroomWeighting = (headroom.get(targetId).getMemorySize()
+          / allocationBookkeeper.totHeadroomMemory) * (ratioHeadroomKnown);
+    }
+    return headroomWeighting;
+  }
+
+  /**
+   * This helper class is used to book-keep the requests made to each
+   * subcluster, and maintain useful statistics to split ANY requests.
+   */
+  private final class AllocationBookkeeper {
+
+    // the answer being accumulated
+    private Map<SubClusterId, List<ResourceRequest>> answer = new TreeMap<>();
+
+    // stores how many containers we have allocated in each RM for localized
+    // asks, used to correctly "spread" the corresponding ANY
+    private Map<Long, Map<SubClusterId, AtomicLong>> countContainersPerRM =
+        new HashMap<>();
+
+    private Set<SubClusterId> activeAndEnabledSC = new HashSet<>();
+    private long totNumLocalizedContainers = 0;
+    private float totHeadroomMemory = 0;
+    private int totHeadRoomEnabledRMs = 0;
+    private float totPolicyWeight = 0;
+
+    private void reinitialize(
+        Map<SubClusterId, SubClusterInfo> activeSubclusters)
+        throws YarnException {
+
+      // reset data structures
+      answer.clear();
+      countContainersPerRM.clear();
+      activeAndEnabledSC.clear();
+      totNumLocalizedContainers = 0;
+      totHeadroomMemory = 0;
+      totHeadRoomEnabledRMs = 0;
+      totPolicyWeight = 0;
+
+      // pre-compute the set of subclusters that are both active and enabled by
+      // the policy weights, and accumulate their total weight
+      for (Map.Entry<SubClusterId, Float> entry : weights.entrySet()) {
+        if (entry.getValue() > 0
+            && activeSubclusters.containsKey(entry.getKey())) {
+          activeAndEnabledSC.add(entry.getKey());
+          totPolicyWeight += entry.getValue();
+        }
+      }
+
+      if (activeAndEnabledSC.size() < 1) {
+        throw new NoActiveSubclustersException(
+            "None of the subclusters enabled in this policy (weight>0) are "
+                + "currently active we cannot forward the ResourceRequest(s)");
+      }
+
+      // pre-compute headroom-based weights for active/enabled subclusters
+      for (Map.Entry<SubClusterId, Resource> r : headroom.entrySet()) {
+        if (activeAndEnabledSC.contains(r.getKey())) {
+          totHeadroomMemory += r.getValue().getMemorySize();
+          totHeadRoomEnabledRMs++;
+        }
+      }
+
+    }
+
+    /**
+     * Add to the answer a localized node request, and keeps track of statistics
+     * on a per-allocation-id and per-subcluster bases.
+     */
+    private void addLocalizedNodeRR(SubClusterId targetId, ResourceRequest rr) {
+      Preconditions.checkArgument(!rr.isAnyLocation(rr.getResourceName()));
+
+      if (!countContainersPerRM.containsKey(rr.getAllocationRequestId())) {
+        countContainersPerRM.put(rr.getAllocationRequestId(), new HashMap<>());
+      }
+      if (!countContainersPerRM.get(rr.getAllocationRequestId())
+          .containsKey(targetId)) {
+        countContainersPerRM.get(rr.getAllocationRequestId()).put(targetId,
+            new AtomicLong(0));
+      }
+      countContainersPerRM.get(rr.getAllocationRequestId()).get(targetId)
+          .addAndGet(rr.getNumContainers());
+
+      totNumLocalizedContainers += rr.getNumContainers();
+
+      internalAddToAnswer(targetId, rr);
+    }
+
+    /**
+     * Add a rack-local request to the final asnwer.
+     */
+    public void addRackRR(SubClusterId targetId, ResourceRequest rr) {
+      Preconditions.checkArgument(!rr.isAnyLocation(rr.getResourceName()));
+      internalAddToAnswer(targetId, rr);
+    }
+
+    /**
+     * Add an ANY request to the final answer.
+     */
+    private void addAnyRR(SubClusterId targetId, ResourceRequest rr) {
+      Preconditions.checkArgument(rr.isAnyLocation(rr.getResourceName()));
+      internalAddToAnswer(targetId, rr);
+    }
+
+    private void internalAddToAnswer(SubClusterId targetId,
+        ResourceRequest partialRR) {
+      if (!answer.containsKey(targetId)) {
+        answer.put(targetId, new ArrayList<ResourceRequest>());
+      }
+      answer.get(targetId).add(partialRR);
+    }
+
+    /**
+     * Return all known subclusters associated with an allocation id.
+     *
+     * @param allocationId the allocation id considered
+     *
+     * @return the list of {@link SubClusterId}s associated with this allocation
+     *         id
+     */
+    private Set<SubClusterId> getSubClustersForId(long allocationId) {
+      if (countContainersPerRM.get(allocationId) == null) {
+        return null;
+      }
+      return countContainersPerRM.get(allocationId).keySet();
+    }
+
+    /**
+     * Return the answer accumulated so far.
+     *
+     * @return the answer
+     */
+    private Map<SubClusterId, List<ResourceRequest>> getAnswer() {
+      return answer;
+    }
+
+    /**
+     * Return the set of sub-clusters that are both active and allowed by our
+     * policy (weight > 0).
+     *
+     * @return a set of active and enabled {@link SubClusterId}s
+     */
+    private Set<SubClusterId> getActiveAndEnabledSC() {
+      return activeAndEnabledSC;
+    }
+
+    /**
+     * Return the total number of container coming from localized requests.
+     */
+    private long getTotNumLocalizedContainers() {
+      return totNumLocalizedContainers;
+    }
+
+    /**
+     * Returns the number of containers matching an allocation Id that are
+     * localized in the targetId subcluster.
+     */
+    private long getNumLocalizedContainers(long allocationId,
+        SubClusterId targetId) {
+      AtomicLong c = countContainersPerRM.get(allocationId).get(targetId);
+      return c == null ? 0 : c.get();
+    }
+
+    /**
+     * Returns true is the subcluster request is both active and enabled.
+     */
+    private boolean isActiveAndEnabled(SubClusterId targetId) {
+      if (targetId == null) {
+        return false;
+      } else {
+        return getActiveAndEnabledSC().contains(targetId);
+      }
+    }
+
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53ca351b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/package-info.java
index 99da20b..ef72647 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/package-info.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/package-info.java
@@ -17,4 +17,3 @@
  */
 /** AMRMPRoxy policies. **/
 package org.apache.hadoop.yarn.server.federation.policies.amrmproxy;
-

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53ca351b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/dao/WeightedPolicyInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/dao/WeightedPolicyInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/dao/WeightedPolicyInfo.java
index a0fa37f..62eb03b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/dao/WeightedPolicyInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/dao/WeightedPolicyInfo.java
@@ -17,10 +17,19 @@
 
 package org.apache.hadoop.yarn.server.federation.policies.dao;
 
-import com.sun.jersey.api.json.JSONConfiguration;
-import com.sun.jersey.api.json.JSONJAXBContext;
-import com.sun.jersey.api.json.JSONMarshaller;
-import com.sun.jersey.api.json.JSONUnmarshaller;
+import java.io.StringReader;
+import java.io.StringWriter;
+import java.nio.ByteBuffer;
+import java.nio.charset.Charset;
+import java.util.HashMap;
+import java.util.Map;
+
+import javax.xml.bind.JAXBException;
+import javax.xml.bind.Marshaller;
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlRootElement;
+
 import org.apache.commons.collections.CollectionUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -29,24 +38,16 @@ import org.apache.hadoop.yarn.server.federation.store.records.SubClusterIdInfo;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import javax.xml.bind.JAXBException;
-import javax.xml.bind.Marshaller;
-import javax.xml.bind.annotation.XmlAccessType;
-import javax.xml.bind.annotation.XmlAccessorType;
-import javax.xml.bind.annotation.XmlRootElement;
-import java.io.StringReader;
-import java.io.StringWriter;
-import java.nio.ByteBuffer;
-import java.nio.charset.Charset;
-import java.util.HashMap;
-import java.util.Map;
+import com.sun.jersey.api.json.JSONConfiguration;
+import com.sun.jersey.api.json.JSONJAXBContext;
+import com.sun.jersey.api.json.JSONMarshaller;
+import com.sun.jersey.api.json.JSONUnmarshaller;
 
 /**
  * This is a DAO class for the configuration of parameteres for federation
  * policies. This generalizes several possible configurations as two lists of
- * {@link SubClusterIdInfo} and corresponding weights as a
- * {@link Float}. The interpretation of the weight is left to the logic in
- * the policy.
+ * {@link SubClusterIdInfo} and corresponding weights as a {@link Float}. The
+ * interpretation of the weight is left to the logic in the policy.
  */
 
 @InterfaceAudience.Private
@@ -57,12 +58,14 @@ public class WeightedPolicyInfo {
 
   private static final Logger LOG =
       LoggerFactory.getLogger(WeightedPolicyInfo.class);
-
+  private static JSONJAXBContext jsonjaxbContext = initContext();
   private Map<SubClusterIdInfo, Float> routerPolicyWeights = new HashMap<>();
   private Map<SubClusterIdInfo, Float> amrmPolicyWeights = new HashMap<>();
   private float headroomAlpha;
 
-  private static JSONJAXBContext jsonjaxbContext = initContext();
+  public WeightedPolicyInfo() {
+    // JAXB needs this
+  }
 
   private static JSONJAXBContext initContext() {
     try {
@@ -74,46 +77,6 @@ public class WeightedPolicyInfo {
     return null;
   }
 
-  public WeightedPolicyInfo() {
-    //JAXB needs this
-  }
-
-  /**
-   * Setter method for Router weights.
-   *
-   * @param policyWeights the router weights.
-   */
-  public void setRouterPolicyWeights(
-      Map<SubClusterIdInfo, Float> policyWeights) {
-    this.routerPolicyWeights = policyWeights;
-  }
-
-  /**
-   * Setter method for ARMRMProxy weights.
-   *
-   * @param policyWeights the amrmproxy weights.
-   */
-  public void setAMRMPolicyWeights(
-      Map<SubClusterIdInfo, Float> policyWeights) {
-    this.amrmPolicyWeights = policyWeights;
-  }
-
-  /**
-   * Getter of the router weights.
-   * @return the router weights.
-   */
-  public Map<SubClusterIdInfo, Float> getRouterPolicyWeights() {
-    return routerPolicyWeights;
-  }
-
-  /**
-   * Getter for AMRMProxy weights.
-   * @return the AMRMProxy weights.
-   */
-  public Map<SubClusterIdInfo, Float> getAMRMPolicyWeights() {
-    return amrmPolicyWeights;
-  }
-
   /**
    * Deserializes a {@link WeightedPolicyInfo} from a byte UTF-8 JSON
    * representation.
@@ -123,14 +86,14 @@ public class WeightedPolicyInfo {
    * @return the {@link WeightedPolicyInfo} represented.
    *
    * @throws FederationPolicyInitializationException if a deserializaiton error
-   *                                                 occurs.
+   *           occurs.
    */
   public static WeightedPolicyInfo fromByteBuffer(ByteBuffer bb)
       throws FederationPolicyInitializationException {
 
     if (jsonjaxbContext == null) {
-      throw new FederationPolicyInitializationException("JSONJAXBContext should"
-          + " not be null.");
+      throw new FederationPolicyInitializationException(
+          "JSONJAXBContext should" + " not be null.");
     }
 
     try {
@@ -139,9 +102,8 @@ public class WeightedPolicyInfo {
       bb.get(bytes);
       String params = new String(bytes, Charset.forName("UTF-8"));
 
-      WeightedPolicyInfo weightedPolicyInfo = unmarshaller
-          .unmarshalFromJSON(new StringReader(params),
-              WeightedPolicyInfo.class);
+      WeightedPolicyInfo weightedPolicyInfo = unmarshaller.unmarshalFromJSON(
+          new StringReader(params), WeightedPolicyInfo.class);
       return weightedPolicyInfo;
     } catch (JAXBException j) {
       throw new FederationPolicyInitializationException(j);
@@ -149,19 +111,56 @@ public class WeightedPolicyInfo {
   }
 
   /**
-   * Converts the policy into a byte array representation in the input {@link
-   * ByteBuffer}.
+   * Getter of the router weights.
+   *
+   * @return the router weights.
+   */
+  public Map<SubClusterIdInfo, Float> getRouterPolicyWeights() {
+    return routerPolicyWeights;
+  }
+
+  /**
+   * Setter method for Router weights.
+   *
+   * @param policyWeights the router weights.
+   */
+  public void setRouterPolicyWeights(
+      Map<SubClusterIdInfo, Float> policyWeights) {
+    this.routerPolicyWeights = policyWeights;
+  }
+
+  /**
+   * Getter for AMRMProxy weights.
+   *
+   * @return the AMRMProxy weights.
+   */
+  public Map<SubClusterIdInfo, Float> getAMRMPolicyWeights() {
+    return amrmPolicyWeights;
+  }
+
+  /**
+   * Setter method for ARMRMProxy weights.
+   *
+   * @param policyWeights the amrmproxy weights.
+   */
+  public void setAMRMPolicyWeights(Map<SubClusterIdInfo, Float> policyWeights) {
+    this.amrmPolicyWeights = policyWeights;
+  }
+
+  /**
+   * Converts the policy into a byte array representation in the input
+   * {@link ByteBuffer}.
    *
    * @return byte array representation of this policy configuration.
    *
    * @throws FederationPolicyInitializationException if a serialization error
-   *                                                 occurs.
+   *           occurs.
    */
   public ByteBuffer toByteBuffer()
       throws FederationPolicyInitializationException {
     if (jsonjaxbContext == null) {
-      throw new FederationPolicyInitializationException("JSONJAXBContext should"
-          + " not be null.");
+      throw new FederationPolicyInitializationException(
+          "JSONJAXBContext should" + " not be null.");
     }
     try {
       String s = toJSONString();
@@ -186,22 +185,21 @@ public class WeightedPolicyInfo {
       return false;
     }
 
-    WeightedPolicyInfo otherPolicy =
-        (WeightedPolicyInfo) other;
+    WeightedPolicyInfo otherPolicy = (WeightedPolicyInfo) other;
     Map<SubClusterIdInfo, Float> otherAMRMWeights =
         otherPolicy.getAMRMPolicyWeights();
     Map<SubClusterIdInfo, Float> otherRouterWeights =
         otherPolicy.getRouterPolicyWeights();
 
-    boolean amrmWeightsMatch = otherAMRMWeights != null &&
-        getAMRMPolicyWeights() != null &&
-        CollectionUtils.isEqualCollection(otherAMRMWeights.entrySet(),
-            getAMRMPolicyWeights().entrySet());
+    boolean amrmWeightsMatch =
+        otherAMRMWeights != null && getAMRMPolicyWeights() != null
+            && CollectionUtils.isEqualCollection(otherAMRMWeights.entrySet(),
+                getAMRMPolicyWeights().entrySet());
 
-    boolean routerWeightsMatch = otherRouterWeights != null &&
-        getRouterPolicyWeights() != null &&
-        CollectionUtils.isEqualCollection(otherRouterWeights.entrySet(),
-            getRouterPolicyWeights().entrySet());
+    boolean routerWeightsMatch =
+        otherRouterWeights != null && getRouterPolicyWeights() != null
+            && CollectionUtils.isEqualCollection(otherRouterWeights.entrySet(),
+                getRouterPolicyWeights().entrySet());
 
     return amrmWeightsMatch && routerWeightsMatch;
   }
@@ -215,10 +213,10 @@ public class WeightedPolicyInfo {
    * Return the parameter headroomAlpha, used by policies that balance
    * weight-based and load-based considerations in their decisions.
    *
-   * For policies that use this parameter, values close to 1 indicate that
-   * most of the decision should be based on currently observed headroom from
-   * various sub-clusters, values close to zero, indicate that the decision
-   * should be mostly based on weights and practically ignore current load.
+   * For policies that use this parameter, values close to 1 indicate that most
+   * of the decision should be based on currently observed headroom from various
+   * sub-clusters, values close to zero, indicate that the decision should be
+   * mostly based on weights and practically ignore current load.
    *
    * @return the value of headroomAlpha.
    */
@@ -227,13 +225,13 @@ public class WeightedPolicyInfo {
   }
 
   /**
-   * Set the parameter headroomAlpha, used by policies that balance
-   * weight-based and load-based considerations in their decisions.
+   * Set the parameter headroomAlpha, used by policies that balance weight-based
+   * and load-based considerations in their decisions.
    *
-   * For policies that use this parameter, values close to 1 indicate that
-   * most of the decision should be based on currently observed headroom from
-   * various sub-clusters, values close to zero, indicate that the decision
-   * should be mostly based on weights and practically ignore current load.
+   * For policies that use this parameter, values close to 1 indicate that most
+   * of the decision should be based on currently observed headroom from various
+   * sub-clusters, values close to zero, indicate that the decision should be
+   * mostly based on weights and practically ignore current load.
    *
    * @param headroomAlpha the value to use for balancing.
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53ca351b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/dao/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/dao/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/dao/package-info.java
index 43f5b83..c292e52 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/dao/package-info.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/dao/package-info.java
@@ -17,4 +17,3 @@
  */
 /** DAO objects for serializing/deserializing policy configurations. **/
 package org.apache.hadoop.yarn.server.federation.policies.dao;
-

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53ca351b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/exceptions/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/exceptions/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/exceptions/package-info.java
index 3318da9..ad2d543 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/exceptions/package-info.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/exceptions/package-info.java
@@ -17,4 +17,3 @@
  */
 /** Exceptions for policies. **/
 package org.apache.hadoop.yarn.server.federation.policies.exceptions;
-

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53ca351b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/package-info.java
index 7d9a121..fa3fcc5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/package-info.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/package-info.java
@@ -17,4 +17,3 @@
  */
 /** Federation Policies. **/
 package org.apache.hadoop.yarn.server.federation.policies;
-

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53ca351b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/AbstractRouterPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/AbstractRouterPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/AbstractRouterPolicy.java
new file mode 100644
index 0000000..f49af1d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/AbstractRouterPolicy.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies.router;
+
+import java.util.Map;
+
+import org.apache.hadoop.yarn.server.federation.policies.AbstractConfigurableFederationPolicy;
+import org.apache.hadoop.yarn.server.federation.policies.dao.WeightedPolicyInfo;
+import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterIdInfo;
+
+/**
+ * Base abstract class for {@link FederationRouterPolicy} implementations, that
+ * provides common validation for reinitialization.
+ */
+public abstract class AbstractRouterPolicy extends
+    AbstractConfigurableFederationPolicy implements FederationRouterPolicy {
+
+  @Override
+  public void validate(WeightedPolicyInfo newPolicyInfo)
+      throws FederationPolicyInitializationException {
+    super.validate(newPolicyInfo);
+    Map<SubClusterIdInfo, Float> newWeights =
+        newPolicyInfo.getRouterPolicyWeights();
+    if (newWeights == null || newWeights.size() < 1) {
+      throw new FederationPolicyInitializationException(
+          "Weight vector cannot be null/empty.");
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53ca351b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/BaseWeightedRouterPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/BaseWeightedRouterPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/BaseWeightedRouterPolicy.java
deleted file mode 100644
index e888979..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/BaseWeightedRouterPolicy.java
+++ /dev/null
@@ -1,150 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.server.federation.policies.router;
-
-import org.apache.hadoop.yarn.exceptions.YarnException;
-import org.apache.hadoop.yarn.server.federation.policies.FederationPolicyInitializationContext;
-import org.apache.hadoop.yarn.server.federation.policies.FederationPolicyInitializationContextValidator;
-import org.apache.hadoop.yarn.server.federation.policies.dao.WeightedPolicyInfo;
-import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
-import org.apache.hadoop.yarn.server.federation.policies.exceptions.NoActiveSubclustersException;
-import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
-import org.apache.hadoop.yarn.server.federation.store.records.SubClusterIdInfo;
-import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
-
-import java.util.Map;
-
-/**
- * Abstract class provides common validation of reinitialize(), for all
- * policies that are "weight-based".
- */
-public abstract class BaseWeightedRouterPolicy
-    implements FederationRouterPolicy {
-
-  private WeightedPolicyInfo policyInfo = null;
-  private FederationPolicyInitializationContext policyContext;
-
-  public BaseWeightedRouterPolicy() {
-  }
-
-  @Override
-  public void reinitialize(FederationPolicyInitializationContext
-      federationPolicyContext)
-      throws FederationPolicyInitializationException {
-    FederationPolicyInitializationContextValidator
-        .validate(federationPolicyContext, this.getClass().getCanonicalName());
-
-    // perform consistency checks
-    WeightedPolicyInfo newPolicyInfo = WeightedPolicyInfo
-        .fromByteBuffer(
-            federationPolicyContext.getSubClusterPolicyConfiguration()
-                .getParams());
-
-    // if nothing has changed skip the rest of initialization
-    if (policyInfo != null && policyInfo.equals(newPolicyInfo)) {
-      return;
-    }
-
-    validate(newPolicyInfo);
-    setPolicyInfo(newPolicyInfo);
-    this.policyContext = federationPolicyContext;
-  }
-
-  /**
-   * Overridable validation step for the policy configuration.
-   * @param newPolicyInfo the configuration to test.
-   * @throws FederationPolicyInitializationException if the configuration is
-   * not valid.
-   */
-  public void validate(WeightedPolicyInfo newPolicyInfo) throws
-      FederationPolicyInitializationException {
-    if (newPolicyInfo == null) {
-      throw new FederationPolicyInitializationException("The policy to "
-          + "validate should not be null.");
-    }
-    Map<SubClusterIdInfo, Float> newWeights =
-        newPolicyInfo.getRouterPolicyWeights();
-    if (newWeights == null || newWeights.size() < 1) {
-      throw new FederationPolicyInitializationException(
-          "Weight vector cannot be null/empty.");
-    }
-  }
-
-
-  /**
-   * Getter method for the configuration weights.
-   *
-   * @return the {@link WeightedPolicyInfo} representing the policy
-   * configuration.
-   */
-  public WeightedPolicyInfo getPolicyInfo() {
-    return policyInfo;
-  }
-
-  /**
-   * Setter method for the configuration weights.
-   *
-   * @param policyInfo the {@link WeightedPolicyInfo} representing the policy
-   *                   configuration.
-   */
-  public void setPolicyInfo(
-      WeightedPolicyInfo policyInfo) {
-    this.policyInfo = policyInfo;
-  }
-
-  /**
-   * Getter method for the {@link FederationPolicyInitializationContext}.
-   * @return the context for this policy.
-   */
-  public FederationPolicyInitializationContext getPolicyContext() {
-    return policyContext;
-  }
-
-  /**
-   * Setter method for the {@link FederationPolicyInitializationContext}.
-   * @param policyContext the context to assign to this policy.
-   */
-  public void setPolicyContext(
-      FederationPolicyInitializationContext policyContext) {
-    this.policyContext = policyContext;
-  }
-
-  /**
-   * This methods gets active subclusters map from the {@code
-   * FederationStateStoreFacade} and validate it not being null/empty.
-   *
-   * @return the map of ids to info for all active subclusters.
-   * @throws YarnException if we can't get the list.
-   */
-  protected Map<SubClusterId, SubClusterInfo> getActiveSubclusters()
-      throws YarnException {
-
-    Map<SubClusterId, SubClusterInfo> activeSubclusters = getPolicyContext()
-        .getFederationStateStoreFacade().getSubClusters(true);
-
-    if (activeSubclusters == null || activeSubclusters.size() < 1) {
-      throw new NoActiveSubclustersException(
-          "Zero active subclusters, cannot pick where to send job.");
-    }
-    return activeSubclusters;
-  }
-
-
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53ca351b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/FederationRouterPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/FederationRouterPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/FederationRouterPolicy.java
index 42c86cc..90ea0a8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/FederationRouterPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/FederationRouterPolicy.java
@@ -35,11 +35,10 @@ public interface FederationRouterPolicy extends ConfigurableFederationPolicy {
    * @param appSubmissionContext the context for the app being submitted.
    *
    * @return the sub-cluster as identified by {@link SubClusterId} to route the
-   * request to.
+   *         request to.
    *
    * @throws YarnException if the policy cannot determine a viable subcluster.
    */
   SubClusterId getHomeSubcluster(
-      ApplicationSubmissionContext appSubmissionContext)
-      throws YarnException;
+      ApplicationSubmissionContext appSubmissionContext) throws YarnException;
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[36/42] hadoop git commit: YARN-6370. Properly handle rack requests for non-active subclusters in LocalityMulticastAMRMProxyPolicy. (Contributed by Botong Huang via curino).

Posted by su...@apache.org.
YARN-6370. Properly handle rack requests for non-active subclusters in LocalityMulticastAMRMProxyPolicy. (Contributed by Botong Huang via curino).


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/54757ae4
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/54757ae4
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/54757ae4

Branch: refs/heads/YARN-2915
Commit: 54757ae41bd2a0230903071ce4a1333a851bd352
Parents: a3450fa
Author: Carlo Curino <cu...@apache.org>
Authored: Wed Mar 22 13:53:47 2017 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon May 8 16:59:13 2017 -0700

----------------------------------------------------------------------
 .../LocalityMulticastAMRMProxyPolicy.java       |  6 ++-
 .../TestLocalityMulticastAMRMProxyPolicy.java   | 53 +++++++++++++-------
 2 files changed, 41 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/54757ae4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/LocalityMulticastAMRMProxyPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/LocalityMulticastAMRMProxyPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/LocalityMulticastAMRMProxyPolicy.java
index 6f97a51..454962f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/LocalityMulticastAMRMProxyPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/LocalityMulticastAMRMProxyPolicy.java
@@ -261,7 +261,11 @@ public class LocalityMulticastAMRMProxyPolicy extends AbstractAMRMProxyPolicy {
 
       // If home-subcluster is not active, ignore node/rack request
       if (bookkeeper.isActiveAndEnabled(homeSubcluster)) {
-        bookkeeper.addLocalizedNodeRR(homeSubcluster, rr);
+        if (targetIds != null && targetIds.size() > 0) {
+          bookkeeper.addRackRR(homeSubcluster, rr);
+        } else {
+          bookkeeper.addLocalizedNodeRR(homeSubcluster, rr);
+        }
       } else {
         if (LOG.isDebugEnabled()) {
           LOG.debug("The homeSubCluster (" + homeSubcluster + ") we are "

http://git-wip-us.apache.org/repos/asf/hadoop/blob/54757ae4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/TestLocalityMulticastAMRMProxyPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/TestLocalityMulticastAMRMProxyPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/TestLocalityMulticastAMRMProxyPolicy.java
index 5b3cf74..6e3a2f1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/TestLocalityMulticastAMRMProxyPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/TestLocalityMulticastAMRMProxyPolicy.java
@@ -339,19 +339,20 @@ public class TestLocalityMulticastAMRMProxyPolicy
     validateSplit(response, resourceRequests);
     prettyPrintRequests(response);
 
-    // we expect 4 entry for home subcluster (3 for request-id 4, and a part
-    // of the broadcast of request-id 2
-    checkExpectedAllocation(response, getHomeSubCluster().getId(), 4, 23);
+    // we expect 7 entries for home subcluster (2 for request-id 4, 3 for
+    // request-id 5, and a part of the broadcast of request-id 2
+    checkExpectedAllocation(response, getHomeSubCluster().getId(), 7, 29);
 
-    // for subcluster0 we expect 3 entry from request-id 0, and 3 from
-    // request-id 3, as well as part of the request-id 2 broadast
-    checkExpectedAllocation(response, "subcluster0", 7, 26);
+    // for subcluster0 we expect 10 entries, 3 from request-id 0, and 3 from
+    // request-id 3, 3 entries from request-id 5, as well as part of the
+    // request-id 2 broadast
+    checkExpectedAllocation(response, "subcluster0", 10, 32);
 
-    // we expect 5 entry for subcluster1 (4 from request-id 1, and part
+    // we expect 5 entries for subcluster1 (4 from request-id 1, and part
     // of the broadcast of request-id 2
     checkExpectedAllocation(response, "subcluster1", 5, 26);
 
-    // sub-cluster 2 should contain 3 entry from request-id 1 and 1 from the
+    // sub-cluster 2 should contain 3 entries from request-id 1 and 1 from the
     // broadcast of request-id 2, and no request-id 0
     checkExpectedAllocation(response, "subcluster2", 4, 23);
 
@@ -364,28 +365,33 @@ public class TestLocalityMulticastAMRMProxyPolicy
 
     // check that the allocations that show up are what expected
     for (ResourceRequest rr : response.get(getHomeSubCluster())) {
-      Assert.assertTrue(rr.getAllocationRequestId() == 4L
-          || rr.getAllocationRequestId() == 2L);
-    }
-
-    for (ResourceRequest rr : response.get(getHomeSubCluster())) {
-      Assert.assertTrue(rr.getAllocationRequestId() != 1L);
+      Assert.assertTrue(
+          rr.getAllocationRequestId() == 2L || rr.getAllocationRequestId() == 4L
+              || rr.getAllocationRequestId() == 5L);
     }
 
     List<ResourceRequest> rrs =
         response.get(SubClusterId.newInstance("subcluster0"));
     for (ResourceRequest rr : rrs) {
       Assert.assertTrue(rr.getAllocationRequestId() != 1L);
+      Assert.assertTrue(rr.getAllocationRequestId() != 4L);
+    }
+
+    for (ResourceRequest rr : response
+        .get(SubClusterId.newInstance("subcluster1"))) {
+      Assert.assertTrue(rr.getAllocationRequestId() == 1L
+          || rr.getAllocationRequestId() == 2L);
     }
 
     for (ResourceRequest rr : response
         .get(SubClusterId.newInstance("subcluster2"))) {
-      Assert.assertTrue(rr.getAllocationRequestId() != 0L);
+      Assert.assertTrue(rr.getAllocationRequestId() == 1L
+          || rr.getAllocationRequestId() == 2L);
     }
 
     for (ResourceRequest rr : response
         .get(SubClusterId.newInstance("subcluster5"))) {
-      Assert.assertTrue(rr.getAllocationRequestId() >= 2);
+      Assert.assertTrue(rr.getAllocationRequestId() == 2);
       Assert.assertTrue(rr.getRelaxLocality());
     }
   }
@@ -555,7 +561,7 @@ public class TestLocalityMulticastAMRMProxyPolicy
     out.add(FederationPoliciesTestUtil.createResourceRequest(1L,
         "subcluster2-rack3", 1024, 1, 1, 1, null, false));
     out.add(FederationPoliciesTestUtil.createResourceRequest(1L,
-        ResourceRequest.ANY, 1024, 1, 1, 2, null, false));
+        ResourceRequest.ANY, 1024, 1, 1, 3, null, false));
 
     // create a non-local ANY request that can span anything
     out.add(FederationPoliciesTestUtil.createResourceRequest(2L,
@@ -578,6 +584,19 @@ public class TestLocalityMulticastAMRMProxyPolicy
     out.add(FederationPoliciesTestUtil.createResourceRequest(4L,
         ResourceRequest.ANY, 1024, 1, 1, 1, null, false));
 
+    // create a request of two hosts, an unknown node and a known node, both in
+    // a known rack, and expect the unknown node to show up in homesubcluster
+    out.add(FederationPoliciesTestUtil.createResourceRequest(5L,
+        "subcluster0-rack0-host0", 1024, 1, 1, 2, null, false));
+    out.add(FederationPoliciesTestUtil.createResourceRequest(5L,
+        "subcluster0-rack0", 1024, 1, 1, 2, null, false));
+    out.add(FederationPoliciesTestUtil.createResourceRequest(5L, "node4", 1024,
+        1, 1, 2, null, false));
+    out.add(FederationPoliciesTestUtil.createResourceRequest(5L, "rack2", 1024,
+        1, 1, 2, null, false));
+    out.add(FederationPoliciesTestUtil.createResourceRequest(5L,
+        ResourceRequest.ANY, 1024, 1, 1, 4, null, false));
+
     return out;
   }
 }
\ No newline at end of file


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[13/42] hadoop git commit: YARN-5634. Simplify initialization/use of RouterPolicy via a RouterPolicyFacade. (Carlo Curino via Subru).

Posted by su...@apache.org.
YARN-5634. Simplify initialization/use of RouterPolicy via a RouterPolicyFacade. (Carlo Curino via Subru).


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/03801af8
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/03801af8
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/03801af8

Branch: refs/heads/YARN-2915
Commit: 03801af865b5bf876e4c68f8bb0701d1c5a8759f
Parents: c8ab396
Author: Subru Krishnan <su...@apache.org>
Authored: Wed Nov 16 19:39:25 2016 -0800
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon May 8 16:59:12 2017 -0700

----------------------------------------------------------------------
 .../dev-support/findbugs-exclude.xml            |   9 +
 .../hadoop/yarn/conf/YarnConfiguration.java     |  13 +
 .../yarn/conf/TestYarnConfigurationFields.java  |  12 +
 ...ionPolicyInitializationContextValidator.java |   2 +-
 .../PriorityBroadcastPolicyManager.java         |  66 +++++
 .../federation/policies/RouterPolicyFacade.java | 266 +++++++++++++++++++
 .../policies/dao/WeightedPolicyInfo.java        |   6 +-
 .../utils/FederationStateStoreFacade.java       |  16 +-
 .../TestPriorityBroadcastPolicyManager.java     |  72 +++++
 .../policies/TestRouterPolicyFacade.java        | 220 +++++++++++++++
 .../utils/FederationStateStoreTestUtil.java     |  22 +-
 11 files changed, 693 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/03801af8/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml b/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
index bbd03a9..ee51094 100644
--- a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
+++ b/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
@@ -310,6 +310,15 @@
     <Bug pattern="IS2_INCONSISTENT_SYNC"/>
   </Match>
 
+  <Match>
+    <Class name="org.apache.hadoop.yarn.server.federation.policies.RouterPolicyFacade"/>
+    <Or>
+      <Field name="globalConfMap"/>
+      <Field name="globalPolicyMap"/>
+    </Or>
+    <Bug pattern="IS2_INCONSISTENT_SYNC"/>
+  </Match>
+
   <!-- Don't care if putIfAbsent value is ignored -->
   <Match>
     <Package name="org.apache.hadoop.yarn.factories.impl.pb" />

http://git-wip-us.apache.org/repos/asf/hadoop/blob/03801af8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
index 9930e3a..a593a2c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
@@ -2572,6 +2572,19 @@ public class YarnConfiguration extends Configuration {
   public static final String FEDERATION_MACHINE_LIST =
       FEDERATION_PREFIX + "machine-list";
 
+  public static final String DEFAULT_FEDERATION_POLICY_KEY = "*";
+
+  public static final String FEDERATION_POLICY_MANAGER = FEDERATION_PREFIX
+      + "policy-manager";
+
+  public static final String DEFAULT_FEDERATION_POLICY_MANAGER = "org.apache"
+      + ".hadoop.yarn.server.federation.policies.UniformBroadcastPolicyManager";
+
+  public static final String FEDERATION_POLICY_MANAGER_PARAMS =
+      FEDERATION_PREFIX + "policy-manager-params";
+
+  public static final String DEFAULT_FEDERATION_POLICY_MANAGER_PARAMS = "";
+
   ////////////////////////////////
   // Other Configs
   ////////////////////////////////

http://git-wip-us.apache.org/repos/asf/hadoop/blob/03801af8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
index 3f3a06c..6e33c0a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
@@ -78,6 +78,18 @@ public class TestYarnConfigurationFields extends TestConfigurationFieldsBase {
     configurationPropsToSkipCompare
         .add(YarnConfiguration.RM_EPOCH);
 
+    // Federation policies configs to be ignored
+    configurationPropsToSkipCompare
+        .add(YarnConfiguration.FEDERATION_POLICY_MANAGER);
+    configurationPropsToSkipCompare
+        .add(YarnConfiguration.FEDERATION_POLICY_MANAGER_PARAMS);
+    configurationPropsToSkipCompare
+        .add(YarnConfiguration.DEFAULT_FEDERATION_POLICY_KEY);
+    configurationPropsToSkipCompare
+        .add(YarnConfiguration.DEFAULT_FEDERATION_POLICY_MANAGER);
+    configurationPropsToSkipCompare
+        .add(YarnConfiguration.DEFAULT_FEDERATION_POLICY_MANAGER_PARAMS);
+
     // Ignore blacklisting nodes for AM failures feature since it is still a
     // "work in progress"
     configurationPropsToSkipCompare.add(YarnConfiguration.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/03801af8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyInitializationContextValidator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyInitializationContextValidator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyInitializationContextValidator.java
index 1b83bbc..3c44e7e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyInitializationContextValidator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyInitializationContextValidator.java
@@ -57,7 +57,7 @@ public final class FederationPolicyInitializationContextValidator {
 
     if (policyContext.getSubClusterPolicyConfiguration() == null) {
       throw new FederationPolicyInitializationException(
-          "The FederationSubclusterResolver provided is null. Cannot "
+          "The SubClusterPolicyConfiguration provided is null. Cannot "
               + "reinitalize successfully.");
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/03801af8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/PriorityBroadcastPolicyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/PriorityBroadcastPolicyManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/PriorityBroadcastPolicyManager.java
new file mode 100644
index 0000000..ebdcf42
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/PriorityBroadcastPolicyManager.java
@@ -0,0 +1,66 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.yarn.server.federation.policies.amrmproxy.BroadcastAMRMProxyPolicy;
+import org.apache.hadoop.yarn.server.federation.policies.dao.WeightedPolicyInfo;
+import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
+import org.apache.hadoop.yarn.server.federation.policies.router.PriorityRouterPolicy;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyConfiguration;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Policy that allows operator to configure "weights" for routing. This picks a
+ * {@link PriorityRouterPolicy} for the router and a
+ * {@link BroadcastAMRMProxyPolicy} for the amrmproxy as they are designed to
+ * work together.
+ */
+public class PriorityBroadcastPolicyManager extends AbstractPolicyManager {
+
+  private WeightedPolicyInfo weightedPolicyInfo;
+
+  public PriorityBroadcastPolicyManager() {
+    // this structurally hard-codes two compatible policies for Router and
+    // AMRMProxy.
+    routerFederationPolicy = PriorityRouterPolicy.class;
+    amrmProxyFederationPolicy = BroadcastAMRMProxyPolicy.class;
+    weightedPolicyInfo = new WeightedPolicyInfo();
+  }
+
+  @Override
+  public SubClusterPolicyConfiguration serializeConf()
+      throws FederationPolicyInitializationException {
+    ByteBuffer buf = weightedPolicyInfo.toByteBuffer();
+    return SubClusterPolicyConfiguration.newInstance(getQueue(),
+        this.getClass().getCanonicalName(), buf);
+  }
+
+  @VisibleForTesting
+  public WeightedPolicyInfo getWeightedPolicyInfo() {
+    return weightedPolicyInfo;
+  }
+
+  @VisibleForTesting
+  public void setWeightedPolicyInfo(WeightedPolicyInfo weightedPolicyInfo) {
+    this.weightedPolicyInfo = weightedPolicyInfo;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/03801af8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/RouterPolicyFacade.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/RouterPolicyFacade.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/RouterPolicyFacade.java
new file mode 100644
index 0000000..a3fd15a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/RouterPolicyFacade.java
@@ -0,0 +1,266 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies;
+
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.federation.policies.amrmproxy.LocalityMulticastAMRMProxyPolicy;
+import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyException;
+import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
+import org.apache.hadoop.yarn.server.federation.policies.router.FederationRouterPolicy;
+import org.apache.hadoop.yarn.server.federation.resolver.SubClusterResolver;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyConfiguration;
+import org.apache.hadoop.yarn.server.federation.utils.FederationStateStoreFacade;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * This class provides a facade to the policy subsystem, and handles the
+ * lifecycle of policies (e.g., refresh from remote, default behaviors etc.).
+ */
+public class RouterPolicyFacade {
+
+  private static final Log LOG =
+      LogFactory.getLog(LocalityMulticastAMRMProxyPolicy.class);
+
+  private final SubClusterResolver subClusterResolver;
+  private final FederationStateStoreFacade federationFacade;
+  private Map<String, SubClusterPolicyConfiguration> globalConfMap;
+
+  @VisibleForTesting
+  Map<String, FederationRouterPolicy> globalPolicyMap;
+
+  public RouterPolicyFacade(YarnConfiguration conf,
+      FederationStateStoreFacade facade, SubClusterResolver resolver,
+      SubClusterId homeSubcluster)
+      throws FederationPolicyInitializationException {
+
+    this.federationFacade = facade;
+    this.subClusterResolver = resolver;
+    this.globalConfMap = new ConcurrentHashMap<>();
+    this.globalPolicyMap = new ConcurrentHashMap<>();
+
+    // load default behavior from store if possible
+    String defaulKey = YarnConfiguration.DEFAULT_FEDERATION_POLICY_KEY;
+    SubClusterPolicyConfiguration configuration = null;
+    try {
+      configuration = federationFacade.getPolicyConfiguration(defaulKey);
+    } catch (YarnException e) {
+      LOG.warn("No fallback behavior defined in store, defaulting to XML "
+          + "configuration fallback behavior.");
+    }
+
+    // or from XML conf otherwise.
+    if (configuration == null) {
+      String defaultFederationPolicyManager =
+          conf.get(YarnConfiguration.FEDERATION_POLICY_MANAGER,
+              YarnConfiguration.DEFAULT_FEDERATION_POLICY_MANAGER);
+      String defaultPolicyParamString =
+          conf.get(YarnConfiguration.FEDERATION_POLICY_MANAGER_PARAMS,
+              YarnConfiguration.DEFAULT_FEDERATION_POLICY_MANAGER_PARAMS);
+      ByteBuffer defaultPolicyParam = ByteBuffer
+          .wrap(defaultPolicyParamString.getBytes(StandardCharsets.UTF_8));
+
+      configuration = SubClusterPolicyConfiguration.newInstance(defaulKey,
+          defaultFederationPolicyManager, defaultPolicyParam);
+    }
+
+    // construct the required policy manager
+    FederationPolicyInitializationContext fallbackContext =
+        new FederationPolicyInitializationContext(configuration,
+            subClusterResolver, federationFacade, homeSubcluster);
+    FederationPolicyManager fallbackPolicyManager =
+        instantiatePolicyManager(configuration.getType());
+    fallbackPolicyManager.setQueue(defaulKey);
+
+    // add to the cache the fallback behavior
+    globalConfMap.put(defaulKey,
+        fallbackContext.getSubClusterPolicyConfiguration());
+    globalPolicyMap.put(defaulKey,
+        fallbackPolicyManager.getRouterPolicy(fallbackContext, null));
+
+  }
+
+  /**
+   * This method provides a wrapper of all policy functionalities for routing .
+   * Internally it manages configuration changes, and policy init/reinit.
+   *
+   * @param appSubmissionContext the application to route.
+   *
+   * @return the id of the subcluster that will be the "home" for this
+   *         application.
+   *
+   * @throws YarnException if there are issues initializing policies, or no
+   *           valid sub-cluster id could be found for this app.
+   */
+  public SubClusterId getHomeSubcluster(
+      ApplicationSubmissionContext appSubmissionContext) throws YarnException {
+
+    // the maps are concurrent, but we need to protect from reset()
+    // reinitialization mid-execution by creating a new reference local to this
+    // method.
+    Map<String, SubClusterPolicyConfiguration> cachedConfs = globalConfMap;
+    Map<String, FederationRouterPolicy> policyMap = globalPolicyMap;
+
+    if (appSubmissionContext == null) {
+      throw new FederationPolicyException(
+          "The ApplicationSubmissionContext " + "cannot be null.");
+    }
+
+    String queue = appSubmissionContext.getQueue();
+
+    // respecting YARN behavior we assume default queue if the queue is not
+    // specified. This also ensures that "null" can be used as a key to get the
+    // default behavior.
+    if (queue == null) {
+      queue = YarnConfiguration.DEFAULT_QUEUE_NAME;
+    }
+
+    // the facade might cache this request, based on its parameterization
+    SubClusterPolicyConfiguration configuration = null;
+
+    try {
+      configuration = federationFacade.getPolicyConfiguration(queue);
+    } catch (YarnException e) {
+      LOG.debug(e);
+    }
+
+    // If there is no policy configured for this queue, fallback to the baseline
+    // policy that is configured either in the store or via XML config (and
+    // cached)
+    if (configuration == null) {
+      try {
+        LOG.warn("There is no policies configured for queue: " + queue + " we"
+            + " fallback to default policy for: "
+            + YarnConfiguration.DEFAULT_FEDERATION_POLICY_KEY);
+
+        queue = YarnConfiguration.DEFAULT_FEDERATION_POLICY_KEY;
+        configuration = federationFacade.getPolicyConfiguration(
+            YarnConfiguration.DEFAULT_FEDERATION_POLICY_KEY);
+      } catch (YarnException e) {
+        // the fallback is not configure via store, but via XML, using
+        // previously loaded configuration.
+        configuration =
+            cachedConfs.get(YarnConfiguration.DEFAULT_FEDERATION_POLICY_KEY);
+      }
+    }
+
+    // if the configuration has changed since last loaded, reinit the policy
+    // based on current configuration
+    if (!cachedConfs.containsKey(queue)
+        || !cachedConfs.get(queue).equals(configuration)) {
+      singlePolicyReinit(policyMap, cachedConfs, queue, configuration);
+    }
+
+    FederationRouterPolicy policy = policyMap.get(queue);
+    if (policy == null) {
+      // this should never happen, as the to maps are updated together
+      throw new FederationPolicyException("No FederationRouterPolicy found "
+          + "for queue: " + appSubmissionContext.getQueue() + " (for "
+          + "application: " + appSubmissionContext.getApplicationId() + ") "
+          + "and no default specified.");
+    }
+
+    return policy.getHomeSubcluster(appSubmissionContext);
+  }
+
+  /**
+   * This method reinitializes a policy and loads it in the policyMap.
+   *
+   * @param queue the queue to initialize a policy for.
+   * @param conf the configuration to use for initalization.
+   *
+   * @throws FederationPolicyInitializationException if initialization fails.
+   */
+  private void singlePolicyReinit(Map<String, FederationRouterPolicy> policyMap,
+      Map<String, SubClusterPolicyConfiguration> cachedConfs, String queue,
+      SubClusterPolicyConfiguration conf)
+      throws FederationPolicyInitializationException {
+
+    FederationPolicyInitializationContext context =
+        new FederationPolicyInitializationContext(conf, subClusterResolver,
+            federationFacade, null);
+    String newType = context.getSubClusterPolicyConfiguration().getType();
+    FederationRouterPolicy routerPolicy = policyMap.get(queue);
+
+    FederationPolicyManager federationPolicyManager =
+        instantiatePolicyManager(newType);
+    // set queue, reinit policy if required (implementation lazily check
+    // content of conf), and cache it
+    federationPolicyManager.setQueue(queue);
+    routerPolicy =
+        federationPolicyManager.getRouterPolicy(context, routerPolicy);
+
+    // we need the two put to be atomic (across multiple threads invoking
+    // this and reset operations)
+    synchronized (this) {
+      policyMap.put(queue, routerPolicy);
+      cachedConfs.put(queue, conf);
+    }
+  }
+
+  private static FederationPolicyManager instantiatePolicyManager(
+      String newType) throws FederationPolicyInitializationException {
+    FederationPolicyManager federationPolicyManager = null;
+    try {
+      // create policy instance and set queue
+      Class c = Class.forName(newType);
+      federationPolicyManager = (FederationPolicyManager) c.newInstance();
+    } catch (ClassNotFoundException e) {
+      throw new FederationPolicyInitializationException(e);
+    } catch (InstantiationException e) {
+      throw new FederationPolicyInitializationException(e);
+    } catch (IllegalAccessException e) {
+      throw new FederationPolicyInitializationException(e);
+    }
+    return federationPolicyManager;
+  }
+
+  /**
+   * This method flushes all cached configurations and policies. This should be
+   * invoked if the facade remains activity after very large churn of queues in
+   * the system.
+   */
+  public synchronized void reset() {
+
+    // remember the fallBack
+    SubClusterPolicyConfiguration conf =
+        globalConfMap.get(YarnConfiguration.DEFAULT_FEDERATION_POLICY_KEY);
+    FederationRouterPolicy policy =
+        globalPolicyMap.get(YarnConfiguration.DEFAULT_FEDERATION_POLICY_KEY);
+
+    globalConfMap = new ConcurrentHashMap<>();
+    globalPolicyMap = new ConcurrentHashMap<>();
+
+    // add to the cache a fallback with keyword null
+    globalConfMap.put(YarnConfiguration.DEFAULT_FEDERATION_POLICY_KEY, conf);
+    globalPolicyMap.put(YarnConfiguration.DEFAULT_FEDERATION_POLICY_KEY,
+        policy);
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/03801af8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/dao/WeightedPolicyInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/dao/WeightedPolicyInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/dao/WeightedPolicyInfo.java
index 62eb03b..e7b8afe 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/dao/WeightedPolicyInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/dao/WeightedPolicyInfo.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.yarn.server.federation.policies.dao;
 import java.io.StringReader;
 import java.io.StringWriter;
 import java.nio.ByteBuffer;
-import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
 import java.util.HashMap;
 import java.util.Map;
 
@@ -100,7 +100,7 @@ public class WeightedPolicyInfo {
       JSONUnmarshaller unmarshaller = jsonjaxbContext.createJSONUnmarshaller();
       final byte[] bytes = new byte[bb.remaining()];
       bb.get(bytes);
-      String params = new String(bytes, Charset.forName("UTF-8"));
+      String params = new String(bytes, StandardCharsets.UTF_8);
 
       WeightedPolicyInfo weightedPolicyInfo = unmarshaller.unmarshalFromJSON(
           new StringReader(params), WeightedPolicyInfo.class);
@@ -164,7 +164,7 @@ public class WeightedPolicyInfo {
     }
     try {
       String s = toJSONString();
-      return ByteBuffer.wrap(s.getBytes(Charset.forName("UTF-8")));
+      return ByteBuffer.wrap(s.getBytes(StandardCharsets.UTF_8));
     } catch (JAXBException j) {
       throw new FederationPolicyInitializationException(j);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/03801af8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java
index 66a0b60..9b794de 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java
@@ -56,6 +56,7 @@ import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterInfoR
 import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterPoliciesConfigurationsRequest;
 import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterPoliciesConfigurationsResponse;
 import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterPolicyConfigurationRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterPolicyConfigurationResponse;
 import org.apache.hadoop.yarn.server.federation.store.records.GetSubClustersInfoRequest;
 import org.apache.hadoop.yarn.server.federation.store.records.GetSubClustersInfoResponse;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
@@ -262,12 +263,17 @@ public final class FederationStateStoreFacade {
     if (isCachingEnabled()) {
       return getPoliciesConfigurations().get(queue);
     } else {
-      return stateStore
-          .getPolicyConfiguration(
-              GetSubClusterPolicyConfigurationRequest.newInstance(queue))
-          .getPolicyConfiguration();
-    }
 
+      GetSubClusterPolicyConfigurationResponse response =
+          stateStore.getPolicyConfiguration(
+              GetSubClusterPolicyConfigurationRequest.newInstance(queue));
+      if (response == null) {
+        throw new YarnException("The stateStore returned a null for "
+            + "GetSubClusterPolicyConfigurationResponse for queue " + queue);
+      } else {
+        return response.getPolicyConfiguration();
+      }
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/03801af8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestPriorityBroadcastPolicyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestPriorityBroadcastPolicyManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestPriorityBroadcastPolicyManager.java
new file mode 100644
index 0000000..5e5bc83
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestPriorityBroadcastPolicyManager.java
@@ -0,0 +1,72 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.yarn.server.federation.policies.amrmproxy.BroadcastAMRMProxyPolicy;
+import org.apache.hadoop.yarn.server.federation.policies.dao.WeightedPolicyInfo;
+import org.apache.hadoop.yarn.server.federation.policies.router.PriorityRouterPolicy;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterIdInfo;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Simple test of {@link PriorityBroadcastPolicyManager}.
+ */
+public class TestPriorityBroadcastPolicyManager extends BasePolicyManagerTest {
+
+  private WeightedPolicyInfo policyInfo;
+
+  @Before
+  public void setup() {
+    // configure a policy
+
+    wfp = new PriorityBroadcastPolicyManager();
+    wfp.setQueue("queue1");
+    SubClusterId sc1 = SubClusterId.newInstance("sc1");
+    SubClusterId sc2 = SubClusterId.newInstance("sc2");
+    policyInfo = new WeightedPolicyInfo();
+
+    Map<SubClusterIdInfo, Float> routerWeights = new HashMap<>();
+    routerWeights.put(new SubClusterIdInfo(sc1), 0.2f);
+    routerWeights.put(new SubClusterIdInfo(sc2), 0.8f);
+    policyInfo.setRouterPolicyWeights(routerWeights);
+
+    ((PriorityBroadcastPolicyManager) wfp).setWeightedPolicyInfo(policyInfo);
+
+    // set expected params that the base test class will use for tests
+    expectedPolicyManager = PriorityBroadcastPolicyManager.class;
+    expectedAMRMProxyPolicy = BroadcastAMRMProxyPolicy.class;
+    expectedRouterPolicy = PriorityRouterPolicy.class;
+  }
+
+  @Test
+  public void testPolicyInfoSetCorrectly() throws Exception {
+    serializeAndDeserializePolicyManager(wfp, expectedPolicyManager,
+        expectedAMRMProxyPolicy, expectedRouterPolicy);
+
+    // check the policyInfo propagates through ser/der correctly
+    Assert.assertEquals(
+        ((PriorityBroadcastPolicyManager) wfp).getWeightedPolicyInfo(),
+        policyInfo);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/03801af8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestRouterPolicyFacade.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestRouterPolicyFacade.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestRouterPolicyFacade.java
new file mode 100644
index 0000000..4975a9f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/TestRouterPolicyFacade.java
@@ -0,0 +1,220 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.federation.policies.dao.WeightedPolicyInfo;
+import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
+import org.apache.hadoop.yarn.server.federation.policies.router.PriorityRouterPolicy;
+import org.apache.hadoop.yarn.server.federation.policies.router.UniformRandomRouterPolicy;
+import org.apache.hadoop.yarn.server.federation.resolver.SubClusterResolver;
+import org.apache.hadoop.yarn.server.federation.store.FederationStateStore;
+import org.apache.hadoop.yarn.server.federation.store.impl.MemoryFederationStateStore;
+import org.apache.hadoop.yarn.server.federation.store.records.SetSubClusterPolicyConfigurationRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterIdInfo;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyConfiguration;
+import org.apache.hadoop.yarn.server.federation.utils.FederationPoliciesTestUtil;
+import org.apache.hadoop.yarn.server.federation.utils.FederationStateStoreFacade;
+import org.apache.hadoop.yarn.server.federation.utils.FederationStateStoreTestUtil;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Simple test of {@link RouterPolicyFacade}.
+ */
+public class TestRouterPolicyFacade {
+
+  private RouterPolicyFacade routerFacade;
+  private List<SubClusterId> subClusterIds;
+  private FederationStateStore store;
+  private String queue1 = "queue1";
+  private String defQueueKey = YarnConfiguration.DEFAULT_FEDERATION_POLICY_KEY;
+
+  @Before
+  public void setup() throws YarnException {
+
+    // setting up a store and its facade (with caching off)
+    FederationStateStoreFacade fedFacade =
+        FederationStateStoreFacade.getInstance();
+    YarnConfiguration conf = new YarnConfiguration();
+    conf.set(YarnConfiguration.FEDERATION_CACHE_TIME_TO_LIVE_SECS, "0");
+    store = new MemoryFederationStateStore();
+    store.init(conf);
+    fedFacade.reinitialize(store, conf);
+
+    FederationStateStoreTestUtil storeTestUtil =
+        new FederationStateStoreTestUtil(store);
+    storeTestUtil.registerSubClusters(10);
+
+    subClusterIds = storeTestUtil.getAllSubClusterIds(true);
+    store.setPolicyConfiguration(SetSubClusterPolicyConfigurationRequest
+        .newInstance(getUniformPolicy(queue1)));
+
+    SubClusterResolver resolver = FederationPoliciesTestUtil.initResolver();
+    routerFacade = new RouterPolicyFacade(new YarnConfiguration(), fedFacade,
+        resolver, subClusterIds.get(0));
+  }
+
+  @Test
+  public void testConfigurationUpdate() throws YarnException {
+
+    // in this test we see what happens when the configuration is changed
+    // between calls. We achieve this by changing what is in the store.
+
+    ApplicationSubmissionContext applicationSubmissionContext =
+        mock(ApplicationSubmissionContext.class);
+    when(applicationSubmissionContext.getQueue()).thenReturn(queue1);
+
+    // first call runs using standard UniformRandomRouterPolicy
+    SubClusterId chosen =
+        routerFacade.getHomeSubcluster(applicationSubmissionContext);
+    Assert.assertTrue(subClusterIds.contains(chosen));
+    Assert.assertTrue(routerFacade.globalPolicyMap
+        .get(queue1) instanceof UniformRandomRouterPolicy);
+
+    // then the operator changes how queue1 is routed setting it to
+    // PriorityRouterPolicy with weights favoring the first subcluster in
+    // subClusterIds.
+    store.setPolicyConfiguration(SetSubClusterPolicyConfigurationRequest
+        .newInstance(getPriorityPolicy(queue1)));
+
+    // second call is routed by new policy PriorityRouterPolicy
+    chosen = routerFacade.getHomeSubcluster(applicationSubmissionContext);
+    Assert.assertTrue(chosen.equals(subClusterIds.get(0)));
+    Assert.assertTrue(routerFacade.globalPolicyMap
+        .get(queue1) instanceof PriorityRouterPolicy);
+  }
+
+  @Test
+  public void testGetHomeSubcluster() throws YarnException {
+
+    ApplicationSubmissionContext applicationSubmissionContext =
+        mock(ApplicationSubmissionContext.class);
+    when(applicationSubmissionContext.getQueue()).thenReturn(queue1);
+
+    // the facade only contains the fallback behavior
+    Assert.assertTrue(routerFacade.globalPolicyMap.containsKey(defQueueKey)
+        && routerFacade.globalPolicyMap.size() == 1);
+
+    // when invoked it returns the expected SubClusterId.
+    SubClusterId chosen =
+        routerFacade.getHomeSubcluster(applicationSubmissionContext);
+    Assert.assertTrue(subClusterIds.contains(chosen));
+
+    // now the caching of policies must have added an entry for this queue
+    Assert.assertTrue(routerFacade.globalPolicyMap.size() == 2);
+
+    // after the facade is used the policyMap contains the expected policy type.
+    Assert.assertTrue(routerFacade.globalPolicyMap
+        .get(queue1) instanceof UniformRandomRouterPolicy);
+
+    // the facade is again empty after reset
+    routerFacade.reset();
+    // the facade only contains the fallback behavior
+    Assert.assertTrue(routerFacade.globalPolicyMap.containsKey(defQueueKey)
+        && routerFacade.globalPolicyMap.size() == 1);
+
+  }
+
+  @Test
+  public void testFallbacks() throws YarnException {
+
+    // this tests the behavior of the system when the queue requested is
+    // not configured (or null) and there is no default policy configured
+    // for DEFAULT_FEDERATION_POLICY_KEY (*). This is our second line of
+    // defense.
+
+    ApplicationSubmissionContext applicationSubmissionContext =
+        mock(ApplicationSubmissionContext.class);
+
+    // The facade answers also for non-initialized policies (using the
+    // defaultPolicy)
+    String uninitQueue = "non-initialized-queue";
+    when(applicationSubmissionContext.getQueue()).thenReturn(uninitQueue);
+    SubClusterId chosen =
+        routerFacade.getHomeSubcluster(applicationSubmissionContext);
+    Assert.assertTrue(subClusterIds.contains(chosen));
+    Assert.assertFalse(routerFacade.globalPolicyMap.containsKey(uninitQueue));
+
+    // empty string
+    when(applicationSubmissionContext.getQueue()).thenReturn("");
+    chosen = routerFacade.getHomeSubcluster(applicationSubmissionContext);
+    Assert.assertTrue(subClusterIds.contains(chosen));
+    Assert.assertFalse(routerFacade.globalPolicyMap.containsKey(uninitQueue));
+
+    // null queue also falls back to default
+    when(applicationSubmissionContext.getQueue()).thenReturn(null);
+    chosen = routerFacade.getHomeSubcluster(applicationSubmissionContext);
+    Assert.assertTrue(subClusterIds.contains(chosen));
+    Assert.assertFalse(routerFacade.globalPolicyMap.containsKey(uninitQueue));
+
+  }
+
+  public static SubClusterPolicyConfiguration getUniformPolicy(String queue)
+      throws FederationPolicyInitializationException {
+
+    // we go through standard lifecycle instantiating a policyManager and
+    // configuring it and serializing it to a conf.
+    UniformBroadcastPolicyManager wfp = new UniformBroadcastPolicyManager();
+    wfp.setQueue(queue);
+
+    SubClusterPolicyConfiguration fpc = wfp.serializeConf();
+
+    return fpc;
+  }
+
+  public SubClusterPolicyConfiguration getPriorityPolicy(String queue)
+      throws FederationPolicyInitializationException {
+
+    // we go through standard lifecycle instantiating a policyManager and
+    // configuring it and serializing it to a conf.
+    PriorityBroadcastPolicyManager wfp = new PriorityBroadcastPolicyManager();
+
+    // equal weight to all subcluster
+    Map<SubClusterIdInfo, Float> routerWeights = new HashMap<>();
+    for (SubClusterId s : subClusterIds) {
+      routerWeights.put(new SubClusterIdInfo(s), 0.9f / subClusterIds.size());
+    }
+
+    // beside the first one who gets more weight
+    SubClusterIdInfo favorite = new SubClusterIdInfo((subClusterIds.get(0)));
+    routerWeights.put(favorite, (0.1f + 0.9f / subClusterIds.size()));
+
+    WeightedPolicyInfo policyInfo = new WeightedPolicyInfo();
+    policyInfo.setRouterPolicyWeights(routerWeights);
+    wfp.setWeightedPolicyInfo(policyInfo);
+    wfp.setQueue(queue);
+
+    // serializeConf it in a context
+    SubClusterPolicyConfiguration fpc = wfp.serializeConf();
+
+    return fpc;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/03801af8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreTestUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreTestUtil.java
index c179521..649a61b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreTestUtil.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreTestUtil.java
@@ -18,6 +18,8 @@
 package org.apache.hadoop.yarn.server.federation.utils;
 
 import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
 
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.exceptions.YarnException;
@@ -29,6 +31,7 @@ import org.apache.hadoop.yarn.server.federation.store.records.GetApplicationHome
 import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterInfoRequest;
 import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterPolicyConfigurationRequest;
 import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterPolicyConfigurationResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClustersInfoRequest;
 import org.apache.hadoop.yarn.server.federation.store.records.SetSubClusterPolicyConfigurationRequest;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
@@ -62,8 +65,8 @@ public class FederationStateStoreTestUtil {
     String webAppAddress = "1.2.3.4:4";
 
     return SubClusterInfo.newInstance(subClusterId, amRMAddress,
-        clientRMAddress, rmAdminAddress, webAppAddress, SubClusterState.SC_NEW,
-        CLOCK.getTime(), "capability");
+        clientRMAddress, rmAdminAddress, webAppAddress,
+        SubClusterState.SC_RUNNING, CLOCK.getTime(), "capability");
   }
 
   private void registerSubCluster(SubClusterId subClusterId)
@@ -97,6 +100,21 @@ public class FederationStateStoreTestUtil {
     }
   }
 
+  public List<SubClusterId> getAllSubClusterIds(
+      boolean filterInactiveSubclusters) throws YarnException {
+
+    List<SubClusterInfo> infos = stateStore
+        .getSubClusters(
+            GetSubClustersInfoRequest.newInstance(filterInactiveSubclusters))
+        .getSubClusters();
+    List<SubClusterId> ids = new ArrayList<>();
+    for (SubClusterInfo s : infos) {
+      ids.add(s.getSubClusterId());
+    }
+
+    return ids;
+  }
+
   private SubClusterPolicyConfiguration createSCPolicyConf(String queueName,
       String policyType) {
     return SubClusterPolicyConfiguration.newInstance(queueName, policyType,


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[31/42] hadoop git commit: YARN-5411. Create a proxy chain for ApplicationClientProtocol in the Router. (Giovanni Matteo Fumarola via Subru).

Posted by su...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/0a93a32e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/TestRouterClientRMService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/TestRouterClientRMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/TestRouterClientRMService.java
new file mode 100644
index 0000000..a9c3729
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/TestRouterClientRMService.java
@@ -0,0 +1,210 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.hadoop.yarn.server.router.clientrm;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeLabelsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.MoveApplicationAcrossQueuesResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationResponse;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.router.clientrm.RouterClientRMService.RequestInterceptorChainWrapper;
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Test class to validate the ClientRM Service inside the Router.
+ */
+public class TestRouterClientRMService extends BaseRouterClientRMTest {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestRouterClientRMService.class);
+
+  /**
+   * Tests if the pipeline is created properly.
+   */
+  @Test
+  public void testRequestInterceptorChainCreation() throws Exception {
+    ClientRequestInterceptor root =
+        super.getRouterClientRMService().createRequestInterceptorChain();
+    int index = 0;
+    while (root != null) {
+      // The current pipeline is:
+      // PassThroughClientRequestInterceptor - index = 0
+      // PassThroughClientRequestInterceptor - index = 1
+      // PassThroughClientRequestInterceptor - index = 2
+      // MockClientRequestInterceptor - index = 3
+      switch (index) {
+      case 0: // Fall to the next case
+      case 1: // Fall to the next case
+      case 2:
+        // If index is equal to 0,1 or 2 we fall in this check
+        Assert.assertEquals(PassThroughClientRequestInterceptor.class.getName(),
+            root.getClass().getName());
+        break;
+      case 3:
+        Assert.assertEquals(MockClientRequestInterceptor.class.getName(),
+            root.getClass().getName());
+        break;
+      default:
+        Assert.fail();
+      }
+      root = root.getNextInterceptor();
+      index++;
+    }
+    Assert.assertEquals("The number of interceptors in chain does not match", 4,
+        index);
+  }
+
+  /**
+   * Test if the RouterClientRM forwards all the requests to the MockRM and get
+   * back the responses.
+   */
+  @Test
+  public void testRouterClientRMServiceE2E() throws Exception {
+
+    String user = "test1";
+
+    LOG.info("testRouterClientRMServiceE2E - Get New Application");
+
+    GetNewApplicationResponse responseGetNewApp = getNewApplication(user);
+    Assert.assertNotNull(responseGetNewApp);
+
+    LOG.info("testRouterClientRMServiceE2E - Submit Application");
+
+    SubmitApplicationResponse responseSubmitApp =
+        submitApplication(responseGetNewApp.getApplicationId(), user);
+    Assert.assertNotNull(responseSubmitApp);
+
+    LOG.info("testRouterClientRMServiceE2E - Kill Application");
+
+    KillApplicationResponse responseKillApp =
+        forceKillApplication(responseGetNewApp.getApplicationId(), user);
+    Assert.assertNotNull(responseKillApp);
+
+    LOG.info("testRouterClientRMServiceE2E - Get Cluster Metrics");
+
+    GetClusterMetricsResponse responseGetClusterMetrics =
+        getClusterMetrics(user);
+    Assert.assertNotNull(responseGetClusterMetrics);
+
+    LOG.info("testRouterClientRMServiceE2E - Get Cluster Nodes");
+
+    GetClusterNodesResponse responseGetClusterNodes = getClusterNodes(user);
+    Assert.assertNotNull(responseGetClusterNodes);
+
+    LOG.info("testRouterClientRMServiceE2E - Get Queue Info");
+
+    GetQueueInfoResponse responseGetQueueInfo = getQueueInfo(user);
+    Assert.assertNotNull(responseGetQueueInfo);
+
+    LOG.info("testRouterClientRMServiceE2E - Get Queue User");
+
+    GetQueueUserAclsInfoResponse responseGetQueueUser = getQueueUserAcls(user);
+    Assert.assertNotNull(responseGetQueueUser);
+
+    LOG.info("testRouterClientRMServiceE2E - Get Cluster Node");
+
+    GetClusterNodeLabelsResponse responseGetClusterNode =
+        getClusterNodeLabels(user);
+    Assert.assertNotNull(responseGetClusterNode);
+
+    LOG.info("testRouterClientRMServiceE2E - Move Application Across Queues");
+
+    MoveApplicationAcrossQueuesResponse responseMoveApp =
+        moveApplicationAcrossQueues(user, responseGetNewApp.getApplicationId());
+    Assert.assertNotNull(responseMoveApp);
+
+    LOG.info("testRouterClientRMServiceE2E - Get New Reservation");
+
+    GetNewReservationResponse getNewReservationResponse =
+        getNewReservation(user);
+
+    LOG.info("testRouterClientRMServiceE2E - Submit Reservation");
+
+    ReservationSubmissionResponse responseSubmitReser =
+        submitReservation(user, getNewReservationResponse.getReservationId());
+    Assert.assertNotNull(responseSubmitReser);
+
+    LOG.info("testRouterClientRMServiceE2E - Update Reservation");
+
+    ReservationUpdateResponse responseUpdateReser =
+        updateReservation(user, getNewReservationResponse.getReservationId());
+    Assert.assertNotNull(responseUpdateReser);
+
+    LOG.info("testRouterClientRMServiceE2E - Delete Reservation");
+
+    ReservationDeleteResponse responseDeleteReser =
+        deleteReservation(user, getNewReservationResponse.getReservationId());
+    Assert.assertNotNull(responseDeleteReser);
+  }
+
+  /**
+   * Test if the different chains for users are generated, and LRU cache is
+   * working as expected.
+   */
+  @Test
+  public void testUsersChainMapWithLRUCache()
+      throws YarnException, IOException, InterruptedException {
+
+    Map<String, RequestInterceptorChainWrapper> pipelines;
+    RequestInterceptorChainWrapper chain;
+
+    getNewApplication("test1");
+    getNewApplication("test2");
+    getNewApplication("test3");
+    getNewApplication("test4");
+    getNewApplication("test5");
+    getNewApplication("test6");
+    getNewApplication("test7");
+    getNewApplication("test8");
+
+    pipelines = super.getRouterClientRMService().getPipelines();
+    Assert.assertEquals(8, pipelines.size());
+
+    getNewApplication("test9");
+    getNewApplication("test10");
+    getNewApplication("test1");
+    getNewApplication("test11");
+
+    // The cache max size is defined in
+    // BaseRouterClientRMTest.TEST_MAX_CACHE_SIZE
+    Assert.assertEquals(10, pipelines.size());
+
+    chain = pipelines.get("test1");
+    Assert.assertNotNull("test1 should not be evicted", chain);
+
+    chain = pipelines.get("test2");
+    Assert.assertNull("test2 should have been evicted", chain);
+  }
+
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[35/42] hadoop git commit: YARN-6093. Minor bugs with AMRMtoken renewal and state store availability when using FederationRMFailoverProxyProvider during RM failover. (Botong Huang via Subru).

Posted by su...@apache.org.
YARN-6093. Minor bugs with AMRMtoken renewal and state store availability when using FederationRMFailoverProxyProvider during RM failover. (Botong Huang via Subru).


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/8f6f1c47
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/8f6f1c47
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/8f6f1c47

Branch: refs/heads/YARN-2915
Commit: 8f6f1c4786bc0b5a52ec6476c673375269c482dc
Parents: 7fbb165
Author: Subru Krishnan <su...@apache.org>
Authored: Wed Feb 22 13:16:22 2017 -0800
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon May 8 16:59:13 2017 -0700

----------------------------------------------------------------------
 .../TestFederationRMFailoverProxyProvider.java  | 69 +++++++++++++++
 .../FederationRMFailoverProxyProvider.java      | 88 +++++++++++---------
 2 files changed, 118 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8f6f1c47/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestFederationRMFailoverProxyProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestFederationRMFailoverProxyProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestFederationRMFailoverProxyProvider.java
index fa3523c..e3f9155 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestFederationRMFailoverProxyProvider.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestFederationRMFailoverProxyProvider.java
@@ -19,17 +19,21 @@ package org.apache.hadoop.yarn.client;
 
 import java.io.IOException;
 import java.net.InetSocketAddress;
+import java.security.PrivilegedExceptionAction;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.ha.HAServiceProtocol;
+import org.apache.hadoop.io.retry.FailoverProxyProvider.ProxyInfo;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
+import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsResponse;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.MiniYARNCluster;
 import org.apache.hadoop.yarn.server.federation.failover.FederationProxyProviderUtil;
+import org.apache.hadoop.yarn.server.federation.failover.FederationRMFailoverProxyProvider;
 import org.apache.hadoop.yarn.server.federation.store.FederationStateStore;
 import org.apache.hadoop.yarn.server.federation.store.impl.MemoryFederationStateStore;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
@@ -44,6 +48,10 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
 /**
  * Unit tests for FederationRMFailoverProxyProvider.
  */
@@ -151,4 +159,65 @@ public class TestFederationRMFailoverProxyProvider {
     }
   }
 
+  @SuppressWarnings({ "rawtypes", "unchecked" })
+  @Test
+  public void testUGIForProxyCreation()
+      throws IOException, InterruptedException {
+    conf.set(YarnConfiguration.RM_CLUSTER_ID, "cluster1");
+
+    UserGroupInformation currentUser = UserGroupInformation.getCurrentUser();
+    UserGroupInformation user1 =
+        UserGroupInformation.createProxyUser("user1", currentUser);
+    UserGroupInformation user2 =
+        UserGroupInformation.createProxyUser("user2", currentUser);
+
+    final TestableFederationRMFailoverProxyProvider provider =
+        new TestableFederationRMFailoverProxyProvider();
+
+    InetSocketAddress addr =
+        conf.getSocketAddr(YarnConfiguration.RM_SCHEDULER_ADDRESS,
+            YarnConfiguration.DEFAULT_RM_SCHEDULER_ADDRESS,
+            YarnConfiguration.DEFAULT_RM_SCHEDULER_PORT);
+    final ClientRMProxy rmProxy = mock(ClientRMProxy.class);
+    when(rmProxy.getRMAddress(any(YarnConfiguration.class), any(Class.class)))
+        .thenReturn(addr);
+
+    user1.doAs(new PrivilegedExceptionAction<Object>() {
+      @Override
+      public Object run() {
+        provider.init(conf, rmProxy, ApplicationMasterProtocol.class);
+        return null;
+      }
+    });
+
+    final ProxyInfo currentProxy = provider.getProxy();
+    Assert.assertEquals("user1", provider.getLastProxyUGI().getUserName());
+
+    user2.doAs(new PrivilegedExceptionAction<Object>() {
+      @Override
+      public Object run() {
+        provider.performFailover(currentProxy.proxy);
+        return null;
+      }
+    });
+    Assert.assertEquals("user1", provider.getLastProxyUGI().getUserName());
+
+    provider.close();
+  }
+
+  protected static class TestableFederationRMFailoverProxyProvider<T>
+      extends FederationRMFailoverProxyProvider<T> {
+
+    private UserGroupInformation lastProxyUGI = null;
+
+    @Override
+    protected T createRMProxy(InetSocketAddress rmAddress) throws IOException {
+      lastProxyUGI = UserGroupInformation.getCurrentUser();
+      return super.createRMProxy(rmAddress);
+    }
+
+    public UserGroupInformation getLastProxyUGI() {
+      return lastProxyUGI;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8f6f1c47/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/failover/FederationRMFailoverProxyProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/failover/FederationRMFailoverProxyProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/failover/FederationRMFailoverProxyProvider.java
index 0ffab0b..7c21882 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/failover/FederationRMFailoverProxyProvider.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/failover/FederationRMFailoverProxyProvider.java
@@ -21,7 +21,7 @@ package org.apache.hadoop.yarn.server.federation.failover;
 import java.io.Closeable;
 import java.io.IOException;
 import java.net.InetSocketAddress;
-import java.util.Collection;
+import java.security.PrivilegedExceptionAction;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
@@ -29,14 +29,12 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
 import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
 import org.apache.hadoop.yarn.client.RMFailoverProxyProvider;
 import org.apache.hadoop.yarn.client.RMProxy;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.server.api.ResourceManagerAdministrationProtocol;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
@@ -44,6 +42,7 @@ import org.apache.hadoop.yarn.server.federation.utils.FederationStateStoreFacade
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 
 /**
@@ -64,7 +63,7 @@ public class FederationRMFailoverProxyProvider<T>
   private YarnConfiguration conf;
   private FederationStateStoreFacade facade;
   private SubClusterId subClusterId;
-  private Collection<Token<? extends TokenIdentifier>> originalTokens;
+  private UserGroupInformation originalUser;
   private boolean federationFailoverEnabled = false;
 
   @Override
@@ -97,59 +96,67 @@ public class FederationRMFailoverProxyProvider<T>
             YarnConfiguration.DEFAULT_CLIENT_FAILOVER_RETRIES_ON_SOCKET_TIMEOUTS));
 
     try {
-      UserGroupInformation currentUser = UserGroupInformation.getCurrentUser();
-      originalTokens = currentUser.getTokens();
+      this.originalUser = UserGroupInformation.getCurrentUser();
       LOG.info("Initialized Federation proxy for user: {}",
-          currentUser.getUserName());
+          this.originalUser.getUserName());
     } catch (IOException e) {
       LOG.warn("Could not get information of requester, ignoring for now.");
+      this.originalUser = null;
     }
 
   }
 
-  private void addOriginalTokens(UserGroupInformation currentUser) {
-    if (originalTokens == null || originalTokens.isEmpty()) {
-      return;
-    }
-    for (Token<? extends TokenIdentifier> token : originalTokens) {
-      currentUser.addToken(token);
-    }
+  @VisibleForTesting
+  protected T createRMProxy(InetSocketAddress rmAddress) throws IOException {
+    return RMProxy.getProxy(conf, protocol, rmAddress);
   }
 
   private T getProxyInternal(boolean isFailover) {
     SubClusterInfo subClusterInfo;
-    UserGroupInformation currentUser = null;
+    // Use the existing proxy as a backup in case getting the new proxy fails.
+    // Note that if the first time it fails, the backup is also null. In that
+    // case we will hit NullPointerException and throw it back to AM.
+    T proxy = this.current;
     try {
       LOG.info("Failing over to the ResourceManager for SubClusterId: {}",
           subClusterId);
       subClusterInfo = facade.getSubCluster(subClusterId, isFailover);
       // updating the conf with the refreshed RM addresses as proxy
-      // creations
-      // are based out of conf
+      // creations are based out of conf
       updateRMAddress(subClusterInfo);
-      currentUser = UserGroupInformation.getCurrentUser();
-      addOriginalTokens(currentUser);
-    } catch (YarnException e) {
+      if (this.originalUser == null) {
+        InetSocketAddress rmAddress = rmProxy.getRMAddress(conf, protocol);
+        LOG.info(
+            "Connecting to {} subClusterId {} with protocol {}"
+                + " without a proxy user",
+            rmAddress, subClusterId, protocol.getSimpleName());
+        proxy = createRMProxy(rmAddress);
+      } else {
+        // If the original ugi exists, always use that to create proxy because
+        // it contains up-to-date AMRMToken
+        proxy = this.originalUser.doAs(new PrivilegedExceptionAction<T>() {
+          @Override
+          public T run() throws IOException {
+            InetSocketAddress rmAddress = rmProxy.getRMAddress(conf, protocol);
+            LOG.info(
+                "Connecting to {} subClusterId {} with protocol {} as user {}",
+                rmAddress, subClusterId, protocol.getSimpleName(),
+                originalUser);
+            return createRMProxy(rmAddress);
+          }
+        });
+      }
+    } catch (Exception e) {
       LOG.error("Exception while trying to create proxy to the ResourceManager"
           + " for SubClusterId: {}", subClusterId, e);
-      return null;
-    } catch (IOException e) {
-      LOG.warn("Could not get information of requester, ignoring for now.");
-    }
-    try {
-      final InetSocketAddress rmAddress = rmProxy.getRMAddress(conf, protocol);
-      LOG.info("Connecting to {} with protocol {} as user: {}", rmAddress,
-          protocol.getSimpleName(), currentUser);
-      LOG.info("Failed over to the RM at {} for SubClusterId: {}", rmAddress,
-          subClusterId);
-      return RMProxy.getProxy(conf, protocol, rmAddress);
-    } catch (IOException ioe) {
-      LOG.error(
-          "IOException while trying to create proxy to the ResourceManager"
-              + " for SubClusterId: {}",
-          subClusterId, ioe);
-      return null;
+      if (proxy == null) {
+        throw new YarnRuntimeException(
+            String.format("Create initial proxy to the ResourceManager for"
+                + " SubClusterId %s failed", subClusterId),
+            e);
+      }
     }
+    return proxy;
   }
 
   private void updateRMAddress(SubClusterInfo subClusterInfo) {
@@ -177,8 +184,11 @@ public class FederationRMFailoverProxyProvider<T>
 
   @Override
   public synchronized void performFailover(T currentProxy) {
-    closeInternal(currentProxy);
+    // It will not return null proxy here
     current = getProxyInternal(federationFailoverEnabled);
+    if (current != currentProxy) {
+      closeInternal(currentProxy);
+    }
   }
 
   @Override


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[38/42] hadoop git commit: YARN-6190. Validation and synchronization fixes in LocalityMulticastAMRMProxyPolicy. (Botong Huang via curino)

Posted by su...@apache.org.
YARN-6190. Validation and synchronization fixes in LocalityMulticastAMRMProxyPolicy. (Botong Huang via curino)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/8221ffab
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/8221ffab
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/8221ffab

Branch: refs/heads/YARN-2915
Commit: 8221ffab0362c99360bdfbe82a1a1d3135829e1b
Parents: c2aab00
Author: Carlo Curino <cu...@apache.org>
Authored: Tue Feb 28 17:04:20 2017 -0800
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon May 8 16:59:13 2017 -0700

----------------------------------------------------------------------
 .../LocalityMulticastAMRMProxyPolicy.java       | 63 +++++++++++++-------
 .../TestLocalityMulticastAMRMProxyPolicy.java   | 21 ++++++-
 .../policies/manager/BasePolicyManagerTest.java |  3 -
 .../resolver/TestDefaultSubClusterResolver.java |  9 ++-
 .../utils/FederationPoliciesTestUtil.java       |  6 +-
 5 files changed, 73 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8221ffab/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/LocalityMulticastAMRMProxyPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/LocalityMulticastAMRMProxyPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/LocalityMulticastAMRMProxyPolicy.java
index 283f89e..6f97a51 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/LocalityMulticastAMRMProxyPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/LocalityMulticastAMRMProxyPolicy.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.server.federation.policies.FederationPolicyInitializationContext;
 import org.apache.hadoop.yarn.server.federation.policies.dao.WeightedPolicyInfo;
 import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
@@ -143,10 +144,9 @@ public class LocalityMulticastAMRMProxyPolicy extends AbstractAMRMProxyPolicy {
     Map<SubClusterId, Float> newWeightsConverted = new HashMap<>();
     boolean allInactive = true;
     WeightedPolicyInfo policy = getPolicyInfo();
-    if (policy.getAMRMPolicyWeights() == null
-        || policy.getAMRMPolicyWeights().size() == 0) {
-      allInactive = false;
-    } else {
+
+    if (policy.getAMRMPolicyWeights() != null
+        && policy.getAMRMPolicyWeights().size() > 0) {
       for (Map.Entry<SubClusterIdInfo, Float> e : policy.getAMRMPolicyWeights()
           .entrySet()) {
         if (e.getValue() > 0) {
@@ -180,7 +180,6 @@ public class LocalityMulticastAMRMProxyPolicy extends AbstractAMRMProxyPolicy {
 
     this.federationFacade =
         policyContext.getFederationStateStoreFacade();
-    this.bookkeeper = new AllocationBookkeeper();
     this.homeSubcluster = policyContext.getHomeSubcluster();
 
   }
@@ -197,7 +196,9 @@ public class LocalityMulticastAMRMProxyPolicy extends AbstractAMRMProxyPolicy {
       List<ResourceRequest> resourceRequests) throws YarnException {
 
     // object used to accumulate statistics about the answer, initialize with
-    // active subclusters.
+    // active subclusters. Create a new instance per call because this method
+    // can be called concurrently.
+    bookkeeper = new AllocationBookkeeper();
     bookkeeper.reinitialize(federationFacade.getSubClusters(true));
 
     List<ResourceRequest> nonLocalizedRequests =
@@ -238,12 +239,16 @@ public class LocalityMulticastAMRMProxyPolicy extends AbstractAMRMProxyPolicy {
         // we log altogether later
       }
       if (targetIds != null && targetIds.size() > 0) {
+        boolean hasActive = false;
         for (SubClusterId tid : targetIds) {
           if (bookkeeper.isActiveAndEnabled(tid)) {
             bookkeeper.addRackRR(tid, rr);
+            hasActive = true;
           }
         }
-        continue;
+        if (hasActive) {
+          continue;
+        }
       }
 
       // Handle node/rack requests that the SubClusterResolver cannot map to
@@ -347,7 +352,7 @@ public class LocalityMulticastAMRMProxyPolicy extends AbstractAMRMProxyPolicy {
                 originalResourceRequest.getExecutionTypeRequest());
         out.setAllocationRequestId(allocationId);
         out.setNumContainers((int) Math.ceil(numContainer));
-        if (out.isAnyLocation(out.getResourceName())) {
+        if (ResourceRequest.isAnyLocation(out.getResourceName())) {
           allocationBookkeeper.addAnyRR(targetId, out);
         } else {
           allocationBookkeeper.addRackRR(targetId, out);
@@ -362,7 +367,7 @@ public class LocalityMulticastAMRMProxyPolicy extends AbstractAMRMProxyPolicy {
    */
   private float getLocalityBasedWeighting(long reqId, SubClusterId targetId,
       AllocationBookkeeper allocationBookkeeper) {
-    float totWeight = allocationBookkeeper.getTotNumLocalizedContainers();
+    float totWeight = allocationBookkeeper.getTotNumLocalizedContainers(reqId);
     float localWeight =
         allocationBookkeeper.getNumLocalizedContainers(reqId, targetId);
     return totWeight > 0 ? localWeight / totWeight : 0;
@@ -375,7 +380,7 @@ public class LocalityMulticastAMRMProxyPolicy extends AbstractAMRMProxyPolicy {
   private float getPolicyConfigWeighting(SubClusterId targetId,
       AllocationBookkeeper allocationBookkeeper) {
     float totWeight = allocationBookkeeper.totPolicyWeight;
-    Float localWeight = weights.get(targetId);
+    Float localWeight = allocationBookkeeper.policyWeights.get(targetId);
     return (localWeight != null && totWeight > 0) ? localWeight / totWeight : 0;
   }
 
@@ -424,29 +429,36 @@ public class LocalityMulticastAMRMProxyPolicy extends AbstractAMRMProxyPolicy {
     // asks, used to correctly "spread" the corresponding ANY
     private Map<Long, Map<SubClusterId, AtomicLong>> countContainersPerRM =
         new HashMap<>();
+    private Map<Long, AtomicLong> totNumLocalizedContainers = new HashMap<>();
 
     private Set<SubClusterId> activeAndEnabledSC = new HashSet<>();
-    private long totNumLocalizedContainers = 0;
     private float totHeadroomMemory = 0;
     private int totHeadRoomEnabledRMs = 0;
+    private Map<SubClusterId, Float> policyWeights;
     private float totPolicyWeight = 0;
 
     private void reinitialize(
         Map<SubClusterId, SubClusterInfo> activeSubclusters)
         throws YarnException {
+      if (activeSubclusters == null) {
+        throw new YarnRuntimeException("null activeSubclusters received");
+      }
 
       // reset data structures
       answer.clear();
       countContainersPerRM.clear();
+      totNumLocalizedContainers.clear();
       activeAndEnabledSC.clear();
-      totNumLocalizedContainers = 0;
       totHeadroomMemory = 0;
       totHeadRoomEnabledRMs = 0;
+      // save the reference locally in case the weights get reinitialized
+      // concurrently
+      policyWeights = weights;
       totPolicyWeight = 0;
 
       // pre-compute the set of subclusters that are both active and enabled by
       // the policy weights, and accumulate their total weight
-      for (Map.Entry<SubClusterId, Float> entry : weights.entrySet()) {
+      for (Map.Entry<SubClusterId, Float> entry : policyWeights.entrySet()) {
         if (entry.getValue() > 0
             && activeSubclusters.containsKey(entry.getKey())) {
           activeAndEnabledSC.add(entry.getKey());
@@ -467,7 +479,6 @@ public class LocalityMulticastAMRMProxyPolicy extends AbstractAMRMProxyPolicy {
           totHeadRoomEnabledRMs++;
         }
       }
-
     }
 
     /**
@@ -475,7 +486,8 @@ public class LocalityMulticastAMRMProxyPolicy extends AbstractAMRMProxyPolicy {
      * on a per-allocation-id and per-subcluster bases.
      */
     private void addLocalizedNodeRR(SubClusterId targetId, ResourceRequest rr) {
-      Preconditions.checkArgument(!rr.isAnyLocation(rr.getResourceName()));
+      Preconditions
+          .checkArgument(!ResourceRequest.isAnyLocation(rr.getResourceName()));
 
       if (!countContainersPerRM.containsKey(rr.getAllocationRequestId())) {
         countContainersPerRM.put(rr.getAllocationRequestId(), new HashMap<>());
@@ -488,7 +500,12 @@ public class LocalityMulticastAMRMProxyPolicy extends AbstractAMRMProxyPolicy {
       countContainersPerRM.get(rr.getAllocationRequestId()).get(targetId)
           .addAndGet(rr.getNumContainers());
 
-      totNumLocalizedContainers += rr.getNumContainers();
+      if (!totNumLocalizedContainers.containsKey(rr.getAllocationRequestId())) {
+        totNumLocalizedContainers.put(rr.getAllocationRequestId(),
+            new AtomicLong(0));
+      }
+      totNumLocalizedContainers.get(rr.getAllocationRequestId())
+          .addAndGet(rr.getNumContainers());
 
       internalAddToAnswer(targetId, rr);
     }
@@ -497,7 +514,8 @@ public class LocalityMulticastAMRMProxyPolicy extends AbstractAMRMProxyPolicy {
      * Add a rack-local request to the final asnwer.
      */
     public void addRackRR(SubClusterId targetId, ResourceRequest rr) {
-      Preconditions.checkArgument(!rr.isAnyLocation(rr.getResourceName()));
+      Preconditions
+          .checkArgument(!ResourceRequest.isAnyLocation(rr.getResourceName()));
       internalAddToAnswer(targetId, rr);
     }
 
@@ -505,7 +523,8 @@ public class LocalityMulticastAMRMProxyPolicy extends AbstractAMRMProxyPolicy {
      * Add an ANY request to the final answer.
      */
     private void addAnyRR(SubClusterId targetId, ResourceRequest rr) {
-      Preconditions.checkArgument(rr.isAnyLocation(rr.getResourceName()));
+      Preconditions
+          .checkArgument(ResourceRequest.isAnyLocation(rr.getResourceName()));
       internalAddToAnswer(targetId, rr);
     }
 
@@ -552,10 +571,12 @@ public class LocalityMulticastAMRMProxyPolicy extends AbstractAMRMProxyPolicy {
     }
 
     /**
-     * Return the total number of container coming from localized requests.
+     * Return the total number of container coming from localized requests
+     * matching an allocation Id.
      */
-    private long getTotNumLocalizedContainers() {
-      return totNumLocalizedContainers;
+    private long getTotNumLocalizedContainers(long allocationId) {
+      AtomicLong c = totNumLocalizedContainers.get(allocationId);
+      return c == null ? 0 : c.get();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8221ffab/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/TestLocalityMulticastAMRMProxyPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/TestLocalityMulticastAMRMProxyPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/TestLocalityMulticastAMRMProxyPolicy.java
index 2654a06..5b3cf74 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/TestLocalityMulticastAMRMProxyPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/TestLocalityMulticastAMRMProxyPolicy.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn.server.federation.policies.amrmproxy;
 
+import static org.junit.Assert.fail;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -40,6 +41,7 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.federation.policies.BaseFederationPoliciesTest;
 import org.apache.hadoop.yarn.server.federation.policies.FederationPolicyInitializationContext;
 import org.apache.hadoop.yarn.server.federation.policies.dao.WeightedPolicyInfo;
+import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
 import org.apache.hadoop.yarn.server.federation.resolver.DefaultSubClusterResolverImpl;
 import org.apache.hadoop.yarn.server.federation.resolver.SubClusterResolver;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
@@ -117,6 +119,21 @@ public class TestLocalityMulticastAMRMProxyPolicy
         getActiveSubclusters());
   }
 
+  @Test(expected = FederationPolicyInitializationException.class)
+  public void testNullWeights() throws Exception {
+    getPolicyInfo().setAMRMPolicyWeights(null);
+    initializePolicy();
+    fail();
+  }
+
+  @Test(expected = FederationPolicyInitializationException.class)
+  public void testEmptyWeights() throws Exception {
+    getPolicyInfo()
+        .setAMRMPolicyWeights(new HashMap<SubClusterIdInfo, Float>());
+    initializePolicy();
+    fail();
+  }
+
   @Test
   public void testSplitBasedOnHeadroom() throws Exception {
 
@@ -154,7 +171,7 @@ public class TestLocalityMulticastAMRMProxyPolicy
     AllocateResponse ar = getAllocateResponseWithTargetHeadroom(100);
     ((FederationAMRMProxyPolicy) getPolicy())
         .notifyOfResponse(SubClusterId.newInstance("subcluster2"), ar);
-    ((FederationAMRMProxyPolicy) getPolicy())
+    response = ((FederationAMRMProxyPolicy) getPolicy())
         .splitResourceRequests(resourceRequests);
 
     LOG.info("After headroom update");
@@ -332,7 +349,7 @@ public class TestLocalityMulticastAMRMProxyPolicy
 
     // we expect 5 entry for subcluster1 (4 from request-id 1, and part
     // of the broadcast of request-id 2
-    checkExpectedAllocation(response, "subcluster1", 5, 25);
+    checkExpectedAllocation(response, "subcluster1", 5, 26);
 
     // sub-cluster 2 should contain 3 entry from request-id 1 and 1 from the
     // broadcast of request-id 2, and no request-id 0

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8221ffab/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/manager/BasePolicyManagerTest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/manager/BasePolicyManagerTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/manager/BasePolicyManagerTest.java
index 3cf73b6..bd99cb5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/manager/BasePolicyManagerTest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/manager/BasePolicyManagerTest.java
@@ -89,9 +89,6 @@ public abstract class BasePolicyManagerTest {
     FederationAMRMProxyPolicy federationAMRMProxyPolicy =
         wfp2.getAMRMPolicy(context, null);
 
-    // needed only for tests (getARMRMPolicy change the "type" in conf)
-    fpc.setType(wfp.getClass().getCanonicalName());
-
     FederationRouterPolicy federationRouterPolicy =
         wfp2.getRouterPolicy(context, null);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8221ffab/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/resolver/TestDefaultSubClusterResolver.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/resolver/TestDefaultSubClusterResolver.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/resolver/TestDefaultSubClusterResolver.java
index 7396942..25d246e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/resolver/TestDefaultSubClusterResolver.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/resolver/TestDefaultSubClusterResolver.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn.server.federation.resolver;
 
+import java.io.File;
 import java.net.URL;
 import java.util.HashSet;
 import java.util.Set;
@@ -46,8 +47,10 @@ public class TestDefaultSubClusterResolver {
       throw new RuntimeException(
           "Could not find 'nodes' dummy file in classpath");
     }
+    // This will get rid of the beginning '/' in the url in Windows env
+    File file = new File(url.getPath());
 
-    conf.set(YarnConfiguration.FEDERATION_MACHINE_LIST, url.getPath());
+    conf.set(YarnConfiguration.FEDERATION_MACHINE_LIST, file.getPath());
     resolver.setConf(conf);
     resolver.load();
   }
@@ -62,8 +65,10 @@ public class TestDefaultSubClusterResolver {
       throw new RuntimeException(
           "Could not find 'nodes-malformed' dummy file in classpath");
     }
+    // This will get rid of the beginning '/' in the url in Windows env
+    File file = new File(url.getPath());
 
-    conf.set(YarnConfiguration.FEDERATION_MACHINE_LIST, url.getPath());
+    conf.set(YarnConfiguration.FEDERATION_MACHINE_LIST, file.getPath());
     resolver.setConf(conf);
     resolver.load();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8221ffab/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/FederationPoliciesTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/FederationPoliciesTestUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/FederationPoliciesTestUtil.java
index 85fdc96..acc14dd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/FederationPoliciesTestUtil.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/FederationPoliciesTestUtil.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.yarn.server.federation.store.FederationStateStore;
 import org.apache.hadoop.yarn.server.federation.store.records.*;
 import org.apache.hadoop.yarn.util.Records;
 
+import java.io.File;
 import java.net.URL;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
@@ -162,7 +163,10 @@ public final class FederationPoliciesTestUtil {
       throw new RuntimeException(
           "Could not find 'nodes' dummy file in classpath");
     }
-    conf.set(YarnConfiguration.FEDERATION_MACHINE_LIST, url.getPath());
+    // This will get rid of the beginning '/' in the url in Windows env
+    File file = new File(url.getPath());
+
+    conf.set(YarnConfiguration.FEDERATION_MACHINE_LIST, file.getPath());
     resolver.setConf(conf);
     resolver.load();
     return resolver;


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[19/42] hadoop git commit: YARN-5905. Update the RM webapp host that is reported as part of Federation membership to current primary RM's IP.

Posted by su...@apache.org.
YARN-5905. Update the RM webapp host that is reported as part of Federation membership to current primary RM's IP.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/a2cdf2cf
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/a2cdf2cf
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/a2cdf2cf

Branch: refs/heads/YARN-2915
Commit: a2cdf2cf74a7de659f30bf010f11d2c2e5708b53
Parents: b370ac6
Author: Subru Krishnan <su...@apache.org>
Authored: Tue Nov 22 18:30:40 2016 -0800
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon May 8 16:59:12 2017 -0700

----------------------------------------------------------------------
 .../federation/FederationStateStoreService.java          |  4 ++--
 .../federation/TestFederationRMStateStoreService.java    | 11 ++++++++++-
 2 files changed, 12 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a2cdf2cf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/federation/FederationStateStoreService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/federation/FederationStateStoreService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/federation/FederationStateStoreService.java
index 9a01d7e..530184f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/federation/FederationStateStoreService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/federation/FederationStateStoreService.java
@@ -177,8 +177,8 @@ public class FederationStateStoreService extends AbstractService
         config.getSocketAddr(YarnConfiguration.RM_ADMIN_ADDRESS,
             YarnConfiguration.DEFAULT_RM_ADMIN_ADDRESS,
             YarnConfiguration.DEFAULT_RM_ADMIN_PORT));
-    String webAppAddress =
-        WebAppUtils.getResolvedRemoteRMWebAppURLWithoutScheme(config);
+    String webAppAddress = getServiceAddress(NetUtils
+        .createSocketAddr(WebAppUtils.getRMWebAppURLWithScheme(config)));
 
     SubClusterInfo subClusterInfo = SubClusterInfo.newInstance(subClusterId,
         amRMAddress, clientRMAddress, rmAdminAddress, webAppAddress,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a2cdf2cf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/federation/TestFederationRMStateStoreService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/federation/TestFederationRMStateStoreService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/federation/TestFederationRMStateStoreService.java
index 30f69b5..d92a793 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/federation/TestFederationRMStateStoreService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/federation/TestFederationRMStateStoreService.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.federation;
 
 import java.io.IOException;
 import java.io.StringReader;
+import java.net.UnknownHostException;
 
 import javax.xml.bind.JAXBException;
 
@@ -157,12 +158,20 @@ public class TestFederationRMStateStoreService {
   }
 
   private String checkSubClusterInfo(SubClusterState state)
-      throws YarnException {
+      throws YarnException, UnknownHostException {
     Assert.assertNotNull(stateStore.getSubCluster(request));
     SubClusterInfo response =
         stateStore.getSubCluster(request).getSubClusterInfo();
     Assert.assertEquals(state, response.getState());
     Assert.assertTrue(response.getLastHeartBeat() >= lastHearbeatTS);
+    String expectedAddress =
+        (response.getClientRMServiceAddress().split(":"))[0];
+    Assert.assertEquals(expectedAddress,
+        (response.getAMRMServiceAddress().split(":"))[0]);
+    Assert.assertEquals(expectedAddress,
+        (response.getRMAdminServiceAddress().split(":"))[0]);
+    Assert.assertEquals(expectedAddress,
+        (response.getRMWebServiceAddress().split(":"))[0]);
     lastHearbeatTS = response.getLastHeartBeat();
     return response.getCapability();
   }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[22/42] hadoop git commit: YARN-5467. InputValidator for the FederationStateStore internal APIs. (Giovanni Matteo Fumarola via Subru)

Posted by su...@apache.org.
YARN-5467. InputValidator for the FederationStateStore internal APIs. (Giovanni Matteo Fumarola via Subru)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/8aa6bda7
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/8aa6bda7
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/8aa6bda7

Branch: refs/heads/YARN-2915
Commit: 8aa6bda7734d1841d03348e7a6af2dfb6b967686
Parents: 865e61b
Author: Subru Krishnan <su...@apache.org>
Authored: Wed Aug 17 12:07:06 2016 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon May 8 16:59:12 2017 -0700

----------------------------------------------------------------------
 .../store/impl/MemoryFederationStateStore.java  |   30 +
 ...cationHomeSubClusterStoreInputValidator.java |  183 +++
 ...ationMembershipStateStoreInputValidator.java |  317 +++++
 .../FederationPolicyStoreInputValidator.java    |  144 ++
 ...derationStateStoreInvalidInputException.java |   48 +
 .../federation/store/utils/package-info.java    |   17 +
 .../impl/FederationStateStoreBaseTest.java      |    6 +-
 .../TestFederationStateStoreInputValidator.java | 1265 ++++++++++++++++++
 8 files changed, 2007 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8aa6bda7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/MemoryFederationStateStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/MemoryFederationStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/MemoryFederationStateStore.java
index 8144435..6e564dc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/MemoryFederationStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/MemoryFederationStateStore.java
@@ -57,6 +57,9 @@ import org.apache.hadoop.yarn.server.federation.store.records.SubClusterRegister
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterRegisterResponse;
 import org.apache.hadoop.yarn.server.federation.store.records.UpdateApplicationHomeSubClusterRequest;
 import org.apache.hadoop.yarn.server.federation.store.records.UpdateApplicationHomeSubClusterResponse;
+import org.apache.hadoop.yarn.server.federation.store.utils.FederationApplicationHomeSubClusterStoreInputValidator;
+import org.apache.hadoop.yarn.server.federation.store.utils.FederationMembershipStateStoreInputValidator;
+import org.apache.hadoop.yarn.server.federation.store.utils.FederationPolicyStoreInputValidator;
 import org.apache.hadoop.yarn.server.records.Version;
 import org.apache.hadoop.yarn.util.MonotonicClock;
 
@@ -88,6 +91,8 @@ public class MemoryFederationStateStore implements FederationStateStore {
   @Override
   public SubClusterRegisterResponse registerSubCluster(
       SubClusterRegisterRequest request) throws YarnException {
+    FederationMembershipStateStoreInputValidator
+        .validateSubClusterRegisterRequest(request);
     SubClusterInfo subClusterInfo = request.getSubClusterInfo();
     membership.put(subClusterInfo.getSubClusterId(), subClusterInfo);
     return SubClusterRegisterResponse.newInstance();
@@ -96,6 +101,8 @@ public class MemoryFederationStateStore implements FederationStateStore {
   @Override
   public SubClusterDeregisterResponse deregisterSubCluster(
       SubClusterDeregisterRequest request) throws YarnException {
+    FederationMembershipStateStoreInputValidator
+        .validateSubClusterDeregisterRequest(request);
     SubClusterInfo subClusterInfo = membership.get(request.getSubClusterId());
     if (subClusterInfo == null) {
       throw new YarnException(
@@ -111,6 +118,8 @@ public class MemoryFederationStateStore implements FederationStateStore {
   public SubClusterHeartbeatResponse subClusterHeartbeat(
       SubClusterHeartbeatRequest request) throws YarnException {
 
+    FederationMembershipStateStoreInputValidator
+        .validateSubClusterHeartbeatRequest(request);
     SubClusterId subClusterId = request.getSubClusterId();
     SubClusterInfo subClusterInfo = membership.get(subClusterId);
 
@@ -129,6 +138,9 @@ public class MemoryFederationStateStore implements FederationStateStore {
   @Override
   public GetSubClusterInfoResponse getSubCluster(
       GetSubClusterInfoRequest request) throws YarnException {
+
+    FederationMembershipStateStoreInputValidator
+        .validateGetSubClusterInfoRequest(request);
     SubClusterId subClusterId = request.getSubClusterId();
     if (!membership.containsKey(subClusterId)) {
       throw new YarnException(
@@ -157,6 +169,9 @@ public class MemoryFederationStateStore implements FederationStateStore {
   @Override
   public AddApplicationHomeSubClusterResponse addApplicationHomeSubCluster(
       AddApplicationHomeSubClusterRequest request) throws YarnException {
+
+    FederationApplicationHomeSubClusterStoreInputValidator
+        .validateAddApplicationHomeSubClusterRequest(request);
     ApplicationId appId =
         request.getApplicationHomeSubCluster().getApplicationId();
 
@@ -172,6 +187,9 @@ public class MemoryFederationStateStore implements FederationStateStore {
   @Override
   public UpdateApplicationHomeSubClusterResponse updateApplicationHomeSubCluster(
       UpdateApplicationHomeSubClusterRequest request) throws YarnException {
+
+    FederationApplicationHomeSubClusterStoreInputValidator
+        .validateUpdateApplicationHomeSubClusterRequest(request);
     ApplicationId appId =
         request.getApplicationHomeSubCluster().getApplicationId();
     if (!applications.containsKey(appId)) {
@@ -186,6 +204,9 @@ public class MemoryFederationStateStore implements FederationStateStore {
   @Override
   public GetApplicationHomeSubClusterResponse getApplicationHomeSubCluster(
       GetApplicationHomeSubClusterRequest request) throws YarnException {
+
+    FederationApplicationHomeSubClusterStoreInputValidator
+        .validateGetApplicationHomeSubClusterRequest(request);
     ApplicationId appId = request.getApplicationId();
     if (!applications.containsKey(appId)) {
       throw new YarnException("Application " + appId + " does not exist");
@@ -212,6 +233,9 @@ public class MemoryFederationStateStore implements FederationStateStore {
   @Override
   public DeleteApplicationHomeSubClusterResponse deleteApplicationHomeSubCluster(
       DeleteApplicationHomeSubClusterRequest request) throws YarnException {
+
+    FederationApplicationHomeSubClusterStoreInputValidator
+        .validateDeleteApplicationHomeSubClusterRequest(request);
     ApplicationId appId = request.getApplicationId();
     if (!applications.containsKey(appId)) {
       throw new YarnException("Application " + appId + " does not exist");
@@ -224,6 +248,9 @@ public class MemoryFederationStateStore implements FederationStateStore {
   @Override
   public GetSubClusterPolicyConfigurationResponse getPolicyConfiguration(
       GetSubClusterPolicyConfigurationRequest request) throws YarnException {
+
+    FederationPolicyStoreInputValidator
+        .validateGetSubClusterPolicyConfigurationRequest(request);
     String queue = request.getQueue();
     if (!policies.containsKey(queue)) {
       throw new YarnException("Policy for queue " + queue + " does not exist");
@@ -236,6 +263,9 @@ public class MemoryFederationStateStore implements FederationStateStore {
   @Override
   public SetSubClusterPolicyConfigurationResponse setPolicyConfiguration(
       SetSubClusterPolicyConfigurationRequest request) throws YarnException {
+
+    FederationPolicyStoreInputValidator
+        .validateSetSubClusterPolicyConfigurationRequest(request);
     policies.put(request.getPolicyConfiguration().getQueue(),
         request.getPolicyConfiguration());
     return SetSubClusterPolicyConfigurationResponse.newInstance();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8aa6bda7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationApplicationHomeSubClusterStoreInputValidator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationApplicationHomeSubClusterStoreInputValidator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationApplicationHomeSubClusterStoreInputValidator.java
new file mode 100644
index 0000000..c14a452
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationApplicationHomeSubClusterStoreInputValidator.java
@@ -0,0 +1,183 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.utils;
+
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.server.federation.store.records.AddApplicationHomeSubClusterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.ApplicationHomeSubCluster;
+import org.apache.hadoop.yarn.server.federation.store.records.DeleteApplicationHomeSubClusterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.GetApplicationHomeSubClusterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.UpdateApplicationHomeSubClusterRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Utility class to validate the inputs to
+ * {@code FederationApplicationHomeSubClusterStore}, allows a fail fast
+ * mechanism for invalid user inputs.
+ *
+ */
+public final class FederationApplicationHomeSubClusterStoreInputValidator {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(FederationApplicationHomeSubClusterStoreInputValidator.class);
+
+  private FederationApplicationHomeSubClusterStoreInputValidator() {
+  }
+
+  /**
+   * Quick validation on the input to check some obvious fail conditions (fail
+   * fast). Check if the provided {@link AddApplicationHomeSubClusterRequest}
+   * for adding a new application is valid or not.
+   *
+   * @param request the {@link AddApplicationHomeSubClusterRequest} to validate
+   *          against
+   * @throws FederationStateStoreInvalidInputException if the request is invalid
+   */
+  public static void validateAddApplicationHomeSubClusterRequest(
+      AddApplicationHomeSubClusterRequest request)
+      throws FederationStateStoreInvalidInputException {
+    if (request == null) {
+      String message = "Missing AddApplicationHomeSubCluster Request."
+          + " Please try again by specifying"
+          + " an AddApplicationHomeSubCluster information.";
+      LOG.warn(message);
+      throw new FederationStateStoreInvalidInputException(message);
+    }
+
+    // validate ApplicationHomeSubCluster info
+    checkApplicationHomeSubCluster(request.getApplicationHomeSubCluster());
+  }
+
+  /**
+   * Quick validation on the input to check some obvious fail conditions (fail
+   * fast). Check if the provided {@link UpdateApplicationHomeSubClusterRequest}
+   * for updating an application is valid or not.
+   *
+   * @param request the {@link UpdateApplicationHomeSubClusterRequest} to
+   *          validate against
+   * @throws FederationStateStoreInvalidInputException if the request is invalid
+   */
+  public static void validateUpdateApplicationHomeSubClusterRequest(
+      UpdateApplicationHomeSubClusterRequest request)
+      throws FederationStateStoreInvalidInputException {
+    if (request == null) {
+      String message = "Missing UpdateApplicationHomeSubCluster Request."
+          + " Please try again by specifying"
+          + " an ApplicationHomeSubCluster information.";
+      LOG.warn(message);
+      throw new FederationStateStoreInvalidInputException(message);
+    }
+
+    // validate ApplicationHomeSubCluster info
+    checkApplicationHomeSubCluster(request.getApplicationHomeSubCluster());
+  }
+
+  /**
+   * Quick validation on the input to check some obvious fail conditions (fail
+   * fast). Check if the provided {@link GetApplicationHomeSubClusterRequest}
+   * for querying application's information is valid or not.
+   *
+   * @param request the {@link GetApplicationHomeSubClusterRequest} to validate
+   *          against
+   * @throws FederationStateStoreInvalidInputException if the request is invalid
+   */
+  public static void validateGetApplicationHomeSubClusterRequest(
+      GetApplicationHomeSubClusterRequest request)
+      throws FederationStateStoreInvalidInputException {
+    if (request == null) {
+      String message = "Missing GetApplicationHomeSubCluster Request."
+          + " Please try again by specifying an Application Id information.";
+      LOG.warn(message);
+      throw new FederationStateStoreInvalidInputException(message);
+    }
+
+    // validate application Id
+    checkApplicationId(request.getApplicationId());
+  }
+
+  /**
+   * Quick validation on the input to check some obvious fail conditions (fail
+   * fast). Check if the provided {@link DeleteApplicationHomeSubClusterRequest}
+   * for deleting an application is valid or not.
+   *
+   * @param request the {@link DeleteApplicationHomeSubClusterRequest} to
+   *          validate against
+   * @throws FederationStateStoreInvalidInputException if the request is invalid
+   */
+  public static void validateDeleteApplicationHomeSubClusterRequest(
+      DeleteApplicationHomeSubClusterRequest request)
+      throws FederationStateStoreInvalidInputException {
+    if (request == null) {
+      String message = "Missing DeleteApplicationHomeSubCluster Request."
+          + " Please try again by specifying"
+          + " an ApplicationHomeSubCluster information.";
+      LOG.warn(message);
+      throw new FederationStateStoreInvalidInputException(message);
+    }
+
+    // validate application Id
+    checkApplicationId(request.getApplicationId());
+  }
+
+  /**
+   * Validate if the ApplicationHomeSubCluster info are present or not.
+   *
+   * @param applicationHomeSubCluster the information of the application to be
+   *          verified
+   * @throws FederationStateStoreInvalidInputException if the SubCluster Info
+   *           are invalid
+   */
+  private static void checkApplicationHomeSubCluster(
+      ApplicationHomeSubCluster applicationHomeSubCluster)
+
+      throws FederationStateStoreInvalidInputException {
+    if (applicationHomeSubCluster == null) {
+      String message = "Missing ApplicationHomeSubCluster Info."
+          + " Please try again by specifying"
+          + " an ApplicationHomeSubCluster information.";
+      LOG.warn(message);
+      throw new FederationStateStoreInvalidInputException(message);
+    }
+    // validate application Id
+    checkApplicationId(applicationHomeSubCluster.getApplicationId());
+
+    // validate subcluster Id
+    FederationMembershipStateStoreInputValidator
+        .checkSubClusterId(applicationHomeSubCluster.getHomeSubCluster());
+
+  }
+
+  /**
+   * Validate if the application id is present or not.
+   *
+   * @param appId the id of the application to be verified
+   * @throws FederationStateStoreInvalidInputException if the application Id is
+   *           invalid
+   */
+  private static void checkApplicationId(ApplicationId appId)
+      throws FederationStateStoreInvalidInputException {
+    if (appId == null) {
+      String message = "Missing Application Id."
+          + " Please try again by specifying an Application Id.";
+      LOG.warn(message);
+      throw new FederationStateStoreInvalidInputException(message);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8aa6bda7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationMembershipStateStoreInputValidator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationMembershipStateStoreInputValidator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationMembershipStateStoreInputValidator.java
new file mode 100644
index 0000000..b587ee5
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationMembershipStateStoreInputValidator.java
@@ -0,0 +1,317 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.utils;
+
+import java.net.URI;
+
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterInfoRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterDeregisterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterHeartbeatRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterRegisterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterState;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Utility class to validate the inputs to
+ * {@code FederationMembershipStateStore}, allows a fail fast mechanism for
+ * invalid user inputs.
+ *
+ */
+public final class FederationMembershipStateStoreInputValidator {
+
+  private static final Logger LOG = LoggerFactory
+      .getLogger(FederationMembershipStateStoreInputValidator.class);
+
+  private FederationMembershipStateStoreInputValidator() {
+  }
+
+  /**
+   * Quick validation on the input to check some obvious fail conditions (fail
+   * fast). Check if the provided {@link SubClusterRegisterRequest} for
+   * registration a new subcluster is valid or not.
+   *
+   * @param request the {@link SubClusterRegisterRequest} to validate against
+   * @throws FederationStateStoreInvalidInputException if the request is invalid
+   */
+  public static void validateSubClusterRegisterRequest(
+      SubClusterRegisterRequest request)
+      throws FederationStateStoreInvalidInputException {
+
+    // check if the request is present
+    if (request == null) {
+      String message = "Missing SubClusterRegister Request."
+          + " Please try again by specifying a"
+          + " SubCluster Register Information.";
+      LOG.warn(message);
+      throw new FederationStateStoreInvalidInputException(message);
+
+    }
+
+    // validate subcluster info
+    checkSubClusterInfo(request.getSubClusterInfo());
+  }
+
+  /**
+   * Quick validation on the input to check some obvious fail conditions (fail
+   * fast). Check if the provided {@link SubClusterDeregisterRequest} for
+   * deregistration a subcluster is valid or not.
+   *
+   * @param request the {@link SubClusterDeregisterRequest} to validate against
+   * @throws FederationStateStoreInvalidInputException if the request is invalid
+   */
+  public static void validateSubClusterDeregisterRequest(
+      SubClusterDeregisterRequest request)
+      throws FederationStateStoreInvalidInputException {
+
+    // check if the request is present
+    if (request == null) {
+      String message = "Missing SubClusterDeregister Request."
+          + " Please try again by specifying a"
+          + " SubCluster Deregister Information.";
+      LOG.warn(message);
+      throw new FederationStateStoreInvalidInputException(message);
+    }
+
+    // validate subcluster id
+    checkSubClusterId(request.getSubClusterId());
+    // validate subcluster state
+    checkSubClusterState(request.getState());
+    if (!request.getState().isFinal()) {
+      String message = "Invalid non-final state: " + request.getState();
+      LOG.warn(message);
+      throw new FederationStateStoreInvalidInputException(message);
+    }
+  }
+
+  /**
+   * Quick validation on the input to check some obvious fail conditions (fail
+   * fast). Check if the provided {@link SubClusterHeartbeatRequest} for
+   * heartbeating a subcluster is valid or not.
+   *
+   * @param request the {@link SubClusterHeartbeatRequest} to validate against
+   * @throws FederationStateStoreInvalidInputException if the request is invalid
+   */
+  public static void validateSubClusterHeartbeatRequest(
+      SubClusterHeartbeatRequest request)
+      throws FederationStateStoreInvalidInputException {
+
+    // check if the request is present
+    if (request == null) {
+      String message = "Missing SubClusterHeartbeat Request."
+          + " Please try again by specifying a"
+          + " SubCluster Heartbeat Information.";
+      LOG.warn(message);
+      throw new FederationStateStoreInvalidInputException(message);
+    }
+
+    // validate subcluster id
+    checkSubClusterId(request.getSubClusterId());
+    // validate last heartbeat timestamp
+    checkTimestamp(request.getLastHeartBeat());
+    // validate subcluster capability
+    checkCapability(request.getCapability());
+    // validate subcluster state
+    checkSubClusterState(request.getState());
+
+  }
+
+  /**
+   * Quick validation on the input to check some obvious fail conditions (fail
+   * fast). Check if the provided {@link GetSubClusterInfoRequest} for querying
+   * subcluster's information is valid or not.
+   *
+   * @param request the {@link GetSubClusterInfoRequest} to validate against
+   * @throws FederationStateStoreInvalidInputException if the request is invalid
+   */
+  public static void validateGetSubClusterInfoRequest(
+      GetSubClusterInfoRequest request)
+      throws FederationStateStoreInvalidInputException {
+
+    // check if the request is present
+    if (request == null) {
+      String message = "Missing GetSubClusterInfo Request."
+          + " Please try again by specifying a Get SubCluster information.";
+      LOG.warn(message);
+      throw new FederationStateStoreInvalidInputException(message);
+    }
+
+    // validate subcluster id
+    checkSubClusterId(request.getSubClusterId());
+  }
+
+  /**
+   * Validate if the SubCluster Info are present or not.
+   *
+   * @param subClusterInfo the information of the subcluster to be verified
+   * @throws FederationStateStoreInvalidInputException if the SubCluster Info
+   *           are invalid
+   */
+  private static void checkSubClusterInfo(SubClusterInfo subClusterInfo)
+      throws FederationStateStoreInvalidInputException {
+    if (subClusterInfo == null) {
+      String message = "Missing SubCluster Information."
+          + " Please try again by specifying SubCluster Information.";
+      LOG.warn(message);
+      throw new FederationStateStoreInvalidInputException(message);
+    }
+
+    // validate subcluster id
+    checkSubClusterId(subClusterInfo.getSubClusterId());
+
+    // validate AMRM Service address
+    checkAddress(subClusterInfo.getAMRMServiceAddress());
+    // validate ClientRM Service address
+    checkAddress(subClusterInfo.getClientRMServiceAddress());
+    // validate RMClient Service address
+    checkAddress(subClusterInfo.getRMAdminServiceAddress());
+    // validate RMWeb Service address
+    checkAddress(subClusterInfo.getRMWebServiceAddress());
+
+    // validate last heartbeat timestamp
+    checkTimestamp(subClusterInfo.getLastHeartBeat());
+    // validate last start timestamp
+    checkTimestamp(subClusterInfo.getLastStartTime());
+
+    // validate subcluster state
+    checkSubClusterState(subClusterInfo.getState());
+
+    // validate subcluster capability
+    checkCapability(subClusterInfo.getCapability());
+  }
+
+  /**
+   * Validate if the timestamp is positive or not.
+   *
+   * @param timestamp the timestamp to be verified
+   * @throws FederationStateStoreInvalidInputException if the timestamp is
+   *           invalid
+   */
+  private static void checkTimestamp(long timestamp)
+      throws FederationStateStoreInvalidInputException {
+    if (timestamp < 0) {
+      String message = "Invalid timestamp information."
+          + " Please try again by specifying valid Timestamp Information.";
+      LOG.warn(message);
+      throw new FederationStateStoreInvalidInputException(message);
+    }
+  }
+
+  /**
+   * Validate if the Capability is present or not.
+   *
+   * @param capability the capability of the subcluster to be verified
+   * @throws FederationStateStoreInvalidInputException if the capability is
+   *           invalid
+   */
+  private static void checkCapability(String capability)
+      throws FederationStateStoreInvalidInputException {
+    if (capability == null || capability.isEmpty()) {
+      String message = "Invalid capability information."
+          + " Please try again by specifying valid Capability Information.";
+      LOG.warn(message);
+      throw new FederationStateStoreInvalidInputException(message);
+    }
+  }
+
+  /**
+   * Validate if the SubCluster Id is present or not.
+   *
+   * @param subClusterId the identifier of the subcluster to be verified
+   * @throws FederationStateStoreInvalidInputException if the SubCluster Id is
+   *           invalid
+   */
+  protected static void checkSubClusterId(SubClusterId subClusterId)
+      throws FederationStateStoreInvalidInputException {
+    // check if cluster id is present
+    if (subClusterId == null) {
+      String message = "Missing SubCluster Id information."
+          + " Please try again by specifying Subcluster Id information.";
+      LOG.warn(message);
+      throw new FederationStateStoreInvalidInputException(message);
+    }
+    // check if cluster id is valid
+    if (subClusterId.getId().isEmpty()) {
+      String message = "Invalid SubCluster Id information."
+          + " Please try again by specifying valid Subcluster Id.";
+      LOG.warn(message);
+      throw new FederationStateStoreInvalidInputException(message);
+    }
+  }
+
+  /**
+   * Validate if the SubCluster Address is a valid URL or not.
+   *
+   * @param address the endpoint of the subcluster to be verified
+   * @throws FederationStateStoreInvalidInputException if the address is invalid
+   */
+  private static void checkAddress(String address)
+      throws FederationStateStoreInvalidInputException {
+    // Ensure url is not null
+    if (address == null || address.isEmpty()) {
+      String message = "Missing SubCluster Endpoint information."
+          + " Please try again by specifying SubCluster Endpoint information.";
+      LOG.warn(message);
+      throw new FederationStateStoreInvalidInputException(message);
+    }
+    // Validate url is well formed
+    boolean hasScheme = address.contains("://");
+    URI uri = null;
+    try {
+      uri = hasScheme ? URI.create(address)
+          : URI.create("dummyscheme://" + address);
+    } catch (IllegalArgumentException e) {
+      String message = "The provided SubCluster Endpoint does not contain a"
+          + " valid host:port authority: " + address;
+      LOG.warn(message);
+      throw new FederationStateStoreInvalidInputException(message);
+    }
+    String host = uri.getHost();
+    int port = uri.getPort();
+    String path = uri.getPath();
+    if ((host == null) || (port < 0)
+        || (!hasScheme && path != null && !path.isEmpty())) {
+      String message = "The provided SubCluster Endpoint does not contain a"
+          + " valid host:port authority: " + address;
+      LOG.warn(message);
+      throw new FederationStateStoreInvalidInputException(message);
+    }
+  }
+
+  /**
+   * Validate if the SubCluster State is present or not.
+   *
+   * @param state the state of the subcluster to be verified
+   * @throws FederationStateStoreInvalidInputException if the SubCluster State
+   *           is invalid
+   */
+  private static void checkSubClusterState(SubClusterState state)
+      throws FederationStateStoreInvalidInputException {
+    // check sub-cluster state is not empty
+    if (state == null) {
+      String message = "Missing SubCluster State information."
+          + " Please try again by specifying SubCluster State information.";
+      LOG.warn(message);
+      throw new FederationStateStoreInvalidInputException(message);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8aa6bda7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationPolicyStoreInputValidator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationPolicyStoreInputValidator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationPolicyStoreInputValidator.java
new file mode 100644
index 0000000..273a8ac
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationPolicyStoreInputValidator.java
@@ -0,0 +1,144 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.utils;
+
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterPolicyConfigurationRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.SetSubClusterPolicyConfigurationRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyConfiguration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Utility class to validate the inputs to {@code FederationPolicyStore}, allows
+ * a fail fast mechanism for invalid user inputs.
+ *
+ */
+public final class FederationPolicyStoreInputValidator {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(FederationPolicyStoreInputValidator.class);
+
+  private FederationPolicyStoreInputValidator() {
+  }
+
+  /**
+   * Quick validation on the input to check some obvious fail conditions (fail
+   * fast). Check if the provided
+   * {@link GetSubClusterPolicyConfigurationRequest} for querying policy's
+   * information is valid or not.
+   *
+   * @param request the {@link GetSubClusterPolicyConfigurationRequest} to
+   *          validate against
+   * @throws FederationStateStoreInvalidInputException if the request is invalid
+   */
+  public static void validateGetSubClusterPolicyConfigurationRequest(
+      GetSubClusterPolicyConfigurationRequest request)
+      throws FederationStateStoreInvalidInputException {
+    if (request == null) {
+      String message = "Missing GetSubClusterPolicyConfiguration Request."
+          + " Please try again by specifying a policy selection information.";
+      LOG.warn(message);
+      throw new FederationStateStoreInvalidInputException(message);
+    }
+
+    // validate queue id
+    checkQueue(request.getQueue());
+  }
+
+  /**
+   * Quick validation on the input to check some obvious fail conditions (fail
+   * fast). Check if the provided
+   * {@link SetSubClusterPolicyConfigurationRequest} for adding a new policy is
+   * valid or not.
+   *
+   * @param request the {@link SetSubClusterPolicyConfigurationRequest} to
+   *          validate against
+   * @throws FederationStateStoreInvalidInputException if the request is invalid
+   */
+  public static void validateSetSubClusterPolicyConfigurationRequest(
+      SetSubClusterPolicyConfigurationRequest request)
+      throws FederationStateStoreInvalidInputException {
+    if (request == null) {
+      String message = "Missing SetSubClusterPolicyConfiguration Request."
+          + " Please try again by specifying an policy insertion information.";
+      LOG.warn(message);
+      throw new FederationStateStoreInvalidInputException(message);
+    }
+
+    // validate subcluster policy configuration
+    checkSubClusterPolicyConfiguration(request.getPolicyConfiguration());
+  }
+
+  /**
+   * Validate if the SubClusterPolicyConfiguration is valid or not.
+   *
+   * @param policyConfiguration the policy information to be verified
+   * @throws FederationStateStoreInvalidInputException if the policy information
+   *           are invalid
+   */
+  private static void checkSubClusterPolicyConfiguration(
+      SubClusterPolicyConfiguration policyConfiguration)
+      throws FederationStateStoreInvalidInputException {
+    if (policyConfiguration == null) {
+      String message = "Missing SubClusterPolicyConfiguration."
+          + " Please try again by specifying a SubClusterPolicyConfiguration.";
+      LOG.warn(message);
+      throw new FederationStateStoreInvalidInputException(message);
+    }
+
+    // validate queue id
+    checkQueue(policyConfiguration.getQueue());
+    // validate policy type
+    checkType(policyConfiguration.getType());
+
+  }
+
+  /**
+   * Validate if the queue id is a valid or not.
+   *
+   * @param queue the queue id of the policy to be verified
+   * @throws FederationStateStoreInvalidInputException if the queue id is
+   *           invalid
+   */
+  private static void checkQueue(String queue)
+      throws FederationStateStoreInvalidInputException {
+    if (queue == null || queue.isEmpty()) {
+      String message = "Missing Queue. Please try again by specifying a Queue.";
+      LOG.warn(message);
+      throw new FederationStateStoreInvalidInputException(message);
+    }
+  }
+
+  /**
+   * Validate if the policy type is a valid or not.
+   *
+   * @param type the type of the policy to be verified
+   * @throws FederationStateStoreInvalidInputException if the policy is invalid
+   */
+  private static void checkType(String type)
+      throws FederationStateStoreInvalidInputException {
+    if (type == null || type.isEmpty()) {
+      String message = "Missing Policy Type."
+          + " Please try again by specifying a Policy Type.";
+      LOG.warn(message);
+      throw new FederationStateStoreInvalidInputException(message);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8aa6bda7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationStateStoreInvalidInputException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationStateStoreInvalidInputException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationStateStoreInvalidInputException.java
new file mode 100644
index 0000000..ea1428d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationStateStoreInvalidInputException.java
@@ -0,0 +1,48 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.utils;
+
+import org.apache.hadoop.yarn.exceptions.YarnException;
+
+/**
+ * Exception thrown by the {@link FederationMembershipStateStoreInputValidator},
+ * {@link FederationApplicationHomeSubClusterStoreInputValidator},
+ * {@link FederationPolicyStoreInputValidator} if the input is invalid.
+ *
+ */
+public class FederationStateStoreInvalidInputException extends YarnException {
+
+  /**
+   * IDE auto-generated.
+   */
+  private static final long serialVersionUID = -7352144682711430801L;
+
+  public FederationStateStoreInvalidInputException(Throwable cause) {
+    super(cause);
+  }
+
+  public FederationStateStoreInvalidInputException(String message) {
+    super(message);
+  }
+
+  public FederationStateStoreInvalidInputException(String message,
+      Throwable cause) {
+    super(message, cause);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8aa6bda7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/package-info.java
new file mode 100644
index 0000000..f4a9c7e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/package-info.java
@@ -0,0 +1,17 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.yarn.server.federation.store.utils;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8aa6bda7/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
----------------------------------------------------------------------
diff --git a/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 b/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
index 414696b..63a5b65 100644
--- a/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
+++ b/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
@@ -162,9 +162,9 @@ public abstract class FederationStateStoreBaseTest {
         SubClusterRegisterRequest.newInstance(subClusterInfo2));
 
     stateStore.subClusterHeartbeat(SubClusterHeartbeatRequest
-        .newInstance(subClusterId1, SubClusterState.SC_RUNNING, ""));
-    stateStore.subClusterHeartbeat(SubClusterHeartbeatRequest
-        .newInstance(subClusterId2, SubClusterState.SC_UNHEALTHY, ""));
+        .newInstance(subClusterId1, SubClusterState.SC_RUNNING, "capability"));
+    stateStore.subClusterHeartbeat(SubClusterHeartbeatRequest.newInstance(
+        subClusterId2, SubClusterState.SC_UNHEALTHY, "capability"));
 
     Assert.assertTrue(
         stateStore.getSubClusters(GetSubClustersInfoRequest.newInstance(true))


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[16/42] hadoop git commit: YARN-5519. Add SubClusterId in AddApplicationHomeSubClusterResponse for Router Failover. (Ellen Hui via Subru)

Posted by su...@apache.org.
YARN-5519. Add SubClusterId in AddApplicationHomeSubClusterResponse for Router Failover. (Ellen Hui via Subru)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/35ffdef5
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/35ffdef5
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/35ffdef5

Branch: refs/heads/YARN-2915
Commit: 35ffdef5d736e9c60c4c693e8904adffa3a531c8
Parents: a82a0df
Author: Subru Krishnan <su...@apache.org>
Authored: Mon Aug 15 14:47:02 2016 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon May 8 16:59:12 2017 -0700

----------------------------------------------------------------------
 ...ederationApplicationHomeSubClusterStore.java | 21 +++---
 .../store/impl/MemoryFederationStateStore.java  | 22 +++---
 .../AddApplicationHomeSubClusterResponse.java   | 29 ++++++--
 ...ApplicationHomeSubClusterResponsePBImpl.java | 39 +++++++++++
 .../proto/yarn_server_federation_protos.proto   |  1 +
 .../impl/FederationStateStoreBaseTest.java      | 71 +++++++++-----------
 6 files changed, 120 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/35ffdef5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/FederationApplicationHomeSubClusterStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/FederationApplicationHomeSubClusterStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/FederationApplicationHomeSubClusterStore.java
index 22bb88a..ace2457 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/FederationApplicationHomeSubClusterStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/FederationApplicationHomeSubClusterStore.java
@@ -51,15 +51,20 @@ public interface FederationApplicationHomeSubClusterStore {
   /**
    * Register the home {@code SubClusterId} of the newly submitted
    * {@code ApplicationId}. Currently response is empty if the operation was
-   * successful, if not an exception reporting reason for a failure.
+   * successful, if not an exception reporting reason for a failure. If a
+   * mapping for the application already existed, the {@code SubClusterId} in
+   * this response will return the existing mapping which might be different
+   * from that in the {@code AddApplicationHomeSubClusterRequest}.
    *
    * @param request the request to register a new application with its home
    *          sub-cluster
-   * @return empty on successful registration of the application in the
-   *         StateStore, if not an exception reporting reason for a failure
+   * @return upon successful registration of the application in the StateStore,
+   *         {@code AddApplicationHomeSubClusterRequest} containing the home
+   *         sub-cluster of the application. Otherwise, an exception reporting
+   *         reason for a failure
    * @throws YarnException if the request is invalid/fails
    */
-  AddApplicationHomeSubClusterResponse addApplicationHomeSubClusterMap(
+  AddApplicationHomeSubClusterResponse addApplicationHomeSubCluster(
       AddApplicationHomeSubClusterRequest request) throws YarnException;
 
   /**
@@ -73,7 +78,7 @@ public interface FederationApplicationHomeSubClusterStore {
    *         not an exception reporting reason for a failure
    * @throws YarnException if the request is invalid/fails
    */
-  UpdateApplicationHomeSubClusterResponse updateApplicationHomeSubClusterMap(
+  UpdateApplicationHomeSubClusterResponse updateApplicationHomeSubCluster(
       UpdateApplicationHomeSubClusterRequest request) throws YarnException;
 
   /**
@@ -85,7 +90,7 @@ public interface FederationApplicationHomeSubClusterStore {
    *         subcluster
    * @throws YarnException if the request is invalid/fails
    */
-  GetApplicationHomeSubClusterResponse getApplicationHomeSubClusterMap(
+  GetApplicationHomeSubClusterResponse getApplicationHomeSubCluster(
       GetApplicationHomeSubClusterRequest request) throws YarnException;
 
   /**
@@ -96,7 +101,7 @@ public interface FederationApplicationHomeSubClusterStore {
    * @return the mapping of all submitted application to it's home sub-cluster
    * @throws YarnException if the request is invalid/fails
    */
-  GetApplicationsHomeSubClusterResponse getApplicationsHomeSubClusterMap(
+  GetApplicationsHomeSubClusterResponse getApplicationsHomeSubCluster(
       GetApplicationsHomeSubClusterRequest request) throws YarnException;
 
   /**
@@ -110,7 +115,7 @@ public interface FederationApplicationHomeSubClusterStore {
    *         not an exception reporting reason for a failure
    * @throws YarnException if the request is invalid/fails
    */
-  DeleteApplicationHomeSubClusterResponse deleteApplicationHomeSubClusterMap(
+  DeleteApplicationHomeSubClusterResponse deleteApplicationHomeSubCluster(
       DeleteApplicationHomeSubClusterRequest request) throws YarnException;
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35ffdef5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/MemoryFederationStateStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/MemoryFederationStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/MemoryFederationStateStore.java
index a540dff..8144435 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/MemoryFederationStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/MemoryFederationStateStore.java
@@ -89,7 +89,6 @@ public class MemoryFederationStateStore implements FederationStateStore {
   public SubClusterRegisterResponse registerSubCluster(
       SubClusterRegisterRequest request) throws YarnException {
     SubClusterInfo subClusterInfo = request.getSubClusterInfo();
-    subClusterInfo.setLastStartTime(clock.getTime());
     membership.put(subClusterInfo.getSubClusterId(), subClusterInfo);
     return SubClusterRegisterResponse.newInstance();
   }
@@ -156,21 +155,22 @@ public class MemoryFederationStateStore implements FederationStateStore {
   // FederationApplicationHomeSubClusterStore methods
 
   @Override
-  public AddApplicationHomeSubClusterResponse addApplicationHomeSubClusterMap(
+  public AddApplicationHomeSubClusterResponse addApplicationHomeSubCluster(
       AddApplicationHomeSubClusterRequest request) throws YarnException {
     ApplicationId appId =
         request.getApplicationHomeSubCluster().getApplicationId();
-    if (applications.containsKey(appId)) {
-      throw new YarnException("Application " + appId + " already exists");
+
+    if (!applications.containsKey(appId)) {
+      applications.put(appId,
+          request.getApplicationHomeSubCluster().getHomeSubCluster());
     }
 
-    applications.put(appId,
-        request.getApplicationHomeSubCluster().getHomeSubCluster());
-    return AddApplicationHomeSubClusterResponse.newInstance();
+    return AddApplicationHomeSubClusterResponse
+        .newInstance(applications.get(appId));
   }
 
   @Override
-  public UpdateApplicationHomeSubClusterResponse updateApplicationHomeSubClusterMap(
+  public UpdateApplicationHomeSubClusterResponse updateApplicationHomeSubCluster(
       UpdateApplicationHomeSubClusterRequest request) throws YarnException {
     ApplicationId appId =
         request.getApplicationHomeSubCluster().getApplicationId();
@@ -184,7 +184,7 @@ public class MemoryFederationStateStore implements FederationStateStore {
   }
 
   @Override
-  public GetApplicationHomeSubClusterResponse getApplicationHomeSubClusterMap(
+  public GetApplicationHomeSubClusterResponse getApplicationHomeSubCluster(
       GetApplicationHomeSubClusterRequest request) throws YarnException {
     ApplicationId appId = request.getApplicationId();
     if (!applications.containsKey(appId)) {
@@ -196,7 +196,7 @@ public class MemoryFederationStateStore implements FederationStateStore {
   }
 
   @Override
-  public GetApplicationsHomeSubClusterResponse getApplicationsHomeSubClusterMap(
+  public GetApplicationsHomeSubClusterResponse getApplicationsHomeSubCluster(
       GetApplicationsHomeSubClusterRequest request) throws YarnException {
     List<ApplicationHomeSubCluster> result =
         new ArrayList<ApplicationHomeSubCluster>();
@@ -210,7 +210,7 @@ public class MemoryFederationStateStore implements FederationStateStore {
   }
 
   @Override
-  public DeleteApplicationHomeSubClusterResponse deleteApplicationHomeSubClusterMap(
+  public DeleteApplicationHomeSubClusterResponse deleteApplicationHomeSubCluster(
       DeleteApplicationHomeSubClusterRequest request) throws YarnException {
     ApplicationId appId = request.getApplicationId();
     if (!applications.containsKey(appId)) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35ffdef5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/AddApplicationHomeSubClusterResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/AddApplicationHomeSubClusterResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/AddApplicationHomeSubClusterResponse.java
index 2145dd1..913f8e6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/AddApplicationHomeSubClusterResponse.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/AddApplicationHomeSubClusterResponse.java
@@ -24,10 +24,13 @@ import org.apache.hadoop.yarn.util.Records;
 /**
  * AddApplicationHomeSubClusterResponse contains the answer from the
  * {@code FederationApplicationHomeSubClusterStore} to a request to insert a
- * newly generated applicationId and its owner. Currently response is empty if
- * the operation was successful, if not an exception reporting reason for a
- * failure.
+ * newly generated applicationId and its owner.
  *
+ * The response contains application's home sub-cluster as it is stored in the
+ * {@code FederationApplicationHomeSubClusterStore}. If a mapping for the
+ * application already existed, the {@code SubClusterId} in this response will
+ * return the existing mapping which might be different from that in the
+ * {@code AddApplicationHomeSubClusterRequest}.
  */
 @Private
 @Unstable
@@ -35,10 +38,28 @@ public abstract class AddApplicationHomeSubClusterResponse {
 
   @Private
   @Unstable
-  public static AddApplicationHomeSubClusterResponse newInstance() {
+  public static AddApplicationHomeSubClusterResponse newInstance(
+      SubClusterId homeSubCluster) {
     AddApplicationHomeSubClusterResponse response =
         Records.newRecord(AddApplicationHomeSubClusterResponse.class);
+    response.setHomeSubCluster(homeSubCluster);
     return response;
   }
 
+  /**
+   * Set the home sub-cluster that this application has been assigned to.
+   *
+   * @param homeSubCluster the {@link SubClusterId} of this application's home
+   *          sub-cluster
+   */
+  public abstract void setHomeSubCluster(SubClusterId homeSubCluster);
+
+  /**
+   * Get the home sub-cluster that this application has been assigned to. This
+   * may not match the {@link SubClusterId} in the corresponding response, if
+   * the mapping for the request's application already existed.
+   *
+   * @return the {@link SubClusterId} of this application's home sub-cluster
+   */
+  public abstract SubClusterId getHomeSubCluster();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35ffdef5/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/AddApplicationHomeSubClusterResponsePBImpl.java
----------------------------------------------------------------------
diff --git a/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/AddApplicationHomeSubClusterResponsePBImpl.java b/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/AddApplicationHomeSubClusterResponsePBImpl.java
index 1415981..b30c41c 100644
--- a/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/AddApplicationHomeSubClusterResponsePBImpl.java
+++ b/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/AddApplicationHomeSubClusterResponsePBImpl.java
@@ -20,7 +20,10 @@ package org.apache.hadoop.yarn.server.federation.store.records.impl.pb;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.AddApplicationHomeSubClusterResponseProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.AddApplicationHomeSubClusterResponseProtoOrBuilder;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SubClusterIdProto;
 import org.apache.hadoop.yarn.server.federation.store.records.AddApplicationHomeSubClusterResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
 
 import com.google.protobuf.TextFormat;
 
@@ -48,6 +51,13 @@ public class AddApplicationHomeSubClusterResponsePBImpl
     viaProto = true;
   }
 
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = AddApplicationHomeSubClusterResponseProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
   public AddApplicationHomeSubClusterResponseProto getProto() {
     proto = viaProto ? proto : builder.build();
     viaProto = true;
@@ -55,6 +65,27 @@ public class AddApplicationHomeSubClusterResponsePBImpl
   }
 
   @Override
+  public void setHomeSubCluster(SubClusterId homeSubCluster) {
+    maybeInitBuilder();
+    if (homeSubCluster == null) {
+      builder.clearHomeSubCluster();
+      return;
+    }
+    builder.setHomeSubCluster(convertToProtoFormat(homeSubCluster));
+  }
+
+  @Override
+  public SubClusterId getHomeSubCluster() {
+    AddApplicationHomeSubClusterResponseProtoOrBuilder p =
+        viaProto ? proto : builder;
+
+    if (!p.hasHomeSubCluster()) {
+      return null;
+    }
+    return convertFromProtoFormat(p.getHomeSubCluster());
+  }
+
+  @Override
   public int hashCode() {
     return getProto().hashCode();
   }
@@ -75,4 +106,12 @@ public class AddApplicationHomeSubClusterResponsePBImpl
     return TextFormat.shortDebugString(getProto());
   }
 
+  private SubClusterId convertFromProtoFormat(SubClusterIdProto sc) {
+    return new SubClusterIdPBImpl(sc);
+  }
+
+  private SubClusterIdProto convertToProtoFormat(SubClusterId sc) {
+    return ((SubClusterIdPBImpl) sc).getProto();
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35ffdef5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_federation_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_federation_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_federation_protos.proto
index 11f786f..cedf482 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_federation_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_federation_protos.proto
@@ -102,6 +102,7 @@ message AddApplicationHomeSubClusterRequestProto {
 }
 
 message AddApplicationHomeSubClusterResponseProto {
+  optional SubClusterIdProto home_sub_cluster = 1;
 }
 
 message UpdateApplicationHomeSubClusterRequestProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35ffdef5/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
----------------------------------------------------------------------
diff --git a/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 b/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
index 165dd78..414696b 100644
--- a/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
+++ b/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
@@ -94,7 +94,7 @@ public abstract class FederationStateStoreBaseTest {
   @Test
   public void testDeregisterSubCluster() throws Exception {
     SubClusterId subClusterId = SubClusterId.newInstance("SC");
-    registerSubCluster(subClusterId);
+    registerSubCluster(createSubClusterInfo(subClusterId));
 
     SubClusterDeregisterRequest deregisterRequest = SubClusterDeregisterRequest
         .newInstance(subClusterId, SubClusterState.SC_UNREGISTERED);
@@ -124,7 +124,7 @@ public abstract class FederationStateStoreBaseTest {
 
     SubClusterId subClusterId = SubClusterId.newInstance("SC");
     SubClusterInfo subClusterInfo = createSubClusterInfo(subClusterId);
-    registerSubCluster(subClusterId);
+    registerSubCluster(subClusterInfo);
 
     GetSubClusterInfoRequest request =
         GetSubClusterInfoRequest.newInstance(subClusterId);
@@ -184,10 +184,10 @@ public abstract class FederationStateStoreBaseTest {
   @Test
   public void testSubClusterHeartbeat() throws Exception {
     SubClusterId subClusterId = SubClusterId.newInstance("SC");
-    registerSubCluster(subClusterId);
+    registerSubCluster(createSubClusterInfo(subClusterId));
 
     SubClusterHeartbeatRequest heartbeatRequest = SubClusterHeartbeatRequest
-        .newInstance(subClusterId, SubClusterState.SC_RUNNING, "cabability");
+        .newInstance(subClusterId, SubClusterState.SC_RUNNING, "capability");
     stateStore.subClusterHeartbeat(heartbeatRequest);
 
     Assert.assertEquals(SubClusterState.SC_RUNNING,
@@ -199,7 +199,7 @@ public abstract class FederationStateStoreBaseTest {
   public void testSubClusterHeartbeatUnknownSubCluster() throws Exception {
     SubClusterId subClusterId = SubClusterId.newInstance("SC");
     SubClusterHeartbeatRequest heartbeatRequest = SubClusterHeartbeatRequest
-        .newInstance(subClusterId, SubClusterState.SC_RUNNING, "cabability");
+        .newInstance(subClusterId, SubClusterState.SC_RUNNING, "capability");
 
     try {
       stateStore.subClusterHeartbeat(heartbeatRequest);
@@ -213,7 +213,7 @@ public abstract class FederationStateStoreBaseTest {
   // Test FederationApplicationHomeSubClusterStore
 
   @Test
-  public void testAddApplicationHomeSubClusterMap() throws Exception {
+  public void testAddApplicationHomeSubCluster() throws Exception {
     ApplicationId appId = ApplicationId.newInstance(1, 1);
     SubClusterId subClusterId = SubClusterId.newInstance("SC");
     ApplicationHomeSubCluster ahsc =
@@ -222,15 +222,15 @@ public abstract class FederationStateStoreBaseTest {
     AddApplicationHomeSubClusterRequest request =
         AddApplicationHomeSubClusterRequest.newInstance(ahsc);
     AddApplicationHomeSubClusterResponse response =
-        stateStore.addApplicationHomeSubClusterMap(request);
+        stateStore.addApplicationHomeSubCluster(request);
 
-    Assert.assertNotNull(response);
+    Assert.assertEquals(subClusterId, response.getHomeSubCluster());
     Assert.assertEquals(subClusterId, queryApplicationHomeSC(appId));
 
   }
 
   @Test
-  public void testAddApplicationHomeSubClusterMapAppAlreadyExists()
+  public void testAddApplicationHomeSubClusterAppAlreadyExists()
       throws Exception {
     ApplicationId appId = ApplicationId.newInstance(1, 1);
     SubClusterId subClusterId1 = SubClusterId.newInstance("SC1");
@@ -240,21 +240,17 @@ public abstract class FederationStateStoreBaseTest {
     ApplicationHomeSubCluster ahsc2 =
         ApplicationHomeSubCluster.newInstance(appId, subClusterId2);
 
-    try {
-      stateStore.addApplicationHomeSubClusterMap(
-          AddApplicationHomeSubClusterRequest.newInstance(ahsc2));
-      Assert.fail();
-    } catch (YarnException e) {
-      Assert.assertTrue(e.getMessage()
-          .startsWith("Application " + appId.toString() + " already exists"));
-    }
+    AddApplicationHomeSubClusterResponse response =
+        stateStore.addApplicationHomeSubCluster(
+            AddApplicationHomeSubClusterRequest.newInstance(ahsc2));
 
+    Assert.assertEquals(subClusterId1, response.getHomeSubCluster());
     Assert.assertEquals(subClusterId1, queryApplicationHomeSC(appId));
 
   }
 
   @Test
-  public void testDeleteApplicationHomeSubClusterMap() throws Exception {
+  public void testDeleteApplicationHomeSubCluster() throws Exception {
     ApplicationId appId = ApplicationId.newInstance(1, 1);
     SubClusterId subClusterId = SubClusterId.newInstance("SC");
     addApplicationHomeSC(appId, subClusterId);
@@ -263,7 +259,7 @@ public abstract class FederationStateStoreBaseTest {
         DeleteApplicationHomeSubClusterRequest.newInstance(appId);
 
     DeleteApplicationHomeSubClusterResponse response =
-        stateStore.deleteApplicationHomeSubClusterMap(delRequest);
+        stateStore.deleteApplicationHomeSubCluster(delRequest);
 
     Assert.assertNotNull(response);
     try {
@@ -277,14 +273,13 @@ public abstract class FederationStateStoreBaseTest {
   }
 
   @Test
-  public void testDeleteApplicationHomeSubClusterMapUnknownApp()
-      throws Exception {
+  public void testDeleteApplicationHomeSubClusterUnknownApp() throws Exception {
     ApplicationId appId = ApplicationId.newInstance(1, 1);
     DeleteApplicationHomeSubClusterRequest delRequest =
         DeleteApplicationHomeSubClusterRequest.newInstance(appId);
 
     try {
-      stateStore.deleteApplicationHomeSubClusterMap(delRequest);
+      stateStore.deleteApplicationHomeSubCluster(delRequest);
       Assert.fail();
     } catch (YarnException e) {
       Assert.assertTrue(e.getMessage()
@@ -293,7 +288,7 @@ public abstract class FederationStateStoreBaseTest {
   }
 
   @Test
-  public void testGetApplicationHomeSubClusterMap() throws Exception {
+  public void testGetApplicationHomeSubCluster() throws Exception {
     ApplicationId appId = ApplicationId.newInstance(1, 1);
     SubClusterId subClusterId = SubClusterId.newInstance("SC");
     addApplicationHomeSC(appId, subClusterId);
@@ -302,7 +297,7 @@ public abstract class FederationStateStoreBaseTest {
         GetApplicationHomeSubClusterRequest.newInstance(appId);
 
     GetApplicationHomeSubClusterResponse result =
-        stateStore.getApplicationHomeSubClusterMap(getRequest);
+        stateStore.getApplicationHomeSubCluster(getRequest);
 
     Assert.assertEquals(appId,
         result.getApplicationHomeSubCluster().getApplicationId());
@@ -311,13 +306,13 @@ public abstract class FederationStateStoreBaseTest {
   }
 
   @Test
-  public void testGetApplicationHomeSubClusterMapUnknownApp() throws Exception {
+  public void testGetApplicationHomeSubClusterUnknownApp() throws Exception {
     ApplicationId appId = ApplicationId.newInstance(1, 1);
     GetApplicationHomeSubClusterRequest request =
         GetApplicationHomeSubClusterRequest.newInstance(appId);
 
     try {
-      stateStore.getApplicationHomeSubClusterMap(request);
+      stateStore.getApplicationHomeSubCluster(request);
       Assert.fail();
     } catch (YarnException e) {
       Assert.assertTrue(e.getMessage()
@@ -326,7 +321,7 @@ public abstract class FederationStateStoreBaseTest {
   }
 
   @Test
-  public void testGetApplicationsHomeSubClusterMap() throws Exception {
+  public void testGetApplicationsHomeSubCluster() throws Exception {
     ApplicationId appId1 = ApplicationId.newInstance(1, 1);
     SubClusterId subClusterId1 = SubClusterId.newInstance("SC1");
     ApplicationHomeSubCluster ahsc1 =
@@ -344,7 +339,7 @@ public abstract class FederationStateStoreBaseTest {
         GetApplicationsHomeSubClusterRequest.newInstance();
 
     GetApplicationsHomeSubClusterResponse result =
-        stateStore.getApplicationsHomeSubClusterMap(getRequest);
+        stateStore.getApplicationsHomeSubCluster(getRequest);
 
     Assert.assertEquals(2, result.getAppsHomeSubClusters().size());
     Assert.assertTrue(result.getAppsHomeSubClusters().contains(ahsc1));
@@ -352,7 +347,7 @@ public abstract class FederationStateStoreBaseTest {
   }
 
   @Test
-  public void testUpdateApplicationHomeSubClusterMap() throws Exception {
+  public void testUpdateApplicationHomeSubCluster() throws Exception {
     ApplicationId appId = ApplicationId.newInstance(1, 1);
     SubClusterId subClusterId1 = SubClusterId.newInstance("SC1");
     addApplicationHomeSC(appId, subClusterId1);
@@ -365,16 +360,14 @@ public abstract class FederationStateStoreBaseTest {
         UpdateApplicationHomeSubClusterRequest.newInstance(ahscUpdate);
 
     UpdateApplicationHomeSubClusterResponse response =
-        stateStore.updateApplicationHomeSubClusterMap(updateRequest);
+        stateStore.updateApplicationHomeSubCluster(updateRequest);
 
     Assert.assertNotNull(response);
-
     Assert.assertEquals(subClusterId2, queryApplicationHomeSC(appId));
   }
 
   @Test
-  public void testUpdateApplicationHomeSubClusterMapUnknownApp()
-      throws Exception {
+  public void testUpdateApplicationHomeSubClusterUnknownApp() throws Exception {
     ApplicationId appId = ApplicationId.newInstance(1, 1);
     SubClusterId subClusterId1 = SubClusterId.newInstance("SC1");
     ApplicationHomeSubCluster ahsc =
@@ -384,7 +377,7 @@ public abstract class FederationStateStoreBaseTest {
         UpdateApplicationHomeSubClusterRequest.newInstance(ahsc);
 
     try {
-      stateStore.updateApplicationHomeSubClusterMap((updateRequest));
+      stateStore.updateApplicationHomeSubCluster((updateRequest));
       Assert.fail();
     } catch (YarnException e) {
       Assert.assertTrue(e.getMessage()
@@ -484,7 +477,7 @@ public abstract class FederationStateStoreBaseTest {
 
     return SubClusterInfo.newInstance(subClusterId, amRMAddress,
         clientRMAddress, rmAdminAddress, webAppAddress, SubClusterState.SC_NEW,
-        CLOCK.getTime(), "cabability");
+        CLOCK.getTime(), "capability");
   }
 
   private SubClusterPolicyConfiguration createSCPolicyConf(String queueName,
@@ -499,7 +492,7 @@ public abstract class FederationStateStoreBaseTest {
         ApplicationHomeSubCluster.newInstance(appId, subClusterId);
     AddApplicationHomeSubClusterRequest request =
         AddApplicationHomeSubClusterRequest.newInstance(ahsc);
-    stateStore.addApplicationHomeSubClusterMap(request);
+    stateStore.addApplicationHomeSubCluster(request);
   }
 
   private void setPolicyConf(String queue, String policyType)
@@ -510,10 +503,8 @@ public abstract class FederationStateStoreBaseTest {
     stateStore.setPolicyConfiguration(request);
   }
 
-  private void registerSubCluster(SubClusterId subClusterId)
+  private void registerSubCluster(SubClusterInfo subClusterInfo)
       throws YarnException {
-
-    SubClusterInfo subClusterInfo = createSubClusterInfo(subClusterId);
     stateStore.registerSubCluster(
         SubClusterRegisterRequest.newInstance(subClusterInfo));
   }
@@ -531,7 +522,7 @@ public abstract class FederationStateStoreBaseTest {
         GetApplicationHomeSubClusterRequest.newInstance(appId);
 
     GetApplicationHomeSubClusterResponse response =
-        stateStore.getApplicationHomeSubClusterMap(request);
+        stateStore.getApplicationHomeSubCluster(request);
 
     return response.getApplicationHomeSubCluster().getHomeSubCluster();
   }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[21/42] hadoop git commit: YARN-5467. InputValidator for the FederationStateStore internal APIs. (Giovanni Matteo Fumarola via Subru)

Posted by su...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/8aa6bda7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/utils/TestFederationStateStoreInputValidator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/utils/TestFederationStateStoreInputValidator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/utils/TestFederationStateStoreInputValidator.java
new file mode 100644
index 0000000..13175ae
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/utils/TestFederationStateStoreInputValidator.java
@@ -0,0 +1,1265 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.utils;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.server.federation.store.records.AddApplicationHomeSubClusterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.ApplicationHomeSubCluster;
+import org.apache.hadoop.yarn.server.federation.store.records.DeleteApplicationHomeSubClusterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.GetApplicationHomeSubClusterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterInfoRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterPolicyConfigurationRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.SetSubClusterPolicyConfigurationRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterDeregisterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterHeartbeatRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyConfiguration;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterRegisterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterState;
+import org.apache.hadoop.yarn.server.federation.store.records.UpdateApplicationHomeSubClusterRequest;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Unit tests for FederationApplicationInputValidator,
+ * FederationMembershipInputValidator, and FederationPolicyInputValidator.
+ */
+public class TestFederationStateStoreInputValidator {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestFederationStateStoreInputValidator.class);
+
+  private static SubClusterId subClusterId;
+  private static String amRMServiceAddress;
+  private static String clientRMServiceAddress;
+  private static String rmAdminServiceAddress;
+  private static String rmWebServiceAddress;
+  private static int lastHeartBeat;
+  private static SubClusterState stateNew;
+  private static SubClusterState stateLost;
+  private static ApplicationId appId;
+  private static int lastStartTime;
+  private static String capability;
+  private static String queue;
+  private static String type;
+  private static ByteBuffer params;
+
+  private static SubClusterId subClusterIdInvalid;
+  private static SubClusterId subClusterIdNull;
+
+  private static int lastHeartBeatNegative;
+  private static int lastStartTimeNegative;
+
+  private static SubClusterState stateNull;
+  private static ApplicationId appIdNull;
+
+  private static String capabilityNull;
+  private static String capabilityEmpty;
+
+  private static String addressNull;
+  private static String addressEmpty;
+  private static String addressWrong;
+  private static String addressWrongPort;
+
+  private static String queueEmpty;
+  private static String queueNull;
+
+  private static String typeEmpty;
+  private static String typeNull;
+
+  @BeforeClass
+  public static void setUp() {
+    subClusterId = SubClusterId.newInstance("abc");
+    amRMServiceAddress = "localhost:8032";
+    clientRMServiceAddress = "localhost:8034";
+    rmAdminServiceAddress = "localhost:8031";
+    rmWebServiceAddress = "localhost:8088";
+    lastHeartBeat = 1000;
+    stateNew = SubClusterState.SC_NEW;
+    stateLost = SubClusterState.SC_LOST;
+    lastStartTime = 1000;
+    capability = "Memory VCores";
+    appId = ApplicationId.newInstance(lastStartTime, 1);
+    queue = "default";
+    type = "random";
+    params = ByteBuffer.allocate(10);
+    params.put((byte) 0xFF);
+
+    subClusterIdInvalid = SubClusterId.newInstance("");
+    subClusterIdNull = null;
+
+    lastHeartBeatNegative = -10;
+    lastStartTimeNegative = -10;
+
+    stateNull = null;
+    appIdNull = null;
+
+    capabilityNull = null;
+    capabilityEmpty = "";
+
+    addressNull = null;
+    addressEmpty = "";
+    addressWrong = "AddressWrong";
+    addressWrongPort = "Address:WrongPort";
+
+    queueEmpty = "";
+    queueNull = null;
+
+    typeEmpty = "";
+    typeNull = null;
+  }
+
+  @Test
+  public void testValidateSubClusterRegisterRequest() {
+
+    // Execution with valid inputs
+
+    SubClusterInfo subClusterInfo =
+        SubClusterInfo.newInstance(subClusterId, amRMServiceAddress,
+            clientRMServiceAddress, rmAdminServiceAddress, rmWebServiceAddress,
+            lastHeartBeat, stateNew, lastStartTime, capability);
+    try {
+      SubClusterRegisterRequest request =
+          SubClusterRegisterRequest.newInstance(subClusterInfo);
+      FederationMembershipStateStoreInputValidator
+          .validateSubClusterRegisterRequest(request);
+    } catch (FederationStateStoreInvalidInputException e) {
+      Assert.fail(e.getMessage());
+    }
+
+    // Execution with null request
+
+    try {
+      SubClusterRegisterRequest request = null;
+      FederationMembershipStateStoreInputValidator
+          .validateSubClusterRegisterRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      LOG.info(e.getMessage());
+      Assert.assertTrue(
+          e.getMessage().startsWith("Missing SubClusterRegister Request."));
+    }
+
+    // Execution with null SubClusterInfo
+
+    subClusterInfo = null;
+    try {
+      SubClusterRegisterRequest request =
+          SubClusterRegisterRequest.newInstance(subClusterInfo);
+      FederationMembershipStateStoreInputValidator
+          .validateSubClusterRegisterRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      LOG.info(e.getMessage());
+      Assert.assertTrue(
+          e.getMessage().startsWith("Missing SubCluster Information."));
+    }
+
+    // Execution with Null SubClusterId
+
+    subClusterInfo =
+        SubClusterInfo.newInstance(subClusterIdNull, amRMServiceAddress,
+            clientRMServiceAddress, rmAdminServiceAddress, rmWebServiceAddress,
+            lastHeartBeat, stateNew, lastStartTime, capability);
+    try {
+      SubClusterRegisterRequest request =
+          SubClusterRegisterRequest.newInstance(subClusterInfo);
+      FederationMembershipStateStoreInputValidator
+          .validateSubClusterRegisterRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      LOG.info(e.getMessage());
+      Assert.assertTrue(
+          e.getMessage().startsWith("Missing SubCluster Id information."));
+    }
+
+    // Execution with Invalid SubClusterId
+
+    subClusterInfo =
+        SubClusterInfo.newInstance(subClusterIdInvalid, amRMServiceAddress,
+            clientRMServiceAddress, rmAdminServiceAddress, rmWebServiceAddress,
+            lastHeartBeat, stateNew, lastStartTime, capability);
+    try {
+      SubClusterRegisterRequest request =
+          SubClusterRegisterRequest.newInstance(subClusterInfo);
+      FederationMembershipStateStoreInputValidator
+          .validateSubClusterRegisterRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      LOG.info(e.getMessage());
+      Assert.assertTrue(
+          e.getMessage().startsWith("Invalid SubCluster Id information."));
+    }
+
+    // Execution with Null State
+
+    subClusterInfo =
+        SubClusterInfo.newInstance(subClusterId, amRMServiceAddress,
+            clientRMServiceAddress, rmAdminServiceAddress, rmWebServiceAddress,
+            lastHeartBeat, stateNull, lastStartTime, capability);
+    try {
+      SubClusterRegisterRequest request =
+          SubClusterRegisterRequest.newInstance(subClusterInfo);
+      FederationMembershipStateStoreInputValidator
+          .validateSubClusterRegisterRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      LOG.info(e.getMessage());
+      Assert.assertTrue(
+          e.getMessage().startsWith("Missing SubCluster State information."));
+    }
+
+    // Execution with Null Capability
+
+    subClusterInfo =
+        SubClusterInfo.newInstance(subClusterId, amRMServiceAddress,
+            clientRMServiceAddress, rmAdminServiceAddress, rmWebServiceAddress,
+            lastHeartBeat, stateNew, lastStartTime, capabilityNull);
+    try {
+      SubClusterRegisterRequest request =
+          SubClusterRegisterRequest.newInstance(subClusterInfo);
+      FederationMembershipStateStoreInputValidator
+          .validateSubClusterRegisterRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      LOG.info(e.getMessage());
+      Assert.assertTrue(
+          e.getMessage().startsWith("Invalid capability information."));
+    }
+
+    // Execution with Empty Capability
+
+    subClusterInfo =
+        SubClusterInfo.newInstance(subClusterId, amRMServiceAddress,
+            clientRMServiceAddress, rmAdminServiceAddress, rmWebServiceAddress,
+            lastHeartBeat, stateNew, lastStartTime, capabilityEmpty);
+    try {
+      SubClusterRegisterRequest request =
+          SubClusterRegisterRequest.newInstance(subClusterInfo);
+      FederationMembershipStateStoreInputValidator
+          .validateSubClusterRegisterRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      LOG.info(e.getMessage());
+      Assert.assertTrue(
+          e.getMessage().startsWith("Invalid capability information."));
+    }
+  }
+
+  @Test
+  public void testValidateSubClusterRegisterRequestTimestamp() {
+
+    // Execution with Negative Last Heartbeat
+
+    SubClusterInfo subClusterInfo =
+        SubClusterInfo.newInstance(subClusterId, amRMServiceAddress,
+            clientRMServiceAddress, rmAdminServiceAddress, rmWebServiceAddress,
+            lastHeartBeatNegative, stateNew, lastStartTime, capability);
+    try {
+      SubClusterRegisterRequest request =
+          SubClusterRegisterRequest.newInstance(subClusterInfo);
+      FederationMembershipStateStoreInputValidator
+          .validateSubClusterRegisterRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      LOG.info(e.getMessage());
+      Assert.assertTrue(
+          e.getMessage().startsWith("Invalid timestamp information."));
+    }
+
+    // Execution with Negative Last StartTime
+
+    subClusterInfo =
+        SubClusterInfo.newInstance(subClusterId, amRMServiceAddress,
+            clientRMServiceAddress, rmAdminServiceAddress, rmWebServiceAddress,
+            lastHeartBeat, stateNew, lastStartTimeNegative, capability);
+    try {
+      SubClusterRegisterRequest request =
+          SubClusterRegisterRequest.newInstance(subClusterInfo);
+      FederationMembershipStateStoreInputValidator
+          .validateSubClusterRegisterRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      LOG.info(e.getMessage());
+      Assert.assertTrue(
+          e.getMessage().startsWith("Invalid timestamp information."));
+    }
+  }
+
+  @Test
+  public void testValidateSubClusterRegisterRequestAddress() {
+    // Execution with Null Address for amRMServiceAddress
+
+    SubClusterInfo subClusterInfo =
+        SubClusterInfo.newInstance(subClusterId, addressNull,
+            clientRMServiceAddress, rmAdminServiceAddress, rmWebServiceAddress,
+            lastHeartBeat, stateNew, lastStartTime, capability);
+    try {
+      SubClusterRegisterRequest request =
+          SubClusterRegisterRequest.newInstance(subClusterInfo);
+      FederationMembershipStateStoreInputValidator
+          .validateSubClusterRegisterRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      LOG.info(e.getMessage());
+      Assert.assertTrue(e.getMessage()
+          .startsWith("Missing SubCluster Endpoint information."));
+    }
+
+    // Execution with Empty Address for amRMServiceAddress
+
+    subClusterInfo = SubClusterInfo.newInstance(subClusterId, addressEmpty,
+        clientRMServiceAddress, rmAdminServiceAddress, rmWebServiceAddress,
+        lastHeartBeat, stateNew, lastStartTime, capability);
+    try {
+      SubClusterRegisterRequest request =
+          SubClusterRegisterRequest.newInstance(subClusterInfo);
+      FederationMembershipStateStoreInputValidator
+          .validateSubClusterRegisterRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      LOG.info(e.getMessage());
+      Assert.assertTrue(e.getMessage()
+          .startsWith("Missing SubCluster Endpoint information."));
+    }
+
+    // Execution with Null Address for clientRMServiceAddress
+
+    subClusterInfo =
+        SubClusterInfo.newInstance(subClusterId, amRMServiceAddress,
+            addressNull, rmAdminServiceAddress, rmWebServiceAddress,
+            lastHeartBeat, stateNew, lastStartTime, capability);
+    try {
+      SubClusterRegisterRequest request =
+          SubClusterRegisterRequest.newInstance(subClusterInfo);
+      FederationMembershipStateStoreInputValidator
+          .validateSubClusterRegisterRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      LOG.info(e.getMessage());
+      Assert.assertTrue(e.getMessage()
+          .startsWith("Missing SubCluster Endpoint information."));
+    }
+
+    // Execution with Empty Address for clientRMServiceAddress
+
+    subClusterInfo =
+        SubClusterInfo.newInstance(subClusterId, amRMServiceAddress,
+            addressEmpty, rmAdminServiceAddress, rmWebServiceAddress,
+            lastHeartBeat, stateNew, lastStartTime, capability);
+    try {
+      SubClusterRegisterRequest request =
+          SubClusterRegisterRequest.newInstance(subClusterInfo);
+      FederationMembershipStateStoreInputValidator
+          .validateSubClusterRegisterRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      LOG.info(e.getMessage());
+      Assert.assertTrue(e.getMessage()
+          .startsWith("Missing SubCluster Endpoint information."));
+    }
+
+    // Execution with Null Address for rmAdminServiceAddress
+
+    subClusterInfo =
+        SubClusterInfo.newInstance(subClusterId, amRMServiceAddress,
+            clientRMServiceAddress, addressNull, rmWebServiceAddress,
+            lastHeartBeat, stateNew, lastStartTime, capability);
+    try {
+      SubClusterRegisterRequest request =
+          SubClusterRegisterRequest.newInstance(subClusterInfo);
+      FederationMembershipStateStoreInputValidator
+          .validateSubClusterRegisterRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      LOG.info(e.getMessage());
+      Assert.assertTrue(e.getMessage()
+          .startsWith("Missing SubCluster Endpoint information."));
+    }
+
+    // Execution with Empty Address for rmAdminServiceAddress
+
+    subClusterInfo =
+        SubClusterInfo.newInstance(subClusterId, amRMServiceAddress,
+            clientRMServiceAddress, addressEmpty, rmWebServiceAddress,
+            lastHeartBeat, stateNew, lastStartTime, capability);
+    try {
+      SubClusterRegisterRequest request =
+          SubClusterRegisterRequest.newInstance(subClusterInfo);
+      FederationMembershipStateStoreInputValidator
+          .validateSubClusterRegisterRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      LOG.info(e.getMessage());
+      Assert.assertTrue(e.getMessage()
+          .startsWith("Missing SubCluster Endpoint information."));
+    }
+
+    // Execution with Null Address for rmWebServiceAddress
+
+    subClusterInfo = SubClusterInfo.newInstance(subClusterId,
+        amRMServiceAddress, clientRMServiceAddress, rmAdminServiceAddress,
+        addressNull, lastHeartBeat, stateNew, lastStartTime, capability);
+    try {
+      SubClusterRegisterRequest request =
+          SubClusterRegisterRequest.newInstance(subClusterInfo);
+      FederationMembershipStateStoreInputValidator
+          .validateSubClusterRegisterRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      LOG.info(e.getMessage());
+      Assert.assertTrue(e.getMessage()
+          .startsWith("Missing SubCluster Endpoint information."));
+    }
+
+    // Execution with Empty Address for rmWebServiceAddress
+
+    subClusterInfo = SubClusterInfo.newInstance(subClusterId,
+        amRMServiceAddress, clientRMServiceAddress, rmAdminServiceAddress,
+        addressEmpty, lastHeartBeat, stateNew, lastStartTime, capability);
+    try {
+      SubClusterRegisterRequest request =
+          SubClusterRegisterRequest.newInstance(subClusterInfo);
+      FederationMembershipStateStoreInputValidator
+          .validateSubClusterRegisterRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      LOG.info(e.getMessage());
+      Assert.assertTrue(e.getMessage()
+          .startsWith("Missing SubCluster Endpoint information."));
+    }
+  }
+
+  @Test
+  public void testValidateSubClusterRegisterRequestAddressInvalid() {
+
+    // Address is not in host:port format for amRMService
+
+    SubClusterInfo subClusterInfo =
+        SubClusterInfo.newInstance(subClusterId, addressWrong,
+            clientRMServiceAddress, rmAdminServiceAddress, rmWebServiceAddress,
+            lastHeartBeat, stateNull, lastStartTime, capability);
+    try {
+      SubClusterRegisterRequest request =
+          SubClusterRegisterRequest.newInstance(subClusterInfo);
+      FederationMembershipStateStoreInputValidator
+          .validateSubClusterRegisterRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      LOG.info(e.getMessage());
+      Assert.assertTrue(e.getMessage().contains("valid host:port authority:"));
+    }
+
+    // Address is not in host:port format for clientRMService
+
+    subClusterInfo =
+        SubClusterInfo.newInstance(subClusterId, amRMServiceAddress,
+            addressWrong, rmAdminServiceAddress, rmWebServiceAddress,
+            lastHeartBeat, stateNull, lastStartTime, capability);
+    try {
+      SubClusterRegisterRequest request =
+          SubClusterRegisterRequest.newInstance(subClusterInfo);
+      FederationMembershipStateStoreInputValidator
+          .validateSubClusterRegisterRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      LOG.info(e.getMessage());
+      Assert.assertTrue(e.getMessage().contains("valid host:port authority:"));
+    }
+
+    // Address is not in host:port format for rmAdminService
+
+    subClusterInfo =
+        SubClusterInfo.newInstance(subClusterId, amRMServiceAddress,
+            clientRMServiceAddress, addressWrong, rmWebServiceAddress,
+            lastHeartBeat, stateNull, lastStartTime, capability);
+    try {
+      SubClusterRegisterRequest request =
+          SubClusterRegisterRequest.newInstance(subClusterInfo);
+      FederationMembershipStateStoreInputValidator
+          .validateSubClusterRegisterRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      LOG.info(e.getMessage());
+      Assert.assertTrue(e.getMessage().contains("valid host:port authority:"));
+    }
+
+    // Address is not in host:port format for rmWebService
+
+    subClusterInfo = SubClusterInfo.newInstance(subClusterId,
+        amRMServiceAddress, clientRMServiceAddress, rmAdminServiceAddress,
+        addressWrong, lastHeartBeat, stateNull, lastStartTime, capability);
+    try {
+      SubClusterRegisterRequest request =
+          SubClusterRegisterRequest.newInstance(subClusterInfo);
+      FederationMembershipStateStoreInputValidator
+          .validateSubClusterRegisterRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      LOG.info(e.getMessage());
+      Assert.assertTrue(e.getMessage().contains("valid host:port authority:"));
+    }
+
+    // Port is not an integer for amRMService
+
+    subClusterInfo = SubClusterInfo.newInstance(subClusterId, addressWrongPort,
+        clientRMServiceAddress, rmAdminServiceAddress, rmWebServiceAddress,
+        lastHeartBeat, stateNull, lastStartTime, capability);
+    try {
+      SubClusterRegisterRequest request =
+          SubClusterRegisterRequest.newInstance(subClusterInfo);
+      FederationMembershipStateStoreInputValidator
+          .validateSubClusterRegisterRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      LOG.info(e.getMessage());
+      Assert.assertTrue(e.getMessage().contains("valid host:port authority:"));
+    }
+
+    // Port is not an integer for clientRMService
+
+    subClusterInfo =
+        SubClusterInfo.newInstance(subClusterId, amRMServiceAddress,
+            addressWrongPort, rmAdminServiceAddress, rmWebServiceAddress,
+            lastHeartBeat, stateNull, lastStartTime, capability);
+    try {
+      SubClusterRegisterRequest request =
+          SubClusterRegisterRequest.newInstance(subClusterInfo);
+      FederationMembershipStateStoreInputValidator
+          .validateSubClusterRegisterRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      LOG.info(e.getMessage());
+      Assert.assertTrue(e.getMessage().contains("valid host:port authority:"));
+    }
+
+    // Port is not an integer for rmAdminService
+
+    subClusterInfo =
+        SubClusterInfo.newInstance(subClusterId, amRMServiceAddress,
+            clientRMServiceAddress, addressWrongPort, rmWebServiceAddress,
+            lastHeartBeat, stateNull, lastStartTime, capability);
+    try {
+      SubClusterRegisterRequest request =
+          SubClusterRegisterRequest.newInstance(subClusterInfo);
+      FederationMembershipStateStoreInputValidator
+          .validateSubClusterRegisterRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      LOG.info(e.getMessage());
+      Assert.assertTrue(e.getMessage().contains("valid host:port authority:"));
+    }
+
+    // Port is not an integer for rmWebService
+
+    subClusterInfo = SubClusterInfo.newInstance(subClusterId,
+        amRMServiceAddress, clientRMServiceAddress, rmAdminServiceAddress,
+        addressWrongPort, lastHeartBeat, stateNull, lastStartTime, capability);
+    try {
+      SubClusterRegisterRequest request =
+          SubClusterRegisterRequest.newInstance(subClusterInfo);
+      FederationMembershipStateStoreInputValidator
+          .validateSubClusterRegisterRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      LOG.info(e.getMessage());
+      Assert.assertTrue(e.getMessage().contains("valid host:port authority:"));
+    }
+
+  }
+
+  @Test
+  public void testValidateSubClusterDeregisterRequest() {
+
+    // Execution with valid inputs
+
+    try {
+      SubClusterDeregisterRequest request =
+          SubClusterDeregisterRequest.newInstance(subClusterId, stateLost);
+      FederationMembershipStateStoreInputValidator
+          .validateSubClusterDeregisterRequest(request);
+    } catch (FederationStateStoreInvalidInputException e) {
+      Assert.fail(e.getMessage());
+    }
+
+    // Execution with null request
+
+    try {
+      SubClusterDeregisterRequest request = null;
+      FederationMembershipStateStoreInputValidator
+          .validateSubClusterDeregisterRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      LOG.info(e.getMessage());
+      Assert.assertTrue(
+          e.getMessage().startsWith("Missing SubClusterDeregister Request."));
+    }
+
+    // Execution with null SubClusterId
+
+    try {
+      SubClusterDeregisterRequest request =
+          SubClusterDeregisterRequest.newInstance(subClusterIdNull, stateLost);
+      FederationMembershipStateStoreInputValidator
+          .validateSubClusterDeregisterRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      LOG.info(e.getMessage());
+      Assert.assertTrue(
+          e.getMessage().startsWith("Missing SubCluster Id information."));
+    }
+
+    // Execution with invalid SubClusterId
+
+    try {
+      SubClusterDeregisterRequest request = SubClusterDeregisterRequest
+          .newInstance(subClusterIdInvalid, stateLost);
+      FederationMembershipStateStoreInputValidator
+          .validateSubClusterDeregisterRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      LOG.info(e.getMessage());
+      Assert.assertTrue(
+          e.getMessage().startsWith("Invalid SubCluster Id information."));
+    }
+
+    // Execution with null SubClusterState
+
+    try {
+      SubClusterDeregisterRequest request =
+          SubClusterDeregisterRequest.newInstance(subClusterId, stateNull);
+      FederationMembershipStateStoreInputValidator
+          .validateSubClusterDeregisterRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      LOG.info(e.getMessage());
+      Assert.assertTrue(
+          e.getMessage().startsWith("Missing SubCluster State information."));
+    }
+
+    // Execution with invalid SubClusterState
+
+    try {
+      SubClusterDeregisterRequest request =
+          SubClusterDeregisterRequest.newInstance(subClusterId, stateNew);
+      FederationMembershipStateStoreInputValidator
+          .validateSubClusterDeregisterRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      LOG.info(e.getMessage());
+      Assert.assertTrue(e.getMessage().startsWith("Invalid non-final state: "));
+    }
+  }
+
+  @Test
+  public void testSubClusterHeartbeatRequest() {
+
+    // Execution with valid inputs
+
+    try {
+      SubClusterHeartbeatRequest request = SubClusterHeartbeatRequest
+          .newInstance(subClusterId, lastHeartBeat, stateLost, capability);
+      FederationMembershipStateStoreInputValidator
+          .validateSubClusterHeartbeatRequest(request);
+    } catch (FederationStateStoreInvalidInputException e) {
+      Assert.fail(e.getMessage());
+    }
+
+    // Execution with null request
+
+    try {
+      SubClusterHeartbeatRequest request = null;
+      FederationMembershipStateStoreInputValidator
+          .validateSubClusterHeartbeatRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      LOG.info(e.getMessage());
+      Assert.assertTrue(
+          e.getMessage().startsWith("Missing SubClusterHeartbeat Request."));
+    }
+
+    // Execution with null SubClusterId
+
+    try {
+      SubClusterHeartbeatRequest request = SubClusterHeartbeatRequest
+          .newInstance(subClusterIdNull, lastHeartBeat, stateLost, capability);
+      FederationMembershipStateStoreInputValidator
+          .validateSubClusterHeartbeatRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      LOG.info(e.getMessage());
+      Assert.assertTrue(
+          e.getMessage().startsWith("Missing SubCluster Id information."));
+    }
+
+    // Execution with invalid SubClusterId
+
+    try {
+      SubClusterHeartbeatRequest request =
+          SubClusterHeartbeatRequest.newInstance(subClusterIdInvalid,
+              lastHeartBeat, stateLost, capability);
+      FederationMembershipStateStoreInputValidator
+          .validateSubClusterHeartbeatRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      LOG.info(e.getMessage());
+      Assert.assertTrue(
+          e.getMessage().startsWith("Invalid SubCluster Id information."));
+    }
+
+    // Execution with null SubClusterState
+
+    try {
+      SubClusterHeartbeatRequest request = SubClusterHeartbeatRequest
+          .newInstance(subClusterId, lastHeartBeat, stateNull, capability);
+      FederationMembershipStateStoreInputValidator
+          .validateSubClusterHeartbeatRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      LOG.info(e.getMessage());
+      Assert.assertTrue(
+          e.getMessage().startsWith("Missing SubCluster State information."));
+    }
+
+    // Execution with negative Last Heartbeat
+
+    try {
+      SubClusterHeartbeatRequest request =
+          SubClusterHeartbeatRequest.newInstance(subClusterId,
+              lastHeartBeatNegative, stateLost, capability);
+      FederationMembershipStateStoreInputValidator
+          .validateSubClusterHeartbeatRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      LOG.info(e.getMessage());
+      Assert.assertTrue(
+          e.getMessage().startsWith("Invalid timestamp information."));
+    }
+
+    // Execution with null Capability
+
+    try {
+      SubClusterHeartbeatRequest request = SubClusterHeartbeatRequest
+          .newInstance(subClusterId, lastHeartBeat, stateLost, capabilityNull);
+      FederationMembershipStateStoreInputValidator
+          .validateSubClusterHeartbeatRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      LOG.info(e.getMessage());
+      Assert.assertTrue(
+          e.getMessage().startsWith("Invalid capability information."));
+    }
+
+    // Execution with empty Capability
+
+    try {
+      SubClusterHeartbeatRequest request = SubClusterHeartbeatRequest
+          .newInstance(subClusterId, lastHeartBeat, stateLost, capabilityEmpty);
+      FederationMembershipStateStoreInputValidator
+          .validateSubClusterHeartbeatRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      LOG.info(e.getMessage());
+      Assert.assertTrue(
+          e.getMessage().startsWith("Invalid capability information."));
+    }
+  }
+
+  @Test
+  public void testGetSubClusterInfoRequest() {
+
+    // Execution with valid inputs
+
+    try {
+      GetSubClusterInfoRequest request =
+          GetSubClusterInfoRequest.newInstance(subClusterId);
+      FederationMembershipStateStoreInputValidator
+          .validateGetSubClusterInfoRequest(request);
+    } catch (FederationStateStoreInvalidInputException e) {
+      Assert.fail(e.getMessage());
+    }
+
+    // Execution with null request
+
+    try {
+      GetSubClusterInfoRequest request = null;
+      FederationMembershipStateStoreInputValidator
+          .validateGetSubClusterInfoRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      LOG.info(e.getMessage());
+      Assert.assertTrue(
+          e.getMessage().startsWith("Missing GetSubClusterInfo Request."));
+    }
+
+    // Execution with null SubClusterId
+
+    try {
+      GetSubClusterInfoRequest request =
+          GetSubClusterInfoRequest.newInstance(subClusterIdNull);
+      FederationMembershipStateStoreInputValidator
+          .validateGetSubClusterInfoRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      LOG.info(e.getMessage());
+      Assert.assertTrue(
+          e.getMessage().startsWith("Missing SubCluster Id information."));
+    }
+
+    // Execution with invalid SubClusterId
+
+    try {
+      GetSubClusterInfoRequest request =
+          GetSubClusterInfoRequest.newInstance(subClusterIdInvalid);
+      FederationMembershipStateStoreInputValidator
+          .validateGetSubClusterInfoRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      LOG.info(e.getMessage());
+      Assert.assertTrue(
+          e.getMessage().startsWith("Invalid SubCluster Id information."));
+    }
+  }
+
+  @Test
+  public void testAddApplicationHomeSubClusterRequest() {
+
+    // Execution with valid inputs
+
+    ApplicationHomeSubCluster applicationHomeSubCluster =
+        ApplicationHomeSubCluster.newInstance(appId, subClusterId);
+    try {
+      AddApplicationHomeSubClusterRequest request =
+          AddApplicationHomeSubClusterRequest
+              .newInstance(applicationHomeSubCluster);
+      FederationApplicationHomeSubClusterStoreInputValidator
+          .validateAddApplicationHomeSubClusterRequest(request);
+    } catch (FederationStateStoreInvalidInputException e) {
+      Assert.fail(e.getMessage());
+    }
+
+    // Execution with null request
+
+    try {
+      AddApplicationHomeSubClusterRequest request = null;
+      FederationApplicationHomeSubClusterStoreInputValidator
+          .validateAddApplicationHomeSubClusterRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      Assert.assertTrue(e.getMessage()
+          .startsWith("Missing AddApplicationHomeSubCluster Request."));
+    }
+
+    // Execution with null ApplicationHomeSubCluster
+
+    applicationHomeSubCluster = null;
+    try {
+      AddApplicationHomeSubClusterRequest request =
+          AddApplicationHomeSubClusterRequest
+              .newInstance(applicationHomeSubCluster);
+      FederationApplicationHomeSubClusterStoreInputValidator
+          .validateAddApplicationHomeSubClusterRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      Assert.assertTrue(
+          e.getMessage().startsWith("Missing ApplicationHomeSubCluster Info."));
+    }
+
+    // Execution with null SubClusterId
+
+    applicationHomeSubCluster =
+        ApplicationHomeSubCluster.newInstance(appId, subClusterIdNull);
+    try {
+      AddApplicationHomeSubClusterRequest request =
+          AddApplicationHomeSubClusterRequest
+              .newInstance(applicationHomeSubCluster);
+      FederationApplicationHomeSubClusterStoreInputValidator
+          .validateAddApplicationHomeSubClusterRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      LOG.info(e.getMessage());
+      Assert.assertTrue(
+          e.getMessage().startsWith("Missing SubCluster Id information."));
+    }
+
+    // Execution with invalid SubClusterId
+
+    applicationHomeSubCluster =
+        ApplicationHomeSubCluster.newInstance(appId, subClusterIdInvalid);
+    try {
+      AddApplicationHomeSubClusterRequest request =
+          AddApplicationHomeSubClusterRequest
+              .newInstance(applicationHomeSubCluster);
+      FederationApplicationHomeSubClusterStoreInputValidator
+          .validateAddApplicationHomeSubClusterRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      LOG.info(e.getMessage());
+      Assert.assertTrue(
+          e.getMessage().startsWith("Invalid SubCluster Id information."));
+    }
+
+    // Execution with Null ApplicationId
+
+    applicationHomeSubCluster =
+        ApplicationHomeSubCluster.newInstance(appIdNull, subClusterId);
+    try {
+      AddApplicationHomeSubClusterRequest request =
+          AddApplicationHomeSubClusterRequest
+              .newInstance(applicationHomeSubCluster);
+      FederationApplicationHomeSubClusterStoreInputValidator
+          .validateAddApplicationHomeSubClusterRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      Assert.assertTrue(e.getMessage().startsWith("Missing Application Id."));
+    }
+  }
+
+  @Test
+  public void testUpdateApplicationHomeSubClusterRequest() {
+
+    // Execution with valid inputs
+
+    ApplicationHomeSubCluster applicationHomeSubCluster =
+        ApplicationHomeSubCluster.newInstance(appId, subClusterId);
+    try {
+      UpdateApplicationHomeSubClusterRequest request =
+          UpdateApplicationHomeSubClusterRequest
+              .newInstance(applicationHomeSubCluster);
+      FederationApplicationHomeSubClusterStoreInputValidator
+          .validateUpdateApplicationHomeSubClusterRequest(request);
+    } catch (FederationStateStoreInvalidInputException e) {
+      Assert.fail(e.getMessage());
+    }
+
+    // Execution with null request
+
+    try {
+      UpdateApplicationHomeSubClusterRequest request = null;
+      FederationApplicationHomeSubClusterStoreInputValidator
+          .validateUpdateApplicationHomeSubClusterRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      Assert.assertTrue(e.getMessage()
+          .startsWith("Missing UpdateApplicationHomeSubCluster Request."));
+    }
+
+    // Execution with null ApplicationHomeSubCluster
+
+    applicationHomeSubCluster = null;
+    try {
+      UpdateApplicationHomeSubClusterRequest request =
+          UpdateApplicationHomeSubClusterRequest
+              .newInstance(applicationHomeSubCluster);
+      FederationApplicationHomeSubClusterStoreInputValidator
+          .validateUpdateApplicationHomeSubClusterRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      Assert.assertTrue(
+          e.getMessage().startsWith("Missing ApplicationHomeSubCluster Info."));
+    }
+
+    // Execution with null SubClusteId
+
+    applicationHomeSubCluster =
+        ApplicationHomeSubCluster.newInstance(appId, subClusterIdNull);
+    try {
+      UpdateApplicationHomeSubClusterRequest request =
+          UpdateApplicationHomeSubClusterRequest
+              .newInstance(applicationHomeSubCluster);
+      FederationApplicationHomeSubClusterStoreInputValidator
+          .validateUpdateApplicationHomeSubClusterRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      LOG.info(e.getMessage());
+      Assert.assertTrue(
+          e.getMessage().startsWith("Missing SubCluster Id information."));
+    }
+
+    // Execution with invalid SubClusterId
+
+    applicationHomeSubCluster =
+        ApplicationHomeSubCluster.newInstance(appId, subClusterIdInvalid);
+    try {
+      UpdateApplicationHomeSubClusterRequest request =
+          UpdateApplicationHomeSubClusterRequest
+              .newInstance(applicationHomeSubCluster);
+      FederationApplicationHomeSubClusterStoreInputValidator
+          .validateUpdateApplicationHomeSubClusterRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      LOG.info(e.getMessage());
+      Assert.assertTrue(
+          e.getMessage().startsWith("Invalid SubCluster Id information."));
+    }
+
+    // Execution with null ApplicationId
+
+    applicationHomeSubCluster =
+        ApplicationHomeSubCluster.newInstance(appIdNull, subClusterId);
+    try {
+      UpdateApplicationHomeSubClusterRequest request =
+          UpdateApplicationHomeSubClusterRequest
+              .newInstance(applicationHomeSubCluster);
+      FederationApplicationHomeSubClusterStoreInputValidator
+          .validateUpdateApplicationHomeSubClusterRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      Assert.assertTrue(e.getMessage().startsWith("Missing Application Id."));
+    }
+  }
+
+  @Test
+  public void testGetApplicationHomeSubClusterRequest() {
+
+    // Execution with valid inputs
+
+    try {
+      GetApplicationHomeSubClusterRequest request =
+          GetApplicationHomeSubClusterRequest.newInstance(appId);
+      FederationApplicationHomeSubClusterStoreInputValidator
+          .validateGetApplicationHomeSubClusterRequest(request);
+    } catch (FederationStateStoreInvalidInputException e) {
+      Assert.fail(e.getMessage());
+    }
+
+    // Execution with null request
+
+    try {
+      GetApplicationHomeSubClusterRequest request = null;
+      FederationApplicationHomeSubClusterStoreInputValidator
+          .validateGetApplicationHomeSubClusterRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      Assert.assertTrue(e.getMessage()
+          .startsWith("Missing GetApplicationHomeSubCluster Request."));
+    }
+
+    // Execution with null ApplicationId
+
+    try {
+      GetApplicationHomeSubClusterRequest request =
+          GetApplicationHomeSubClusterRequest.newInstance(appIdNull);
+      FederationApplicationHomeSubClusterStoreInputValidator
+          .validateGetApplicationHomeSubClusterRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      Assert.assertTrue(e.getMessage().startsWith("Missing Application Id."));
+    }
+
+  }
+
+  @Test
+  public void testDeleteApplicationHomeSubClusterRequestNull() {
+
+    // Execution with valid inputs
+
+    try {
+      DeleteApplicationHomeSubClusterRequest request =
+          DeleteApplicationHomeSubClusterRequest.newInstance(appId);
+      FederationApplicationHomeSubClusterStoreInputValidator
+          .validateDeleteApplicationHomeSubClusterRequest(request);
+    } catch (FederationStateStoreInvalidInputException e) {
+      Assert.fail(e.getMessage());
+    }
+
+    // Execution with null request
+
+    try {
+      DeleteApplicationHomeSubClusterRequest request = null;
+      FederationApplicationHomeSubClusterStoreInputValidator
+          .validateDeleteApplicationHomeSubClusterRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      Assert.assertTrue(e.getMessage()
+          .startsWith("Missing DeleteApplicationHomeSubCluster Request."));
+    }
+
+    // Execution with null ApplicationId
+
+    try {
+      DeleteApplicationHomeSubClusterRequest request =
+          DeleteApplicationHomeSubClusterRequest.newInstance(appIdNull);
+      FederationApplicationHomeSubClusterStoreInputValidator
+          .validateDeleteApplicationHomeSubClusterRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      Assert.assertTrue(e.getMessage().startsWith("Missing Application Id."));
+    }
+
+  }
+
+  @Test
+  public void testGetSubClusterPolicyConfigurationRequest() {
+
+    // Execution with valid inputs
+
+    try {
+      GetSubClusterPolicyConfigurationRequest request =
+          GetSubClusterPolicyConfigurationRequest.newInstance(queue);
+      FederationPolicyStoreInputValidator
+          .validateGetSubClusterPolicyConfigurationRequest(request);
+    } catch (FederationStateStoreInvalidInputException e) {
+      Assert.fail(e.getMessage());
+    }
+
+    // Execution with null request
+
+    try {
+      GetSubClusterPolicyConfigurationRequest request = null;
+      FederationPolicyStoreInputValidator
+          .validateGetSubClusterPolicyConfigurationRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      Assert.assertTrue(e.getMessage()
+          .startsWith("Missing GetSubClusterPolicyConfiguration Request."));
+    }
+
+    // Execution with null queue id
+
+    try {
+      GetSubClusterPolicyConfigurationRequest request =
+          GetSubClusterPolicyConfigurationRequest.newInstance(queueNull);
+      FederationPolicyStoreInputValidator
+          .validateGetSubClusterPolicyConfigurationRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      Assert.assertTrue(e.getMessage().startsWith("Missing Queue."));
+    }
+
+    // Execution with empty queue id
+
+    try {
+      GetSubClusterPolicyConfigurationRequest request =
+          GetSubClusterPolicyConfigurationRequest.newInstance(queueEmpty);
+      FederationPolicyStoreInputValidator
+          .validateGetSubClusterPolicyConfigurationRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      Assert.assertTrue(e.getMessage().startsWith("Missing Queue."));
+    }
+
+  }
+
+  @Test
+  public void testSetSubClusterPolicyConfigurationRequest() {
+
+    // Execution with valid inputs
+
+    try {
+      SubClusterPolicyConfiguration policy =
+          SubClusterPolicyConfiguration.newInstance(queue, type, params);
+      SetSubClusterPolicyConfigurationRequest request =
+          SetSubClusterPolicyConfigurationRequest.newInstance(policy);
+      FederationPolicyStoreInputValidator
+          .validateSetSubClusterPolicyConfigurationRequest(request);
+    } catch (FederationStateStoreInvalidInputException e) {
+      Assert.fail(e.getMessage());
+    }
+
+    // Execution with null request
+
+    try {
+      SetSubClusterPolicyConfigurationRequest request = null;
+      FederationPolicyStoreInputValidator
+          .validateSetSubClusterPolicyConfigurationRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      Assert.assertTrue(e.getMessage()
+          .startsWith("Missing SetSubClusterPolicyConfiguration Request."));
+    }
+
+    // Execution with null SubClusterPolicyConfiguration
+
+    try {
+      SubClusterPolicyConfiguration policy = null;
+      SetSubClusterPolicyConfigurationRequest request =
+          SetSubClusterPolicyConfigurationRequest.newInstance(policy);
+      FederationPolicyStoreInputValidator
+          .validateSetSubClusterPolicyConfigurationRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      Assert.assertTrue(
+          e.getMessage().startsWith("Missing SubClusterPolicyConfiguration."));
+    }
+
+    // Execution with null queue id
+
+    try {
+      SubClusterPolicyConfiguration policy =
+          SubClusterPolicyConfiguration.newInstance(queueNull, type, params);
+      SetSubClusterPolicyConfigurationRequest request =
+          SetSubClusterPolicyConfigurationRequest.newInstance(policy);
+      FederationPolicyStoreInputValidator
+          .validateSetSubClusterPolicyConfigurationRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      Assert.assertTrue(e.getMessage().startsWith("Missing Queue."));
+    }
+
+    // Execution with empty queue id
+
+    try {
+      SubClusterPolicyConfiguration policy =
+          SubClusterPolicyConfiguration.newInstance(queueEmpty, type, params);
+      SetSubClusterPolicyConfigurationRequest request =
+          SetSubClusterPolicyConfigurationRequest.newInstance(policy);
+      FederationPolicyStoreInputValidator
+          .validateSetSubClusterPolicyConfigurationRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      Assert.assertTrue(e.getMessage().startsWith("Missing Queue."));
+    }
+
+    // Execution with null policy type
+
+    try {
+      SubClusterPolicyConfiguration policy =
+          SubClusterPolicyConfiguration.newInstance(queue, typeNull, params);
+      SetSubClusterPolicyConfigurationRequest request =
+          SetSubClusterPolicyConfigurationRequest.newInstance(policy);
+      FederationPolicyStoreInputValidator
+          .validateSetSubClusterPolicyConfigurationRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      Assert.assertTrue(e.getMessage().startsWith("Missing Policy Type."));
+    }
+
+    // Execution with empty policy type
+
+    try {
+      SubClusterPolicyConfiguration policy =
+          SubClusterPolicyConfiguration.newInstance(queue, typeEmpty, params);
+      SetSubClusterPolicyConfigurationRequest request =
+          SetSubClusterPolicyConfigurationRequest.newInstance(policy);
+      FederationPolicyStoreInputValidator
+          .validateSetSubClusterPolicyConfigurationRequest(request);
+      Assert.fail();
+    } catch (FederationStateStoreInvalidInputException e) {
+      Assert.assertTrue(e.getMessage().startsWith("Missing Policy Type."));
+    }
+  }
+
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[39/42] hadoop git commit: YARN-5602. Utils for Federation State and Policy Store. (Giovanni Matteo Fumarola via Subru).

Posted by su...@apache.org.
YARN-5602. Utils for Federation State and Policy Store. (Giovanni Matteo Fumarola via Subru).


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/fa7e31ea
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/fa7e31ea
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/fa7e31ea

Branch: refs/heads/YARN-2915
Commit: fa7e31ea5d138ad2738d00f848d093c977ddfde4
Parents: 54757ae
Author: Subru Krishnan <su...@apache.org>
Authored: Wed Apr 5 15:02:00 2017 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon May 8 16:59:13 2017 -0700

----------------------------------------------------------------------
 .../FederationStateStoreErrorCode.java          | 105 +++++++++++++
 .../FederationStateStoreException.java          |  45 ++++++
 ...derationStateStoreInvalidInputException.java |  48 ++++++
 .../FederationStateStoreRetriableException.java |  44 ++++++
 .../store/exception/package-info.java           |  17 ++
 .../store/impl/MemoryFederationStateStore.java  |  56 +++++--
 .../store/records/SubClusterInfo.java           |  62 ++++++++
 .../records/impl/pb/SubClusterInfoPBImpl.java   |  16 --
 ...cationHomeSubClusterStoreInputValidator.java |   1 +
 ...ationMembershipStateStoreInputValidator.java |   1 +
 .../FederationPolicyStoreInputValidator.java    |   1 +
 ...derationStateStoreInvalidInputException.java |  48 ------
 .../store/utils/FederationStateStoreUtils.java  | 155 +++++++++++++++++++
 .../utils/FederationStateStoreFacade.java       |  23 ++-
 .../impl/FederationStateStoreBaseTest.java      |  91 ++++++-----
 .../impl/TestMemoryFederationStateStore.java    |   4 +-
 .../TestFederationStateStoreInputValidator.java |   1 +
 .../TestFederationStateStoreFacadeRetry.java    | 125 +++++++++++++++
 18 files changed, 730 insertions(+), 113 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7e31ea/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/exception/FederationStateStoreErrorCode.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/exception/FederationStateStoreErrorCode.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/exception/FederationStateStoreErrorCode.java
new file mode 100644
index 0000000..88e2d3a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/exception/FederationStateStoreErrorCode.java
@@ -0,0 +1,105 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.exception;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+
+/**
+ * <p>
+ * Logical error codes from <code>FederationStateStore</code>.
+ * </p>
+ */
+@Public
+@Unstable
+public enum FederationStateStoreErrorCode {
+
+  MEMBERSHIP_INSERT_FAIL(1101, "Fail to insert a tuple into Membership table."),
+
+  MEMBERSHIP_DELETE_FAIL(1102, "Fail to delete a tuple from Membership table."),
+
+  MEMBERSHIP_SINGLE_SELECT_FAIL(1103,
+      "Fail to select a tuple from Membership table."),
+
+  MEMBERSHIP_MULTIPLE_SELECT_FAIL(1104,
+      "Fail to select multiple tuples from Membership table."),
+
+  MEMBERSHIP_UPDATE_DEREGISTER_FAIL(1105,
+      "Fail to update/deregister a tuple in Membership table."),
+
+  MEMBERSHIP_UPDATE_HEARTBEAT_FAIL(1106,
+      "Fail to update/heartbeat a tuple in Membership table."),
+
+  APPLICATIONS_INSERT_FAIL(1201,
+      "Fail to insert a tuple into ApplicationsHomeSubCluster table."),
+
+  APPLICATIONS_DELETE_FAIL(1202,
+      "Fail to delete a tuple from ApplicationsHomeSubCluster table"),
+
+  APPLICATIONS_SINGLE_SELECT_FAIL(1203,
+      "Fail to select a tuple from ApplicationsHomeSubCluster table."),
+
+  APPLICATIONS_MULTIPLE_SELECT_FAIL(1204,
+      "Fail to select multiple tuple from ApplicationsHomeSubCluster table."),
+
+  APPLICATIONS_UPDATE_FAIL(1205,
+      "Fail to update a tuple in ApplicationsHomeSubCluster table."),
+
+  POLICY_INSERT_FAIL(1301, "Fail to insert a tuple into Policy table."),
+
+  POLICY_DELETE_FAIL(1302, "Fail to delete a tuple from Membership table."),
+
+  POLICY_SINGLE_SELECT_FAIL(1303, "Fail to select a tuple from Policy table."),
+
+  POLICY_MULTIPLE_SELECT_FAIL(1304,
+      "Fail to select multiple tuples from Policy table."),
+
+  POLICY_UPDATE_FAIL(1305, "Fail to update a tuple in Policy table.");
+
+  private final int id;
+  private final String msg;
+
+  FederationStateStoreErrorCode(int id, String msg) {
+    this.id = id;
+    this.msg = msg;
+  }
+
+  /**
+   * Get the error code related to the FederationStateStore failure.
+   *
+   * @return the error code related to the FederationStateStore failure.
+   */
+  public int getId() {
+    return this.id;
+  }
+
+  /**
+   * Get the error message related to the FederationStateStore failure.
+   *
+   * @return the error message related to the FederationStateStore failure.
+   */
+  public String getMsg() {
+    return this.msg;
+  }
+
+  @Override
+  public String toString() {
+    return "\nError Code: " + this.id + "\nError Message: " + this.msg;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7e31ea/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/exception/FederationStateStoreException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/exception/FederationStateStoreException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/exception/FederationStateStoreException.java
new file mode 100644
index 0000000..81a9e99
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/exception/FederationStateStoreException.java
@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.exception;
+
+import org.apache.hadoop.yarn.exceptions.YarnException;
+
+/**
+ * Exception thrown by the <code>FederationStateStore</code>.
+ *
+ */
+public class FederationStateStoreException extends YarnException {
+
+  /**
+   * IDE auto-generated.
+   */
+  private static final long serialVersionUID = -6453353714832159296L;
+
+  private FederationStateStoreErrorCode code;
+
+  public FederationStateStoreException(FederationStateStoreErrorCode code) {
+    super();
+    this.code = code;
+  }
+
+  public FederationStateStoreErrorCode getCode() {
+    return code;
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7e31ea/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/exception/FederationStateStoreInvalidInputException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/exception/FederationStateStoreInvalidInputException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/exception/FederationStateStoreInvalidInputException.java
new file mode 100644
index 0000000..edf7837
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/exception/FederationStateStoreInvalidInputException.java
@@ -0,0 +1,48 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.exception;
+
+import org.apache.hadoop.yarn.exceptions.YarnException;
+
+/**
+ * Exception thrown by the {@code FederationMembershipStateStoreInputValidator},
+ * {@code FederationApplicationHomeSubClusterStoreInputValidator},
+ * {@code FederationPolicyStoreInputValidator} if the input is invalid.
+ *
+ */
+public class FederationStateStoreInvalidInputException extends YarnException {
+
+  /**
+   * IDE auto-generated.
+   */
+  private static final long serialVersionUID = -7352144682711430801L;
+
+  public FederationStateStoreInvalidInputException(Throwable cause) {
+    super(cause);
+  }
+
+  public FederationStateStoreInvalidInputException(String message) {
+    super(message);
+  }
+
+  public FederationStateStoreInvalidInputException(String message,
+      Throwable cause) {
+    super(message, cause);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7e31ea/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/exception/FederationStateStoreRetriableException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/exception/FederationStateStoreRetriableException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/exception/FederationStateStoreRetriableException.java
new file mode 100644
index 0000000..19d6750
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/exception/FederationStateStoreRetriableException.java
@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.exception;
+
+import org.apache.hadoop.yarn.exceptions.YarnException;
+
+/**
+ * Exception thrown by the {@code FederationStateStore}, if it is a retriable
+ * exception.
+ *
+ */
+public class FederationStateStoreRetriableException extends YarnException {
+
+  private static final long serialVersionUID = 1L;
+
+  public FederationStateStoreRetriableException(Throwable cause) {
+    super(cause);
+  }
+
+  public FederationStateStoreRetriableException(String message) {
+    super(message);
+  }
+
+  public FederationStateStoreRetriableException(String message,
+      Throwable cause) {
+    super(message, cause);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7e31ea/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/exception/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/exception/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/exception/package-info.java
new file mode 100644
index 0000000..727606f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/exception/package-info.java
@@ -0,0 +1,17 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.yarn.server.federation.store.exception;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7e31ea/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/MemoryFederationStateStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/MemoryFederationStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/MemoryFederationStateStore.java
index 6e564dc..127bf82 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/MemoryFederationStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/MemoryFederationStateStore.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyConfiguration;
 import org.apache.hadoop.yarn.server.federation.store.FederationStateStore;
+import org.apache.hadoop.yarn.server.federation.store.exception.FederationStateStoreErrorCode;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterDeregisterRequest;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterDeregisterResponse;
 import org.apache.hadoop.yarn.server.federation.store.records.AddApplicationHomeSubClusterRequest;
@@ -60,8 +61,11 @@ import org.apache.hadoop.yarn.server.federation.store.records.UpdateApplicationH
 import org.apache.hadoop.yarn.server.federation.store.utils.FederationApplicationHomeSubClusterStoreInputValidator;
 import org.apache.hadoop.yarn.server.federation.store.utils.FederationMembershipStateStoreInputValidator;
 import org.apache.hadoop.yarn.server.federation.store.utils.FederationPolicyStoreInputValidator;
+import org.apache.hadoop.yarn.server.federation.store.utils.FederationStateStoreUtils;
 import org.apache.hadoop.yarn.server.records.Version;
 import org.apache.hadoop.yarn.util.MonotonicClock;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * In-memory implementation of {@link FederationStateStore}.
@@ -74,6 +78,9 @@ public class MemoryFederationStateStore implements FederationStateStore {
 
   private final MonotonicClock clock = new MonotonicClock();
 
+  public static final Logger LOG =
+      LoggerFactory.getLogger(MemoryFederationStateStore.class);
+
   @Override
   public void init(Configuration conf) {
     membership = new ConcurrentHashMap<SubClusterId, SubClusterInfo>();
@@ -94,7 +101,17 @@ public class MemoryFederationStateStore implements FederationStateStore {
     FederationMembershipStateStoreInputValidator
         .validateSubClusterRegisterRequest(request);
     SubClusterInfo subClusterInfo = request.getSubClusterInfo();
-    membership.put(subClusterInfo.getSubClusterId(), subClusterInfo);
+
+    SubClusterInfo subClusterInfoToSave =
+        SubClusterInfo.newInstance(subClusterInfo.getSubClusterId(),
+            subClusterInfo.getAMRMServiceAddress(),
+            subClusterInfo.getClientRMServiceAddress(),
+            subClusterInfo.getRMAdminServiceAddress(),
+            subClusterInfo.getRMWebServiceAddress(), clock.getTime(),
+            subClusterInfo.getState(), subClusterInfo.getLastStartTime(),
+            subClusterInfo.getCapability());
+
+    membership.put(subClusterInfo.getSubClusterId(), subClusterInfoToSave);
     return SubClusterRegisterResponse.newInstance();
   }
 
@@ -105,8 +122,11 @@ public class MemoryFederationStateStore implements FederationStateStore {
         .validateSubClusterDeregisterRequest(request);
     SubClusterInfo subClusterInfo = membership.get(request.getSubClusterId());
     if (subClusterInfo == null) {
-      throw new YarnException(
-          "SubCluster " + request.getSubClusterId().toString() + " not found");
+      String errMsg =
+          "SubCluster " + request.getSubClusterId().toString() + " not found";
+      FederationStateStoreUtils.logAndThrowStoreException(LOG,
+          FederationStateStoreErrorCode.MEMBERSHIP_UPDATE_DEREGISTER_FAIL,
+          errMsg);
     } else {
       subClusterInfo.setState(request.getState());
     }
@@ -124,8 +144,11 @@ public class MemoryFederationStateStore implements FederationStateStore {
     SubClusterInfo subClusterInfo = membership.get(subClusterId);
 
     if (subClusterInfo == null) {
-      throw new YarnException("Subcluster " + subClusterId.toString()
-          + " does not exist; cannot heartbeat");
+      String errMsg = "Subcluster " + subClusterId.toString()
+          + " does not exist; cannot heartbeat";
+      FederationStateStoreUtils.logAndThrowStoreException(LOG,
+          FederationStateStoreErrorCode.MEMBERSHIP_UPDATE_HEARTBEAT_FAIL,
+          errMsg);
     }
 
     subClusterInfo.setLastHeartBeat(clock.getTime());
@@ -143,8 +166,10 @@ public class MemoryFederationStateStore implements FederationStateStore {
         .validateGetSubClusterInfoRequest(request);
     SubClusterId subClusterId = request.getSubClusterId();
     if (!membership.containsKey(subClusterId)) {
-      throw new YarnException(
-          "Subcluster " + subClusterId.toString() + " does not exist");
+      String errMsg =
+          "Subcluster " + subClusterId.toString() + " does not exist";
+      FederationStateStoreUtils.logAndThrowStoreException(LOG,
+          FederationStateStoreErrorCode.MEMBERSHIP_SINGLE_SELECT_FAIL, errMsg);
     }
 
     return GetSubClusterInfoResponse.newInstance(membership.get(subClusterId));
@@ -193,7 +218,9 @@ public class MemoryFederationStateStore implements FederationStateStore {
     ApplicationId appId =
         request.getApplicationHomeSubCluster().getApplicationId();
     if (!applications.containsKey(appId)) {
-      throw new YarnException("Application " + appId + " does not exist");
+      String errMsg = "Application " + appId + " does not exist";
+      FederationStateStoreUtils.logAndThrowStoreException(LOG,
+          FederationStateStoreErrorCode.APPLICATIONS_UPDATE_FAIL, errMsg);
     }
 
     applications.put(appId,
@@ -209,7 +236,10 @@ public class MemoryFederationStateStore implements FederationStateStore {
         .validateGetApplicationHomeSubClusterRequest(request);
     ApplicationId appId = request.getApplicationId();
     if (!applications.containsKey(appId)) {
-      throw new YarnException("Application " + appId + " does not exist");
+      String errMsg = "Application " + appId + " does not exist";
+      FederationStateStoreUtils.logAndThrowStoreException(LOG,
+          FederationStateStoreErrorCode.APPLICATIONS_SINGLE_SELECT_FAIL,
+          errMsg);
     }
 
     return GetApplicationHomeSubClusterResponse.newInstance(
@@ -238,7 +268,9 @@ public class MemoryFederationStateStore implements FederationStateStore {
         .validateDeleteApplicationHomeSubClusterRequest(request);
     ApplicationId appId = request.getApplicationId();
     if (!applications.containsKey(appId)) {
-      throw new YarnException("Application " + appId + " does not exist");
+      String errMsg = "Application " + appId + " does not exist";
+      FederationStateStoreUtils.logAndThrowStoreException(LOG,
+          FederationStateStoreErrorCode.APPLICATIONS_DELETE_FAIL, errMsg);
     }
 
     applications.remove(appId);
@@ -253,7 +285,9 @@ public class MemoryFederationStateStore implements FederationStateStore {
         .validateGetSubClusterPolicyConfigurationRequest(request);
     String queue = request.getQueue();
     if (!policies.containsKey(queue)) {
-      throw new YarnException("Policy for queue " + queue + " does not exist");
+      String errMsg = "Policy for queue " + queue + " does not exist";
+      FederationStateStoreUtils.logAndThrowStoreException(LOG,
+          FederationStateStoreErrorCode.POLICY_SINGLE_SELECT_FAIL, errMsg);
     }
 
     return GetSubClusterPolicyConfigurationResponse

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7e31ea/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterInfo.java
index f13c8f1..cbf64e6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/SubClusterInfo.java
@@ -260,4 +260,66 @@ public abstract class SubClusterInfo {
         + ", getState() = " + getState() + ", getLastStartTime() = "
         + getLastStartTime() + ", getCapability() = " + getCapability() + "]";
   }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (getClass() != obj.getClass()) {
+      return false;
+    }
+    SubClusterInfo other = (SubClusterInfo) obj;
+    if (!this.getSubClusterId().equals(other.getSubClusterId())) {
+      return false;
+    }
+    if (!this.getAMRMServiceAddress().equals(other.getAMRMServiceAddress())) {
+      return false;
+    }
+    if (!this.getClientRMServiceAddress()
+        .equals(other.getClientRMServiceAddress())) {
+      return false;
+    }
+    if (!this.getRMAdminServiceAddress()
+        .equals(other.getRMAdminServiceAddress())) {
+      return false;
+    }
+    if (!this.getRMWebServiceAddress().equals(other.getRMWebServiceAddress())) {
+      return false;
+    }
+    if (!this.getState().equals(other.getState())) {
+      return false;
+    }
+    return this.getLastStartTime() == other.getLastStartTime();
+    // Capability and HeartBeat fields are not included as they are temporal
+    // (i.e. timestamps), so they change during the lifetime of the same
+    // sub-cluster
+  }
+
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result
+        + ((getSubClusterId() == null) ? 0 : getSubClusterId().hashCode());
+    result = prime * result + ((getAMRMServiceAddress() == null) ? 0
+        : getAMRMServiceAddress().hashCode());
+    result = prime * result + ((getClientRMServiceAddress() == null) ? 0
+        : getClientRMServiceAddress().hashCode());
+    result = prime * result + ((getRMAdminServiceAddress() == null) ? 0
+        : getRMAdminServiceAddress().hashCode());
+    result = prime * result + ((getRMWebServiceAddress() == null) ? 0
+        : getRMWebServiceAddress().hashCode());
+    result =
+        prime * result + ((getState() == null) ? 0 : getState().hashCode());
+    result = prime * result
+        + (int) (getLastStartTime() ^ (getLastStartTime() >>> 32));
+    return result;
+    // Capability and HeartBeat fields are not included as they are temporal
+    // (i.e. timestamps), so they change during the lifetime of the same
+    // sub-cluster
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7e31ea/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/SubClusterInfoPBImpl.java
----------------------------------------------------------------------
diff --git a/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/SubClusterInfoPBImpl.java b/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/SubClusterInfoPBImpl.java
index b650b5f..cfdd038 100644
--- a/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/SubClusterInfoPBImpl.java
+++ b/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/SubClusterInfoPBImpl.java
@@ -82,22 +82,6 @@ public class SubClusterInfoPBImpl extends SubClusterInfo {
   }
 
   @Override
-  public int hashCode() {
-    return getProto().hashCode();
-  }
-
-  @Override
-  public boolean equals(Object other) {
-    if (other == null) {
-      return false;
-    }
-    if (other.getClass().isAssignableFrom(this.getClass())) {
-      return this.getProto().equals(this.getClass().cast(other).getProto());
-    }
-    return false;
-  }
-
-  @Override
   public String toString() {
     return TextFormat.shortDebugString(getProto());
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7e31ea/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationApplicationHomeSubClusterStoreInputValidator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationApplicationHomeSubClusterStoreInputValidator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationApplicationHomeSubClusterStoreInputValidator.java
index c14a452..d920144 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationApplicationHomeSubClusterStoreInputValidator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationApplicationHomeSubClusterStoreInputValidator.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.yarn.server.federation.store.utils;
 
 import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.server.federation.store.exception.FederationStateStoreInvalidInputException;
 import org.apache.hadoop.yarn.server.federation.store.records.AddApplicationHomeSubClusterRequest;
 import org.apache.hadoop.yarn.server.federation.store.records.ApplicationHomeSubCluster;
 import org.apache.hadoop.yarn.server.federation.store.records.DeleteApplicationHomeSubClusterRequest;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7e31ea/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationMembershipStateStoreInputValidator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationMembershipStateStoreInputValidator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationMembershipStateStoreInputValidator.java
index ff9d8e9..ebe622b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationMembershipStateStoreInputValidator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationMembershipStateStoreInputValidator.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.server.federation.store.utils;
 
 import java.net.URI;
 
+import org.apache.hadoop.yarn.server.federation.store.exception.FederationStateStoreInvalidInputException;
 import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterInfoRequest;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterDeregisterRequest;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterHeartbeatRequest;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7e31ea/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationPolicyStoreInputValidator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationPolicyStoreInputValidator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationPolicyStoreInputValidator.java
index 273a8ac..0df2d85 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationPolicyStoreInputValidator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationPolicyStoreInputValidator.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn.server.federation.store.utils;
 
+import org.apache.hadoop.yarn.server.federation.store.exception.FederationStateStoreInvalidInputException;
 import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterPolicyConfigurationRequest;
 import org.apache.hadoop.yarn.server.federation.store.records.SetSubClusterPolicyConfigurationRequest;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyConfiguration;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7e31ea/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationStateStoreInvalidInputException.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationStateStoreInvalidInputException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationStateStoreInvalidInputException.java
deleted file mode 100644
index ea1428d..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationStateStoreInvalidInputException.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.server.federation.store.utils;
-
-import org.apache.hadoop.yarn.exceptions.YarnException;
-
-/**
- * Exception thrown by the {@link FederationMembershipStateStoreInputValidator},
- * {@link FederationApplicationHomeSubClusterStoreInputValidator},
- * {@link FederationPolicyStoreInputValidator} if the input is invalid.
- *
- */
-public class FederationStateStoreInvalidInputException extends YarnException {
-
-  /**
-   * IDE auto-generated.
-   */
-  private static final long serialVersionUID = -7352144682711430801L;
-
-  public FederationStateStoreInvalidInputException(Throwable cause) {
-    super(cause);
-  }
-
-  public FederationStateStoreInvalidInputException(String message) {
-    super(message);
-  }
-
-  public FederationStateStoreInvalidInputException(String message,
-      Throwable cause) {
-    super(message, cause);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7e31ea/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationStateStoreUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationStateStoreUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationStateStoreUtils.java
new file mode 100644
index 0000000..7dbb20a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/utils/FederationStateStoreUtils.java
@@ -0,0 +1,155 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store.utils;
+
+import java.sql.CallableStatement;
+import java.sql.Connection;
+import java.sql.SQLException;
+
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.federation.store.exception.FederationStateStoreErrorCode;
+import org.apache.hadoop.yarn.server.federation.store.exception.FederationStateStoreException;
+import org.apache.hadoop.yarn.server.federation.store.exception.FederationStateStoreInvalidInputException;
+import org.apache.hadoop.yarn.server.federation.store.exception.FederationStateStoreRetriableException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Common utility methods used by the store implementations.
+ *
+ */
+public final class FederationStateStoreUtils {
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(FederationStateStoreUtils.class);
+
+  private FederationStateStoreUtils() {
+  }
+
+  /**
+   * Returns the SQL <code>FederationStateStore</code> connection to the pool.
+   *
+   * @param log the logger interface
+   * @param cstmt the interface used to execute SQL stored procedures
+   * @param conn the SQL connection
+   * @throws YarnException on failure
+   */
+  public static void returnToPool(Logger log, CallableStatement cstmt,
+      Connection conn) throws YarnException {
+    if (cstmt != null) {
+      try {
+        cstmt.close();
+      } catch (SQLException e) {
+        logAndThrowException(log, "Exception while trying to close Statement",
+            e);
+      }
+    }
+
+    if (conn != null) {
+      try {
+        conn.close();
+      } catch (SQLException e) {
+        logAndThrowException(log, "Exception while trying to close Connection",
+            e);
+      }
+    }
+  }
+
+  /**
+   * Throws an exception due to an error in <code>FederationStateStore</code>.
+   *
+   * @param log the logger interface
+   * @param errMsg the error message
+   * @param t the throwable raised in the called class.
+   * @throws YarnException on failure
+   */
+  public static void logAndThrowException(Logger log, String errMsg,
+      Throwable t) throws YarnException {
+    if (t != null) {
+      log.error(errMsg, t);
+      throw new YarnException(errMsg, t);
+    } else {
+      log.error(errMsg);
+      throw new YarnException(errMsg);
+    }
+  }
+
+  /**
+   * Throws an <code>FederationStateStoreException</code> due to an error in
+   * <code>FederationStateStore</code>.
+   *
+   * @param log the logger interface
+   * @param code FederationStateStoreErrorCode of the error
+   * @param errMsg the error message
+   * @throws YarnException on failure
+   */
+  public static void logAndThrowStoreException(Logger log,
+      FederationStateStoreErrorCode code, String errMsg) throws YarnException {
+    log.error(errMsg + " " + code.toString());
+    throw new FederationStateStoreException(code);
+  }
+
+  /**
+   * Throws an <code>FederationStateStoreException</code> due to an error in
+   * <code>FederationStateStore</code>.
+   *
+   * @param log the logger interface
+   * @param code FederationStateStoreErrorCode of the error
+   * @throws YarnException on failure
+   */
+  public static void logAndThrowStoreException(Logger log,
+      FederationStateStoreErrorCode code) throws YarnException {
+    log.error(code.toString());
+    throw new FederationStateStoreException(code);
+  }
+
+  /**
+   * Throws an <code>FederationStateStoreInvalidInputException</code> due to an
+   * error in <code>FederationStateStore</code>.
+   *
+   * @param log the logger interface
+   * @param errMsg the error message
+   * @throws YarnException on failure
+   */
+  public static void logAndThrowInvalidInputException(Logger log, String errMsg)
+      throws YarnException {
+    LOG.error(errMsg);
+    throw new FederationStateStoreInvalidInputException(errMsg);
+  }
+
+  /**
+   * Throws an <code>FederationStateStoreRetriableException</code> due to an
+   * error in <code>FederationStateStore</code>.
+   *
+   * @param log the logger interface
+   * @param errMsg the error message
+   * @param t the throwable raised in the called class.
+   * @throws YarnException on failure
+   */
+  public static void logAndThrowRetriableException(Logger log, String errMsg,
+      Throwable t) throws YarnException {
+    if (t != null) {
+      LOG.error(errMsg, t);
+      throw new FederationStateStoreRetriableException(errMsg, t);
+    } else {
+      LOG.error(errMsg);
+      throw new FederationStateStoreRetriableException(errMsg);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7e31ea/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java
index e8f245e..5693342 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationStateStoreFacade.java
@@ -48,6 +48,7 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.server.federation.resolver.SubClusterResolver;
 import org.apache.hadoop.yarn.server.federation.store.FederationStateStore;
+import org.apache.hadoop.yarn.server.federation.store.exception.FederationStateStoreRetriableException;
 import org.apache.hadoop.yarn.server.federation.store.records.AddApplicationHomeSubClusterRequest;
 import org.apache.hadoop.yarn.server.federation.store.records.AddApplicationHomeSubClusterResponse;
 import org.apache.hadoop.yarn.server.federation.store.records.ApplicationHomeSubCluster;
@@ -137,14 +138,32 @@ public final class FederationStateStoreFacade {
     initCache();
   }
 
+  /**
+   * Create a RetryPolicy for {@code FederationStateStoreFacade}. In case of
+   * failure, it retries for:
+   * <ul>
+   * <li>{@code FederationStateStoreRetriableException}</li>
+   * <li>{@code CacheLoaderException}</li>
+   * </ul>
+   *
+   * @param conf the updated configuration
+   * @return the RetryPolicy for FederationStateStoreFacade
+   */
   public static RetryPolicy createRetryPolicy(Configuration conf) {
     // Retry settings for StateStore
-    RetryPolicy retryPolicy = RetryPolicies.exponentialBackoffRetry(
+    RetryPolicy basePolicy = RetryPolicies.exponentialBackoffRetry(
         conf.getInt(YarnConfiguration.CLIENT_FAILOVER_RETRIES, Integer.SIZE),
         conf.getLong(YarnConfiguration.CLIENT_FAILOVER_SLEEPTIME_BASE_MS,
             YarnConfiguration.DEFAULT_RESOURCEMANAGER_CONNECT_RETRY_INTERVAL_MS),
         TimeUnit.MILLISECONDS);
-
+    Map<Class<? extends Exception>, RetryPolicy> exceptionToPolicyMap =
+        new HashMap<Class<? extends Exception>, RetryPolicy>();
+    exceptionToPolicyMap.put(FederationStateStoreRetriableException.class,
+        basePolicy);
+    exceptionToPolicyMap.put(CacheLoaderException.class, basePolicy);
+
+    RetryPolicy retryPolicy = RetryPolicies.retryByException(
+        RetryPolicies.TRY_ONCE_THEN_FAIL, exceptionToPolicyMap);
     return retryPolicy;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7e31ea/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
----------------------------------------------------------------------
diff --git a/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 b/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
index 63a5b65..80b00ef 100644
--- a/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
+++ b/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
@@ -19,11 +19,14 @@ package org.apache.hadoop.yarn.server.federation.store.impl;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.federation.store.FederationStateStore;
+import org.apache.hadoop.yarn.server.federation.store.exception.FederationStateStoreErrorCode;
+import org.apache.hadoop.yarn.server.federation.store.exception.FederationStateStoreException;
 import org.apache.hadoop.yarn.server.federation.store.records.AddApplicationHomeSubClusterRequest;
 import org.apache.hadoop.yarn.server.federation.store.records.AddApplicationHomeSubClusterResponse;
 import org.apache.hadoop.yarn.server.federation.store.records.ApplicationHomeSubCluster;
@@ -67,9 +70,11 @@ public abstract class FederationStateStoreBaseTest {
 
   protected abstract FederationStateStore createStateStore();
 
+  private Configuration conf;
+
   @Before
   public void before() throws IOException, YarnException {
-    stateStore.init(new Configuration());
+    stateStore.init(conf);
   }
 
   @After
@@ -114,8 +119,10 @@ public abstract class FederationStateStoreBaseTest {
     try {
       stateStore.deregisterSubCluster(deregisterRequest);
       Assert.fail();
-    } catch (YarnException e) {
-      Assert.assertTrue(e.getMessage().startsWith("SubCluster SC not found"));
+    } catch (FederationStateStoreException e) {
+      Assert.assertEquals(
+          FederationStateStoreErrorCode.MEMBERSHIP_UPDATE_DEREGISTER_FAIL,
+          e.getCode());
     }
   }
 
@@ -141,9 +148,10 @@ public abstract class FederationStateStoreBaseTest {
     try {
       stateStore.getSubCluster(request).getSubClusterInfo();
       Assert.fail();
-    } catch (YarnException e) {
-      Assert.assertTrue(
-          e.getMessage().startsWith("Subcluster SC does not exist"));
+    } catch (FederationStateStoreException e) {
+      Assert.assertEquals(
+          FederationStateStoreErrorCode.MEMBERSHIP_SINGLE_SELECT_FAIL,
+          e.getCode());
     }
   }
 
@@ -166,19 +174,25 @@ public abstract class FederationStateStoreBaseTest {
     stateStore.subClusterHeartbeat(SubClusterHeartbeatRequest.newInstance(
         subClusterId2, SubClusterState.SC_UNHEALTHY, "capability"));
 
-    Assert.assertTrue(
-        stateStore.getSubClusters(GetSubClustersInfoRequest.newInstance(true))
-            .getSubClusters().contains(subClusterInfo1));
-    Assert.assertFalse(
+    List<SubClusterInfo> subClustersActive =
         stateStore.getSubClusters(GetSubClustersInfoRequest.newInstance(true))
-            .getSubClusters().contains(subClusterInfo2));
-
-    Assert.assertTrue(
+            .getSubClusters();
+    List<SubClusterInfo> subClustersAll =
         stateStore.getSubClusters(GetSubClustersInfoRequest.newInstance(false))
-            .getSubClusters().contains(subClusterInfo1));
-    Assert.assertTrue(
-        stateStore.getSubClusters(GetSubClustersInfoRequest.newInstance(false))
-            .getSubClusters().contains(subClusterInfo2));
+            .getSubClusters();
+
+    // SC1 is the only active
+    Assert.assertEquals(1, subClustersActive.size());
+    SubClusterInfo sc1 = subClustersActive.get(0);
+    Assert.assertEquals(subClusterId1, sc1.getSubClusterId());
+
+    // SC1 and SC2 are the SubCluster present into the StateStore
+
+    Assert.assertEquals(2, subClustersAll.size());
+    Assert.assertTrue(subClustersAll.contains(sc1));
+    subClustersAll.remove(sc1);
+    SubClusterInfo sc2 = subClustersAll.get(0);
+    Assert.assertEquals(subClusterId2, sc2.getSubClusterId());
   }
 
   @Test
@@ -204,9 +218,10 @@ public abstract class FederationStateStoreBaseTest {
     try {
       stateStore.subClusterHeartbeat(heartbeatRequest);
       Assert.fail();
-    } catch (YarnException e) {
-      Assert.assertTrue(e.getMessage()
-          .startsWith("Subcluster SC does not exist; cannot heartbeat"));
+    } catch (FederationStateStoreException e) {
+      Assert.assertEquals(
+          FederationStateStoreErrorCode.MEMBERSHIP_UPDATE_HEARTBEAT_FAIL,
+          e.getCode());
     }
   }
 
@@ -265,9 +280,10 @@ public abstract class FederationStateStoreBaseTest {
     try {
       queryApplicationHomeSC(appId);
       Assert.fail();
-    } catch (YarnException e) {
-      Assert.assertTrue(e.getMessage()
-          .startsWith("Application " + appId + " does not exist"));
+    } catch (FederationStateStoreException e) {
+      Assert.assertEquals(
+          FederationStateStoreErrorCode.APPLICATIONS_SINGLE_SELECT_FAIL,
+          e.getCode());
     }
 
   }
@@ -281,9 +297,9 @@ public abstract class FederationStateStoreBaseTest {
     try {
       stateStore.deleteApplicationHomeSubCluster(delRequest);
       Assert.fail();
-    } catch (YarnException e) {
-      Assert.assertTrue(e.getMessage()
-          .startsWith("Application " + appId.toString() + " does not exist"));
+    } catch (FederationStateStoreException e) {
+      Assert.assertEquals(
+          FederationStateStoreErrorCode.APPLICATIONS_DELETE_FAIL, e.getCode());
     }
   }
 
@@ -314,9 +330,10 @@ public abstract class FederationStateStoreBaseTest {
     try {
       stateStore.getApplicationHomeSubCluster(request);
       Assert.fail();
-    } catch (YarnException e) {
-      Assert.assertTrue(e.getMessage()
-          .startsWith("Application " + appId.toString() + " does not exist"));
+    } catch (FederationStateStoreException e) {
+      Assert.assertEquals(
+          FederationStateStoreErrorCode.APPLICATIONS_SINGLE_SELECT_FAIL,
+          e.getCode());
     }
   }
 
@@ -379,9 +396,9 @@ public abstract class FederationStateStoreBaseTest {
     try {
       stateStore.updateApplicationHomeSubCluster((updateRequest));
       Assert.fail();
-    } catch (YarnException e) {
-      Assert.assertTrue(e.getMessage()
-          .startsWith("Application " + appId.toString() + " does not exist"));
+    } catch (FederationStateStoreException e) {
+      Assert.assertEquals(
+          FederationStateStoreErrorCode.APPLICATIONS_UPDATE_FAIL, e.getCode());
     }
   }
 
@@ -440,9 +457,9 @@ public abstract class FederationStateStoreBaseTest {
     try {
       stateStore.getPolicyConfiguration(request);
       Assert.fail();
-    } catch (YarnException e) {
-      Assert.assertTrue(
-          e.getMessage().startsWith("Policy for queue Queue does not exist"));
+    } catch (FederationStateStoreException e) {
+      Assert.assertEquals(
+          FederationStateStoreErrorCode.POLICY_SINGLE_SELECT_FAIL, e.getCode());
     }
   }
 
@@ -537,4 +554,8 @@ public abstract class FederationStateStoreBaseTest {
     return result.getPolicyConfiguration();
   }
 
+  protected void setConf(Configuration conf) {
+    this.conf = conf;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7e31ea/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
----------------------------------------------------------------------
diff --git a/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 b/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
index 74404c7..64adab8 100644
--- a/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
+++ b/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
@@ -17,6 +17,7 @@
 
 package org.apache.hadoop.yarn.server.federation.store.impl;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.server.federation.store.FederationStateStore;
 
 /**
@@ -27,6 +28,7 @@ public class TestMemoryFederationStateStore
 
   @Override
   protected FederationStateStore createStateStore() {
+    super.setConf(new Configuration());
     return new MemoryFederationStateStore();
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7e31ea/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/utils/TestFederationStateStoreInputValidator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/utils/TestFederationStateStoreInputValidator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/utils/TestFederationStateStoreInputValidator.java
index b95f17a..8ac5e81 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/utils/TestFederationStateStoreInputValidator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/utils/TestFederationStateStoreInputValidator.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.yarn.server.federation.store.utils;
 import java.nio.ByteBuffer;
 
 import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.server.federation.store.exception.FederationStateStoreInvalidInputException;
 import org.apache.hadoop.yarn.server.federation.store.records.AddApplicationHomeSubClusterRequest;
 import org.apache.hadoop.yarn.server.federation.store.records.ApplicationHomeSubCluster;
 import org.apache.hadoop.yarn.server.federation.store.records.DeleteApplicationHomeSubClusterRequest;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa7e31ea/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/TestFederationStateStoreFacadeRetry.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/TestFederationStateStoreFacadeRetry.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/TestFederationStateStoreFacadeRetry.java
new file mode 100644
index 0000000..632e865
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/TestFederationStateStoreFacadeRetry.java
@@ -0,0 +1,125 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.utils;
+
+import javax.cache.integration.CacheLoaderException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.retry.RetryPolicy;
+import org.apache.hadoop.io.retry.RetryPolicy.RetryAction;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.federation.store.exception.FederationStateStoreErrorCode;
+import org.apache.hadoop.yarn.server.federation.store.exception.FederationStateStoreException;
+import org.apache.hadoop.yarn.server.federation.store.exception.FederationStateStoreInvalidInputException;
+import org.apache.hadoop.yarn.server.federation.store.exception.FederationStateStoreRetriableException;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test class to validate FederationStateStoreFacade retry policy.
+ */
+public class TestFederationStateStoreFacadeRetry {
+
+  private int maxRetries = 4;
+  private Configuration conf;
+
+  /*
+   * Test to validate that FederationStateStoreRetriableException is a retriable
+   * exception.
+   */
+  @Test
+  public void testFacadeRetriableException() throws Exception {
+    conf = new Configuration();
+    conf.setInt(YarnConfiguration.CLIENT_FAILOVER_RETRIES, maxRetries);
+    RetryPolicy policy = FederationStateStoreFacade.createRetryPolicy(conf);
+    RetryAction action = policy.shouldRetry(
+        new FederationStateStoreRetriableException(""), 0, 0, false);
+    // We compare only the action, since delay and the reason are random values
+    // during this test
+    Assert.assertEquals(RetryAction.RETRY.action, action.action);
+
+    // After maxRetries we stop to retry
+    action = policy.shouldRetry(new FederationStateStoreRetriableException(""),
+        maxRetries, 0, false);
+    Assert.assertEquals(RetryAction.FAIL.action, action.action);
+  }
+
+  /*
+   * Test to validate that YarnException is not a retriable exception.
+   */
+  @Test
+  public void testFacadeYarnException() throws Exception {
+
+    conf = new Configuration();
+    conf.setInt(YarnConfiguration.CLIENT_FAILOVER_RETRIES, maxRetries);
+    RetryPolicy policy = FederationStateStoreFacade.createRetryPolicy(conf);
+    RetryAction action = policy.shouldRetry(new YarnException(), 0, 0, false);
+    Assert.assertEquals(RetryAction.FAIL.action, action.action);
+  }
+
+  /*
+   * Test to validate that FederationStateStoreException is not a retriable
+   * exception.
+   */
+  @Test
+  public void testFacadeStateStoreException() throws Exception {
+    conf = new Configuration();
+    conf.setInt(YarnConfiguration.CLIENT_FAILOVER_RETRIES, maxRetries);
+    RetryPolicy policy = FederationStateStoreFacade.createRetryPolicy(conf);
+    RetryAction action = policy.shouldRetry(
+        new FederationStateStoreException(
+            FederationStateStoreErrorCode.APPLICATIONS_INSERT_FAIL),
+        0, 0, false);
+    Assert.assertEquals(RetryAction.FAIL.action, action.action);
+  }
+
+  /*
+   * Test to validate that FederationStateStoreInvalidInputException is not a
+   * retriable exception.
+   */
+  @Test
+  public void testFacadeInvalidInputException() throws Exception {
+    conf = new Configuration();
+    conf.setInt(YarnConfiguration.CLIENT_FAILOVER_RETRIES, maxRetries);
+    RetryPolicy policy = FederationStateStoreFacade.createRetryPolicy(conf);
+    RetryAction action = policy.shouldRetry(
+        new FederationStateStoreInvalidInputException(""), 0, 0, false);
+    Assert.assertEquals(RetryAction.FAIL.action, action.action);
+  }
+
+  /*
+   * Test to validate that CacheLoaderException is a retriable exception.
+   */
+  @Test
+  public void testFacadeCacheRetriableException() throws Exception {
+    conf = new Configuration();
+    conf.setInt(YarnConfiguration.CLIENT_FAILOVER_RETRIES, maxRetries);
+    RetryPolicy policy = FederationStateStoreFacade.createRetryPolicy(conf);
+    RetryAction action =
+        policy.shouldRetry(new CacheLoaderException(""), 0, 0, false);
+    // We compare only the action, since delay and the reason are random values
+    // during this test
+    Assert.assertEquals(RetryAction.RETRY.action, action.action);
+
+    // After maxRetries we stop to retry
+    action =
+        policy.shouldRetry(new CacheLoaderException(""), maxRetries, 0, false);
+    Assert.assertEquals(RetryAction.FAIL.action, action.action);
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[12/42] hadoop git commit: YARN-5390. Federation Subcluster Resolver. Contributed by Ellen Hui.

Posted by su...@apache.org.
YARN-5390. Federation Subcluster Resolver. Contributed by Ellen Hui.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/7986923e
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/7986923e
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/7986923e

Branch: refs/heads/YARN-2915
Commit: 7986923edc798a6c3c82238d8a853dbc198b61b9
Parents: 38ddea0
Author: Subru Krishnan <su...@apache.org>
Authored: Thu Aug 4 15:58:31 2016 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon May 8 16:59:12 2017 -0700

----------------------------------------------------------------------
 .../hadoop/yarn/conf/YarnConfiguration.java     |   8 +
 .../src/main/resources/yarn-default.xml         |   7 +
 .../hadoop-yarn-server-common/pom.xml           |  10 +
 .../resolver/AbstractSubClusterResolver.java    |  67 +++++++
 .../resolver/DefaultSubClusterResolverImpl.java | 164 +++++++++++++++++
 .../federation/resolver/SubClusterResolver.java |  58 ++++++
 .../federation/resolver/package-info.java       |  17 ++
 .../resolver/TestDefaultSubClusterResolver.java | 184 +++++++++++++++++++
 .../src/test/resources/nodes                    |   4 +
 .../src/test/resources/nodes-malformed          |   3 +
 10 files changed, 522 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7986923e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
index 82274fe..a7a9641 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
@@ -2534,6 +2534,14 @@ public class YarnConfiguration extends Configuration {
   public static final int DEFAULT_SHARED_CACHE_NM_UPLOADER_THREAD_COUNT = 20;
 
   ////////////////////////////////
+  // Federation Configs
+  ////////////////////////////////
+
+  public static final String FEDERATION_PREFIX = YARN_PREFIX + "federation.";
+  public static final String FEDERATION_MACHINE_LIST =
+      FEDERATION_PREFIX + "machine-list";
+
+  ////////////////////////////////
   // Other Configs
   ////////////////////////////////
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7986923e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
index e687eef..a3a2465 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
@@ -2671,6 +2671,13 @@
   <!-- Other Configuration -->
 
   <property>
+    <description>
+      Machine list file to be loaded by the FederationSubCluster Resolver
+    </description>
+    <name>yarn.federation.machine-list</name>
+  </property>
+
+  <property>
     <description>The interval that the yarn client library uses to poll the
     completion status of the asynchronous API of application client protocol.
     </description>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7986923e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml
index 9cc3cae..6d2fbef 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml
@@ -177,6 +177,16 @@
           </filesets>
         </configuration>
       </plugin>
+      <plugin>
+        <groupId>org.apache.rat</groupId>
+        <artifactId>apache-rat-plugin</artifactId>
+        <configuration>
+          <excludes>
+            <exclude>src/test/resources/nodes</exclude>
+            <exclude>src/test/resources/nodes-malformed</exclude>
+          </excludes>
+        </configuration>
+      </plugin>
     </plugins>
   </build>
 </project>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7986923e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/resolver/AbstractSubClusterResolver.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/resolver/AbstractSubClusterResolver.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/resolver/AbstractSubClusterResolver.java
new file mode 100644
index 0000000..8238633
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/resolver/AbstractSubClusterResolver.java
@@ -0,0 +1,67 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.hadoop.yarn.server.federation.resolver;
+
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+
+import java.util.HashMap;
+import java.util.Set;
+import java.util.Map;
+
+/**
+ * Partial implementation of {@link SubClusterResolver}, containing basic
+ * implementations of the read methods.
+ */
+public abstract class AbstractSubClusterResolver implements SubClusterResolver {
+  private Map<String, SubClusterId> nodeToSubCluster =
+      new HashMap<String, SubClusterId>();
+  private Map<String, Set<SubClusterId>> rackToSubClusters =
+      new HashMap<String, Set<SubClusterId>>();
+
+  @Override
+  public SubClusterId getSubClusterForNode(String nodename)
+      throws YarnException {
+    SubClusterId subClusterId = this.nodeToSubCluster.get(nodename);
+
+    if (subClusterId == null) {
+      throw new YarnException("Cannot find subClusterId for node " + nodename);
+    }
+
+    return subClusterId;
+  }
+
+  @Override
+  public Set<SubClusterId> getSubClustersForRack(String rackname)
+      throws YarnException {
+    if (!rackToSubClusters.containsKey(rackname)) {
+      throw new YarnException("Cannot resolve rack " + rackname);
+    }
+
+    return rackToSubClusters.get(rackname);
+  }
+
+  protected Map<String, SubClusterId> getNodeToSubCluster() {
+    return nodeToSubCluster;
+  }
+
+  protected Map<String, Set<SubClusterId>> getRackToSubClusters() {
+    return rackToSubClusters;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7986923e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/resolver/DefaultSubClusterResolverImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/resolver/DefaultSubClusterResolverImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/resolver/DefaultSubClusterResolverImpl.java
new file mode 100644
index 0000000..d3c5c26
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/resolver/DefaultSubClusterResolverImpl.java
@@ -0,0 +1,164 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.hadoop.yarn.server.federation.resolver;
+
+import java.io.BufferedReader;
+import java.nio.charset.Charset;
+import java.nio.file.Files;
+import java.nio.file.InvalidPathException;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ *
+ * Default simple sub-cluster and rack resolver class.
+ *
+ * This class expects a three-column comma separated file, specified in
+ * yarn.federation.machine-list. Each line of the file should be of the format:
+ *
+ * nodeName, subClusterId, rackName
+ *
+ * Lines that do not follow this format will be ignored. This resolver only
+ * loads the file when load() is explicitly called; it will not react to changes
+ * to the file.
+ *
+ * It is case-insensitive on the rack and node names and ignores
+ * leading/trailing whitespace.
+ *
+ */
+public class DefaultSubClusterResolverImpl extends AbstractSubClusterResolver
+    implements SubClusterResolver {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(DefaultSubClusterResolverImpl.class);
+  private Configuration conf;
+
+  // Index of the node hostname in the machine info file.
+  private static final int NODE_NAME_INDEX = 0;
+
+  // Index of the sub-cluster ID in the machine info file.
+  private static final int SUBCLUSTER_ID_INDEX = 1;
+
+  // Index of the rack name ID in the machine info file.
+  private static final int RACK_NAME_INDEX = 2;
+
+  @Override
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+  }
+
+  @Override
+  public Configuration getConf() {
+    return this.conf;
+  }
+
+  @Override
+  public SubClusterId getSubClusterForNode(String nodename)
+      throws YarnException {
+    return super.getSubClusterForNode(nodename.toUpperCase());
+  }
+
+  @Override
+  public void load() {
+    String fileName =
+        this.conf.get(YarnConfiguration.FEDERATION_MACHINE_LIST, "");
+
+    try {
+      if (fileName == null || fileName.trim().length() == 0) {
+        LOG.info(
+            "The machine list file path is not specified in the configuration");
+        return;
+      }
+
+      Path file = null;
+      BufferedReader reader = null;
+
+      try {
+        file = Paths.get(fileName);
+      } catch (InvalidPathException e) {
+        LOG.info("The configured machine list file path {} does not exist",
+            fileName);
+        return;
+      }
+
+      try {
+        reader = Files.newBufferedReader(file, Charset.defaultCharset());
+        String line = null;
+        while ((line = reader.readLine()) != null) {
+          String[] tokens = line.split(",");
+          if (tokens.length == 3) {
+
+            String nodeName = tokens[NODE_NAME_INDEX].trim().toUpperCase();
+            SubClusterId subClusterId =
+                SubClusterId.newInstance(tokens[SUBCLUSTER_ID_INDEX].trim());
+            String rackName = tokens[RACK_NAME_INDEX].trim().toUpperCase();
+
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("Loading node into resolver: {} --> {}", nodeName,
+                  subClusterId);
+              LOG.debug("Loading rack into resolver: {} --> {} ", rackName,
+                  subClusterId);
+            }
+
+            this.getNodeToSubCluster().put(nodeName, subClusterId);
+            loadRackToSubCluster(rackName, subClusterId);
+          } else {
+            LOG.warn("Skipping malformed line in machine list: " + line);
+          }
+        }
+      } finally {
+        if (reader != null) {
+          reader.close();
+        }
+      }
+      LOG.info("Successfully loaded file {}", fileName);
+
+    } catch (Exception e) {
+      LOG.error("Failed to parse file " + fileName, e);
+    }
+  }
+
+  private void loadRackToSubCluster(String rackName,
+      SubClusterId subClusterId) {
+    String rackNameUpper = rackName.toUpperCase();
+
+    if (!this.getRackToSubClusters().containsKey(rackNameUpper)) {
+      this.getRackToSubClusters().put(rackNameUpper,
+          new HashSet<SubClusterId>());
+    }
+
+    this.getRackToSubClusters().get(rackNameUpper).add(subClusterId);
+
+  }
+
+  @Override
+  public Set<SubClusterId> getSubClustersForRack(String rackname)
+      throws YarnException {
+    return super.getSubClustersForRack(rackname.toUpperCase());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7986923e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/resolver/SubClusterResolver.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/resolver/SubClusterResolver.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/resolver/SubClusterResolver.java
new file mode 100644
index 0000000..c6adfa6
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/resolver/SubClusterResolver.java
@@ -0,0 +1,58 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.hadoop.yarn.server.federation.resolver;
+
+import java.util.Set;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+
+/**
+ * An utility that helps to determine the sub-cluster that a specified node
+ * belongs to.
+ */
+public interface SubClusterResolver extends Configurable {
+
+  /**
+   * Obtain the sub-cluster that a specified node belongs to.
+   *
+   * @param nodename the node whose sub-cluster is to be determined
+   * @return the sub-cluster as identified by the {@link SubClusterId} that the
+   *         node belongs to
+   * @throws YarnException if the node's sub-cluster cannot be resolved
+   */
+  SubClusterId getSubClusterForNode(String nodename) throws YarnException;
+
+  /**
+   * Obtain the sub-clusters that have nodes on a specified rack.
+   *
+   * @param rackname the name of the rack
+   * @return the sub-clusters as identified by the {@link SubClusterId} that
+   *         have nodes on the given rack
+   * @throws YarnException if the sub-cluster of any node on the rack cannot be
+   *           resolved, or if the rack name is not recognized
+   */
+  Set<SubClusterId> getSubClustersForRack(String rackname) throws YarnException;
+
+  /**
+   * Load the nodes to subCluster mapping from the file.
+   */
+  void load();
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7986923e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/resolver/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/resolver/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/resolver/package-info.java
new file mode 100644
index 0000000..c042660
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/resolver/package-info.java
@@ -0,0 +1,17 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.yarn.server.federation.resolver;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7986923e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/resolver/TestDefaultSubClusterResolver.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/resolver/TestDefaultSubClusterResolver.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/resolver/TestDefaultSubClusterResolver.java
new file mode 100644
index 0000000..7396942
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/resolver/TestDefaultSubClusterResolver.java
@@ -0,0 +1,184 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.hadoop.yarn.server.federation.resolver;
+
+import java.net.URL;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test {@link SubClusterResolver} against correct and malformed Federation
+ * machine lists.
+ */
+public class TestDefaultSubClusterResolver {
+  private static YarnConfiguration conf;
+  private static SubClusterResolver resolver;
+
+  public static void setUpGoodFile() {
+    conf = new YarnConfiguration();
+    resolver = new DefaultSubClusterResolverImpl();
+
+    URL url =
+        Thread.currentThread().getContextClassLoader().getResource("nodes");
+    if (url == null) {
+      throw new RuntimeException(
+          "Could not find 'nodes' dummy file in classpath");
+    }
+
+    conf.set(YarnConfiguration.FEDERATION_MACHINE_LIST, url.getPath());
+    resolver.setConf(conf);
+    resolver.load();
+  }
+
+  private void setUpMalformedFile() {
+    conf = new YarnConfiguration();
+    resolver = new DefaultSubClusterResolverImpl();
+
+    URL url = Thread.currentThread().getContextClassLoader()
+        .getResource("nodes-malformed");
+    if (url == null) {
+      throw new RuntimeException(
+          "Could not find 'nodes-malformed' dummy file in classpath");
+    }
+
+    conf.set(YarnConfiguration.FEDERATION_MACHINE_LIST, url.getPath());
+    resolver.setConf(conf);
+    resolver.load();
+  }
+
+  private void setUpNonExistentFile() {
+    conf = new YarnConfiguration();
+    resolver = new DefaultSubClusterResolverImpl();
+
+    conf.set(YarnConfiguration.FEDERATION_MACHINE_LIST, "fileDoesNotExist");
+    resolver.setConf(conf);
+    resolver.load();
+  }
+
+  @Test
+  public void testGetSubClusterForNode() throws YarnException {
+    setUpGoodFile();
+
+    // All lowercase, no whitespace in machine list file
+    Assert.assertEquals(SubClusterId.newInstance("subcluster1"),
+        resolver.getSubClusterForNode("node1"));
+    // Leading and trailing whitespace in machine list file
+    Assert.assertEquals(SubClusterId.newInstance("subcluster2"),
+        resolver.getSubClusterForNode("node2"));
+    // Node name capitalization in machine list file
+    Assert.assertEquals(SubClusterId.newInstance("subcluster3"),
+        resolver.getSubClusterForNode("node3"));
+
+    try {
+      resolver.getSubClusterForNode("nodeDoesNotExist");
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertTrue(
+          e.getMessage().startsWith("Cannot find subClusterId for node"));
+    }
+  }
+
+  @Test
+  public void testGetSubClusterForNodeMalformedFile() throws YarnException {
+    setUpMalformedFile();
+
+    try {
+      resolver.getSubClusterForNode("node1");
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertTrue(
+          e.getMessage().startsWith("Cannot find subClusterId for node"));
+    }
+
+    try {
+      resolver.getSubClusterForNode("node2");
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertTrue(
+          e.getMessage().startsWith("Cannot find subClusterId for node"));
+    }
+
+    Assert.assertEquals(SubClusterId.newInstance("subcluster3"),
+        resolver.getSubClusterForNode("node3"));
+
+    try {
+      resolver.getSubClusterForNode("nodeDoesNotExist");
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertTrue(
+          e.getMessage().startsWith("Cannot find subClusterId for node"));
+    }
+  }
+
+  @Test
+  public void testGetSubClusterForNodeNoFile() throws YarnException {
+    setUpNonExistentFile();
+
+    try {
+      resolver.getSubClusterForNode("node1");
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertTrue(
+          e.getMessage().startsWith("Cannot find subClusterId for node"));
+    }
+  }
+
+  @Test
+  public void testGetSubClustersForRack() throws YarnException {
+    setUpGoodFile();
+
+    Set<SubClusterId> rack1Expected = new HashSet<SubClusterId>();
+    rack1Expected.add(SubClusterId.newInstance("subcluster1"));
+    rack1Expected.add(SubClusterId.newInstance("subcluster2"));
+
+    Set<SubClusterId> rack2Expected = new HashSet<SubClusterId>();
+    rack2Expected.add(SubClusterId.newInstance("subcluster3"));
+
+    // Two subclusters have nodes in rack1
+    Assert.assertEquals(rack1Expected, resolver.getSubClustersForRack("rack1"));
+
+    // Two nodes are in rack2, but both belong to subcluster3
+    Assert.assertEquals(rack2Expected, resolver.getSubClustersForRack("rack2"));
+
+    try {
+      resolver.getSubClustersForRack("rackDoesNotExist");
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertTrue(e.getMessage().startsWith("Cannot resolve rack"));
+    }
+  }
+
+  @Test
+  public void testGetSubClustersForRackNoFile() throws YarnException {
+    setUpNonExistentFile();
+
+    try {
+      resolver.getSubClustersForRack("rack1");
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertTrue(e.getMessage().startsWith("Cannot resolve rack"));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7986923e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/resources/nodes
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/resources/nodes b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/resources/nodes
new file mode 100644
index 0000000..e4d6112
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/resources/nodes
@@ -0,0 +1,4 @@
+node1,subcluster1,rack1
+ node2 , subcluster2, RACK1
+noDE3,subcluster3, rack2
+node4, subcluster3, rack2
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7986923e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/resources/nodes-malformed
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/resources/nodes-malformed b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/resources/nodes-malformed
new file mode 100644
index 0000000..6d0aa39
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/resources/nodes-malformed
@@ -0,0 +1,3 @@
+node1,
+node2,subcluster2,subCluster2, rack1
+node3,subcluster3, rack2
\ No newline at end of file


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[28/42] hadoop git commit: YARN-5300. Exclude generated federation protobuf sources from YARN Javadoc/findbugs build

Posted by su...@apache.org.
YARN-5300. Exclude generated federation protobuf sources from YARN Javadoc/findbugs build


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/7fe7ebb8
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/7fe7ebb8
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/7fe7ebb8

Branch: refs/heads/YARN-2915
Commit: 7fe7ebb814c5c74d5dba736dff78ec495a2744e7
Parents: 749e5c0
Author: Subru Krishnan <su...@apache.org>
Authored: Tue Jul 19 15:08:25 2016 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon May 8 16:59:12 2017 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml | 3 +++
 hadoop-yarn-project/hadoop-yarn/pom.xml                          | 2 +-
 2 files changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7fe7ebb8/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml b/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
index c090749..2f5451d 100644
--- a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
+++ b/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
@@ -21,6 +21,9 @@
     <Package name="org.apache.hadoop.yarn.proto" />
   </Match>
   <Match>
+    <Package name="org.apache.hadoop.yarn.federation.proto" />
+  </Match>
+  <Match>
     <Class name="~org\.apache\.hadoop\.yarn\.ipc\.RpcProtos.*" />
   </Match>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7fe7ebb8/hadoop-yarn-project/hadoop-yarn/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/pom.xml b/hadoop-yarn-project/hadoop-yarn/pom.xml
index c43588a..99b8b5f 100644
--- a/hadoop-yarn-project/hadoop-yarn/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/pom.xml
@@ -75,7 +75,7 @@
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-javadoc-plugin</artifactId>
         <configuration>
-          <excludePackageNames>org.apache.hadoop.yarn.proto</excludePackageNames>
+          <excludePackageNames>org.apache.hadoop.yarn.proto:org.apache.hadoop.yarn.federation.proto</excludePackageNames>
         </configuration>
       </plugin>
     </plugins>


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[14/42] hadoop git commit: YARN-5408. Compose Federation membership/application/policy APIs into an uber FederationStateStore API. (Ellen Hui via Subru).

Posted by su...@apache.org.
YARN-5408. Compose Federation membership/application/policy APIs into an uber FederationStateStore API. (Ellen Hui via Subru).


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/27b3ce0b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/27b3ce0b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/27b3ce0b

Branch: refs/heads/YARN-2915
Commit: 27b3ce0b38593c112cb09a152d2d35e42f154a8b
Parents: 33dfdb7
Author: Subru Krishnan <su...@apache.org>
Authored: Mon Aug 8 14:53:38 2016 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon May 8 16:59:12 2017 -0700

----------------------------------------------------------------------
 ...ederationApplicationHomeSubClusterStore.java | 18 ++----
 .../store/FederationMembershipStateStore.java   | 14 +----
 .../federation/store/FederationStateStore.java  | 64 ++++++++++++++++++++
 .../store/impl/MemoryFederationStateStore.java  | 19 ------
 .../impl/FederationStateStoreBaseTest.java      | 57 +++++++++--------
 .../impl/TestMemoryFederationStateStore.java    | 21 +------
 6 files changed, 99 insertions(+), 94 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/27b3ce0b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/FederationApplicationHomeSubClusterStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/FederationApplicationHomeSubClusterStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/FederationApplicationHomeSubClusterStore.java
index 217ee2e..22bb88a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/FederationApplicationHomeSubClusterStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/FederationApplicationHomeSubClusterStore.java
@@ -30,7 +30,6 @@ import org.apache.hadoop.yarn.server.federation.store.records.GetApplicationsHom
 import org.apache.hadoop.yarn.server.federation.store.records.GetApplicationsHomeSubClusterResponse;
 import org.apache.hadoop.yarn.server.federation.store.records.UpdateApplicationHomeSubClusterRequest;
 import org.apache.hadoop.yarn.server.federation.store.records.UpdateApplicationHomeSubClusterResponse;
-import org.apache.hadoop.yarn.server.records.Version;
 
 /**
  * FederationApplicationHomeSubClusterStore maintains the state of all
@@ -50,15 +49,6 @@ import org.apache.hadoop.yarn.server.records.Version;
 public interface FederationApplicationHomeSubClusterStore {
 
   /**
-   * Get the {@link Version} of the underlying federation application state
-   * store.
-   *
-   * @return the {@link Version} of the underlying federation application state
-   *         store
-   */
-  Version getApplicationStateStoreVersion();
-
-  /**
    * Register the home {@code SubClusterId} of the newly submitted
    * {@code ApplicationId}. Currently response is empty if the operation was
    * successful, if not an exception reporting reason for a failure.
@@ -91,16 +81,16 @@ public interface FederationApplicationHomeSubClusterStore {
    * {@code ApplicationId}.
    *
    * @param request contains the application queried
-   * @return {@code ApplicationHomeSubCluster} containing the application's
-   *         home subcluster
+   * @return {@code ApplicationHomeSubCluster} containing the application's home
+   *         subcluster
    * @throws YarnException if the request is invalid/fails
    */
   GetApplicationHomeSubClusterResponse getApplicationHomeSubClusterMap(
       GetApplicationHomeSubClusterRequest request) throws YarnException;
 
   /**
-   * Get the {@code ApplicationHomeSubCluster} list representing the mapping
-   * of all submitted applications to it's home sub-cluster.
+   * Get the {@code ApplicationHomeSubCluster} list representing the mapping of
+   * all submitted applications to it's home sub-cluster.
    *
    * @param request empty representing all applications
    * @return the mapping of all submitted application to it's home sub-cluster

http://git-wip-us.apache.org/repos/asf/hadoop/blob/27b3ce0b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/FederationMembershipStateStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/FederationMembershipStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/FederationMembershipStateStore.java
index 378eadc..7778d5f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/FederationMembershipStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/FederationMembershipStateStore.java
@@ -30,7 +30,6 @@ import org.apache.hadoop.yarn.server.federation.store.records.SubClusterHeartbea
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterHeartbeatResponse;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterRegisterRequest;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterRegisterResponse;
-import org.apache.hadoop.yarn.server.records.Version;
 
 /**
  * FederationMembershipStateStore maintains the state of all
@@ -42,15 +41,6 @@ import org.apache.hadoop.yarn.server.records.Version;
 public interface FederationMembershipStateStore {
 
   /**
-   * Get the {@link Version} of the underlying federation membership state
-   * store.
-   *
-   * @return the {@link Version} of the underlying federation membership state
-   *         store
-   */
-  Version getMembershipStateStoreVersion();
-
-  /**
    * Register a <em>subcluster</em> by publishing capabilities as represented by
    * {@code SubClusterInfo} to indicate participation in federation. This is
    * typically done during initialization or restart/failover of the
@@ -80,7 +70,7 @@ public interface FederationMembershipStateStore {
    */
   SubClusterDeregisterResponse deregisterSubCluster(
       SubClusterDeregisterRequest subClusterDeregisterRequest)
-      throws YarnException;
+          throws YarnException;
 
   /**
    * Periodic heartbeat from a <code>ResourceManager</code> participating in
@@ -96,7 +86,7 @@ public interface FederationMembershipStateStore {
    */
   SubClusterHeartbeatResponse subClusterHeartbeat(
       SubClusterHeartbeatRequest subClusterHeartbeatRequest)
-      throws YarnException;
+          throws YarnException;
 
   /**
    * Get the membership information of <em>subcluster</em> as identified by

http://git-wip-us.apache.org/repos/asf/hadoop/blob/27b3ce0b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/FederationStateStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/FederationStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/FederationStateStore.java
new file mode 100644
index 0000000..9397e9c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/FederationStateStore.java
@@ -0,0 +1,64 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.store;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.records.Version;
+
+/**
+ * FederationStore extends the three interfaces used to coordinate the state of
+ * a federated cluster: {@link FederationApplicationHomeSubClusterStore},
+ * {@link FederationMembershipStateStore}, and {@link FederationPolicyStore}.
+ *
+ */
+public interface FederationStateStore
+    extends FederationApplicationHomeSubClusterStore,
+    FederationMembershipStateStore, FederationPolicyStore {
+
+  /**
+   * Initialize the FederationStore.
+   *
+   * @param conf the cluster configuration
+   * @throws YarnException if initialization fails
+   */
+  void init(Configuration conf) throws YarnException;
+
+  /**
+   * Perform any cleanup operations of the StateStore.
+   *
+   * @throws Exception if cleanup fails
+   */
+  void close() throws Exception;
+
+  /**
+   * Get the {@link Version} of the underlying federation state store client.
+   *
+   * @return the {@link Version} of the underlying federation store client
+   */
+  Version getCurrentVersion();
+
+  /**
+   * Load the version information from the federation state store.
+   *
+   * @return the {@link Version} of the federation state store
+   */
+  Version loadVersion();
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/27b3ce0b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/MemoryFederationStateStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/MemoryFederationStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/MemoryFederationStateStore.java
index 7fdc4a9..cea4ac2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/MemoryFederationStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/MemoryFederationStateStore.java
@@ -36,11 +36,8 @@ import org.apache.hadoop.yarn.server.federation.store.records.SubClusterHeartbea
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterHeartbeatResponse;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterRegisterRequest;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterRegisterResponse;
-import org.apache.hadoop.yarn.server.records.Version;
 import org.apache.hadoop.yarn.util.MonotonicClock;
 
-import com.google.common.annotations.VisibleForTesting;
-
 /**
  * In-memory implementation of FederationMembershipStateStore.
  */
@@ -52,11 +49,6 @@ public class MemoryFederationStateStore
   private final MonotonicClock clock = new MonotonicClock();
 
   @Override
-  public Version getMembershipStateStoreVersion() {
-    return null;
-  }
-
-  @Override
   public SubClusterRegisterResponse registerSubCluster(
       SubClusterRegisterRequest request) throws YarnException {
     SubClusterInfo subClusterInfo = request.getSubClusterInfo();
@@ -122,17 +114,6 @@ public class MemoryFederationStateStore
       }
     }
     return GetSubClustersInfoResponse.newInstance(result);
-
-  }
-
-  @VisibleForTesting
-  public Map<SubClusterId, SubClusterInfo> getMembershipTable() {
-    return membership;
-  }
-
-  @VisibleForTesting
-  public void clearMembershipTable() {
-    membership.clear();
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/27b3ce0b/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
----------------------------------------------------------------------
diff --git a/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 b/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
index 7eb1c86..c76a485 100644
--- a/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
+++ b/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
@@ -19,25 +19,21 @@ package org.apache.hadoop.yarn.server.federation.store.impl;
 
 import java.io.IOException;
 
-import org.junit.Before;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.Map;
-
-import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
-import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
-import org.apache.hadoop.yarn.server.federation.store.records.SubClusterState;
-import org.apache.hadoop.yarn.server.federation.store.records.SubClusterDeregisterRequest;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.federation.store.FederationMembershipStateStore;
 import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterInfoRequest;
 import org.apache.hadoop.yarn.server.federation.store.records.GetSubClustersInfoRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterDeregisterRequest;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterHeartbeatRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterRegisterRequest;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterRegisterResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterState;
 import org.apache.hadoop.yarn.util.MonotonicClock;
+import org.junit.After;
 import org.junit.Assert;
+import org.junit.Before;
 import org.junit.Test;
 
 /**
@@ -45,17 +41,22 @@ import org.junit.Test;
  */
 public abstract class FederationStateStoreBaseTest {
 
-  static final Logger LOG =
-      LoggerFactory.getLogger(FederationStateStoreBaseTest.class);
   private static final MonotonicClock CLOCK = new MonotonicClock();
 
-  private FederationMembershipStateStore stateStore = getStateStore();
+  private FederationMembershipStateStore stateStore;
 
   @Before
   public void before() throws IOException {
-    clearMembership();
+    stateStore = getCleanStateStore();
   }
 
+  @After
+  public void after() {
+    stateStore = null;
+  }
+
+  protected abstract FederationMembershipStateStore getCleanStateStore();
+
   @Test
   public void testRegisterSubCluster() throws Exception {
     SubClusterId subClusterId = SubClusterId.newInstance("SC");
@@ -63,11 +64,9 @@ public abstract class FederationStateStoreBaseTest {
 
     SubClusterRegisterResponse result = stateStore.registerSubCluster(
         SubClusterRegisterRequest.newInstance(subClusterInfo));
-    Map<SubClusterId, SubClusterInfo> membership = getMembership();
 
-    Assert.assertNotNull(membership.get(subClusterId));
     Assert.assertNotNull(result);
-    Assert.assertEquals(subClusterInfo, membership.get(subClusterId));
+    Assert.assertEquals(subClusterInfo, querySubClusterInfo(subClusterId));
   }
 
   @Test
@@ -83,10 +82,8 @@ public abstract class FederationStateStoreBaseTest {
 
     stateStore.deregisterSubCluster(deregisterRequest);
 
-    Map<SubClusterId, SubClusterInfo> membership = getMembership();
-    Assert.assertNotNull(membership.get(subClusterId));
-    Assert.assertEquals(membership.get(subClusterId).getState(),
-        SubClusterState.SC_UNREGISTERED);
+    Assert.assertEquals(SubClusterState.SC_UNREGISTERED,
+        querySubClusterInfo(subClusterId).getState());
   }
 
   @Test
@@ -179,10 +176,9 @@ public abstract class FederationStateStoreBaseTest {
         .newInstance(subClusterId, SubClusterState.SC_RUNNING, "cabability");
     stateStore.subClusterHeartbeat(heartbeatRequest);
 
-    Map<SubClusterId, SubClusterInfo> membership = getMembership();
-    Assert.assertEquals(membership.get(subClusterId).getState(),
-        SubClusterState.SC_RUNNING);
-    Assert.assertNotNull(membership.get(subClusterId).getLastHeartBeat());
+    Assert.assertEquals(SubClusterState.SC_RUNNING,
+        querySubClusterInfo(subClusterId).getState());
+    Assert.assertNotNull(querySubClusterInfo(subClusterId).getLastHeartBeat());
   }
 
   @Test
@@ -212,10 +208,11 @@ public abstract class FederationStateStoreBaseTest {
         CLOCK.getTime(), "cabability");
   }
 
-  protected abstract Map<SubClusterId, SubClusterInfo> getMembership();
-
-  protected abstract void clearMembership();
-
-  protected abstract FederationMembershipStateStore getStateStore();
+  private SubClusterInfo querySubClusterInfo(SubClusterId subClusterId)
+      throws YarnException {
+    GetSubClusterInfoRequest request =
+        GetSubClusterInfoRequest.newInstance(subClusterId);
+    return stateStore.getSubCluster(request).getSubClusterInfo();
+  }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/27b3ce0b/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
----------------------------------------------------------------------
diff --git a/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 b/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
index b74ffbd..9396eda 100644
--- a/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
+++ b/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
@@ -17,11 +17,7 @@
 
 package org.apache.hadoop.yarn.server.federation.store.impl;
 
-import java.util.Map;
-
 import org.apache.hadoop.yarn.server.federation.store.FederationMembershipStateStore;
-import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
-import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
 
 /**
  * Unit tests for MemoryFederationStateStore.
@@ -29,21 +25,8 @@ import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
 public class TestMemoryFederationStateStore
     extends FederationStateStoreBaseTest {
 
-  private static final MemoryFederationStateStore STATESTORE =
-      new MemoryFederationStateStore();
-
-  @Override
-  protected Map<SubClusterId, SubClusterInfo> getMembership() {
-    return STATESTORE.getMembershipTable();
-  }
-
-  @Override
-  protected void clearMembership() {
-    STATESTORE.clearMembershipTable();
-  }
-
   @Override
-  protected FederationMembershipStateStore getStateStore() {
-    return STATESTORE;
+  protected FederationMembershipStateStore getCleanStateStore() {
+    return new MemoryFederationStateStore();
   }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[40/42] hadoop git commit: YARN-5872. Add AlwayReject policies for router and amrmproxy. (Carlo Curino via Subru).

Posted by su...@apache.org.
YARN-5872. Add AlwayReject policies for router and amrmproxy. (Carlo Curino via Subru).


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/7fbb165c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/7fbb165c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/7fbb165c

Branch: refs/heads/YARN-2915
Commit: 7fbb165c5c37ff76a8b5630f41551917a755b188
Parents: a2cdf2c
Author: Subru Krishnan <su...@apache.org>
Authored: Tue Nov 22 18:37:30 2016 -0800
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon May 8 16:59:13 2017 -0700

----------------------------------------------------------------------
 .../amrmproxy/RejectAMRMProxyPolicy.java        | 67 +++++++++++++++++
 .../manager/RejectAllPolicyManager.java         | 40 ++++++++++
 .../policies/router/RejectRouterPolicy.java     | 66 +++++++++++++++++
 .../amrmproxy/TestRejectAMRMProxyPolicy.java    | 78 ++++++++++++++++++++
 .../manager/TestRejectAllPolicyManager.java     | 40 ++++++++++
 .../policies/router/TestRejectRouterPolicy.java | 63 ++++++++++++++++
 6 files changed, 354 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7fbb165c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/RejectAMRMProxyPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/RejectAMRMProxyPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/RejectAMRMProxyPolicy.java
new file mode 100644
index 0000000..3783df6
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/RejectAMRMProxyPolicy.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies.amrmproxy;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
+import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.federation.policies.FederationPolicyInitializationContext;
+import org.apache.hadoop.yarn.server.federation.policies.FederationPolicyInitializationContextValidator;
+import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyException;
+import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+
+/**
+ * An implementation of the {@link FederationAMRMProxyPolicy} that simply
+ * rejects all requests. Useful to prevent apps from accessing any sub-cluster.
+ */
+public class RejectAMRMProxyPolicy extends AbstractAMRMProxyPolicy {
+
+  private Set<SubClusterId> knownClusterIds = new HashSet<>();
+
+  @Override
+  public void reinitialize(FederationPolicyInitializationContext policyContext)
+      throws FederationPolicyInitializationException {
+    // overrides initialize to avoid weight checks that do no apply for
+    // this policy.
+    FederationPolicyInitializationContextValidator.validate(policyContext,
+        this.getClass().getCanonicalName());
+    setPolicyContext(policyContext);
+  }
+
+  @Override
+  public Map<SubClusterId, List<ResourceRequest>> splitResourceRequests(
+      List<ResourceRequest> resourceRequests) throws YarnException {
+    throw new FederationPolicyException("The policy configured for this queue "
+        + "rejects all routing requests by construction.");
+  }
+
+  @Override
+  public void notifyOfResponse(SubClusterId subClusterId,
+      AllocateResponse response) throws YarnException {
+    // This might be invoked for applications started with a previous policy,
+    // do nothing for this policy.
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7fbb165c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/manager/RejectAllPolicyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/manager/RejectAllPolicyManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/manager/RejectAllPolicyManager.java
new file mode 100644
index 0000000..7bd7a1b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/manager/RejectAllPolicyManager.java
@@ -0,0 +1,40 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies.manager;
+
+import org.apache.hadoop.yarn.server.federation.policies.amrmproxy.RejectAMRMProxyPolicy;
+import org.apache.hadoop.yarn.server.federation.policies.router.RejectRouterPolicy;
+
+/**
+ * This class represents a simple implementation of a {@code
+ * FederationPolicyManager}.
+ *
+ * This policy rejects all reuqests for both router and amrmproxy routing. This
+ * is to be used to prevent applications in a specific queue (or if used as
+ * default for non-configured queues) from accessing cluster resources.
+ */
+public class RejectAllPolicyManager extends AbstractPolicyManager {
+
+  public RejectAllPolicyManager() {
+    // this structurally hard-codes two compatible policies for Router and
+    // AMRMProxy.
+    routerFederationPolicy = RejectRouterPolicy.class;
+    amrmProxyFederationPolicy = RejectAMRMProxyPolicy.class;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7fbb165c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/RejectRouterPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/RejectRouterPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/RejectRouterPolicy.java
new file mode 100644
index 0000000..faf3279
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/router/RejectRouterPolicy.java
@@ -0,0 +1,66 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies.router;
+
+import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.federation.policies.FederationPolicyInitializationContext;
+import org.apache.hadoop.yarn.server.federation.policies.FederationPolicyInitializationContextValidator;
+import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyException;
+import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+
+/**
+ * This {@link FederationRouterPolicy} simply rejects all incoming requests.
+ * This is useful to prevent applications running in a queue to be run
+ * anywhere in the federated cluster.
+ */
+public class RejectRouterPolicy extends AbstractRouterPolicy {
+
+  @Override
+  public void reinitialize(
+      FederationPolicyInitializationContext federationPolicyContext)
+      throws FederationPolicyInitializationException {
+    FederationPolicyInitializationContextValidator
+        .validate(federationPolicyContext, this.getClass().getCanonicalName());
+    setPolicyContext(federationPolicyContext);
+  }
+
+  /**
+   * The policy always reject requests.
+   *
+   * @param appSubmissionContext the context for the app being submitted.
+   *
+   * @return (never).
+   *
+   * @throws YarnException (always) to prevent applications in this queue to
+   * be run anywhere in the federated cluster.
+   */
+  public SubClusterId getHomeSubcluster(
+      ApplicationSubmissionContext appSubmissionContext) throws YarnException {
+
+    // run standard validation, as error might differ
+    validate(appSubmissionContext);
+
+    throw new FederationPolicyException("The policy configured for this queue"
+        + " (" + appSubmissionContext.getQueue() + ") reject all routing "
+        + "requests by construction. Application " + appSubmissionContext
+        .getApplicationId() + " cannot be routed to any RM.");
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7fbb165c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/TestRejectAMRMProxyPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/TestRejectAMRMProxyPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/TestRejectAMRMProxyPolicy.java
new file mode 100644
index 0000000..41e7fed
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/amrmproxy/TestRejectAMRMProxyPolicy.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies.amrmproxy;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.server.federation.policies.BaseFederationPoliciesTest;
+import org.apache.hadoop.yarn.server.federation.policies.dao.WeightedPolicyInfo;
+import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyException;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterIdInfo;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterState;
+import org.apache.hadoop.yarn.server.federation.utils.FederationPoliciesTestUtil;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Simple test class for the {@link RejectAMRMProxyPolicy}.
+ */
+public class TestRejectAMRMProxyPolicy
+    extends BaseFederationPoliciesTest {
+
+  @Before
+  public void setUp() throws Exception {
+    setPolicy(new RejectAMRMProxyPolicy());
+    // needed for base test to work
+    setPolicyInfo(mock(WeightedPolicyInfo.class));
+
+    for (int i = 1; i <= 2; i++) {
+      SubClusterIdInfo sc = new SubClusterIdInfo("sc" + i);
+      SubClusterInfo sci = mock(SubClusterInfo.class);
+      when(sci.getState()).thenReturn(SubClusterState.SC_RUNNING);
+      when(sci.getSubClusterId()).thenReturn(sc.toId());
+      getActiveSubclusters().put(sc.toId(), sci);
+    }
+
+    FederationPoliciesTestUtil.initializePolicyContext(getPolicy(),
+        mock(WeightedPolicyInfo.class), getActiveSubclusters());
+
+  }
+
+  @Test (expected = FederationPolicyException.class)
+  public void testSplitAllocateRequest() throws Exception {
+    // verify the request is broadcasted to all subclusters
+    String[] hosts = new String[] {"host1", "host2" };
+    List<ResourceRequest> resourceRequests = FederationPoliciesTestUtil
+        .createResourceRequests(hosts, 2 * 1024, 2, 1, 3, null, false);
+
+    Map<SubClusterId, List<ResourceRequest>> response =
+        ((FederationAMRMProxyPolicy) getPolicy())
+            .splitResourceRequests(resourceRequests);
+  }
+
+
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7fbb165c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/manager/TestRejectAllPolicyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/manager/TestRejectAllPolicyManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/manager/TestRejectAllPolicyManager.java
new file mode 100644
index 0000000..e4dc7f4
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/manager/TestRejectAllPolicyManager.java
@@ -0,0 +1,40 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies.manager;
+
+import org.apache.hadoop.yarn.server.federation.policies.amrmproxy.RejectAMRMProxyPolicy;
+import org.apache.hadoop.yarn.server.federation.policies.router.RejectRouterPolicy;
+import org.junit.Before;
+
+/**
+ * Simple test of {@link RejectAllPolicyManager}.
+ */
+public class TestRejectAllPolicyManager extends BasePolicyManagerTest {
+
+  @Before
+  public void setup() {
+    // config policy
+    wfp = new RejectAllPolicyManager();
+    wfp.setQueue("queue1");
+
+    // set expected params that the base test class will use for tests
+    expectedPolicyManager = RejectAllPolicyManager.class;
+    expectedAMRMProxyPolicy = RejectAMRMProxyPolicy.class;
+    expectedRouterPolicy = RejectRouterPolicy.class;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7fbb165c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestRejectRouterPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestRejectRouterPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestRejectRouterPolicy.java
new file mode 100644
index 0000000..049ebbf
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/policies/router/TestRejectRouterPolicy.java
@@ -0,0 +1,63 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.federation.policies.router;
+
+import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyException;
+import org.apache.hadoop.yarn.server.federation.utils.FederationPoliciesTestUtil;
+import org.apache.hadoop.yarn.util.resource.Resources;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Simple test class for the {@link RejectRouterPolicy}. Tests that one of the
+ * active subcluster is chosen.
+ */
+public class TestRejectRouterPolicy extends BaseRouterPoliciesTest {
+
+  @Before
+  public void setUp() throws Exception {
+    setPolicy(new RejectRouterPolicy());
+
+    // setting up the active sub-clusters for this test
+    setMockActiveSubclusters(2);
+
+    // initialize policy with context
+    FederationPoliciesTestUtil.initializePolicyContext(getPolicy(),
+        getPolicyInfo(), getActiveSubclusters());
+
+  }
+
+  @Test(expected = FederationPolicyException.class)
+  public void testNoClusterIsChosen() throws YarnException {
+    ((FederationRouterPolicy) getPolicy())
+        .getHomeSubcluster(getApplicationSubmissionContext());
+  }
+
+  @Override
+  @Test(expected = FederationPolicyException.class)
+  public void testNullQueueRouting() throws YarnException {
+    FederationRouterPolicy localPolicy = (FederationRouterPolicy) getPolicy();
+    ApplicationSubmissionContext applicationSubmissionContext =
+        ApplicationSubmissionContext.newInstance(null, null, null, null, null,
+            false, false, 0, Resources.none(), null, false, null, null);
+    localPolicy.getHomeSubcluster(applicationSubmissionContext);
+  }
+
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org