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/07/25 23:59:06 UTC

[01/50] [abbrv] hadoop git commit: YARN-5408. Compose Federation membership/application/policy APIs into an uber FederationStateStore API. (Ellen Hui via Subru). [Forced Update!]

Repository: hadoop
Updated Branches:
  refs/heads/YARN-2915 4e890f254 -> d4cb18005 (forced update)


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/c7c8ca67
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/c7c8ca67
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/c7c8ca67

Branch: refs/heads/YARN-2915
Commit: c7c8ca67ac7d59b28b9ee5602538fa4cf839c331
Parents: 7f7a33d
Author: Subru Krishnan <su...@apache.org>
Authored: Mon Aug 8 14:53:38 2016 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Tue Jul 25 16:56:31 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/c7c8ca67/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/c7c8ca67/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/c7c8ca67/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/c7c8ca67/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/c7c8ca67/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/c7c8ca67/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


[13/50] [abbrv] 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/5627b4e3
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/5627b4e3
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/5627b4e3

Branch: refs/heads/YARN-2915
Commit: 5627b4e3750381ebbd82265e3aeecd6dea41bcaa
Parents: 47f2986
Author: Carlo Curino <cu...@apache.org>
Authored: Thu Apr 13 12:09:39 2017 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Tue Jul 25 16:56:32 2017 -0700

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5627b4e3/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


[02/50] [abbrv] 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/7f7a33d2
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/7f7a33d2
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/7f7a33d2

Branch: refs/heads/YARN-2915
Commit: 7f7a33d2c502b8db23e8f71a4a420ea01674a583
Parents: cfa1ed9
Author: Subru Krishnan <su...@apache.org>
Authored: Fri Aug 5 12:34:58 2016 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Tue Jul 25 16:56:31 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/7f7a33d2/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/7f7a33d2/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/7f7a33d2/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/7f7a33d2/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/7f7a33d2/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/7f7a33d2/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/7f7a33d2/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/7f7a33d2/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/7f7a33d2/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/7f7a33d2/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/7f7a33d2/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/7f7a33d2/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/7f7a33d2/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/7f7a33d2/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/7f7a33d2/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/7f7a33d2/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/7f7a33d2/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/7f7a33d2/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/50] [abbrv] 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/a359ab0b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/a359ab0b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/a359ab0b

Branch: refs/heads/YARN-2915
Commit: a359ab0b3744b3e529a68834545d20dcf7daf7f5
Parents: 6f1a6f7
Author: Subru Krishnan <su...@apache.org>
Authored: Thu Sep 22 17:06:57 2016 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Tue Jul 25 16:56:32 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/a359ab0b/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/a359ab0b/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/a359ab0b/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/a359ab0b/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/a359ab0b/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/a359ab0b/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/a359ab0b/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/a359ab0b/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/a359ab0b/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/a359ab0b/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/a359ab0b/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/a359ab0b/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/a359ab0b/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/a359ab0b/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/a359ab0b/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/a359ab0b/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/a359ab0b/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/a359ab0b/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


[47/50] [abbrv] 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/b08cbca9
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/b08cbca9
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/b08cbca9

Branch: refs/heads/YARN-2915
Commit: b08cbca9f6bdd17dcb3e5ef633f922aeedd6142a
Parents: d28d850
Author: Subru Krishnan <su...@apache.org>
Authored: Fri Feb 24 12:08:53 2017 -0800
Committer: Subru Krishnan <su...@apache.org>
Committed: Tue Jul 25 16:56:32 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/b08cbca9/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
old mode 100644
new mode 100755
index 93bbcf8..5395a10
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -407,6 +407,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/b08cbca9/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/b08cbca9/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/b08cbca9/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/b08cbca9/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/b08cbca9/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 517326b..8335fc8 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/b08cbca9/hadoop-yarn-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/pom.xml b/hadoop-yarn-project/pom.xml
index df87417..2798192 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


[12/50] [abbrv] 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/d28d8507
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/d28d8507
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/d28d8507

Branch: refs/heads/YARN-2915
Commit: d28d8507b8ab7e812d7a9369e518b1c9081b993a
Parents: b6a4fb6
Author: Subru Krishnan <su...@apache.org>
Authored: Wed Feb 22 13:16:22 2017 -0800
Committer: Subru Krishnan <su...@apache.org>
Committed: Tue Jul 25 16:56:32 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/d28d8507/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/d28d8507/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 1915f67..e00f8d1 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


[03/50] [abbrv] 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/cfa1ed98/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/cfa1ed98/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/cfa1ed98/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/cfa1ed98/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/cfa1ed98/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/cfa1ed98/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


[10/50] [abbrv] 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/a359ab0b/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


[29/50] [abbrv] 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/cb84a60c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/cb84a60c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/cb84a60c

Branch: refs/heads/YARN-2915
Commit: cb84a60ce79d96768b2fe79e77d1239f88f7d599
Parents: b08cbca
Author: Carlo Curino <cu...@apache.org>
Authored: Tue Feb 28 17:04:20 2017 -0800
Committer: Subru Krishnan <su...@apache.org>
Committed: Tue Jul 25 16:56:32 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/cb84a60c/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/cb84a60c/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/cb84a60c/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/cb84a60c/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/cb84a60c/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


[41/50] [abbrv] 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/765f7a68
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/765f7a68
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/765f7a68

Branch: refs/heads/YARN-2915
Commit: 765f7a68fb5f08de3b5a81849aaa74c8e6b07cf7
Parents: d38a4de
Author: Carlo Curino <cu...@apache.org>
Authored: Wed Mar 22 13:53:47 2017 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Tue Jul 25 16:56:32 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/765f7a68/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/765f7a68/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


[23/50] [abbrv] hadoop git commit: YARN-6666. Fix unit test failure in TestRouterClientRMService. (Botong Huang via Subru).

Posted by su...@apache.org.
YARN-6666. Fix unit test failure in TestRouterClientRMService. (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/d06bbbff
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/d06bbbff
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/d06bbbff

Branch: refs/heads/YARN-2915
Commit: d06bbbfff856b49f10679dbf397969a4338fc49c
Parents: 81b980b
Author: Subru Krishnan <su...@apache.org>
Authored: Tue May 30 13:37:37 2017 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Tue Jul 25 16:56:32 2017 -0700

----------------------------------------------------------------------
 .../yarn/server/MockResourceManagerFacade.java  | 38 ++++++++------------
 1 file changed, 14 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d06bbbff/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
index 9535ed5..65c12c6 100644
--- 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
@@ -21,8 +21,8 @@ package org.apache.hadoop.yarn.server;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
-import java.util.Map.Entry;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
 
@@ -118,6 +118,7 @@ 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.ApplicationMasterNotRegisteredException;
+import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException;
 import org.apache.hadoop.yarn.exceptions.InvalidApplicationMasterRequestException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
@@ -167,6 +168,7 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
   private static final Logger LOG =
       LoggerFactory.getLogger(MockResourceManagerFacade.class);
 
+  private HashSet<ApplicationId> applicationMap = new HashSet<>();
   private HashMap<String, List<ContainerId>> applicationContainerIdMap =
       new HashMap<String, List<ContainerId>>();
   private HashMap<ContainerId, Container> allocatedContainerMap =
@@ -424,38 +426,26 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
   @Override
   public SubmitApplicationResponse submitApplication(
       SubmitApplicationRequest request) throws YarnException, IOException {
+    ApplicationId appId = null;
+    if (request.getApplicationSubmissionContext() != null) {
+      appId = request.getApplicationSubmissionContext().getApplicationId();
+    }
+    LOG.info("Application submitted: " + appId);
+    applicationMap.add(appId);
     return SubmitApplicationResponse.newInstance();
   }
 
   @Override
   public KillApplicationResponse forceKillApplication(
       KillApplicationRequest request) throws YarnException, IOException {
-    String appId = "";
-    boolean foundApp = false;
+    ApplicationId appId = null;
     if (request.getApplicationId() != null) {
-      appId = request.getApplicationId().toString();
-      synchronized (applicationContainerIdMap) {
-        for (Entry<String, List<ContainerId>> entry : applicationContainerIdMap
-            .entrySet()) {
-          ApplicationAttemptId attemptId =
-              ApplicationAttemptId.fromString(entry.getKey());
-          if (attemptId.getApplicationId().equals(request.getApplicationId())) {
-            // Remove the apptempt and the containers that were being tracked
-            List<ContainerId> ids =
-                applicationContainerIdMap.remove(entry.getKey());
-            if (ids != null) {
-              for (ContainerId c : ids) {
-                allocatedContainerMap.remove(c);
-              }
-            }
-            foundApp = true;
-          }
-        }
+      appId = request.getApplicationId();
+      if (!applicationMap.remove(appId)) {
+        throw new ApplicationNotFoundException(
+            "Trying to kill an absent application: " + appId);
       }
     }
-    if (!foundApp) {
-      throw new YarnException("The application id is NOT registered: " + appId);
-    }
     LOG.info("Force killing application: " + appId);
     return KillApplicationResponse.newInstance(true);
   }


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


[08/50] [abbrv] hadoop git commit: YARN-5413. Create a proxy chain for ResourceManager Admin API in the Router. (Giovanni Matteo Fumarola via Subru).

Posted by su...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/f4b45eb6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/rmadmin/TestRouterRMAdminService.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/rmadmin/TestRouterRMAdminService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/rmadmin/TestRouterRMAdminService.java
new file mode 100644
index 0000000..11786e6
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/rmadmin/TestRouterRMAdminService.java
@@ -0,0 +1,219 @@
+/**
+* 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.rmadmin;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.api.protocolrecords.AddToClusterNodeLabelsResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.CheckForDecommissioningNodesResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshAdminAclsResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshClusterMaxPriorityResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshNodesResourcesResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshNodesResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshQueuesResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshServiceAclsResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshSuperUserGroupsConfigurationResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshUserToGroupsMappingsResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RemoveFromClusterNodeLabelsResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.ReplaceLabelsOnNodeResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.UpdateNodeResourceResponse;
+import org.apache.hadoop.yarn.server.router.rmadmin.RouterRMAdminService.RequestInterceptorChainWrapper;
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Test class to validate the RMAdmin Service inside the Router.
+ */
+public class TestRouterRMAdminService extends BaseRouterRMAdminTest {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestRouterRMAdminService.class);
+
+  /**
+   * Tests if the pipeline is created properly.
+   */
+  @Test
+  public void testRequestInterceptorChainCreation() throws Exception {
+    RMAdminRequestInterceptor root =
+        super.getRouterRMAdminService().createRequestInterceptorChain();
+    int index = 0;
+    while (root != null) {
+      // The current pipeline is:
+      // PassThroughRMAdminRequestInterceptor - index = 0
+      // PassThroughRMAdminRequestInterceptor - index = 1
+      // PassThroughRMAdminRequestInterceptor - 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(
+            PassThroughRMAdminRequestInterceptor.class.getName(),
+            root.getClass().getName());
+        break;
+      case 3:
+        Assert.assertEquals(MockRMAdminRequestInterceptor.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 RouterRMAdmin forwards all the requests to the MockRM and get
+   * back the responses.
+   */
+  @Test
+  public void testRouterRMAdminServiceE2E() throws Exception {
+
+    String user = "test1";
+
+    LOG.info("testRouterRMAdminServiceE2E - Refresh Queues");
+
+    RefreshQueuesResponse responseRefreshQueues = refreshQueues(user);
+    Assert.assertNotNull(responseRefreshQueues);
+
+    LOG.info("testRouterRMAdminServiceE2E - Refresh Nodes");
+
+    RefreshNodesResponse responseRefreshNodes = refreshNodes(user);
+    Assert.assertNotNull(responseRefreshNodes);
+
+    LOG.info("testRouterRMAdminServiceE2E - Refresh Super User");
+
+    RefreshSuperUserGroupsConfigurationResponse responseRefreshSuperUser =
+        refreshSuperUserGroupsConfiguration(user);
+    Assert.assertNotNull(responseRefreshSuperUser);
+
+    LOG.info("testRouterRMAdminServiceE2E - Refresh User to Group");
+
+    RefreshUserToGroupsMappingsResponse responseRefreshUserToGroup =
+        refreshUserToGroupsMappings(user);
+    Assert.assertNotNull(responseRefreshUserToGroup);
+
+    LOG.info("testRouterRMAdminServiceE2E - Refresh Admin Acls");
+
+    RefreshAdminAclsResponse responseRefreshAdminAcls = refreshAdminAcls(user);
+    Assert.assertNotNull(responseRefreshAdminAcls);
+
+    LOG.info("testRouterRMAdminServiceE2E - Refresh Service Acls");
+
+    RefreshServiceAclsResponse responseRefreshServiceAcls =
+        refreshServiceAcls(user);
+    Assert.assertNotNull(responseRefreshServiceAcls);
+
+    LOG.info("testRouterRMAdminServiceE2E - Update Node Resource");
+
+    UpdateNodeResourceResponse responseUpdateNodeResource =
+        updateNodeResource(user);
+    Assert.assertNotNull(responseUpdateNodeResource);
+
+    LOG.info("testRouterRMAdminServiceE2E - Refresh Nodes Resource");
+
+    RefreshNodesResourcesResponse responseRefreshNodesResources =
+        refreshNodesResources(user);
+    Assert.assertNotNull(responseRefreshNodesResources);
+
+    LOG.info("testRouterRMAdminServiceE2E - Add To Cluster NodeLabels");
+
+    AddToClusterNodeLabelsResponse responseAddToClusterNodeLabels =
+        addToClusterNodeLabels(user);
+    Assert.assertNotNull(responseAddToClusterNodeLabels);
+
+    LOG.info("testRouterRMAdminServiceE2E - Remove To Cluster NodeLabels");
+
+    RemoveFromClusterNodeLabelsResponse responseRemoveFromClusterNodeLabels =
+        removeFromClusterNodeLabels(user);
+    Assert.assertNotNull(responseRemoveFromClusterNodeLabels);
+
+    LOG.info("testRouterRMAdminServiceE2E - Replace Labels On Node");
+
+    ReplaceLabelsOnNodeResponse responseReplaceLabelsOnNode =
+        replaceLabelsOnNode(user);
+    Assert.assertNotNull(responseReplaceLabelsOnNode);
+
+    LOG.info("testRouterRMAdminServiceE2E - Check For Decommissioning Nodes");
+
+    CheckForDecommissioningNodesResponse responseCheckForDecom =
+        checkForDecommissioningNodes(user);
+    Assert.assertNotNull(responseCheckForDecom);
+
+    LOG.info("testRouterRMAdminServiceE2E - Refresh Cluster Max Priority");
+
+    RefreshClusterMaxPriorityResponse responseRefreshClusterMaxPriority =
+        refreshClusterMaxPriority(user);
+    Assert.assertNotNull(responseRefreshClusterMaxPriority);
+
+    LOG.info("testRouterRMAdminServiceE2E - Get Groups For User");
+
+    String[] responseGetGroupsForUser = getGroupsForUser(user);
+    Assert.assertNotNull(responseGetGroupsForUser);
+
+  }
+
+  /**
+   * 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;
+
+    refreshQueues("test1");
+    refreshQueues("test2");
+    refreshQueues("test3");
+    refreshQueues("test4");
+    refreshQueues("test5");
+    refreshQueues("test6");
+    refreshQueues("test7");
+    refreshQueues("test8");
+
+    pipelines = super.getRouterRMAdminService().getPipelines();
+    Assert.assertEquals(8, pipelines.size());
+
+    refreshQueues("test9");
+    refreshQueues("test10");
+    refreshQueues("test1");
+    refreshQueues("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


[44/50] [abbrv] 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/469a5c93/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


[43/50] [abbrv] 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/47f29860
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/47f29860
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/47f29860

Branch: refs/heads/YARN-2915
Commit: 47f29860145573a26207961816d5aaf85a9b2fee
Parents: 765f7a6
Author: Subru Krishnan <su...@apache.org>
Authored: Wed Apr 5 15:02:00 2017 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Tue Jul 25 16:56:32 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/47f29860/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/47f29860/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/47f29860/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/47f29860/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/47f29860/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/47f29860/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/47f29860/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/47f29860/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/47f29860/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/47f29860/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/47f29860/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/47f29860/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/47f29860/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/47f29860/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/47f29860/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/47f29860/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/47f29860/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/47f29860/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


[42/50] [abbrv] 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/d38a4de8
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/d38a4de8
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/d38a4de8

Branch: refs/heads/YARN-2915
Commit: d38a4de81aa49bacde3adee4fdb1d619c2693b94
Parents: cb84a60
Author: Subru Krishnan <su...@apache.org>
Authored: Thu Mar 2 18:54:53 2017 -0800
Committer: Subru Krishnan <su...@apache.org>
Committed: Tue Jul 25 16:56:32 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/d38a4de8/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 18c3444..987f8cf 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
@@ -2594,6 +2594,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/d38a4de8/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 96d73bc..6af7321 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
@@ -2701,6 +2701,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/d38a4de8/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/d38a4de8/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/d38a4de8/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


[34/50] [abbrv] 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/c1990752/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/c1990752/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/c1990752/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/c1990752/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/c1990752/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/c1990752/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/c1990752/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/c1990752/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


[30/50] [abbrv] hadoop git commit: YARN-3666. Federation Intercepting and propagating AM- home RM communications. (Botong Huang via Subru).

Posted by su...@apache.org.
YARN-3666. Federation Intercepting and propagating AM- home RM communications. (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/376dc4ae
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/376dc4ae
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/376dc4ae

Branch: refs/heads/YARN-2915
Commit: 376dc4ae0e992b6f11c60fd48f71d10e36bffbb7
Parents: 09c5c5f
Author: Subru Krishnan <su...@apache.org>
Authored: Wed May 31 13:21:09 2017 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Tue Jul 25 16:56:32 2017 -0700

----------------------------------------------------------------------
 .../dev-support/findbugs-exclude.xml            |   7 +
 .../amrmproxy/FederationInterceptor.java        | 510 +++++++++++++++++++
 .../amrmproxy/TestAMRMProxyService.java         |   1 +
 .../amrmproxy/TestFederationInterceptor.java    | 167 ++++++
 .../TestableFederationInterceptor.java          | 133 +++++
 5 files changed, 818 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/376dc4ae/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 ee51094..034f03c 100644
--- a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
+++ b/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
@@ -594,4 +594,11 @@
     <Bug pattern="UL_UNRELEASED_LOCK_EXCEPTION_PATH" />
   </Match>
 
+  <!-- Ignore false alert for RCN_REDUNDANT_NULLCHECK_OF_NULL_VALUE -->
+  <Match>
+    <Class name="org.apache.hadoop.yarn.server.nodemanager.amrmproxy.FederationInterceptor" />
+    <Method name="registerApplicationMaster" />
+    <Bug pattern="RCN_REDUNDANT_NULLCHECK_OF_NULL_VALUE" />
+  </Match>
+
 </FindBugsFilter>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/376dc4ae/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/FederationInterceptor.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/FederationInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/FederationInterceptor.java
new file mode 100644
index 0000000..5f82d69
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/FederationInterceptor.java
@@ -0,0 +1,510 @@
+/**
+ * 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.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
+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.FinishApplicationMasterRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
+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.ResourceBlacklistRequest;
+import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.InvalidApplicationMasterRequestException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.hadoop.yarn.server.federation.failover.FederationProxyProviderUtil;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.utils.AMRMClientUtils;
+import org.apache.hadoop.yarn.server.utils.YarnServerSecurityUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Extends the AbstractRequestInterceptor and provides an implementation for
+ * federation of YARN RM and scaling an application across multiple YARN
+ * sub-clusters. All the federation specific implementation is encapsulated in
+ * this class. This is always the last intercepter in the chain.
+ */
+public class FederationInterceptor extends AbstractRequestInterceptor {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(FederationInterceptor.class);
+
+  /**
+   * The home sub-cluster is the sub-cluster where the AM container is running
+   * in.
+   */
+  private ApplicationMasterProtocol homeRM;
+  private SubClusterId homeSubClusterId;
+
+  /**
+   * Used to keep track of the container Id and the sub cluster RM that created
+   * the container, so that we know which sub-cluster to forward later requests
+   * about existing containers to.
+   */
+  private Map<ContainerId, SubClusterId> containerIdToSubClusterIdMap;
+
+  /**
+   * The original registration request that was sent by the AM. This instance is
+   * reused to register/re-register with all the sub-cluster RMs.
+   */
+  private RegisterApplicationMasterRequest amRegistrationRequest;
+
+  /**
+   * The original registration response from home RM. This instance is reused
+   * for duplicate register request from AM, triggered by timeout between AM and
+   * AMRMProxy.
+   */
+  private RegisterApplicationMasterResponse amRegistrationResponse;
+
+  /** The proxy ugi used to talk to home RM. */
+  private UserGroupInformation appOwner;
+
+  /**
+   * Creates an instance of the FederationInterceptor class.
+   */
+  public FederationInterceptor() {
+    this.containerIdToSubClusterIdMap = new ConcurrentHashMap<>();
+    this.amRegistrationResponse = null;
+  }
+
+  /**
+   * Initializes the instance using specified context.
+   */
+  @Override
+  public void init(AMRMProxyApplicationContext appContext) {
+    super.init(appContext);
+    LOG.info("Initializing Federation Interceptor");
+
+    // Update the conf if available
+    Configuration conf = appContext.getConf();
+    if (conf == null) {
+      conf = getConf();
+    } else {
+      setConf(conf);
+    }
+
+    try {
+      this.appOwner = UserGroupInformation.createProxyUser(appContext.getUser(),
+          UserGroupInformation.getCurrentUser());
+    } catch (Exception ex) {
+      throw new YarnRuntimeException(ex);
+    }
+
+    this.homeSubClusterId =
+        SubClusterId.newInstance(YarnConfiguration.getClusterId(conf));
+    this.homeRM = createHomeRMProxy(appContext);
+  }
+
+  /**
+   * Sends the application master's registration request to the home RM.
+   *
+   * Between AM and AMRMProxy, FederationInterceptor modifies the RM behavior,
+   * so that when AM registers more than once, it returns the same register
+   * success response instead of throwing
+   * {@link InvalidApplicationMasterRequestException}. Furthermore, we present
+   * to AM as if we are the RM that never fails over. When actual RM fails over,
+   * we always re-register automatically.
+   *
+   * We did this because FederationInterceptor can receive concurrent register
+   * requests from AM because of timeout between AM and AMRMProxy, which is
+   * shorter than the timeout + failOver between FederationInterceptor
+   * (AMRMProxy) and RM.
+   */
+  @Override
+  public RegisterApplicationMasterResponse registerApplicationMaster(
+      RegisterApplicationMasterRequest request)
+      throws YarnException, IOException {
+    // If AM is calling with a different request, complain
+    if (this.amRegistrationRequest != null
+        && !this.amRegistrationRequest.equals(request)) {
+      throw new YarnException("A different request body recieved. AM should"
+          + " not call registerApplicationMaster with different request body");
+    }
+
+    // Save the registration request. This will be used for registering with
+    // secondary sub-clusters using UAMs, as well as re-register later
+    this.amRegistrationRequest = request;
+
+    /*
+     * Present to AM as if we are the RM that never fails over. When actual RM
+     * fails over, we always re-register automatically.
+     *
+     * We did this because it is possible for AM to send duplicate register
+     * request because of timeout. When it happens, it is fine to simply return
+     * the success message. Out of all outstanding register threads, only the
+     * last one will still have an unbroken RPC connection and successfully
+     * return the response.
+     */
+    if (this.amRegistrationResponse != null) {
+      return this.amRegistrationResponse;
+    }
+
+    /*
+     * Send a registration request to the home resource manager. Note that here
+     * we don't register with other sub-cluster resource managers because that
+     * will prevent us from using new sub-clusters that get added while the AM
+     * is running and will breaks the elasticity feature. The registration with
+     * the other sub-cluster RM will be done lazily as needed later.
+     */
+    try {
+      this.amRegistrationResponse =
+          this.homeRM.registerApplicationMaster(request);
+    } catch (InvalidApplicationMasterRequestException e) {
+      if (e.getMessage()
+          .contains(AMRMClientUtils.APP_ALREADY_REGISTERED_MESSAGE)) {
+        // Some other register thread might have succeeded in the meantime
+        if (this.amRegistrationResponse != null) {
+          LOG.info("Other concurrent thread registered successfully, "
+              + "simply return the same success register response");
+          return this.amRegistrationResponse;
+        }
+      }
+      // This is a real issue, throw back to AM
+      throw e;
+    }
+
+    // the queue this application belongs will be used for getting
+    // AMRMProxy policy from state store.
+    String queue = this.amRegistrationResponse.getQueue();
+    if (queue == null) {
+      LOG.warn("Received null queue for application "
+          + getApplicationContext().getApplicationAttemptId().getApplicationId()
+          + " from home subcluster. Will use default queue name "
+          + YarnConfiguration.DEFAULT_QUEUE_NAME
+          + " for getting AMRMProxyPolicy");
+    } else {
+      LOG.info("Application "
+          + getApplicationContext().getApplicationAttemptId().getApplicationId()
+          + " belongs to queue " + queue);
+    }
+
+    return this.amRegistrationResponse;
+  }
+
+  /**
+   * Sends the heart beats to the home RM and the secondary sub-cluster RMs that
+   * are being used by the application.
+   */
+  @Override
+  public AllocateResponse allocate(AllocateRequest request)
+      throws YarnException {
+
+    try {
+      // Split the heart beat request into multiple requests, one for each
+      // sub-cluster RM that is used by this application.
+      Map<SubClusterId, AllocateRequest> requests =
+          splitAllocateRequest(request);
+
+      // Send the request to the home RM and get the response
+      AllocateResponse homeResponse = AMRMClientUtils.allocateWithReRegister(
+          requests.get(this.homeSubClusterId), this.homeRM,
+          this.amRegistrationRequest,
+          getApplicationContext().getApplicationAttemptId());
+
+      // If the resource manager sent us a new token, add to the current user
+      if (homeResponse.getAMRMToken() != null) {
+        LOG.debug("Received new AMRMToken");
+        YarnServerSecurityUtils.updateAMRMToken(homeResponse.getAMRMToken(),
+            this.appOwner, getConf());
+      }
+
+      // Merge the responses from home and secondary sub-cluster RMs
+      homeResponse = mergeAllocateResponses(homeResponse);
+
+      // return the final response to the application master.
+      return homeResponse;
+    } catch (IOException ex) {
+      LOG.error("Exception encountered while processing heart beat", ex);
+      throw new YarnException(ex);
+    }
+  }
+
+  /**
+   * Sends the finish application master request to all the resource managers
+   * used by the application.
+   */
+  @Override
+  public FinishApplicationMasterResponse finishApplicationMaster(
+      FinishApplicationMasterRequest request)
+      throws YarnException, IOException {
+
+    FinishApplicationMasterResponse homeResponse =
+        AMRMClientUtils.finishAMWithReRegister(request, this.homeRM,
+            this.amRegistrationRequest,
+            getApplicationContext().getApplicationAttemptId());
+    return homeResponse;
+  }
+
+  @Override
+  public void setNextInterceptor(RequestInterceptor next) {
+    throw new YarnRuntimeException(
+        "setNextInterceptor is being called on FederationInterceptor. "
+            + "It should always be used as the last interceptor in the chain");
+  }
+
+  /**
+   * This is called when the application pipeline is being destroyed. We will
+   * release all the resources that we are holding in this call.
+   */
+  @Override
+  public void shutdown() {
+    super.shutdown();
+  }
+
+  /**
+   * Returns instance of the ApplicationMasterProtocol proxy class that is used
+   * to connect to the Home resource manager.
+   *
+   * @param appContext AMRMProxyApplicationContext
+   * @return the proxy created
+   */
+  protected ApplicationMasterProtocol createHomeRMProxy(
+      AMRMProxyApplicationContext appContext) {
+    try {
+      return FederationProxyProviderUtil.createRMProxy(appContext.getConf(),
+          ApplicationMasterProtocol.class, this.homeSubClusterId, this.appOwner,
+          appContext.getAMRMToken());
+    } catch (Exception ex) {
+      throw new YarnRuntimeException(ex);
+    }
+  }
+
+  /**
+   * In federation, the heart beat request needs to be sent to all the sub
+   * clusters from which the AM has requested containers. This method splits the
+   * specified AllocateRequest from the AM and creates a new request for each
+   * sub-cluster RM.
+   */
+  private Map<SubClusterId, AllocateRequest> splitAllocateRequest(
+      AllocateRequest request) throws YarnException {
+    Map<SubClusterId, AllocateRequest> requestMap =
+        new HashMap<SubClusterId, AllocateRequest>();
+
+    // Create heart beat request for home sub-cluster resource manager
+    findOrCreateAllocateRequestForSubCluster(this.homeSubClusterId, request,
+        requestMap);
+
+    if (!isNullOrEmpty(request.getAskList())) {
+      AllocateRequest newRequest = findOrCreateAllocateRequestForSubCluster(
+          this.homeSubClusterId, request, requestMap);
+      newRequest.getAskList().addAll(request.getAskList());
+    }
+
+    if (request.getResourceBlacklistRequest() != null && !isNullOrEmpty(
+        request.getResourceBlacklistRequest().getBlacklistAdditions())) {
+      for (String resourceName : request.getResourceBlacklistRequest()
+          .getBlacklistAdditions()) {
+        AllocateRequest newRequest = findOrCreateAllocateRequestForSubCluster(
+            this.homeSubClusterId, request, requestMap);
+        newRequest.getResourceBlacklistRequest().getBlacklistAdditions()
+            .add(resourceName);
+      }
+    }
+
+    if (request.getResourceBlacklistRequest() != null && !isNullOrEmpty(
+        request.getResourceBlacklistRequest().getBlacklistRemovals())) {
+      for (String resourceName : request.getResourceBlacklistRequest()
+          .getBlacklistRemovals()) {
+        AllocateRequest newRequest = findOrCreateAllocateRequestForSubCluster(
+            this.homeSubClusterId, request, requestMap);
+        newRequest.getResourceBlacklistRequest().getBlacklistRemovals()
+            .add(resourceName);
+      }
+    }
+
+    if (!isNullOrEmpty(request.getReleaseList())) {
+      for (ContainerId cid : request.getReleaseList()) {
+        if (warnIfNotExists(cid, "release")) {
+          SubClusterId subClusterId =
+              this.containerIdToSubClusterIdMap.get(cid);
+          AllocateRequest newRequest = requestMap.get(subClusterId);
+          newRequest.getReleaseList().add(cid);
+        }
+      }
+    }
+
+    if (!isNullOrEmpty(request.getUpdateRequests())) {
+      for (UpdateContainerRequest ucr : request.getUpdateRequests()) {
+        if (warnIfNotExists(ucr.getContainerId(), "update")) {
+          SubClusterId subClusterId =
+              this.containerIdToSubClusterIdMap.get(ucr.getContainerId());
+          AllocateRequest newRequest = requestMap.get(subClusterId);
+          newRequest.getUpdateRequests().add(ucr);
+        }
+      }
+    }
+
+    return requestMap;
+  }
+
+  /**
+   * Merges the responses from other sub-clusters that we received
+   * asynchronously with the specified home cluster response and keeps track of
+   * the containers received from each sub-cluster resource managers.
+   */
+  private AllocateResponse mergeAllocateResponses(
+      AllocateResponse homeResponse) {
+    // Timing issue, we need to remove the completed and then save the new ones.
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Remove containers: "
+          + homeResponse.getCompletedContainersStatuses());
+      LOG.debug("Adding containers: " + homeResponse.getAllocatedContainers());
+    }
+    removeFinishedContainersFromCache(
+        homeResponse.getCompletedContainersStatuses());
+    cacheAllocatedContainers(homeResponse.getAllocatedContainers(),
+        this.homeSubClusterId);
+
+    return homeResponse;
+  }
+
+  /**
+   * Removes the finished containers from the local cache.
+   */
+  private void removeFinishedContainersFromCache(
+      List<ContainerStatus> finishedContainers) {
+    for (ContainerStatus container : finishedContainers) {
+      if (containerIdToSubClusterIdMap
+          .containsKey(container.getContainerId())) {
+        containerIdToSubClusterIdMap.remove(container.getContainerId());
+      }
+    }
+  }
+
+  /**
+   * Add allocated containers to cache mapping.
+   */
+  private void cacheAllocatedContainers(List<Container> containers,
+      SubClusterId subClusterId) {
+    for (Container container : containers) {
+      if (containerIdToSubClusterIdMap.containsKey(container.getId())) {
+        SubClusterId existingSubClusterId =
+            containerIdToSubClusterIdMap.get(container.getId());
+        if (existingSubClusterId.equals(subClusterId)) {
+          // When RM fails over, the new RM master might send out the same
+          // container allocation more than once. Just move on in this case.
+          LOG.warn(
+              "Duplicate containerID: {} found in the allocated containers"
+                  + " from same subcluster: {}, so ignoring.",
+              container.getId(), subClusterId);
+        } else {
+          // The same container allocation from different subclusters,
+          // something is wrong.
+          // TODO: YARN-6667 if some subcluster RM is configured wrong, we
+          // should not fail the entire heartbeat.
+          throw new YarnRuntimeException(
+              "Duplicate containerID found in the allocated containers. This"
+                  + " can happen if the RM epoch is not configured properly."
+                  + " ContainerId: " + container.getId().toString()
+                  + " ApplicationId: "
+                  + getApplicationContext().getApplicationAttemptId()
+                  + " From RM: " + subClusterId
+                  + " . Previous container was from subcluster: "
+                  + existingSubClusterId);
+        }
+      }
+
+      containerIdToSubClusterIdMap.put(container.getId(), subClusterId);
+    }
+  }
+
+  /**
+   * Check to see if an AllocateRequest exists in the Map for the specified sub
+   * cluster. If not found, create a new one, copy the value of responseId and
+   * progress from the orignialAMRequest, save it in the specified Map and
+   * return the new instance. If found, just return the old instance.
+   */
+  private static AllocateRequest findOrCreateAllocateRequestForSubCluster(
+      SubClusterId subClusterId, AllocateRequest originalAMRequest,
+      Map<SubClusterId, AllocateRequest> requestMap) {
+    AllocateRequest newRequest = null;
+    if (requestMap.containsKey(subClusterId)) {
+      newRequest = requestMap.get(subClusterId);
+    } else {
+      newRequest = createAllocateRequest();
+      newRequest.setResponseId(originalAMRequest.getResponseId());
+      newRequest.setProgress(originalAMRequest.getProgress());
+      requestMap.put(subClusterId, newRequest);
+    }
+
+    return newRequest;
+  }
+
+  /**
+   * Create an empty AllocateRequest instance.
+   */
+  private static AllocateRequest createAllocateRequest() {
+    AllocateRequest request =
+        AllocateRequest.newInstance(0, 0, null, null, null);
+    request.setAskList(new ArrayList<ResourceRequest>());
+    request.setReleaseList(new ArrayList<ContainerId>());
+    ResourceBlacklistRequest blackList =
+        ResourceBlacklistRequest.newInstance(null, null);
+    blackList.setBlacklistAdditions(new ArrayList<String>());
+    blackList.setBlacklistRemovals(new ArrayList<String>());
+    request.setResourceBlacklistRequest(blackList);
+    request.setUpdateRequests(new ArrayList<UpdateContainerRequest>());
+    return request;
+  }
+
+  /**
+   * Check to see if the specified containerId exists in the cache and log an
+   * error if not found.
+   *
+   * @param containerId the container id
+   * @param actionName the name of the action
+   * @return true if the container exists in the map, false otherwise
+   */
+  private boolean warnIfNotExists(ContainerId containerId, String actionName) {
+    if (!this.containerIdToSubClusterIdMap.containsKey(containerId)) {
+      LOG.error("AM is trying to {} a container {} that does not exist. ",
+          actionName, containerId.toString());
+      return false;
+    }
+    return true;
+  }
+
+  /**
+   * Utility method to check if the specified Collection is null or empty
+   *
+   * @param c the collection object
+   * @param <T> element type of the collection
+   * @return whether is it is null or empty
+   */
+  public static <T> boolean isNullOrEmpty(Collection<T> c) {
+    return (c == null || c.size() == 0);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/376dc4ae/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 e734bdd..72e5f53 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
@@ -42,6 +42,7 @@ 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.security.AMRMTokenIdentifier;
+import org.apache.hadoop.yarn.server.MockResourceManagerFacade;
 import org.apache.hadoop.yarn.server.nodemanager.amrmproxy.AMRMProxyService.RequestInterceptorChainWrapper;
 import org.apache.hadoop.yarn.util.Records;
 import org.junit.Assert;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/376dc4ae/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestFederationInterceptor.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/TestFederationInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestFederationInterceptor.java
new file mode 100644
index 0000000..3b564f0
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestFederationInterceptor.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
+ *
+ *     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 org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.InvalidApplicationMasterRequestException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.util.Records;
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Extends the TestAMRMProxyService and overrides methods in order to use the
+ * AMRMProxyService's pipeline test cases for testing the FederationInterceptor
+ * class. The tests for AMRMProxyService has been written cleverly so that it
+ * can be reused to validate different request intercepter chains.
+ */
+public class TestFederationInterceptor extends BaseAMRMProxyTest {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestFederationInterceptor.class);
+
+  public static final String HOME_SC_ID = "SC-home";
+
+  private TestableFederationInterceptor interceptor;
+
+  private int testAppId;
+  private ApplicationAttemptId attemptId;
+
+  @Override
+  public void setUp() throws IOException {
+    super.setUp();
+    interceptor = new TestableFederationInterceptor();
+
+    testAppId = 1;
+    attemptId = getApplicationAttemptId(testAppId);
+    interceptor.init(new AMRMProxyApplicationContextImpl(null, getConf(),
+        attemptId, "test-user", null, null));
+  }
+
+  @Override
+  public void tearDown() {
+    interceptor.shutdown();
+    super.tearDown();
+  }
+
+  @Override
+  protected YarnConfiguration createConfiguration() {
+    YarnConfiguration conf = new YarnConfiguration();
+    conf.setBoolean(YarnConfiguration.AMRM_PROXY_ENABLED, true);
+    conf.setBoolean(YarnConfiguration.FEDERATION_ENABLED, true);
+    String mockPassThroughInterceptorClass =
+        PassThroughRequestInterceptor.class.getName();
+
+    // Create a request intercepter pipeline for testing. The last one in the
+    // chain is the federation intercepter that calls the mock resource manager.
+    // The others in the chain will simply forward it to the next one in the
+    // chain
+    conf.set(YarnConfiguration.AMRM_PROXY_INTERCEPTOR_CLASS_PIPELINE,
+        mockPassThroughInterceptorClass + "," + mockPassThroughInterceptorClass
+            + "," + TestableFederationInterceptor.class.getName());
+
+    conf.set(YarnConfiguration.RM_CLUSTER_ID, HOME_SC_ID);
+
+    return conf;
+  }
+
+  @Test
+  public void testRequestInterceptorChainCreation() throws Exception {
+    RequestInterceptor root =
+        super.getAMRMProxyService().createRequestInterceptorChain();
+    int index = 0;
+    while (root != null) {
+      switch (index) {
+      case 0:
+      case 1:
+        Assert.assertEquals(PassThroughRequestInterceptor.class.getName(),
+            root.getClass().getName());
+        break;
+      case 2:
+        Assert.assertEquals(TestableFederationInterceptor.class.getName(),
+            root.getClass().getName());
+        break;
+      default:
+        Assert.fail();
+      }
+      root = root.getNextInterceptor();
+      index++;
+    }
+    Assert.assertEquals("The number of interceptors in chain does not match",
+        Integer.toString(3), Integer.toString(index));
+  }
+
+  /**
+   * Between AM and AMRMProxy, FederationInterceptor modifies the RM behavior,
+   * so that when AM registers more than once, it returns the same register
+   * success response instead of throwing
+   * {@link InvalidApplicationMasterRequestException}
+   *
+   * We did this because FederationInterceptor can receive concurrent register
+   * requests from AM because of timeout between AM and AMRMProxy. This can
+   * possible since the timeout between FederationInterceptor and RM longer
+   * because of performFailover + timeout.
+   */
+  @Test
+  public void testTwoIdenticalRegisterRequest() throws Exception {
+    // Register the application twice
+    RegisterApplicationMasterRequest registerReq =
+        Records.newRecord(RegisterApplicationMasterRequest.class);
+    registerReq.setHost(Integer.toString(testAppId));
+    registerReq.setRpcPort(testAppId);
+    registerReq.setTrackingUrl("");
+
+    for (int i = 0; i < 2; i++) {
+      RegisterApplicationMasterResponse registerResponse =
+          interceptor.registerApplicationMaster(registerReq);
+      Assert.assertNotNull(registerResponse);
+    }
+  }
+
+  @Test
+  public void testTwoDifferentRegisterRequest() throws Exception {
+    // Register the application first time
+    RegisterApplicationMasterRequest registerReq =
+        Records.newRecord(RegisterApplicationMasterRequest.class);
+    registerReq.setHost(Integer.toString(testAppId));
+    registerReq.setRpcPort(testAppId);
+    registerReq.setTrackingUrl("");
+
+    RegisterApplicationMasterResponse registerResponse =
+        interceptor.registerApplicationMaster(registerReq);
+    Assert.assertNotNull(registerResponse);
+
+    // Register the application second time with a different request obj
+    registerReq = Records.newRecord(RegisterApplicationMasterRequest.class);
+    registerReq.setHost(Integer.toString(testAppId));
+    registerReq.setRpcPort(testAppId);
+    registerReq.setTrackingUrl("different");
+    try {
+      registerResponse = interceptor.registerApplicationMaster(registerReq);
+      Assert.fail("Should throw if a different request obj is used");
+    } catch (YarnException e) {
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/376dc4ae/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestableFederationInterceptor.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/TestableFederationInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestableFederationInterceptor.java
new file mode 100644
index 0000000..0ca7488
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestableFederationInterceptor.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
+ *
+ *     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.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
+import org.apache.hadoop.yarn.server.MockResourceManagerFacade;
+import org.apache.hadoop.yarn.server.uam.UnmanagedAMPoolManager;
+import org.apache.hadoop.yarn.server.uam.UnmanagedApplicationManager;
+
+/**
+ * Extends the FederationInterceptor and overrides methods to provide a testable
+ * implementation of FederationInterceptor.
+ */
+public class TestableFederationInterceptor extends FederationInterceptor {
+  private ConcurrentHashMap<String, MockResourceManagerFacade>
+      secondaryResourceManagers = new ConcurrentHashMap<>();
+  private AtomicInteger runningIndex = new AtomicInteger(0);
+  private MockResourceManagerFacade mockRm;
+
+  @Override
+  protected ApplicationMasterProtocol createHomeRMProxy(
+      AMRMProxyApplicationContext appContext) {
+    synchronized (this) {
+      if (mockRm == null) {
+        mockRm = new MockResourceManagerFacade(
+            new YarnConfiguration(super.getConf()), 0);
+      }
+    }
+    return mockRm;
+  }
+
+  @SuppressWarnings("unchecked")
+  protected <T> T createSecondaryRMProxy(Class<T> proxyClass,
+      Configuration conf, String subClusterId) throws IOException {
+    // We create one instance of the mock resource manager per sub cluster. Keep
+    // track of the instances of the RMs in the map keyed by the sub cluster id
+    synchronized (this.secondaryResourceManagers) {
+      if (this.secondaryResourceManagers.contains(subClusterId)) {
+        return (T) this.secondaryResourceManagers.get(subClusterId);
+      } else {
+        // The running index here is used to simulate different RM_EPOCH to
+        // generate unique container identifiers in a federation environment
+        MockResourceManagerFacade rm = new MockResourceManagerFacade(
+            new Configuration(conf), runningIndex.addAndGet(10000));
+        this.secondaryResourceManagers.put(subClusterId, rm);
+        return (T) rm;
+      }
+    }
+  }
+
+  protected void setShouldReRegisterNext() {
+    if (mockRm != null) {
+      mockRm.setShouldReRegisterNext();
+    }
+    for (MockResourceManagerFacade subCluster : secondaryResourceManagers
+        .values()) {
+      subCluster.setShouldReRegisterNext();
+    }
+  }
+
+  /**
+   * Extends the UnmanagedAMPoolManager and overrides methods to provide a
+   * testable implementation of UnmanagedAMPoolManager.
+   */
+  protected class TestableUnmanagedAMPoolManager
+      extends UnmanagedAMPoolManager {
+    public TestableUnmanagedAMPoolManager(ExecutorService threadpool) {
+      super(threadpool);
+    }
+
+    @Override
+    public UnmanagedApplicationManager createUAM(Configuration conf,
+        ApplicationId appId, String queueName, String submitter,
+        String appNameSuffix) {
+      return new TestableUnmanagedApplicationManager(conf, appId, queueName,
+          submitter, appNameSuffix);
+    }
+  }
+
+  /**
+   * Extends the UnmanagedApplicationManager and overrides methods to provide a
+   * testable implementation.
+   */
+  protected class TestableUnmanagedApplicationManager
+      extends UnmanagedApplicationManager {
+
+    public TestableUnmanagedApplicationManager(Configuration conf,
+        ApplicationId appId, String queueName, String submitter,
+        String appNameSuffix) {
+      super(conf, appId, queueName, submitter, appNameSuffix);
+    }
+
+    /**
+     * We override this method here to return a mock RM instances. The base
+     * class returns the proxy to the real RM which will not work in case of
+     * stand alone test cases.
+     */
+    @Override
+    protected <T> T createRMProxy(Class<T> protocol, Configuration config,
+        UserGroupInformation user, Token<AMRMTokenIdentifier> token)
+        throws IOException {
+      return createSecondaryRMProxy(protocol, config,
+          YarnConfiguration.getClusterId(config));
+    }
+  }
+}


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


[39/50] [abbrv] 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/13a87bf6/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/13a87bf6/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/13a87bf6/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/13a87bf6/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/13a87bf6/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/13a87bf6/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/13a87bf6/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/13a87bf6/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/13a87bf6/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/13a87bf6/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/13a87bf6/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/13a87bf6/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/13a87bf6/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/13a87bf6/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


[26/50] [abbrv] hadoop git commit: YARN-3659. Federation: routing client invocations transparently to multiple RMs. (Giovanni Matteo Fumarola via Subru).

Posted by su...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/9edf7208/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/TestFederationClientInterceptor.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/TestFederationClientInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/TestFederationClientInterceptor.java
new file mode 100644
index 0000000..87dfc95
--- /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/TestFederationClientInterceptor.java
@@ -0,0 +1,403 @@
+/**
+ * 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.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationResponse;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+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.manager.UniformBroadcastPolicyManager;
+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.utils.FederationStateStoreFacade;
+import org.apache.hadoop.yarn.server.federation.utils.FederationStateStoreTestUtil;
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Extends the {@code BaseRouterClientRMTest} and overrides methods in order to
+ * use the {@code RouterClientRMService} pipeline test cases for testing the
+ * {@code FederationInterceptor} class. The tests for
+ * {@code RouterClientRMService} has been written cleverly so that it can be
+ * reused to validate different request intercepter chains.
+ */
+public class TestFederationClientInterceptor extends BaseRouterClientRMTest {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestFederationClientInterceptor.class);
+
+  private TestableFederationClientInterceptor interceptor;
+  private MemoryFederationStateStore stateStore;
+  private FederationStateStoreTestUtil stateStoreUtil;
+  private List<SubClusterId> subClusters;
+
+  private String user = "test-user";
+
+  private final static int NUM_SUBCLUSTER = 4;
+
+  @Override
+  public void setUp() {
+    super.setUpConfig();
+    interceptor = new TestableFederationClientInterceptor();
+
+    stateStore = new MemoryFederationStateStore();
+    stateStore.init(this.getConf());
+    FederationStateStoreFacade.getInstance().reinitialize(stateStore,
+        getConf());
+    stateStoreUtil = new FederationStateStoreTestUtil(stateStore);
+
+    interceptor.setConf(this.getConf());
+    interceptor.init(user);
+
+    subClusters = new ArrayList<SubClusterId>();
+
+    try {
+      for (int i = 0; i < NUM_SUBCLUSTER; i++) {
+        SubClusterId sc = SubClusterId.newInstance(Integer.toString(i));
+        stateStoreUtil.registerSubCluster(sc);
+        subClusters.add(sc);
+      }
+    } catch (YarnException e) {
+      LOG.error(e.getMessage());
+      Assert.fail();
+    }
+
+  }
+
+  @Override
+  public void tearDown() {
+    interceptor.shutdown();
+    super.tearDown();
+  }
+
+  @Override
+  protected YarnConfiguration createConfiguration() {
+    YarnConfiguration conf = new YarnConfiguration();
+    conf.setBoolean(YarnConfiguration.FEDERATION_ENABLED, true);
+    String mockPassThroughInterceptorClass =
+        PassThroughClientRequestInterceptor.class.getName();
+
+    // Create a request intercepter pipeline for testing. The last one in the
+    // chain is the federation intercepter that calls the mock resource manager.
+    // The others in the chain will simply forward it to the next one in the
+    // chain
+    conf.set(YarnConfiguration.ROUTER_CLIENTRM_INTERCEPTOR_CLASS_PIPELINE,
+        mockPassThroughInterceptorClass + "," + mockPassThroughInterceptorClass
+            + "," + TestableFederationClientInterceptor.class.getName());
+
+    conf.set(YarnConfiguration.FEDERATION_POLICY_MANAGER,
+        UniformBroadcastPolicyManager.class.getName());
+
+    // Disable StateStoreFacade cache
+    conf.setInt(YarnConfiguration.FEDERATION_CACHE_TIME_TO_LIVE_SECS, 0);
+
+    return conf;
+  }
+
+  /**
+   * This test validates the correctness of GetNewApplication. The return
+   * ApplicationId has to belong to one of the SubCluster in the cluster.
+   */
+  @Test
+  public void testGetNewApplication()
+      throws YarnException, IOException, InterruptedException {
+    System.out.println("Test FederationClientInterceptor: Get New Application");
+
+    GetNewApplicationRequest request = GetNewApplicationRequest.newInstance();
+    GetNewApplicationResponse response = interceptor.getNewApplication(request);
+
+    Assert.assertNotNull(response);
+    Assert.assertNotNull(response.getApplicationId());
+    Assert.assertTrue(
+        response.getApplicationId().getClusterTimestamp() < NUM_SUBCLUSTER);
+    Assert.assertTrue(response.getApplicationId().getClusterTimestamp() >= 0);
+  }
+
+  /**
+   * This test validates the correctness of SubmitApplication. The application
+   * has to be submitted to one of the SubCluster in the cluster.
+   */
+  @Test
+  public void testSubmitApplication()
+      throws YarnException, IOException, InterruptedException {
+    System.out.println("Test FederationClientInterceptor: Submit Application");
+
+    ApplicationId appId =
+        ApplicationId.newInstance(System.currentTimeMillis(), 1);
+    ApplicationSubmissionContext context = ApplicationSubmissionContext
+        .newInstance(appId, "", "", null, null, false, false, -1, null, null);
+    SubmitApplicationRequest request =
+        SubmitApplicationRequest.newInstance(context);
+
+    SubmitApplicationResponse response = interceptor.submitApplication(request);
+
+    Assert.assertNotNull(response);
+    SubClusterId scIdResult = stateStoreUtil.queryApplicationHomeSC(appId);
+    Assert.assertNotNull(scIdResult);
+    Assert.assertTrue(subClusters.contains(scIdResult));
+  }
+
+  /**
+   * This test validates the correctness of SubmitApplication in case of
+   * multiple submission. The first retry has to be submitted to the same
+   * SubCluster of the first attempt.
+   */
+  @Test
+  public void testSubmitApplicationMultipleSubmission()
+      throws YarnException, IOException, InterruptedException {
+    System.out.println(
+        "Test FederationClientInterceptor: Submit Application - Multiple");
+
+    ApplicationId appId =
+        ApplicationId.newInstance(System.currentTimeMillis(), 1);
+    ApplicationSubmissionContext context = ApplicationSubmissionContext
+        .newInstance(appId, "", "", null, null, false, false, -1, null, null);
+    SubmitApplicationRequest request =
+        SubmitApplicationRequest.newInstance(context);
+
+    // First attempt
+    SubmitApplicationResponse response = interceptor.submitApplication(request);
+
+    Assert.assertNotNull(response);
+    SubClusterId scIdResult = stateStoreUtil.queryApplicationHomeSC(appId);
+    Assert.assertNotNull(scIdResult);
+
+    // First retry
+    response = interceptor.submitApplication(request);
+
+    Assert.assertNotNull(response);
+    SubClusterId scIdResult2 = stateStoreUtil.queryApplicationHomeSC(appId);
+    Assert.assertNotNull(scIdResult2);
+    Assert.assertEquals(scIdResult, scIdResult);
+  }
+
+  /**
+   * This test validates the correctness of SubmitApplication in case of empty
+   * request.
+   */
+  @Test
+  public void testSubmitApplicationEmptyRequest()
+      throws YarnException, IOException, InterruptedException {
+    System.out.println(
+        "Test FederationClientInterceptor: Submit Application - Empty");
+    try {
+      interceptor.submitApplication(null);
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertTrue(
+          e.getMessage().startsWith("Missing submitApplication request or "
+              + "applicationSubmissionContex information."));
+    }
+    try {
+      interceptor.submitApplication(SubmitApplicationRequest.newInstance(null));
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertTrue(
+          e.getMessage().startsWith("Missing submitApplication request or "
+              + "applicationSubmissionContex information."));
+    }
+    try {
+      ApplicationSubmissionContext context = ApplicationSubmissionContext
+          .newInstance(null, "", "", null, null, false, false, -1, null, null);
+      SubmitApplicationRequest request =
+          SubmitApplicationRequest.newInstance(context);
+      interceptor.submitApplication(request);
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertTrue(
+          e.getMessage().startsWith("Missing submitApplication request or "
+              + "applicationSubmissionContex information."));
+    }
+  }
+
+  /**
+   * This test validates the correctness of ForceKillApplication in case the
+   * application exists in the cluster.
+   */
+  @Test
+  public void testForceKillApplication()
+      throws YarnException, IOException, InterruptedException {
+    System.out
+        .println("Test FederationClientInterceptor: Force Kill Application");
+
+    ApplicationId appId =
+        ApplicationId.newInstance(System.currentTimeMillis(), 1);
+    ApplicationSubmissionContext context = ApplicationSubmissionContext
+        .newInstance(appId, "", "", null, null, false, false, -1, null, null);
+
+    SubmitApplicationRequest request =
+        SubmitApplicationRequest.newInstance(context);
+    // Submit the application we are going to kill later
+    SubmitApplicationResponse response = interceptor.submitApplication(request);
+
+    Assert.assertNotNull(response);
+    Assert.assertNotNull(stateStoreUtil.queryApplicationHomeSC(appId));
+
+    KillApplicationRequest requestKill =
+        KillApplicationRequest.newInstance(appId);
+    KillApplicationResponse responseKill =
+        interceptor.forceKillApplication(requestKill);
+    Assert.assertNotNull(responseKill);
+  }
+
+  /**
+   * This test validates the correctness of ForceKillApplication in case of
+   * application does not exist in StateStore.
+   */
+  @Test
+  public void testForceKillApplicationNotExists()
+      throws YarnException, IOException, InterruptedException {
+    System.out.println("Test FederationClientInterceptor: "
+        + "Force Kill Application - Not Exists");
+
+    ApplicationId appId =
+        ApplicationId.newInstance(System.currentTimeMillis(), 1);
+    KillApplicationRequest requestKill =
+        KillApplicationRequest.newInstance(appId);
+    try {
+      interceptor.forceKillApplication(requestKill);
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertTrue(e.getMessage().equals(
+          "Application " + appId + " does not exist in FederationStateStore"));
+    }
+  }
+
+  /**
+   * This test validates the correctness of ForceKillApplication in case of
+   * empty request.
+   */
+  @Test
+  public void testForceKillApplicationEmptyRequest()
+      throws YarnException, IOException, InterruptedException {
+    System.out.println(
+        "Test FederationClientInterceptor: Force Kill Application - Empty");
+    try {
+      interceptor.forceKillApplication(null);
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertTrue(e.getMessage().startsWith(
+          "Missing forceKillApplication request or ApplicationId."));
+    }
+    try {
+      interceptor
+          .forceKillApplication(KillApplicationRequest.newInstance(null));
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertTrue(e.getMessage().startsWith(
+          "Missing forceKillApplication request or ApplicationId."));
+    }
+  }
+
+  /**
+   * This test validates the correctness of GetApplicationReport in case the
+   * application exists in the cluster.
+   */
+  @Test
+  public void testGetApplicationReport()
+      throws YarnException, IOException, InterruptedException {
+    System.out
+        .println("Test FederationClientInterceptor: Get Application Report");
+
+    ApplicationId appId =
+        ApplicationId.newInstance(System.currentTimeMillis(), 1);
+    ApplicationSubmissionContext context = ApplicationSubmissionContext
+        .newInstance(appId, "", "", null, null, false, false, -1, null, null);
+
+    SubmitApplicationRequest request =
+        SubmitApplicationRequest.newInstance(context);
+    // Submit the application we want the report later
+    SubmitApplicationResponse response = interceptor.submitApplication(request);
+
+    Assert.assertNotNull(response);
+    Assert.assertNotNull(stateStoreUtil.queryApplicationHomeSC(appId));
+
+    GetApplicationReportRequest requestGet =
+        GetApplicationReportRequest.newInstance(appId);
+
+    GetApplicationReportResponse responseGet =
+        interceptor.getApplicationReport(requestGet);
+
+    Assert.assertNotNull(responseGet);
+  }
+
+  /**
+   * This test validates the correctness of GetApplicationReport in case the
+   * application does not exist in StateStore.
+   */
+  @Test
+  public void testGetApplicationNotExists()
+      throws YarnException, IOException, InterruptedException {
+    System.out.println(
+        "Test ApplicationClientProtocol: Get Application Report - Not Exists");
+    ApplicationId appId =
+        ApplicationId.newInstance(System.currentTimeMillis(), 1);
+    GetApplicationReportRequest requestGet =
+        GetApplicationReportRequest.newInstance(appId);
+    try {
+      interceptor.getApplicationReport(requestGet);
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertTrue(e.getMessage().equals(
+          "Application " + appId + " does not exist in FederationStateStore"));
+    }
+  }
+
+  /**
+   * This test validates the correctness of GetApplicationReport in case of
+   * empty request.
+   */
+  @Test
+  public void testGetApplicationEmptyRequest()
+      throws YarnException, IOException, InterruptedException {
+    System.out.println(
+        "Test FederationClientInterceptor: Get Application Report - Empty");
+    try {
+      interceptor.getApplicationReport(null);
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertTrue(
+          e.getMessage().startsWith("Missing getApplicationReport request or "
+              + "applicationId information."));
+    }
+    try {
+      interceptor
+          .getApplicationReport(GetApplicationReportRequest.newInstance(null));
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertTrue(
+          e.getMessage().startsWith("Missing getApplicationReport request or "
+              + "applicationId information."));
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9edf7208/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/TestFederationClientInterceptorRetry.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/TestFederationClientInterceptorRetry.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/TestFederationClientInterceptorRetry.java
new file mode 100644
index 0000000..a655c16
--- /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/TestFederationClientInterceptorRetry.java
@@ -0,0 +1,295 @@
+/**
+ * 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.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+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.FederationPolicyUtils;
+import org.apache.hadoop.yarn.server.federation.policies.manager.UniformBroadcastPolicyManager;
+import org.apache.hadoop.yarn.server.federation.store.impl.MemoryFederationStateStore;
+import org.apache.hadoop.yarn.server.federation.store.records.GetApplicationHomeSubClusterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+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.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Extends the {@code BaseRouterClientRMTest} and overrides methods in order to
+ * use the {@code RouterClientRMService} pipeline test cases for testing the
+ * {@code FederationInterceptor} class. The tests for
+ * {@code RouterClientRMService} has been written cleverly so that it can be
+ * reused to validate different request intercepter chains.
+ *
+ * It tests the case with SubClusters down and the Router logic of retries. We
+ * have 1 good SubCluster and 2 bad ones for all the tests.
+ */
+public class TestFederationClientInterceptorRetry
+    extends BaseRouterClientRMTest {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestFederationClientInterceptorRetry.class);
+
+  private TestableFederationClientInterceptor interceptor;
+  private MemoryFederationStateStore stateStore;
+  private FederationStateStoreTestUtil stateStoreUtil;
+
+  private String user = "test-user";
+
+  // running and registered
+  private static SubClusterId good;
+
+  // registered but not running
+  private static SubClusterId bad1;
+  private static SubClusterId bad2;
+
+  private static List<SubClusterId> scs = new ArrayList<SubClusterId>();
+
+  @Override
+  public void setUp() {
+    super.setUpConfig();
+    interceptor = new TestableFederationClientInterceptor();
+
+    stateStore = new MemoryFederationStateStore();
+    stateStore.init(this.getConf());
+    FederationStateStoreFacade.getInstance().reinitialize(stateStore,
+        getConf());
+    stateStoreUtil = new FederationStateStoreTestUtil(stateStore);
+
+    interceptor.setConf(this.getConf());
+    interceptor.init(user);
+
+    // Create SubClusters
+    good = SubClusterId.newInstance("0");
+    bad1 = SubClusterId.newInstance("1");
+    bad2 = SubClusterId.newInstance("2");
+    scs.add(good);
+    scs.add(bad1);
+    scs.add(bad2);
+
+    // The mock RM will not start in these SubClusters, this is done to simulate
+    // a SubCluster down
+
+    interceptor.registerBadSubCluster(bad1);
+    interceptor.registerBadSubCluster(bad2);
+  }
+
+  @Override
+  public void tearDown() {
+    interceptor.shutdown();
+    super.tearDown();
+  }
+
+  private void setupCluster(List<SubClusterId> scsToRegister)
+      throws YarnException {
+
+    try {
+      // Clean up the StateStore before every test
+      stateStoreUtil.deregisterAllSubClusters();
+
+      for (SubClusterId sc : scsToRegister) {
+        stateStoreUtil.registerSubCluster(sc);
+      }
+    } catch (YarnException e) {
+      LOG.error(e.getMessage());
+      Assert.fail();
+    }
+  }
+
+  @Override
+  protected YarnConfiguration createConfiguration() {
+    YarnConfiguration conf = new YarnConfiguration();
+    conf.setBoolean(YarnConfiguration.FEDERATION_ENABLED, true);
+    String mockPassThroughInterceptorClass =
+        PassThroughClientRequestInterceptor.class.getName();
+
+    // Create a request intercepter pipeline for testing. The last one in the
+    // chain is the federation intercepter that calls the mock resource manager.
+    // The others in the chain will simply forward it to the next one in the
+    // chain
+    conf.set(YarnConfiguration.ROUTER_CLIENTRM_INTERCEPTOR_CLASS_PIPELINE,
+        mockPassThroughInterceptorClass + "," + mockPassThroughInterceptorClass
+            + "," + TestableFederationClientInterceptor.class.getName());
+
+    conf.set(YarnConfiguration.FEDERATION_POLICY_MANAGER,
+        UniformBroadcastPolicyManager.class.getName());
+
+    // Disable StateStoreFacade cache
+    conf.setInt(YarnConfiguration.FEDERATION_CACHE_TIME_TO_LIVE_SECS, 0);
+
+    return conf;
+  }
+
+  /**
+   * This test validates the correctness of GetNewApplication in case the
+   * cluster is composed of only 1 bad SubCluster.
+   */
+  @Test
+  public void testGetNewApplicationOneBadSC()
+      throws YarnException, IOException, InterruptedException {
+
+    System.out.println("Test getNewApplication with one bad SubCluster");
+    setupCluster(Arrays.asList(bad2));
+
+    try {
+      interceptor.getNewApplication(GetNewApplicationRequest.newInstance());
+      Assert.fail();
+    } catch (Exception e) {
+      System.out.println(e.toString());
+      Assert.assertTrue(e.getMessage()
+          .equals(FederationPolicyUtils.NO_ACTIVE_SUBCLUSTER_AVAILABLE));
+    }
+  }
+
+  /**
+   * This test validates the correctness of GetNewApplication in case the
+   * cluster is composed of only 2 bad SubClusters.
+   */
+  @Test
+  public void testGetNewApplicationTwoBadSCs()
+      throws YarnException, IOException, InterruptedException {
+    System.out.println("Test getNewApplication with two bad SubClusters");
+    setupCluster(Arrays.asList(bad1, bad2));
+
+    try {
+      interceptor.getNewApplication(GetNewApplicationRequest.newInstance());
+      Assert.fail();
+    } catch (Exception e) {
+      System.out.println(e.toString());
+      Assert.assertTrue(e.getMessage()
+          .equals(FederationPolicyUtils.NO_ACTIVE_SUBCLUSTER_AVAILABLE));
+    }
+  }
+
+  /**
+   * This test validates the correctness of GetNewApplication in case the
+   * cluster is composed of only 1 bad SubCluster and 1 good one.
+   */
+  @Test
+  public void testGetNewApplicationOneBadOneGood()
+      throws YarnException, IOException, InterruptedException {
+    System.out.println("Test getNewApplication with one bad, one good SC");
+    setupCluster(Arrays.asList(good, bad2));
+    GetNewApplicationResponse response = null;
+    try {
+      response =
+          interceptor.getNewApplication(GetNewApplicationRequest.newInstance());
+    } catch (Exception e) {
+      Assert.fail();
+    }
+    Assert.assertEquals(Integer.parseInt(good.getId()),
+        response.getApplicationId().getClusterTimestamp());
+  }
+
+  /**
+   * This test validates the correctness of SubmitApplication in case the
+   * cluster is composed of only 1 bad SubCluster.
+   */
+  @Test
+  public void testSubmitApplicationOneBadSC()
+      throws YarnException, IOException, InterruptedException {
+
+    System.out.println("Test submitApplication with one bad SubCluster");
+    setupCluster(Arrays.asList(bad2));
+
+    final ApplicationId appId =
+        ApplicationId.newInstance(System.currentTimeMillis(), 1);
+
+    ApplicationSubmissionContext context = ApplicationSubmissionContext
+        .newInstance(appId, "", "", null, null, false, false, -1, null, null);
+    final SubmitApplicationRequest request =
+        SubmitApplicationRequest.newInstance(context);
+    try {
+      interceptor.submitApplication(request);
+      Assert.fail();
+    } catch (Exception e) {
+      System.out.println(e.toString());
+      Assert.assertTrue(e.getMessage()
+          .equals(FederationPolicyUtils.NO_ACTIVE_SUBCLUSTER_AVAILABLE));
+    }
+  }
+
+  /**
+   * This test validates the correctness of SubmitApplication in case the
+   * cluster is composed of only 2 bad SubClusters.
+   */
+  @Test
+  public void testSubmitApplicationTwoBadSCs()
+      throws YarnException, IOException, InterruptedException {
+    System.out.println("Test submitApplication with two bad SubClusters");
+    setupCluster(Arrays.asList(bad1, bad2));
+
+    final ApplicationId appId =
+        ApplicationId.newInstance(System.currentTimeMillis(), 1);
+
+    ApplicationSubmissionContext context = ApplicationSubmissionContext
+        .newInstance(appId, "", "", null, null, false, false, -1, null, null);
+    final SubmitApplicationRequest request =
+        SubmitApplicationRequest.newInstance(context);
+    try {
+      interceptor.submitApplication(request);
+      Assert.fail();
+    } catch (Exception e) {
+      System.out.println(e.toString());
+      Assert.assertTrue(e.getMessage()
+          .equals(FederationPolicyUtils.NO_ACTIVE_SUBCLUSTER_AVAILABLE));
+    }
+  }
+
+  /**
+   * This test validates the correctness of SubmitApplication in case the
+   * cluster is composed of only 1 bad SubCluster and a good one.
+   */
+  @Test
+  public void testSubmitApplicationOneBadOneGood()
+      throws YarnException, IOException, InterruptedException {
+    System.out.println("Test submitApplication with one bad, one good SC");
+    setupCluster(Arrays.asList(good, bad2));
+
+    final ApplicationId appId =
+        ApplicationId.newInstance(System.currentTimeMillis(), 1);
+
+    ApplicationSubmissionContext context = ApplicationSubmissionContext
+        .newInstance(appId, "", "", null, null, false, false, -1, null, null);
+    final SubmitApplicationRequest request =
+        SubmitApplicationRequest.newInstance(context);
+    try {
+      interceptor.submitApplication(request);
+    } catch (Exception e) {
+      Assert.fail();
+    }
+    Assert.assertEquals(good,
+        stateStore
+            .getApplicationHomeSubCluster(
+                GetApplicationHomeSubClusterRequest.newInstance(appId))
+            .getApplicationHomeSubCluster().getHomeSubCluster());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9edf7208/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/TestableFederationClientInterceptor.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/TestableFederationClientInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/TestableFederationClientInterceptor.java
new file mode 100644
index 0000000..e4a1a42
--- /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/TestableFederationClientInterceptor.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.router.clientrm;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.MockResourceManagerFacade;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+
+/**
+ * Extends the FederationClientInterceptor and overrides methods to provide a
+ * testable implementation of FederationClientInterceptor.
+ */
+public class TestableFederationClientInterceptor
+    extends FederationClientInterceptor {
+
+  private ConcurrentHashMap<SubClusterId, MockResourceManagerFacade> mockRMs =
+      new ConcurrentHashMap<>();
+
+  private List<SubClusterId> badSubCluster = new ArrayList<SubClusterId>();
+
+  @Override
+  protected ApplicationClientProtocol getClientRMProxyForSubCluster(
+      SubClusterId subClusterId) throws YarnException {
+
+    MockResourceManagerFacade mockRM = null;
+    synchronized (this) {
+      if (mockRMs.containsKey(subClusterId)) {
+        mockRM = mockRMs.get(subClusterId);
+      } else {
+        mockRM = new MockResourceManagerFacade(super.getConf(), 0,
+            Integer.parseInt(subClusterId.getId()),
+            !badSubCluster.contains(subClusterId));
+        mockRMs.put(subClusterId, mockRM);
+
+      }
+      return mockRM;
+    }
+  }
+
+  /**
+   * For testing purpose, some subclusters has to be down to simulate particular
+   * scenarios as RM Failover, network issues. For this reason we keep track of
+   * these bad subclusters. This method make the subcluster unusable.
+   *
+   * @param badSC the subcluster to make unusable
+   */
+  protected void registerBadSubCluster(SubClusterId badSC) {
+    badSubCluster.add(badSC);
+    if (mockRMs.contains(badSC)) {
+      mockRMs.get(badSC).setRunningMode(false);
+    }
+  }
+
+}


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


[48/50] [abbrv] hadoop git commit: Bumping up yarn-server-router (new) module pom to beta1 and fixing imports post rebase.

Posted by su...@apache.org.
Bumping up yarn-server-router (new) module pom to beta1 and fixing imports post rebase.


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

Branch: refs/heads/YARN-2915
Commit: 1c49b10dee483a8b824ec54014c734d207436d0b
Parents: 3ec835b
Author: Subru Krishnan <su...@apache.org>
Authored: Fri Jul 14 12:02:38 2017 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Tue Jul 25 16:56:32 2017 -0700

----------------------------------------------------------------------
 .../yarn/server/resourcemanager/ApplicationMasterService.java      | 1 +
 .../hadoop/yarn/server/resourcemanager/recovery/RMStateStore.java  | 1 +
 .../hadoop-yarn-server/hadoop-yarn-server-router/pom.xml           | 2 +-
 3 files changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c49b10d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.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/ApplicationMasterService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java
index 7638207..f77d09e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java
@@ -70,6 +70,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.security
     .AMRMTokenSecretManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.authorize.RMPolicyProvider;
 import org.apache.hadoop.yarn.server.security.MasterKeyData;
+import org.apache.hadoop.yarn.server.utils.AMRMClientUtils;
 import org.apache.hadoop.yarn.server.utils.YarnServerSecurityUtils;
 import org.apache.hadoop.yarn.util.resource.Resources;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c49b10d/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 465ff64..35340e6 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
@@ -48,6 +48,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationSubmissionContextPBImpl;
 import org.apache.hadoop.yarn.conf.HAUtil;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.event.EventHandler;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c49b10d/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 f27b2b2..78e5e59 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
@@ -24,7 +24,7 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>org.apache.hadoop</groupId>
   <artifactId>hadoop-yarn-server-router</artifactId>
-  <version>3.0.0-alpha4-SNAPSHOT</version>
+  <version>3.0.0-beta1-SNAPSHOT</version>
   <name>Apache Hadoop YARN Router</name>
 
   <properties>


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


[37/50] [abbrv] hadoop git commit: Bumping up hadoop-yarn-server-router module to 3.0.0-alpha4 post rebase.

Posted by su...@apache.org.
Bumping up hadoop-yarn-server-router module to 3.0.0-alpha4 post rebase.


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

Branch: refs/heads/YARN-2915
Commit: 81b980b11eb1656cc01bd74799b65f0d311c3f64
Parents: c73abec
Author: Subru Krishnan <su...@apache.org>
Authored: Fri May 26 17:10:03 2017 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Tue Jul 25 16:56:32 2017 -0700

----------------------------------------------------------------------
 .../hadoop-yarn-server/hadoop-yarn-server-router/pom.xml         | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/81b980b1/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 89813de..f9169e1 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
@@ -19,12 +19,12 @@
   <parent>
     <artifactId>hadoop-yarn-server</artifactId>
     <groupId>org.apache.hadoop</groupId>
-    <version>3.0.0-alpha3-SNAPSHOT</version>
+    <version>3.0.0-alpha4-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>
+  <version>3.0.0-alpha4-SNAPSHOT</version>
   <name>Apache Hadoop YARN Router</name>
 
   <properties>


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


[46/50] [abbrv] 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/469a5c93
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/469a5c93
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/469a5c93

Branch: refs/heads/YARN-2915
Commit: 469a5c932d9add09fdb9b42a626963918c73080d
Parents: c199075
Author: Subru Krishnan <su...@apache.org>
Authored: Wed May 3 18:26:15 2017 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Tue Jul 25 16:56:32 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/469a5c93/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 15bd1fa..7301e90 100755
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -324,6 +324,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/469a5c93/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 7bcb123..cf9c237 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
@@ -2637,6 +2637,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/469a5c93/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/469a5c93/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 6af7321..94dccd1 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
@@ -3167,6 +3167,24 @@
 
   <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>
+
+  <property>
+    <description>
       Comma-separated list of PlacementRules to determine how applications
       submitted by certain users get mapped to certain queues. Default is
       user-group, which corresponds to UserGroupMappingPlacementRule.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/469a5c93/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/469a5c93/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 c9f6d79..5f85097 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
@@ -209,6 +209,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/469a5c93/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/469a5c93/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 094519a..28ee0d9 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
@@ -172,6 +172,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/469a5c93/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/469a5c93/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/469a5c93/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/469a5c93/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/469a5c93/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/469a5c93/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


[36/50] [abbrv] 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/6f1a6f71
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/6f1a6f71
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/6f1a6f71

Branch: refs/heads/YARN-2915
Commit: 6f1a6f71a5aa2f72a5e28ffe36827e6764d26fac
Parents: 225acfb
Author: Subru Krishnan <su...@apache.org>
Authored: Wed Sep 7 17:33:34 2016 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Tue Jul 25 16:56:32 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/6f1a6f71/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/6f1a6f71/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/6f1a6f71/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/6f1a6f71/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/6f1a6f71/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/6f1a6f71/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/6f1a6f71/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/6f1a6f71/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/6f1a6f71/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/6f1a6f71/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/6f1a6f71/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/6f1a6f71/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/6f1a6f71/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/6f1a6f71/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/6f1a6f71/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/6f1a6f71/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/6f1a6f71/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


[07/50] [abbrv] hadoop git commit: YARN-6724. Add ability to blacklist sub-clusters when invoking Routing policies. (Giovanni Matteo Fumarola via Subru).

Posted by su...@apache.org.
YARN-6724. Add ability to blacklist sub-clusters when invoking Routing policies. (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/a4f9fdfc
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/a4f9fdfc
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/a4f9fdfc

Branch: refs/heads/YARN-2915
Commit: a4f9fdfc9dc16edb2c1306af72d37174811f405e
Parents: 474c0a6
Author: Subru Krishnan <su...@apache.org>
Authored: Wed Jun 21 19:08:47 2017 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Tue Jul 25 16:56:32 2017 -0700

----------------------------------------------------------------------
 .../federation/policies/RouterPolicyFacade.java | 15 +++++--
 .../policies/router/FederationRouterPolicy.java | 18 +++++---
 .../policies/router/HashBasedRouterPolicy.java  | 22 ++++++++--
 .../policies/router/LoadBasedRouterPolicy.java  |  7 +++-
 .../policies/router/PriorityRouterPolicy.java   |  7 +++-
 .../policies/router/RejectRouterPolicy.java     | 26 ++++++++----
 .../router/UniformRandomRouterPolicy.java       | 23 +++++++++--
 .../router/WeightedRandomRouterPolicy.java      | 11 ++++-
 .../policies/BaseFederationPoliciesTest.java    |  2 +-
 .../policies/TestRouterPolicyFacade.java        | 12 +++---
 .../policies/router/BaseRouterPoliciesTest.java | 43 +++++++++++++++++++-
 .../router/TestHashBasedRouterPolicy.java       |  2 +-
 .../router/TestLoadBasedRouterPolicy.java       |  2 +-
 .../router/TestPriorityRouterPolicy.java        |  2 +-
 .../policies/router/TestRejectRouterPolicy.java |  4 +-
 .../router/TestUniformRandomRouterPolicy.java   |  2 +-
 .../router/TestWeightedRandomRouterPolicy.java  |  2 +-
 17 files changed, 157 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4f9fdfc/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 5e31a08..44c1b10 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
@@ -19,6 +19,7 @@ package org.apache.hadoop.yarn.server.federation.policies;
 
 import java.nio.ByteBuffer;
 import java.nio.charset.StandardCharsets;
+import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 
@@ -110,16 +111,22 @@ public class RouterPolicyFacade {
    * 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.
+   * @param appSubmissionContext the {@link ApplicationSubmissionContext} that
+   *          has to be routed to an appropriate subCluster for execution.
    *
-   * @return the id of the subcluster that will be the "home" for this
+   * @param blackListSubClusters the list of subClusters as identified by
+   *          {@link SubClusterId} to blackList from the selection of the home
+   *          subCluster.
+   *
+   * @return the {@link SubClusterId} 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 {
+      ApplicationSubmissionContext appSubmissionContext,
+      List<SubClusterId> blackListSubClusters) throws YarnException {
 
     // the maps are concurrent, but we need to protect from reset()
     // reinitialization mid-execution by creating a new reference local to this
@@ -186,7 +193,7 @@ public class RouterPolicyFacade {
           + "and no default specified.");
     }
 
-    return policy.getHomeSubcluster(appSubmissionContext);
+    return policy.getHomeSubcluster(appSubmissionContext, blackListSubClusters);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4f9fdfc/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 90ea0a8..9325bd8 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
@@ -17,6 +17,8 @@
 
 package org.apache.hadoop.yarn.server.federation.policies.router;
 
+import java.util.List;
+
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.federation.policies.ConfigurableFederationPolicy;
@@ -29,16 +31,22 @@ import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
 public interface FederationRouterPolicy extends ConfigurableFederationPolicy {
 
   /**
-   * Determines the sub-cluster that the user application submision should be
+   * Determines the sub-cluster that the user application submission should be
    * routed to.
    *
-   * @param appSubmissionContext the context for the app being submitted.
+   * @param appSubmissionContext the {@link ApplicationSubmissionContext} that
+   *          has to be routed to an appropriate subCluster for execution.
+   *
+   * @param blackListSubClusters the list of subClusters as identified by
+   *          {@link SubClusterId} to blackList from the selection of the home
+   *          subCluster.
    *
-   * @return the sub-cluster as identified by {@link SubClusterId} to route the
-   *         request to.
+   * @return the {@link SubClusterId} that will be the "home" for this
+   *         application.
    *
    * @throws YarnException if the policy cannot determine a viable subcluster.
    */
   SubClusterId getHomeSubcluster(
-      ApplicationSubmissionContext appSubmissionContext) throws YarnException;
+      ApplicationSubmissionContext appSubmissionContext,
+      List<SubClusterId> blackListSubClusters) throws YarnException;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4f9fdfc/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
index e40e87e..257a9fe 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/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
@@ -55,19 +55,35 @@ public class HashBasedRouterPolicy extends AbstractRouterPolicy {
    * 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.
+   * @param appSubmissionContext the {@link ApplicationSubmissionContext} that
+   *          has to be routed to an appropriate subCluster for execution.
    *
-   * @return a hash-based chosen subcluster.
+   * @param blackListSubClusters the list of subClusters as identified by
+   *          {@link SubClusterId} to blackList from the selection of the home
+   *          subCluster.
+   *
+   * @return a hash-based chosen {@link SubClusterId} that will be the "home"
+   *         for this application.
    *
    * @throws YarnException if there are no active subclusters.
    */
+  @Override
   public SubClusterId getHomeSubcluster(
-      ApplicationSubmissionContext appSubmissionContext) throws YarnException {
+      ApplicationSubmissionContext appSubmissionContext,
+      List<SubClusterId> blackListSubClusters) throws YarnException {
 
     // throws if no active subclusters available
     Map<SubClusterId, SubClusterInfo> activeSubclusters =
         getActiveSubclusters();
 
+    if (blackListSubClusters != null) {
+
+      // Remove from the active SubClusters from StateStore the blacklisted ones
+      for (SubClusterId scId : blackListSubClusters) {
+        activeSubclusters.remove(scId);
+      }
+    }
+
     validate(appSubmissionContext);
 
     int chosenPosition = Math.abs(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4f9fdfc/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 2ca15bf..c124001 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,6 +17,7 @@
 
 package org.apache.hadoop.yarn.server.federation.policies.router;
 
+import java.util.List;
 import java.util.Map;
 
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
@@ -62,7 +63,8 @@ public class LoadBasedRouterPolicy extends AbstractRouterPolicy {
 
   @Override
   public SubClusterId getHomeSubcluster(
-      ApplicationSubmissionContext appSubmissionContext) throws YarnException {
+      ApplicationSubmissionContext appSubmissionContext,
+      List<SubClusterId> blacklist) throws YarnException {
 
     // null checks and default-queue behavior
     validate(appSubmissionContext);
@@ -76,6 +78,9 @@ public class LoadBasedRouterPolicy extends AbstractRouterPolicy {
     long currBestMem = -1;
     for (Map.Entry<SubClusterId, SubClusterInfo> entry : activeSubclusters
         .entrySet()) {
+      if (blacklist != null && blacklist.contains(entry.getKey())) {
+        continue;
+      }
       SubClusterIdInfo id = new SubClusterIdInfo(entry.getKey());
       if (weights.containsKey(id) && weights.get(id) > 0) {
         long availableMemory = getAvailableMemory(entry.getValue());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4f9fdfc/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 13d9140..59f8767 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,6 +17,7 @@
 
 package org.apache.hadoop.yarn.server.federation.policies.router;
 
+import java.util.List;
 import java.util.Map;
 
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
@@ -34,7 +35,8 @@ public class PriorityRouterPolicy extends AbstractRouterPolicy {
 
   @Override
   public SubClusterId getHomeSubcluster(
-      ApplicationSubmissionContext appSubmissionContext) throws YarnException {
+      ApplicationSubmissionContext appSubmissionContext,
+      List<SubClusterId> blacklist) throws YarnException {
 
     // null checks and default-queue behavior
     validate(appSubmissionContext);
@@ -50,6 +52,9 @@ public class PriorityRouterPolicy extends AbstractRouterPolicy {
     Float currentBest = Float.MIN_VALUE;
     for (SubClusterId id : activeSubclusters.keySet()) {
       SubClusterIdInfo idInfo = new SubClusterIdInfo(id);
+      if (blacklist != null && blacklist.contains(id)) {
+        continue;
+      }
       if (weights.containsKey(idInfo) && weights.get(idInfo) > currentBest) {
         currentBest = weights.get(idInfo);
         chosen = id;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4f9fdfc/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
index faf3279..b4c0192 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/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
@@ -17,6 +17,8 @@
 
 package org.apache.hadoop.yarn.server.federation.policies.router;
 
+import java.util.List;
+
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.federation.policies.FederationPolicyInitializationContext;
@@ -27,8 +29,8 @@ 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.
+ * This is useful to prevent applications running in a queue to be run anywhere
+ * in the federated cluster.
  */
 public class RejectRouterPolicy extends AbstractRouterPolicy {
 
@@ -44,23 +46,31 @@ public class RejectRouterPolicy extends AbstractRouterPolicy {
   /**
    * The policy always reject requests.
    *
-   * @param appSubmissionContext the context for the app being submitted.
+   * @param appSubmissionContext the {@link ApplicationSubmissionContext} that
+   *          has to be routed to an appropriate subCluster for execution.
+   *
+   * @param blackListSubClusters the list of subClusters as identified by
+   *          {@link SubClusterId} to blackList from the selection of the home
+   *          subCluster.
    *
    * @return (never).
    *
-   * @throws YarnException (always) to prevent applications in this queue to
-   * be run anywhere in the federated cluster.
+   * @throws YarnException (always) to prevent applications in this queue to be
+   *           run anywhere in the federated cluster.
    */
+  @Override
   public SubClusterId getHomeSubcluster(
-      ApplicationSubmissionContext appSubmissionContext) throws YarnException {
+      ApplicationSubmissionContext appSubmissionContext,
+      List<SubClusterId> blackListSubClusters) 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.");
+        + "requests by construction. Application "
+        + appSubmissionContext.getApplicationId()
+        + " cannot be routed to any RM.");
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4f9fdfc/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 d820449..bc729b7 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
@@ -59,18 +59,24 @@ public class UniformRandomRouterPolicy extends AbstractRouterPolicy {
   }
 
   /**
-   * Simply picks a random active subcluster to start the AM (this does NOT
+   * 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).
+   * @param appSubmissionContext the {@link ApplicationSubmissionContext} that
+   *          has to be routed to an appropriate subCluster for execution.
+   *
+   * @param blackListSubClusters the list of subClusters as identified by
+   *          {@link SubClusterId} to blackList from the selection of the home
+   *          subCluster.
    *
    * @return a randomly chosen subcluster.
    *
    * @throws YarnException if there are no active subclusters.
    */
+  @Override
   public SubClusterId getHomeSubcluster(
-      ApplicationSubmissionContext appSubmissionContext) throws YarnException {
+      ApplicationSubmissionContext appSubmissionContext,
+      List<SubClusterId> blackListSubClusters) throws YarnException {
 
     // null checks and default-queue behavior
     validate(appSubmissionContext);
@@ -79,6 +85,15 @@ public class UniformRandomRouterPolicy extends AbstractRouterPolicy {
         getActiveSubclusters();
 
     List<SubClusterId> list = new ArrayList<>(activeSubclusters.keySet());
+
+    if (blackListSubClusters != null) {
+
+      // Remove from the active SubClusters from StateStore the blacklisted ones
+      for (SubClusterId scId : blackListSubClusters) {
+        list.remove(scId);
+      }
+    }
+
     return list.get(rand.nextInt(list.size()));
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4f9fdfc/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 5727134..7f230a7 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,6 +18,7 @@
 
 package org.apache.hadoop.yarn.server.federation.policies.router;
 
+import java.util.List;
 import java.util.Map;
 import java.util.Random;
 
@@ -41,7 +42,8 @@ public class WeightedRandomRouterPolicy extends AbstractRouterPolicy {
 
   @Override
   public SubClusterId getHomeSubcluster(
-      ApplicationSubmissionContext appSubmissionContext) throws YarnException {
+      ApplicationSubmissionContext appSubmissionContext,
+      List<SubClusterId> blacklist) throws YarnException {
 
     // null checks and default-queue behavior
     validate(appSubmissionContext);
@@ -58,6 +60,9 @@ public class WeightedRandomRouterPolicy extends AbstractRouterPolicy {
 
     float totActiveWeight = 0;
     for (Map.Entry<SubClusterIdInfo, Float> entry : weights.entrySet()) {
+      if (blacklist != null && blacklist.contains(entry.getKey().toId())) {
+        continue;
+      }
       if (entry.getKey() != null
           && activeSubclusters.containsKey(entry.getKey().toId())) {
         totActiveWeight += entry.getValue();
@@ -66,6 +71,9 @@ public class WeightedRandomRouterPolicy extends AbstractRouterPolicy {
     float lookupValue = rand.nextFloat() * totActiveWeight;
 
     for (SubClusterId id : activeSubclusters.keySet()) {
+      if (blacklist != null && blacklist.contains(id)) {
+        continue;
+      }
       SubClusterIdInfo idInfo = new SubClusterIdInfo(id);
       if (weights.containsKey(idInfo)) {
         lookupValue -= weights.get(idInfo);
@@ -77,4 +85,5 @@ public class WeightedRandomRouterPolicy extends AbstractRouterPolicy {
     // should never happen
     return null;
   }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4f9fdfc/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 6bd8bf0..23978ed 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
@@ -104,7 +104,7 @@ public abstract class BaseFederationPoliciesTest {
     ConfigurableFederationPolicy localPolicy = getPolicy();
     if (localPolicy instanceof FederationRouterPolicy) {
       ((FederationRouterPolicy) localPolicy)
-          .getHomeSubcluster(getApplicationSubmissionContext());
+          .getHomeSubcluster(getApplicationSubmissionContext(), null);
     } else {
       String[] hosts = new String[] {"host1", "host2"};
       List<ResourceRequest> resourceRequests = FederationPoliciesTestUtil

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4f9fdfc/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 5fa02d6..d0e2dec 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
@@ -95,7 +95,7 @@ public class TestRouterPolicyFacade {
 
     // first call runs using standard UniformRandomRouterPolicy
     SubClusterId chosen =
-        routerFacade.getHomeSubcluster(applicationSubmissionContext);
+        routerFacade.getHomeSubcluster(applicationSubmissionContext, null);
     Assert.assertTrue(subClusterIds.contains(chosen));
     Assert.assertTrue(routerFacade.globalPolicyMap
         .get(queue1) instanceof UniformRandomRouterPolicy);
@@ -107,7 +107,7 @@ public class TestRouterPolicyFacade {
         .newInstance(getPriorityPolicy(queue1)));
 
     // second call is routed by new policy PriorityRouterPolicy
-    chosen = routerFacade.getHomeSubcluster(applicationSubmissionContext);
+    chosen = routerFacade.getHomeSubcluster(applicationSubmissionContext, null);
     Assert.assertTrue(chosen.equals(subClusterIds.get(0)));
     Assert.assertTrue(routerFacade.globalPolicyMap
         .get(queue1) instanceof PriorityRouterPolicy);
@@ -126,7 +126,7 @@ public class TestRouterPolicyFacade {
 
     // when invoked it returns the expected SubClusterId.
     SubClusterId chosen =
-        routerFacade.getHomeSubcluster(applicationSubmissionContext);
+        routerFacade.getHomeSubcluster(applicationSubmissionContext, null);
     Assert.assertTrue(subClusterIds.contains(chosen));
 
     // now the caching of policies must have added an entry for this queue
@@ -160,19 +160,19 @@ public class TestRouterPolicyFacade {
     String uninitQueue = "non-initialized-queue";
     when(applicationSubmissionContext.getQueue()).thenReturn(uninitQueue);
     SubClusterId chosen =
-        routerFacade.getHomeSubcluster(applicationSubmissionContext);
+        routerFacade.getHomeSubcluster(applicationSubmissionContext, null);
     Assert.assertTrue(subClusterIds.contains(chosen));
     Assert.assertFalse(routerFacade.globalPolicyMap.containsKey(uninitQueue));
 
     // empty string
     when(applicationSubmissionContext.getQueue()).thenReturn("");
-    chosen = routerFacade.getHomeSubcluster(applicationSubmissionContext);
+    chosen = routerFacade.getHomeSubcluster(applicationSubmissionContext, null);
     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);
+    chosen = routerFacade.getHomeSubcluster(applicationSubmissionContext, null);
     Assert.assertTrue(subClusterIds.contains(chosen));
     Assert.assertFalse(routerFacade.globalPolicyMap.containsKey(uninitQueue));
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4f9fdfc/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
index 2e7a0af..c7a7767 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/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
@@ -18,11 +18,19 @@
 
 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.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.server.federation.store.records.SubClusterIdInfo;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
+import org.apache.hadoop.yarn.server.federation.utils.FederationPoliciesTestUtil;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.junit.Assert;
 import org.junit.Test;
@@ -40,12 +48,43 @@ public abstract class BaseRouterPoliciesTest
         ApplicationSubmissionContext.newInstance(null, null, null, null, null,
             false, false, 0, Resources.none(), null, false, null, null);
     SubClusterId chosen =
-        localPolicy.getHomeSubcluster(applicationSubmissionContext);
+        localPolicy.getHomeSubcluster(applicationSubmissionContext, null);
     Assert.assertNotNull(chosen);
   }
 
   @Test(expected = FederationPolicyException.class)
   public void testNullAppContext() throws YarnException {
-    ((FederationRouterPolicy) getPolicy()).getHomeSubcluster(null);
+    ((FederationRouterPolicy) getPolicy()).getHomeSubcluster(null, null);
+  }
+
+  @Test
+  public void testBlacklistSubcluster() throws YarnException {
+    FederationRouterPolicy localPolicy = (FederationRouterPolicy) getPolicy();
+    ApplicationSubmissionContext applicationSubmissionContext =
+        ApplicationSubmissionContext.newInstance(null, null, null, null, null,
+            false, false, 0, Resources.none(), null, false, null, null);
+    Map<SubClusterId, SubClusterInfo> activeSubClusters =
+        getActiveSubclusters();
+    if (activeSubClusters != null && activeSubClusters.size() > 1
+        && !(localPolicy instanceof RejectRouterPolicy)) {
+      // blacklist all the active subcluster but one.
+      Random random = new Random();
+      List<SubClusterId> blacklistSubclusters =
+          new ArrayList<SubClusterId>(activeSubClusters.keySet());
+      SubClusterId removed = blacklistSubclusters
+          .remove(random.nextInt(blacklistSubclusters.size()));
+      // bias LoadBasedRouterPolicy
+      getPolicyInfo().getRouterPolicyWeights()
+          .put(new SubClusterIdInfo(removed), 1.0f);
+      FederationPoliciesTestUtil.initializePolicyContext(getPolicy(),
+          getPolicyInfo(), getActiveSubclusters());
+
+      SubClusterId chosen = localPolicy.getHomeSubcluster(
+          applicationSubmissionContext, blacklistSubclusters);
+
+      // check that the selected sub-cluster is only one not blacklisted
+      Assert.assertNotNull(chosen);
+      Assert.assertEquals(removed, chosen);
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4f9fdfc/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
index af7fe43..ee3e09d 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/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
@@ -70,7 +70,7 @@ public class TestHashBasedRouterPolicy extends BaseRouterPoliciesTest {
     for (int i = 0; i < jobPerSub * numSubclusters; i++) {
       when(applicationSubmissionContext.getQueue()).thenReturn("queue" + i);
       chosen = ((FederationRouterPolicy) getPolicy())
-          .getHomeSubcluster(applicationSubmissionContext);
+          .getHomeSubcluster(applicationSubmissionContext, null);
       counter.get(chosen).addAndGet(1);
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4f9fdfc/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 b70b4aa..dc8f99b 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
@@ -97,7 +97,7 @@ public class TestLoadBasedRouterPolicy extends BaseRouterPoliciesTest {
   public void testLoadIsRespected() throws YarnException {
 
     SubClusterId chosen = ((FederationRouterPolicy) getPolicy())
-        .getHomeSubcluster(getApplicationSubmissionContext());
+        .getHomeSubcluster(getApplicationSubmissionContext(), null);
 
     // check the "planted" best cluster is chosen
     Assert.assertEquals("sc05", chosen.getId());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4f9fdfc/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 42d919d..3c036c1 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
@@ -78,7 +78,7 @@ public class TestPriorityRouterPolicy extends BaseRouterPoliciesTest {
   @Test
   public void testPickLowestWeight() throws YarnException {
     SubClusterId chosen = ((FederationRouterPolicy) getPolicy())
-        .getHomeSubcluster(getApplicationSubmissionContext());
+        .getHomeSubcluster(getApplicationSubmissionContext(), null);
     Assert.assertEquals("sc5", chosen.getId());
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4f9fdfc/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
index 049ebbf..1747f73 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/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
@@ -47,7 +47,7 @@ public class TestRejectRouterPolicy extends BaseRouterPoliciesTest {
   @Test(expected = FederationPolicyException.class)
   public void testNoClusterIsChosen() throws YarnException {
     ((FederationRouterPolicy) getPolicy())
-        .getHomeSubcluster(getApplicationSubmissionContext());
+        .getHomeSubcluster(getApplicationSubmissionContext(), null);
   }
 
   @Override
@@ -57,7 +57,7 @@ public class TestRejectRouterPolicy extends BaseRouterPoliciesTest {
     ApplicationSubmissionContext applicationSubmissionContext =
         ApplicationSubmissionContext.newInstance(null, null, null, null, null,
             false, false, 0, Resources.none(), null, false, null, null);
-    localPolicy.getHomeSubcluster(applicationSubmissionContext);
+    localPolicy.getHomeSubcluster(applicationSubmissionContext, null);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4f9fdfc/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 b45aa2a..05490ab 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
@@ -57,7 +57,7 @@ public class TestUniformRandomRouterPolicy extends BaseRouterPoliciesTest {
   @Test
   public void testOneSubclusterIsChosen() throws YarnException {
     SubClusterId chosen = ((FederationRouterPolicy) getPolicy())
-        .getHomeSubcluster(getApplicationSubmissionContext());
+        .getHomeSubcluster(getApplicationSubmissionContext(), null);
     Assert.assertTrue(getActiveSubclusters().keySet().contains(chosen));
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4f9fdfc/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 09173e6..c969a30 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
@@ -98,7 +98,7 @@ public class TestWeightedRandomRouterPolicy extends BaseRouterPoliciesTest {
 
     for (float i = 0; i < numberOfDraws; i++) {
       SubClusterId chosenId = ((FederationRouterPolicy) getPolicy())
-          .getHomeSubcluster(getApplicationSubmissionContext());
+          .getHomeSubcluster(getApplicationSubmissionContext(), null);
       counter.get(chosenId).incrementAndGet();
     }
 


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


[16/50] [abbrv] 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/225acfb8
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/225acfb8
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/225acfb8

Branch: refs/heads/YARN-2915
Commit: 225acfb86b3edca04b1e55d6a01be7e1d4ab717a
Parents: 8f2cae3
Author: Jian He <ji...@apache.org>
Authored: Fri Sep 2 12:23:57 2016 +0800
Committer: Subru Krishnan <su...@apache.org>
Committed: Tue Jul 25 16:56:32 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/225acfb8/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/225acfb8/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 46ed221..5dcb993 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/225acfb8/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/225acfb8/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 9e78cd7..9691885 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
@@ -772,6 +772,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/225acfb8/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/225acfb8/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/225acfb8/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/225acfb8/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 d0a8cf5..465ff64 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
@@ -98,6 +98,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;
@@ -690,6 +691,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/225acfb8/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 3b986d1..1b3b367 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
@@ -438,7 +438,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/225acfb8/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/225acfb8/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/225acfb8/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/225acfb8/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 942e9e8..5ae8239 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
@@ -189,6 +189,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


[14/50] [abbrv] 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/c62772fa
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/c62772fa
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/c62772fa

Branch: refs/heads/YARN-2915
Commit: c62772fab4ac18bc0e9db983efb92d17b7d9a23c
Parents: f419f02
Author: Jian He <ji...@apache.org>
Authored: Tue Aug 30 12:20:52 2016 +0800
Committer: Subru Krishnan <su...@apache.org>
Committed: Tue Jul 25 16:56:32 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/c62772fa/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 612d89b..46ed221 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
@@ -2568,9 +2568,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";
 
@@ -2583,6 +2580,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/c62772fa/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/c62772fa/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/c62772fa/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 c70362c..1915f67 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/c62772fa/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/c62772fa/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/c62772fa/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 b63b60d..9e78cd7 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
@@ -64,6 +64,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;
@@ -185,6 +186,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;
@@ -499,6 +501,10 @@ public class ResourceManager extends CompositeService implements Recoverable {
     return new RMTimelineCollectorManager(this);
   }
 
+  private FederationStateStoreService createFederationStateStoreService() {
+    return new FederationStateStoreService(rmContext);
+  }
+
   protected SystemMetricsPublisher createSystemMetricsPublisher() {
     SystemMetricsPublisher publisher;
     if (YarnConfiguration.timelineServiceEnabled(conf) &&
@@ -724,6 +730,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);
@@ -1350,6 +1370,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/c62772fa/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/c62772fa/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/c62772fa/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/c62772fa/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/c62772fa/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


[05/50] [abbrv] 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/1112595a
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/1112595a
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/1112595a

Branch: refs/heads/YARN-2915
Commit: 1112595afdc4d4d1afddc7ea5c70afc25746d3a0
Parents: dc374d3
Author: Jian He <ji...@apache.org>
Authored: Wed Aug 17 11:13:19 2016 +0800
Committer: Subru Krishnan <su...@apache.org>
Committed: Tue Jul 25 16:56:31 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/1112595a/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index b9819b4..93bbcf8 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>
 
@@ -1265,6 +1268,16 @@
           <artifactId>kerb-simplekdc</artifactId>
           <version>1.0.0</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/1112595a/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 3e778ee..fe6c7b8 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
@@ -2560,6 +2560,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/1112595a/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/1112595a/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 e20aad5..0b0a160 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
@@ -2686,8 +2686,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
@@ -2696,6 +2696,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/1112595a/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 89dec30..def5357 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
@@ -102,6 +102,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/1112595a/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/1112595a/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/1112595a/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/1112595a/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


[31/50] [abbrv] hadoop git commit: YARN-6807. Adding required missing configs to Federation configuration guide based on e2e testing. (Tanuj Nayak via Subru).

Posted by su...@apache.org.
YARN-6807. Adding required missing configs to Federation configuration guide based on e2e testing. (Tanuj Nayak via Subru).


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

Branch: refs/heads/YARN-2915
Commit: f20d855ad285d6c25ce182c0dc7da9597b0796e3
Parents: 9edf720
Author: Subru Krishnan <su...@apache.org>
Authored: Thu Jul 13 18:44:32 2017 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Tue Jul 25 16:56:32 2017 -0700

----------------------------------------------------------------------
 .../src/site/markdown/Federation.md             | 53 ++++++++++++++++++--
 1 file changed, 49 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f20d855a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/Federation.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/Federation.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/Federation.md
index c50ba76..79225b4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/Federation.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/Federation.md
@@ -86,6 +86,8 @@ of the desirable properties of balance, optimal cluster utilization and global i
 
 *NOTE*: In the current implementation the GPG is a manual tuning process, simply exposed via a CLI (YARN-3657).
 
+This part of the federation system is part of future work in [YARN-5597](https://issues.apache.org/jira/browse/YARN-5597).
+
 
 ###Federation State-Store
 The Federation State defines the additional state that needs to be maintained to loosely couple multiple individual sub-clusters into a single large federated cluster. This includes the following information:
@@ -159,7 +161,7 @@ These are common configurations that should appear in the **conf/yarn-site.xml**
 |:---- |:---- |
 |`yarn.federation.enabled` | `true` | Whether federation is enabled or not |
 |`yarn.federation.state-store.class` | `org.apache.hadoop.yarn.server.federation.store.impl.SQLFederationStateStore` | The type of state-store to use. |
-|`yarn.federation.state-store.sql.url` | `jdbc:sqlserver://<host>:<port>;database` | For SQLFederationStateStore the name of the DB where the state is stored. |
+|`yarn.federation.state-store.sql.url` | `jdbc:sqlserver://<host>:<port>;databaseName=FederationStateStore` | For SQLFederationStateStore the name of the DB where the state is stored. |
 |`yarn.federation.state-store.sql.jdbc-class` | `com.microsoft.sqlserver.jdbc.SQLServerDataSource` | For SQLFederationStateStore the jdbc class to use. |
 |`yarn.federation.state-store.sql.username` | `<dbuser>` | For SQLFederationStateStore the username for the DB connection. |
 |`yarn.federation.state-store.sql.password` | `<dbpass>` | For SQLFederationStateStore the password for the DB connection. |
@@ -175,7 +177,7 @@ Optional:
 |`yarn.federation.policy-manager` | `org.apache.hadoop.yarn.server.federation.policies.manager.WeightedLocalityPolicyManager` | The choice of policy manager determines how Applications and ResourceRequests are routed through the system. |
 |`yarn.federation.policy-manager-params` | `<binary>` | The payload that configures the policy. In our example a set of weights for router and amrmproxy policies. This is typically generated by serializing a policymanager that has been configured programmatically, or by populating the state-store with the .json serialized form of it. |
 |`yarn.federation.subcluster-resolver.class` | `org.apache.hadoop.yarn.server.federation.resolver.DefaultSubClusterResolverImpl` | The class used to resolve which subcluster a node belongs to, and which subcluster(s) a rack belongs to. |
-| `yarn.federation.machine-list` | `node1,subcluster1,rack1\n node2 , subcluster2, RACK1\n noDE3,subcluster3, rack2\n node4, subcluster3, rack2\n` | a list of Nodes, Sub-clusters, Rack, used by the `DefaultSubClusterResolverImpl` |
+| `yarn.federation.machine-list` | `node1,subcluster1,rack1\n node2 , subcluster2, RACK1\n node3,subcluster3, rack2\n node4, subcluster3, rack2\n` | a list of Nodes, Sub-clusters, Rack, used by the `DefaultSubClusterResolverImpl` |
 
 ###ON RMs:
 
@@ -200,6 +202,7 @@ These are extra configurations that should appear in the **conf/yarn-site.xml**
 | Property | Example | Description |
 |:---- |:---- |
 |`yarn.router.bind-host` | `0.0.0.0` | Host IP to bind the router to.  The actual address the server will bind to. If this optional address is set, the RPC and webapp servers will bind to this address and the port specified in yarn.router.*.address respectively. This is most useful for making Router listen to all interfaces by setting to 0.0.0.0. |
+| `yarn.router.clientrm.interceptor-class.pipeline` | `org.apache.hadoop.yarn.server.router.clientrm.FederationClientInterceptor` | A comma-seperated list of interceptor classes to be run at the router when interfacing with the client. The last step of this pipeline must be the Federation Client Interceptor. |
 
 Optional:
 
@@ -222,11 +225,53 @@ These are extra configurations that should appear in the **conf/yarn-site.xml**
 
 | Property | Example | Description |
 |:---- |:---- |
-|`yarn.nodemanager.amrmproxy.interceptor-class.pipeline` | `org.apache.hadoop.yarn.server.nodemanager.amrmproxy.FederationInterceptor` | A coma-separated list of interceptors to be run at the amrmproxy. For federation the last step in the pipeline should be the FederationInterceptor. |
+| `yarn.nodemanager.amrmproxy.enabled` | `true` | Whether or not the AMRMProxy is enabled.
+|`yarn.nodemanager.amrmproxy.interceptor-class.pipeline` | `org.apache.hadoop.yarn.server.nodemanager.amrmproxy.FederationInterceptor` | A comma-separated list of interceptors to be run at the amrmproxy. For federation the last step in the pipeline should be the FederationInterceptor.
+| `yarn.client.failover.proxy-provider` | `org.apache.hadoop.yarn.server.federation.failover.FederationRMFailoverProxyProvider` | The class used to connect to the RMs by looking up the membership information in federation state-store. This must be set if federation is enabled, even if RM HA is not enabled.|
 
 Optional:
 
 | Property | Example | Description |
 |:---- |:---- |
 |`yarn.federation.statestore.max-connections` | `1` | The maximum number of parallel connections from each AMRMProxy to the state-store. This value is typically lower than the router one, since we have many AMRMProxy that could burn-through many DB connections quickly. |
-|`yarn.federation.cache-ttl.secs` | `300` | The time to leave for the AMRMProxy cache. Typically larger than at the router, as the number of AMRMProxy is large, and we want to limit the load to the centralized state-store. |
\ No newline at end of file
+|`yarn.federation.cache-ttl.secs` | `300` | The time to leave for the AMRMProxy cache. Typically larger than at the router, as the number of AMRMProxy is large, and we want to limit the load to the centralized state-store. |
+
+###State-Store:
+
+Currently, the only supported implementation of the state-store is Microsoft SQL Server. After [setting up](https://www.microsoft.com/en-us/sql-server/sql-server-downloads) such an instance of SQL Server, set up the database for use by the federation system. This can be done by running the following SQL files in the database: **sbin/FederationStateStore/SQLServer/FederationStateStoreStoreProcs.sql** and **sbin/FederationStateStore/SQLServer/FederationStateStoreStoreTables.sql**
+
+Running a Sample Job
+--------------------
+In order to submit jobs to a Federation cluster one must create a seperate set of configs for the client from which said jobs will be submitted. In these, the **conf/yarn-site.xml** should have the following additional configurations:
+
+| Property | Example | Description |
+|:--- |:--- |
+| `yarn.resourcemanager.address` | `<router_host>:8050` | Redirects jobs launched at the client to the router's client RM port. |
+| `yarn.resourcemanger.scheduler.address` | `localhost:8049` | Redirects jobs to the federation AMRMProxy port.|
+
+Any YARN jobs for the cluster can be submitted from the client configurations described above. In order to launch a job through federation, first start up all the clusters involved in the federation as described [here](../../hadoop-project-dist/hadoop-common/ClusterSetup.html). Next, start up the router on the router machine with the following command:
+
+      $HADOOP_HOME/bin/yarn --daemon start router
+
+Now with $HADOOP_CONF_DIR pointing to the client configurations folder that is described above, run your job the usual way. The configurations in the client configurations folder described above will direct the job to the router's client RM port where the router should be listening after being started. Here is an example run of a Pi job on a federation cluster from the client:
+
+      $HADOOP_HOME/bin/yarn jar hadoop-mapreduce-examples-3.0.0.jar pi 16 1000
+
+This job is submitted to the router which as described above, uses a generated policy from the [GPG](#Global_Policy_Generator) to pick a home RM for the job to which it is submitted.
+
+The output from this particular example job should be something like:
+
+      2017-07-13 16:29:25,055 INFO mapreduce.Job: Job job_1499988226739_0001 running in uber mode : false
+      2017-07-13 16:29:25,056 INFO mapreduce.Job:  map 0% reduce 0%
+      2017-07-13 16:29:33,131 INFO mapreduce.Job:  map 38% reduce 0%
+      2017-07-13 16:29:39,176 INFO mapreduce.Job:  map 75% reduce 0%
+      2017-07-13 16:29:45,217 INFO mapreduce.Job:  map 94% reduce 0%
+      2017-07-13 16:29:46,228 INFO mapreduce.Job:  map 100% reduce 100%
+      2017-07-13 16:29:46,235 INFO mapreduce.Job: Job job_1499988226739_0001 completed successfully
+      .
+      .
+      .
+      Job Finished in 30.586 seconds
+      Estimated value of Pi is 3.14250000......
+
+Note that no change in the code or recompilation of the input jar was required to use federation. Also, the output of this job is the exact same as it would be when run without federation. Also, in order to get the full benefit of federation, use a large enough number of mappers such that more than one cluster is required. That number happens to be 16 in the case of the above example.


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


[04/50] [abbrv] 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/cfa1ed98
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/cfa1ed98
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/cfa1ed98

Branch: refs/heads/YARN-2915
Commit: cfa1ed989d83bab28f2fffe9ddf0902e95024e19
Parents: 1d5dacb
Author: Subru Krishnan <su...@apache.org>
Authored: Fri Aug 5 11:52:44 2016 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Tue Jul 25 16:56:31 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/cfa1ed98/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/cfa1ed98/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/cfa1ed98/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/cfa1ed98/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/cfa1ed98/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/cfa1ed98/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/cfa1ed98/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/cfa1ed98/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/cfa1ed98/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/cfa1ed98/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/cfa1ed98/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/cfa1ed98/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/cfa1ed98/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/cfa1ed98/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/cfa1ed98/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/cfa1ed98/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/cfa1ed98/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/cfa1ed98/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/cfa1ed98/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/cfa1ed98/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


[20/50] [abbrv] hadoop git commit: YARN-5531. UnmanagedAM pool manager for federating application across clusters. (Botong Huang via Subru).

Posted by su...@apache.org.
YARN-5531. UnmanagedAM pool manager for federating application across clusters. (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/c73abec4
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/c73abec4
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/c73abec4

Branch: refs/heads/YARN-2915
Commit: c73abec4ac1ad6c48691b4fc14a2e8eaa41c094f
Parents: f4b45eb
Author: Subru Krishnan <su...@apache.org>
Authored: Fri May 26 16:23:38 2017 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Tue Jul 25 16:56:32 2017 -0700

----------------------------------------------------------------------
 .../apache/hadoop/yarn/util/AsyncCallback.java  |  35 ++
 .../failover/FederationProxyProviderUtil.java   | 114 ++--
 .../yarn/server/uam/UnmanagedAMPoolManager.java | 311 ++++++++++
 .../server/uam/UnmanagedApplicationManager.java | 607 +++++++++++++++++++
 .../hadoop/yarn/server/uam/package-info.java    |  18 +
 .../yarn/server/utils/AMRMClientUtils.java      | 189 ++++++
 .../server/utils/YarnServerSecurityUtils.java   |  41 +-
 .../yarn/server/MockResourceManagerFacade.java  |  83 ++-
 .../uam/TestUnmanagedApplicationManager.java    | 335 ++++++++++
 .../amrmproxy/DefaultRequestInterceptor.java    |  30 +-
 .../ApplicationMasterService.java               |  12 +-
 .../TestApplicationMasterLauncher.java          |   6 +-
 12 files changed, 1661 insertions(+), 120 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c73abec4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/AsyncCallback.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/AsyncCallback.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/AsyncCallback.java
new file mode 100644
index 0000000..b4f75c9
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/AsyncCallback.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.util;
+
+/**
+ * Generic interface that can be used for calling back when a corresponding
+ * asynchronous operation completes.
+ *
+ * @param <T> parameter type for the callback
+ */
+public interface AsyncCallback<T> {
+  /**
+   * This method is called back when the corresponding asynchronous operation
+   * completes.
+   *
+   * @param response response of the callback
+   */
+  void callback(T response);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c73abec4/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 18f1338..3931f2b 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
@@ -19,22 +19,20 @@
 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.security.token.TokenIdentifier;
 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.apache.hadoop.yarn.server.utils.AMRMClientUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -51,10 +49,15 @@ public final class FederationProxyProviderUtil {
   public static final Logger LOG =
       LoggerFactory.getLogger(FederationProxyProviderUtil.class);
 
+  // Disable constructor
+  private FederationProxyProviderUtil() {
+  }
+
   /**
-   * 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.
+   * Create a proxy for the specified protocol in the context of Federation. 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
@@ -67,15 +70,16 @@ public final class FederationProxyProviderUtil {
   @Public
   @Unstable
   public static <T> T createRMProxy(Configuration configuration,
-      final Class<T> protocol, SubClusterId subClusterId,
-      UserGroupInformation user) throws IOException {
+      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.
+   * Create a proxy for the specified protocol in the context of Federation. 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
@@ -88,65 +92,35 @@ public final class FederationProxyProviderUtil {
    */
   @Public
   @Unstable
-  @SuppressWarnings("unchecked")
-  public static <T> T createRMProxy(final Configuration configuration,
+  public static <T> T createRMProxy(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);
-    }
+      UserGroupInformation user, Token<? extends TokenIdentifier> token)
+      throws IOException {
+    final YarnConfiguration config = new YarnConfiguration(configuration);
+    updateConfForFederation(config, subClusterId.getId());
+    return AMRMClientUtils.createRMProxy(config, protocol, user, token);
   }
 
-  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.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
-    // 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.
+  /**
+   * Updating the conf with Federation as long as certain subclusterId.
+   *
+   * @param conf configuration
+   * @param subClusterId subclusterId for the conf
+   */
+  public static void updateConfForFederation(Configuration conf,
+      String subClusterId) {
+    conf.set(YarnConfiguration.RM_CLUSTER_ID, subClusterId);
+    /*
+     * 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);
@@ -156,8 +130,4 @@ public final class FederationProxyProviderUtil {
     }
   }
 
-  // disable instantiation
-  private FederationProxyProviderUtil() {
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c73abec4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/uam/UnmanagedAMPoolManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/uam/UnmanagedAMPoolManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/uam/UnmanagedAMPoolManager.java
new file mode 100644
index 0000000..08aee77
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/uam/UnmanagedAMPoolManager.java
@@ -0,0 +1,311 @@
+/**
+ * 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.uam;
+
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.utils.AMRMClientUtils;
+import org.apache.hadoop.yarn.util.AsyncCallback;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * A service that manages a pool of UAM managers in
+ * {@link UnmanagedApplicationManager}.
+ */
+@Public
+@Unstable
+public class UnmanagedAMPoolManager extends AbstractService {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(UnmanagedAMPoolManager.class);
+
+  // Map from uamId to UAM instances
+  private Map<String, UnmanagedApplicationManager> unmanagedAppMasterMap;
+
+  private Map<String, ApplicationAttemptId> attemptIdMap;
+
+  private ExecutorService threadpool;
+
+  public UnmanagedAMPoolManager(ExecutorService threadpool) {
+    super(UnmanagedAMPoolManager.class.getName());
+    this.threadpool = threadpool;
+  }
+
+  @Override
+  protected void serviceStart() throws Exception {
+    if (this.threadpool == null) {
+      this.threadpool = Executors.newCachedThreadPool();
+    }
+    this.unmanagedAppMasterMap = new ConcurrentHashMap<>();
+    this.attemptIdMap = new ConcurrentHashMap<>();
+    super.serviceStart();
+  }
+
+  /**
+   * Normally we should finish all applications before stop. If there are still
+   * UAMs running, force kill all of them. Do parallel kill because of
+   * performance reasons.
+   *
+   * TODO: move waiting for the kill to finish into a separate thread, without
+   * blocking the serviceStop.
+   */
+  @Override
+  protected void serviceStop() throws Exception {
+    ExecutorCompletionService<KillApplicationResponse> completionService =
+        new ExecutorCompletionService<>(this.threadpool);
+    if (this.unmanagedAppMasterMap.isEmpty()) {
+      return;
+    }
+
+    // Save a local copy of the key set so that it won't change with the map
+    Set<String> addressList =
+        new HashSet<>(this.unmanagedAppMasterMap.keySet());
+    LOG.warn("Abnormal shutdown of UAMPoolManager, still {} UAMs in map",
+        addressList.size());
+
+    for (final String uamId : addressList) {
+      completionService.submit(new Callable<KillApplicationResponse>() {
+        @Override
+        public KillApplicationResponse call() throws Exception {
+          try {
+            LOG.info("Force-killing UAM id " + uamId + " for application "
+                + attemptIdMap.get(uamId));
+            return unmanagedAppMasterMap.remove(uamId).forceKillApplication();
+          } catch (Exception e) {
+            LOG.error("Failed to kill unmanaged application master", e);
+            return null;
+          }
+        }
+      });
+    }
+
+    for (int i = 0; i < addressList.size(); ++i) {
+      try {
+        Future<KillApplicationResponse> future = completionService.take();
+        future.get();
+      } catch (Exception e) {
+        LOG.error("Failed to kill unmanaged application master", e);
+      }
+    }
+    this.attemptIdMap.clear();
+    super.serviceStop();
+  }
+
+  /**
+   * Create a new UAM and register the application, without specifying uamId and
+   * appId. We will ask for an appId from RM and use it as the uamId.
+   *
+   * @param registerRequest RegisterApplicationMasterRequest
+   * @param conf configuration for this UAM
+   * @param queueName queue of the application
+   * @param submitter submitter name of the UAM
+   * @param appNameSuffix application name suffix for the UAM
+   * @return uamId for the UAM
+   * @throws YarnException if registerApplicationMaster fails
+   * @throws IOException if registerApplicationMaster fails
+   */
+  public String createAndRegisterNewUAM(
+      RegisterApplicationMasterRequest registerRequest, Configuration conf,
+      String queueName, String submitter, String appNameSuffix)
+      throws YarnException, IOException {
+    ApplicationId appId = null;
+    ApplicationClientProtocol rmClient;
+    try {
+      UserGroupInformation appSubmitter =
+          UserGroupInformation.createRemoteUser(submitter);
+      rmClient = AMRMClientUtils.createRMProxy(conf,
+          ApplicationClientProtocol.class, appSubmitter, null);
+
+      // Get a new appId from RM
+      GetNewApplicationResponse response =
+          rmClient.getNewApplication(GetNewApplicationRequest.newInstance());
+      if (response == null) {
+        throw new YarnException("getNewApplication got null response");
+      }
+      appId = response.getApplicationId();
+      LOG.info("Received new application ID {} from RM", appId);
+    } finally {
+      rmClient = null;
+    }
+
+    createAndRegisterNewUAM(appId.toString(), registerRequest, conf, appId,
+        queueName, submitter, appNameSuffix);
+    return appId.toString();
+  }
+
+  /**
+   * Create a new UAM and register the application, using the provided uamId and
+   * appId.
+   *
+   * @param uamId identifier for the UAM
+   * @param registerRequest RegisterApplicationMasterRequest
+   * @param conf configuration for this UAM
+   * @param appId application id for the UAM
+   * @param queueName queue of the application
+   * @param submitter submitter name of the UAM
+   * @param appNameSuffix application name suffix for the UAM
+   * @return RegisterApplicationMasterResponse
+   * @throws YarnException if registerApplicationMaster fails
+   * @throws IOException if registerApplicationMaster fails
+   */
+  public RegisterApplicationMasterResponse createAndRegisterNewUAM(String uamId,
+      RegisterApplicationMasterRequest registerRequest, Configuration conf,
+      ApplicationId appId, String queueName, String submitter,
+      String appNameSuffix) throws YarnException, IOException {
+
+    if (this.unmanagedAppMasterMap.containsKey(uamId)) {
+      throw new YarnException("UAM " + uamId + " already exists");
+    }
+    UnmanagedApplicationManager uam =
+        createUAM(conf, appId, queueName, submitter, appNameSuffix);
+    // Put the UAM into map first before initializing it to avoid additional UAM
+    // for the same uamId being created concurrently
+    this.unmanagedAppMasterMap.put(uamId, uam);
+
+    RegisterApplicationMasterResponse response = null;
+    try {
+      LOG.info("Creating and registering UAM id {} for application {}", uamId,
+          appId);
+      response = uam.createAndRegisterApplicationMaster(registerRequest);
+    } catch (Exception e) {
+      // Add the map earlier and remove here if register failed because we want
+      // to make sure there is only one uam instance per uamId at any given time
+      this.unmanagedAppMasterMap.remove(uamId);
+      throw e;
+    }
+
+    this.attemptIdMap.put(uamId, uam.getAttemptId());
+    return response;
+  }
+
+  /**
+   * Creates the UAM instance. Pull out to make unit test easy.
+   *
+   * @param conf Configuration
+   * @param appId application id
+   * @param queueName queue of the application
+   * @param submitter submitter name of the application
+   * @param appNameSuffix application name suffix
+   * @return the UAM instance
+   */
+  @VisibleForTesting
+  protected UnmanagedApplicationManager createUAM(Configuration conf,
+      ApplicationId appId, String queueName, String submitter,
+      String appNameSuffix) {
+    return new UnmanagedApplicationManager(conf, appId, queueName, submitter,
+        appNameSuffix);
+  }
+
+  /**
+   * AllocateAsync to an UAM.
+   *
+   * @param uamId identifier for the UAM
+   * @param request AllocateRequest
+   * @param callback callback for response
+   * @throws YarnException if allocate fails
+   * @throws IOException if allocate fails
+   */
+  public void allocateAsync(String uamId, AllocateRequest request,
+      AsyncCallback<AllocateResponse> callback)
+      throws YarnException, IOException {
+    if (!this.unmanagedAppMasterMap.containsKey(uamId)) {
+      throw new YarnException("UAM " + uamId + " does not exist");
+    }
+    this.unmanagedAppMasterMap.get(uamId).allocateAsync(request, callback);
+  }
+
+  /**
+   * Finish an UAM/application.
+   *
+   * @param uamId identifier for the UAM
+   * @param request FinishApplicationMasterRequest
+   * @return FinishApplicationMasterResponse
+   * @throws YarnException if finishApplicationMaster call fails
+   * @throws IOException if finishApplicationMaster call fails
+   */
+  public FinishApplicationMasterResponse finishApplicationMaster(String uamId,
+      FinishApplicationMasterRequest request)
+      throws YarnException, IOException {
+    if (!this.unmanagedAppMasterMap.containsKey(uamId)) {
+      throw new YarnException("UAM " + uamId + " does not exist");
+    }
+    LOG.info("Finishing application for UAM id {} ", uamId);
+    FinishApplicationMasterResponse response =
+        this.unmanagedAppMasterMap.get(uamId).finishApplicationMaster(request);
+
+    if (response.getIsUnregistered()) {
+      // Only remove the UAM when the unregister finished
+      this.unmanagedAppMasterMap.remove(uamId);
+      this.attemptIdMap.remove(uamId);
+      LOG.info("UAM id {} is unregistered", uamId);
+    }
+    return response;
+  }
+
+  /**
+   * Get the id of all running UAMs.
+   *
+   * @return uamId set
+   */
+  public Set<String> getAllUAMIds() {
+    // Return a clone of the current id set for concurrency reasons, so that the
+    // returned map won't change with the actual map
+    return new HashSet<String>(this.unmanagedAppMasterMap.keySet());
+  }
+
+  /**
+   * Return whether an UAM exists.
+   *
+   * @param uamId identifier for the UAM
+   * @return UAM exists or not
+   */
+  public boolean hasUAMId(String uamId) {
+    return this.unmanagedAppMasterMap.containsKey(uamId);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c73abec4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/uam/UnmanagedApplicationManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/uam/UnmanagedApplicationManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/uam/UnmanagedApplicationManager.java
new file mode 100644
index 0000000..60a9a27
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/uam/UnmanagedApplicationManager.java
@@ -0,0 +1,607 @@
+/**
+ * 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.uam;
+
+import java.io.IOException;
+import java.lang.Thread.UncaughtExceptionHandler;
+import java.util.EnumSet;
+import java.util.Set;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+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.FinishApplicationMasterRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
+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.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
+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.factories.RecordFactory;
+import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
+import org.apache.hadoop.yarn.server.utils.AMRMClientUtils;
+import org.apache.hadoop.yarn.server.utils.BuilderUtils;
+import org.apache.hadoop.yarn.server.utils.YarnServerSecurityUtils;
+import org.apache.hadoop.yarn.util.AsyncCallback;
+import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+
+/**
+ * UnmanagedApplicationManager is used to register unmanaged application and
+ * negotiate for resources from resource managers. An unmanagedAM is an AM that
+ * is not launched and managed by the RM. Allocate calls are handled
+ * asynchronously using {@link AsyncCallback}.
+ */
+@Public
+@Unstable
+public class UnmanagedApplicationManager {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(UnmanagedApplicationManager.class);
+  private static final long AM_STATE_WAIT_TIMEOUT_MS = 10000;
+  private static final String APP_NAME = "UnmanagedAM";
+  private static final String DEFAULT_QUEUE_CONFIG = "uam.default.queue.name";
+
+  private BlockingQueue<AsyncAllocateRequestInfo> requestQueue;
+  private AMRequestHandlerThread handlerThread;
+  private ApplicationMasterProtocol rmProxy;
+  private ApplicationId applicationId;
+  private ApplicationAttemptId attemptId;
+  private String submitter;
+  private String appNameSuffix;
+  private Configuration conf;
+  private String queueName;
+  private UserGroupInformation userUgi;
+  private RegisterApplicationMasterRequest registerRequest;
+  private int lastResponseId;
+  private ApplicationClientProtocol rmClient;
+  private long asyncApiPollIntervalMillis;
+  private RecordFactory recordFactory;
+
+  public UnmanagedApplicationManager(Configuration conf, ApplicationId appId,
+      String queueName, String submitter, String appNameSuffix) {
+    Preconditions.checkNotNull(conf, "Configuration cannot be null");
+    Preconditions.checkNotNull(appId, "ApplicationId cannot be null");
+    Preconditions.checkNotNull(submitter, "App submitter cannot be null");
+
+    this.conf = conf;
+    this.applicationId = appId;
+    this.queueName = queueName;
+    this.submitter = submitter;
+    this.appNameSuffix = appNameSuffix;
+    this.handlerThread = new AMRequestHandlerThread();
+    this.requestQueue = new LinkedBlockingQueue<>();
+    this.rmProxy = null;
+    this.registerRequest = null;
+    this.recordFactory = RecordFactoryProvider.getRecordFactory(conf);
+    this.asyncApiPollIntervalMillis = conf.getLong(
+        YarnConfiguration.
+            YARN_CLIENT_APPLICATION_CLIENT_PROTOCOL_POLL_INTERVAL_MS,
+        YarnConfiguration.
+            DEFAULT_YARN_CLIENT_APPLICATION_CLIENT_PROTOCOL_POLL_INTERVAL_MS);
+  }
+
+  /**
+   * Registers this {@link UnmanagedApplicationManager} with the resource
+   * manager.
+   *
+   * @param request the register request
+   * @return the register response
+   * @throws YarnException if register fails
+   * @throws IOException if register fails
+   */
+  public RegisterApplicationMasterResponse createAndRegisterApplicationMaster(
+      RegisterApplicationMasterRequest request)
+      throws YarnException, IOException {
+    // This need to be done first in this method, because it is used as an
+    // indication that this method is called (and perhaps blocked due to RM
+    // connection and not finished yet)
+    this.registerRequest = request;
+
+    // attemptId will be available after this call
+    UnmanagedAMIdentifier identifier =
+        initializeUnmanagedAM(this.applicationId);
+
+    try {
+      this.userUgi = UserGroupInformation.createProxyUser(
+          identifier.getAttemptId().toString(),
+          UserGroupInformation.getCurrentUser());
+    } catch (IOException e) {
+      LOG.error("Exception while trying to get current user", e);
+      throw new YarnRuntimeException(e);
+    }
+
+    this.rmProxy = createRMProxy(ApplicationMasterProtocol.class, this.conf,
+        this.userUgi, identifier.getToken());
+
+    LOG.info("Registering the Unmanaged application master {}", this.attemptId);
+    RegisterApplicationMasterResponse response =
+        this.rmProxy.registerApplicationMaster(this.registerRequest);
+
+    // Only when register succeed that we start the heartbeat thread
+    this.handlerThread.setUncaughtExceptionHandler(
+        new HeartBeatThreadUncaughtExceptionHandler());
+    this.handlerThread.setDaemon(true);
+    this.handlerThread.start();
+
+    this.lastResponseId = 0;
+    return response;
+  }
+
+  /**
+   * Unregisters from the resource manager and stops the request handler thread.
+   *
+   * @param request the finishApplicationMaster request
+   * @return the response
+   * @throws YarnException if finishAM call fails
+   * @throws IOException if finishAM call fails
+   */
+  public FinishApplicationMasterResponse finishApplicationMaster(
+      FinishApplicationMasterRequest request)
+      throws YarnException, IOException {
+
+    this.handlerThread.shutdown();
+
+    if (this.rmProxy == null) {
+      if (this.registerRequest != null) {
+        // This is possible if the async registerApplicationMaster is still
+        // blocked and retrying. Return a dummy response in this case.
+        LOG.warn("Unmanaged AM still not successfully launched/registered yet."
+            + " Stopping the UAM client thread anyways.");
+        return FinishApplicationMasterResponse.newInstance(false);
+      } else {
+        throw new YarnException("finishApplicationMaster should not "
+            + "be called before createAndRegister");
+      }
+    }
+    return AMRMClientUtils.finishAMWithReRegister(request, this.rmProxy,
+        this.registerRequest, this.attemptId);
+  }
+
+  /**
+   * Force kill the UAM.
+   *
+   * @return kill response
+   * @throws IOException if fails to create rmProxy
+   * @throws YarnException if force kill fails
+   */
+  public KillApplicationResponse forceKillApplication()
+      throws IOException, YarnException {
+    KillApplicationRequest request =
+        KillApplicationRequest.newInstance(this.attemptId.getApplicationId());
+
+    this.handlerThread.shutdown();
+
+    if (this.rmClient == null) {
+      this.rmClient = createRMProxy(ApplicationClientProtocol.class, this.conf,
+          UserGroupInformation.createRemoteUser(this.submitter), null);
+    }
+    return this.rmClient.forceKillApplication(request);
+  }
+
+  /**
+   * Sends the specified heart beat request to the resource manager and invokes
+   * the callback asynchronously with the response.
+   *
+   * @param request the allocate request
+   * @param callback the callback method for the request
+   * @throws YarnException if registerAM is not called yet
+   */
+  public void allocateAsync(AllocateRequest request,
+      AsyncCallback<AllocateResponse> callback) throws YarnException {
+    try {
+      this.requestQueue.put(new AsyncAllocateRequestInfo(request, callback));
+    } catch (InterruptedException ex) {
+      // Should not happen as we have MAX_INT queue length
+      LOG.debug("Interrupted while waiting to put on response queue", ex);
+    }
+    // Two possible cases why the UAM is not successfully registered yet:
+    // 1. registerApplicationMaster is not called at all. Should throw here.
+    // 2. registerApplicationMaster is called but hasn't successfully returned.
+    //
+    // In case 2, we have already save the allocate request above, so if the
+    // registration succeed later, no request is lost.
+    if (this.rmProxy == null) {
+      if (this.registerRequest != null) {
+        LOG.info("Unmanaged AM still not successfully launched/registered yet."
+            + " Saving the allocate request and send later.");
+      } else {
+        throw new YarnException(
+            "AllocateAsync should not be called before createAndRegister");
+      }
+    }
+  }
+
+  /**
+   * Returns the application attempt id of the UAM.
+   *
+   * @return attempt id of the UAM
+   */
+  public ApplicationAttemptId getAttemptId() {
+    return this.attemptId;
+  }
+
+  /**
+   * Returns RM proxy for the specified protocol type. Unit test cases can
+   * override this method and return mock proxy instances.
+   *
+   * @param protocol protocal of the proxy
+   * @param config configuration
+   * @param user ugi for the proxy connection
+   * @param token token for the connection
+   * @param <T> type of the proxy
+   * @return the proxy instance
+   * @throws IOException if fails to create the proxy
+   */
+  protected <T> T createRMProxy(Class<T> protocol, Configuration config,
+      UserGroupInformation user, Token<AMRMTokenIdentifier> token)
+      throws IOException {
+    return AMRMClientUtils.createRMProxy(config, protocol, user, token);
+  }
+
+  /**
+   * Launch and initialize an unmanaged AM. First, it creates a new application
+   * on the RM and negotiates a new attempt id. Then it waits for the RM
+   * application attempt state to reach YarnApplicationAttemptState.LAUNCHED
+   * after which it returns the AM-RM token and the attemptId.
+   *
+   * @param appId application id
+   * @return the UAM identifier
+   * @throws IOException if initialize fails
+   * @throws YarnException if initialize fails
+   */
+  protected UnmanagedAMIdentifier initializeUnmanagedAM(ApplicationId appId)
+      throws IOException, YarnException {
+    try {
+      UserGroupInformation appSubmitter =
+          UserGroupInformation.createRemoteUser(this.submitter);
+      this.rmClient = createRMProxy(ApplicationClientProtocol.class, this.conf,
+          appSubmitter, null);
+
+      // Submit the application
+      submitUnmanagedApp(appId);
+
+      // Monitor the application attempt to wait for launch state
+      ApplicationAttemptReport attemptReport = monitorCurrentAppAttempt(appId,
+          EnumSet.of(YarnApplicationState.ACCEPTED,
+              YarnApplicationState.RUNNING, YarnApplicationState.KILLED,
+              YarnApplicationState.FAILED, YarnApplicationState.FINISHED),
+          YarnApplicationAttemptState.LAUNCHED);
+      this.attemptId = attemptReport.getApplicationAttemptId();
+      return getUAMIdentifier();
+    } finally {
+      this.rmClient = null;
+    }
+  }
+
+  private void submitUnmanagedApp(ApplicationId appId)
+      throws YarnException, IOException {
+    SubmitApplicationRequest submitRequest =
+        this.recordFactory.newRecordInstance(SubmitApplicationRequest.class);
+
+    ApplicationSubmissionContext context = this.recordFactory
+        .newRecordInstance(ApplicationSubmissionContext.class);
+
+    context.setApplicationId(appId);
+    context.setApplicationName(APP_NAME + "-" + appNameSuffix);
+    if (StringUtils.isBlank(this.queueName)) {
+      context.setQueue(this.conf.get(DEFAULT_QUEUE_CONFIG,
+          YarnConfiguration.DEFAULT_QUEUE_NAME));
+    } else {
+      context.setQueue(this.queueName);
+    }
+
+    ContainerLaunchContext amContainer =
+        this.recordFactory.newRecordInstance(ContainerLaunchContext.class);
+    Resource resource = BuilderUtils.newResource(1024, 1);
+    context.setResource(resource);
+    context.setAMContainerSpec(amContainer);
+    submitRequest.setApplicationSubmissionContext(context);
+
+    context.setUnmanagedAM(true);
+
+    LOG.info("Submitting unmanaged application {}", appId);
+    this.rmClient.submitApplication(submitRequest);
+  }
+
+  /**
+   * Monitor the submitted application and attempt until it reaches certain
+   * states.
+   *
+   * @param appId Application Id of application to be monitored
+   * @param appStates acceptable application state
+   * @param attemptState acceptable application attempt state
+   * @return the application report
+   * @throws YarnException if getApplicationReport fails
+   * @throws IOException if getApplicationReport fails
+   */
+  private ApplicationAttemptReport monitorCurrentAppAttempt(ApplicationId appId,
+      Set<YarnApplicationState> appStates,
+      YarnApplicationAttemptState attemptState)
+      throws YarnException, IOException {
+
+    long startTime = System.currentTimeMillis();
+    ApplicationAttemptId appAttemptId = null;
+    while (true) {
+      if (appAttemptId == null) {
+        // Get application report for the appId we are interested in
+        ApplicationReport report = getApplicationReport(appId);
+        YarnApplicationState state = report.getYarnApplicationState();
+        if (appStates.contains(state)) {
+          if (state != YarnApplicationState.ACCEPTED) {
+            throw new YarnRuntimeException(
+                "Received non-accepted application state: " + state
+                    + ". Application " + appId + " not the first attempt?");
+          }
+          appAttemptId =
+              getApplicationReport(appId).getCurrentApplicationAttemptId();
+        } else {
+          LOG.info("Current application state of {} is {}, will retry later.",
+              appId, state);
+        }
+      }
+
+      if (appAttemptId != null) {
+        GetApplicationAttemptReportRequest req = this.recordFactory
+            .newRecordInstance(GetApplicationAttemptReportRequest.class);
+        req.setApplicationAttemptId(appAttemptId);
+        ApplicationAttemptReport attemptReport = this.rmClient
+            .getApplicationAttemptReport(req).getApplicationAttemptReport();
+        if (attemptState
+            .equals(attemptReport.getYarnApplicationAttemptState())) {
+          return attemptReport;
+        }
+        LOG.info("Current attempt state of " + appAttemptId + " is "
+            + attemptReport.getYarnApplicationAttemptState()
+            + ", waiting for current attempt to reach " + attemptState);
+      }
+
+      try {
+        Thread.sleep(this.asyncApiPollIntervalMillis);
+      } catch (InterruptedException e) {
+        LOG.warn("Interrupted while waiting for current attempt of " + appId
+            + " to reach " + attemptState);
+      }
+
+      if (System.currentTimeMillis() - startTime > AM_STATE_WAIT_TIMEOUT_MS) {
+        throw new RuntimeException("Timeout for waiting current attempt of "
+            + appId + " to reach " + attemptState);
+      }
+    }
+  }
+
+  /**
+   * Gets the identifier of the unmanaged AM.
+   *
+   * @return the identifier of the unmanaged AM.
+   * @throws IOException if getApplicationReport fails
+   * @throws YarnException if getApplicationReport fails
+   */
+  protected UnmanagedAMIdentifier getUAMIdentifier()
+      throws IOException, YarnException {
+    Token<AMRMTokenIdentifier> token = null;
+    org.apache.hadoop.yarn.api.records.Token amrmToken =
+        getApplicationReport(this.attemptId.getApplicationId()).getAMRMToken();
+    if (amrmToken != null) {
+      token = ConverterUtils.convertFromYarn(amrmToken, (Text) null);
+    } else {
+      LOG.warn(
+          "AMRMToken not found in the application report for application: {}",
+          this.attemptId.getApplicationId());
+    }
+    return new UnmanagedAMIdentifier(this.attemptId, token);
+  }
+
+  private ApplicationReport getApplicationReport(ApplicationId appId)
+      throws YarnException, IOException {
+    GetApplicationReportRequest request =
+        this.recordFactory.newRecordInstance(GetApplicationReportRequest.class);
+    request.setApplicationId(appId);
+    return this.rmClient.getApplicationReport(request).getApplicationReport();
+  }
+
+  /**
+   * Data structure that encapsulates the application attempt identifier and the
+   * AMRMTokenIdentifier. Make it public because clients with HA need it.
+   */
+  public static class UnmanagedAMIdentifier {
+    private ApplicationAttemptId attemptId;
+    private Token<AMRMTokenIdentifier> token;
+
+    public UnmanagedAMIdentifier(ApplicationAttemptId attemptId,
+        Token<AMRMTokenIdentifier> token) {
+      this.attemptId = attemptId;
+      this.token = token;
+    }
+
+    public ApplicationAttemptId getAttemptId() {
+      return this.attemptId;
+    }
+
+    public Token<AMRMTokenIdentifier> getToken() {
+      return this.token;
+    }
+  }
+
+  /**
+   * Data structure that encapsulates AllocateRequest and AsyncCallback
+   * instance.
+   */
+  public static class AsyncAllocateRequestInfo {
+    private AllocateRequest request;
+    private AsyncCallback<AllocateResponse> callback;
+
+    public AsyncAllocateRequestInfo(AllocateRequest request,
+        AsyncCallback<AllocateResponse> callback) {
+      Preconditions.checkArgument(request != null,
+          "AllocateRequest cannot be null");
+      Preconditions.checkArgument(callback != null, "Callback cannot be null");
+
+      this.request = request;
+      this.callback = callback;
+    }
+
+    public AsyncCallback<AllocateResponse> getCallback() {
+      return this.callback;
+    }
+
+    public AllocateRequest getRequest() {
+      return this.request;
+    }
+  }
+
+  @VisibleForTesting
+  public int getRequestQueueSize() {
+    return this.requestQueue.size();
+  }
+
+  /**
+   * Extends Thread and provides an implementation that is used for processing
+   * the AM heart beat request asynchronously and sending back the response
+   * using the callback method registered with the system.
+   */
+  public class AMRequestHandlerThread extends Thread {
+
+    // Indication flag for the thread to keep running
+    private volatile boolean keepRunning;
+
+    public AMRequestHandlerThread() {
+      super("UnmanagedApplicationManager Heartbeat Handler Thread");
+      this.keepRunning = true;
+    }
+
+    /**
+     * Shutdown the thread.
+     */
+    public void shutdown() {
+      this.keepRunning = false;
+      this.interrupt();
+    }
+
+    @Override
+    public void run() {
+      while (keepRunning) {
+        AsyncAllocateRequestInfo requestInfo;
+        try {
+          requestInfo = requestQueue.take();
+          if (requestInfo == null) {
+            throw new YarnException(
+                "Null requestInfo taken from request queue");
+          }
+          if (!keepRunning) {
+            break;
+          }
+
+          // change the response id before forwarding the allocate request as we
+          // could have different values for each UAM
+          AllocateRequest request = requestInfo.getRequest();
+          if (request == null) {
+            throw new YarnException("Null allocateRequest from requestInfo");
+          }
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Sending Heartbeat to Unmanaged AM. AskList:"
+                + ((request.getAskList() == null) ? " empty"
+                    : request.getAskList().size()));
+          }
+
+          request.setResponseId(lastResponseId);
+          AllocateResponse response = AMRMClientUtils.allocateWithReRegister(
+              request, rmProxy, registerRequest, attemptId);
+          if (response == null) {
+            throw new YarnException("Null allocateResponse from allocate");
+          }
+
+          lastResponseId = response.getResponseId();
+          // update token if RM has reissued/renewed
+          if (response.getAMRMToken() != null) {
+            LOG.debug("Received new AMRMToken");
+            YarnServerSecurityUtils.updateAMRMToken(response.getAMRMToken(),
+                userUgi, conf);
+          }
+
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Received Heartbeat reply from RM. Allocated Containers:"
+                + ((response.getAllocatedContainers() == null) ? " empty"
+                    : response.getAllocatedContainers().size()));
+          }
+
+          if (requestInfo.getCallback() == null) {
+            throw new YarnException("Null callback from requestInfo");
+          }
+          requestInfo.getCallback().callback(response);
+        } catch (InterruptedException ex) {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Interrupted while waiting for queue", ex);
+          }
+        } catch (IOException ex) {
+          LOG.warn(
+              "IO Error occurred while processing heart beat for " + attemptId,
+              ex);
+        } catch (Throwable ex) {
+          LOG.warn(
+              "Error occurred while processing heart beat for " + attemptId,
+              ex);
+        }
+      }
+
+      LOG.info("UnmanagedApplicationManager has been stopped for {}. "
+          + "AMRequestHandlerThread thread is exiting", attemptId);
+    }
+  }
+
+  /**
+   * Uncaught exception handler for the background heartbeat thread.
+   */
+  protected class HeartBeatThreadUncaughtExceptionHandler
+      implements UncaughtExceptionHandler {
+    @Override
+    public void uncaughtException(Thread t, Throwable e) {
+      LOG.error("Heartbeat thread {} for application attempt {} crashed!",
+          t.getName(), attemptId, e);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c73abec4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/uam/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/uam/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/uam/package-info.java
new file mode 100644
index 0000000..0e78094
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/uam/package-info.java
@@ -0,0 +1,18 @@
+/**
+ * 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.uam;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c73abec4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/AMRMClientUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/AMRMClientUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/AMRMClientUtils.java
new file mode 100644
index 0000000..7993bd8
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/AMRMClientUtils.java
@@ -0,0 +1,189 @@
+/**
+ * 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.utils;
+
+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.security.token.TokenIdentifier;
+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.FinishApplicationMasterRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.client.ClientRMProxy;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.ApplicationMasterNotRegisteredException;
+import org.apache.hadoop.yarn.exceptions.InvalidApplicationMasterRequestException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Utility class for AMRMClient.
+ */
+@Private
+public final class AMRMClientUtils {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(AMRMClientUtils.class);
+
+  public static final String APP_ALREADY_REGISTERED_MESSAGE =
+      "Application Master is already registered : ";
+
+  private AMRMClientUtils() {
+  }
+
+  /**
+   * Handle ApplicationNotRegistered exception and re-register.
+   *
+   * @param attemptId app attemptId
+   * @param rmProxy RM proxy instance
+   * @param registerRequest the AM re-register request
+   * @throws YarnException if re-register fails
+   */
+  public static void handleNotRegisteredExceptionAndReRegister(
+      ApplicationAttemptId attemptId, ApplicationMasterProtocol rmProxy,
+      RegisterApplicationMasterRequest registerRequest) throws YarnException {
+    LOG.info("App attempt {} not registered, most likely due to RM failover. "
+        + " Trying to re-register.", attemptId);
+    try {
+      rmProxy.registerApplicationMaster(registerRequest);
+    } catch (Exception e) {
+      if (e instanceof InvalidApplicationMasterRequestException
+          && e.getMessage().contains(APP_ALREADY_REGISTERED_MESSAGE)) {
+        LOG.info("Concurrent thread successfully registered, moving on.");
+      } else {
+        LOG.error("Error trying to re-register AM", e);
+        throw new YarnException(e);
+      }
+    }
+  }
+
+  /**
+   * Helper method for client calling ApplicationMasterProtocol.allocate that
+   * handles re-register if RM fails over.
+   *
+   * @param request allocate request
+   * @param rmProxy RM proxy
+   * @param registerRequest the register request for re-register
+   * @param attemptId application attempt id
+   * @return allocate response
+   * @throws YarnException if RM call fails
+   * @throws IOException if RM call fails
+   */
+  public static AllocateResponse allocateWithReRegister(AllocateRequest request,
+      ApplicationMasterProtocol rmProxy,
+      RegisterApplicationMasterRequest registerRequest,
+      ApplicationAttemptId attemptId) throws YarnException, IOException {
+    try {
+      return rmProxy.allocate(request);
+    } catch (ApplicationMasterNotRegisteredException e) {
+      handleNotRegisteredExceptionAndReRegister(attemptId, rmProxy,
+          registerRequest);
+      // reset responseId after re-register
+      request.setResponseId(0);
+      // retry allocate
+      return allocateWithReRegister(request, rmProxy, registerRequest,
+          attemptId);
+    }
+  }
+
+  /**
+   * Helper method for client calling
+   * ApplicationMasterProtocol.finishApplicationMaster that handles re-register
+   * if RM fails over.
+   *
+   * @param request finishApplicationMaster request
+   * @param rmProxy RM proxy
+   * @param registerRequest the register request for re-register
+   * @param attemptId application attempt id
+   * @return finishApplicationMaster response
+   * @throws YarnException if RM call fails
+   * @throws IOException if RM call fails
+   */
+  public static FinishApplicationMasterResponse finishAMWithReRegister(
+      FinishApplicationMasterRequest request, ApplicationMasterProtocol rmProxy,
+      RegisterApplicationMasterRequest registerRequest,
+      ApplicationAttemptId attemptId) throws YarnException, IOException {
+    try {
+      return rmProxy.finishApplicationMaster(request);
+    } catch (ApplicationMasterNotRegisteredException ex) {
+      handleNotRegisteredExceptionAndReRegister(attemptId, rmProxy,
+          registerRequest);
+      // retry finishAM after re-register
+      return finishAMWithReRegister(request, rmProxy, registerRequest,
+          attemptId);
+    }
+  }
+
+  /**
+   * Create a proxy for the specified protocol.
+   *
+   * @param configuration Configuration to generate {@link ClientRMProxy}
+   * @param protocol Protocol for the proxy
+   * @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
+  public static <T> T createRMProxy(final Configuration configuration,
+      final Class<T> protocol, UserGroupInformation user,
+      final Token<? extends TokenIdentifier> token) throws IOException {
+    try {
+      String rmClusterId = configuration.get(YarnConfiguration.RM_CLUSTER_ID,
+          YarnConfiguration.DEFAULT_RM_CLUSTER_ID);
+      LOG.info("Creating RMProxy to RM {} for protocol {} for user {}",
+          rmClusterId, protocol.getSimpleName(), user);
+      if (token != null) {
+        token.setService(ClientRMProxy.getAMRMTokenService(configuration));
+        user.addToken(token);
+        setAuthModeInConf(configuration);
+      }
+      final T proxyConnection = user.doAs(new PrivilegedExceptionAction<T>() {
+        @Override
+        public T run() throws Exception {
+          return ClientRMProxy.createRMProxy(configuration, protocol);
+        }
+      });
+      return proxyConnection;
+
+    } catch (InterruptedException e) {
+      throw new YarnRuntimeException(e);
+    }
+  }
+
+  private static void setAuthModeInConf(Configuration conf) {
+    conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION,
+        SaslRpcServer.AuthMethod.TOKEN.toString());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c73abec4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/YarnServerSecurityUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/YarnServerSecurityUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/YarnServerSecurityUtils.java
index 9af556e..e61798d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/YarnServerSecurityUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/YarnServerSecurityUtils.java
@@ -23,13 +23,16 @@ import java.nio.ByteBuffer;
 import java.util.Set;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.DataInputByteBuffer;
+import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.hadoop.yarn.client.ClientRMProxy;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.ipc.RPCUtil;
 import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
@@ -42,8 +45,8 @@ import org.slf4j.LoggerFactory;
  */
 @Private
 public final class YarnServerSecurityUtils {
-  private static final Logger LOG = LoggerFactory
-      .getLogger(YarnServerSecurityUtils.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(YarnServerSecurityUtils.class);
 
   private YarnServerSecurityUtils() {
   }
@@ -55,8 +58,7 @@ public final class YarnServerSecurityUtils {
    * @return the AMRMTokenIdentifier instance for the current user
    * @throws YarnException
    */
-  public static AMRMTokenIdentifier authorizeRequest()
-      throws YarnException {
+  public static AMRMTokenIdentifier authorizeRequest() throws YarnException {
 
     UserGroupInformation remoteUgi;
     try {
@@ -82,9 +84,8 @@ public final class YarnServerSecurityUtils {
       }
     } catch (IOException e) {
       tokenFound = false;
-      message =
-          "Got exception while looking for AMRMToken for user "
-              + remoteUgi.getUserName();
+      message = "Got exception while looking for AMRMToken for user "
+          + remoteUgi.getUserName();
     }
 
     if (!tokenFound) {
@@ -113,8 +114,29 @@ public final class YarnServerSecurityUtils {
   }
 
   /**
+   * Update the new AMRMToken into the ugi used for RM proxy.
+   *
+   * @param token the new AMRMToken sent by RM
+   * @param user ugi used for RM proxy
+   * @param conf configuration
+   */
+  public static void updateAMRMToken(
+      org.apache.hadoop.yarn.api.records.Token token, UserGroupInformation user,
+      Configuration conf) {
+    Token<AMRMTokenIdentifier> amrmToken = new Token<AMRMTokenIdentifier>(
+        token.getIdentifier().array(), token.getPassword().array(),
+        new Text(token.getKind()), new Text(token.getService()));
+    // Preserve the token service sent by the RM when adding the token
+    // to ensure we replace the previous token setup by the RM.
+    // Afterwards we can update the service address for the RPC layer.
+    user.addToken(amrmToken);
+    amrmToken.setService(ClientRMProxy.getAMRMTokenService(conf));
+  }
+
+  /**
    * Parses the container launch context and returns a Credential instance that
-   * contains all the tokens from the launch context. 
+   * contains all the tokens from the launch context.
+   *
    * @param launchContext
    * @return the credential instance
    * @throws IOException
@@ -130,8 +152,7 @@ public final class YarnServerSecurityUtils {
       buf.reset(tokens);
       credentials.readTokenStorageStream(buf);
       if (LOG.isDebugEnabled()) {
-        for (Token<? extends TokenIdentifier> tk : credentials
-            .getAllTokens()) {
+        for (Token<? extends TokenIdentifier> tk : credentials.getAllTokens()) {
           LOG.debug(tk.getService() + " = " + tk.toString());
         }
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c73abec4/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
index 696188b..9535ed5 100644
--- 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
@@ -22,6 +22,7 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
+import java.util.Map.Entry;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
 
@@ -116,6 +117,8 @@ 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.ApplicationMasterNotRegisteredException;
+import org.apache.hadoop.yarn.exceptions.InvalidApplicationMasterRequestException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
 import org.apache.hadoop.yarn.server.api.ResourceManagerAdministrationProtocol;
@@ -145,11 +148,11 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.ReplaceLabelsOnNodeRequ
 import org.apache.hadoop.yarn.server.api.protocolrecords.ReplaceLabelsOnNodeResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.UpdateNodeResourceRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.UpdateNodeResourceResponse;
+import org.apache.hadoop.yarn.server.utils.AMRMClientUtils;
 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;
 
 /**
@@ -171,12 +174,25 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
   private AtomicInteger containerIndex = new AtomicInteger(0);
   private Configuration conf;
 
+  private boolean shouldReRegisterNext = false;
+
+  // For unit test synchronization
+  private static Object syncObj = new Object();
+
+  public static Object getSyncObj() {
+    return syncObj;
+  }
+
   public MockResourceManagerFacade(Configuration conf,
       int startContainerIndex) {
     this.conf = conf;
     this.containerIndex.set(startContainerIndex);
   }
 
+  public void setShouldReRegisterNext() {
+    shouldReRegisterNext = true;
+  }
+
   private static String getAppIdentifier() throws IOException {
     AMRMTokenIdentifier result = null;
     UserGroupInformation remoteUgi = UserGroupInformation.getCurrentUser();
@@ -197,14 +213,31 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
     String amrmToken = getAppIdentifier();
     LOG.info("Registering application attempt: " + amrmToken);
 
+    shouldReRegisterNext = false;
+
+    synchronized (syncObj) {
+      syncObj.notifyAll();
+      // We reuse the port number to indicate whether the unit test want us to
+      // wait here
+      if (request.getRpcPort() > 1000) {
+        LOG.info("Register call in RM start waiting");
+        try {
+          syncObj.wait();
+          LOG.info("Register call in RM wait finished");
+        } catch (InterruptedException e) {
+          LOG.info("Register call in RM wait interrupted", e);
+        }
+      }
+    }
+
     synchronized (applicationContainerIdMap) {
-      Assert.assertFalse(
-          "The application id is already registered: " + amrmToken,
-          applicationContainerIdMap.containsKey(amrmToken));
+      if (applicationContainerIdMap.containsKey(amrmToken)) {
+        throw new InvalidApplicationMasterRequestException(
+            AMRMClientUtils.APP_ALREADY_REGISTERED_MESSAGE);
+      }
       // 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);
   }
@@ -216,6 +249,12 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
     String amrmToken = getAppIdentifier();
     LOG.info("Finishing application attempt: " + amrmToken);
 
+    if (shouldReRegisterNext) {
+      String message = "AM is not registered, should re-register.";
+      LOG.warn(message);
+      throw new ApplicationMasterNotRegisteredException(message);
+    }
+
     synchronized (applicationContainerIdMap) {
       // Remove the containers that were being tracked for this application
       Assert.assertTrue("The application id is NOT registered: " + amrmToken,
@@ -251,6 +290,13 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
     }
 
     String amrmToken = getAppIdentifier();
+    LOG.info("Allocate from application attempt: " + amrmToken);
+
+    if (shouldReRegisterNext) {
+      String message = "AM is not registered, should re-register.";
+      LOG.warn(message);
+      throw new ApplicationMasterNotRegisteredException(message);
+    }
 
     ArrayList<Container> containerList = new ArrayList<Container>();
     if (request.getAskList() != null) {
@@ -384,6 +430,33 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
   @Override
   public KillApplicationResponse forceKillApplication(
       KillApplicationRequest request) throws YarnException, IOException {
+    String appId = "";
+    boolean foundApp = false;
+    if (request.getApplicationId() != null) {
+      appId = request.getApplicationId().toString();
+      synchronized (applicationContainerIdMap) {
+        for (Entry<String, List<ContainerId>> entry : applicationContainerIdMap
+            .entrySet()) {
+          ApplicationAttemptId attemptId =
+              ApplicationAttemptId.fromString(entry.getKey());
+          if (attemptId.getApplicationId().equals(request.getApplicationId())) {
+            // Remove the apptempt and the containers that were being tracked
+            List<ContainerId> ids =
+                applicationContainerIdMap.remove(entry.getKey());
+            if (ids != null) {
+              for (ContainerId c : ids) {
+                allocatedContainerMap.remove(c);
+              }
+            }
+            foundApp = true;
+          }
+        }
+      }
+    }
+    if (!foundApp) {
+      throw new YarnException("The application id is NOT registered: " + appId);
+    }
+    LOG.info("Force killing application: " + appId);
     return KillApplicationResponse.newInstance(true);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c73abec4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/uam/TestUnmanagedApplicationManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/uam/TestUnmanagedApplicationManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/uam/TestUnmanagedApplicationManager.java
new file mode 100644
index 0000000..9159cf7
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/uam/TestUnmanagedApplicationManager.java
@@ -0,0 +1,335 @@
+/**
+ * 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.uam;
+
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+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.security.AMRMTokenIdentifier;
+import org.apache.hadoop.yarn.server.MockResourceManagerFacade;
+import org.apache.hadoop.yarn.util.AsyncCallback;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Unit test for UnmanagedApplicationManager.
+ */
+public class TestUnmanagedApplicationManager {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestUnmanagedApplicationManager.class);
+
+  private TestableUnmanagedApplicationManager uam;
+  private Configuration conf = new YarnConfiguration();
+  private CountingCallback callback;
+
+  private ApplicationAttemptId attemptId;
+
+  @Before
+  public void setup() {
+    conf.set(YarnConfiguration.RM_CLUSTER_ID, "subclusterId");
+    callback = new CountingCallback();
+
+    attemptId =
+        ApplicationAttemptId.newInstance(ApplicationId.newInstance(0, 1), 1);
+
+    uam = new TestableUnmanagedApplicationManager(conf,
+        attemptId.getApplicationId(), null, "submitter", "appNameSuffix");
+  }
+
+  protected void waitForCallBackCountAndCheckZeroPending(
+      CountingCallback callBack, int expectCallBackCount) {
+    synchronized (callBack) {
+      while (callBack.callBackCount != expectCallBackCount) {
+        try {
+          callBack.wait();
+        } catch (InterruptedException e) {
+        }
+      }
+      Assert.assertEquals(
+          "Non zero pending requests when number of allocate callbacks reaches "
+              + expectCallBackCount,
+          0, callBack.requestQueueSize);
+    }
+  }
+
+  @Test(timeout = 5000)
+  public void testBasicUsage()
+      throws YarnException, IOException, InterruptedException {
+
+    createAndRegisterApplicationMaster(
+        RegisterApplicationMasterRequest.newInstance(null, 0, null), attemptId);
+
+    allocateAsync(AllocateRequest.newInstance(0, 0, null, null, null), callback,
+        attemptId);
+
+    // Wait for outstanding async allocate callback
+    waitForCallBackCountAndCheckZeroPending(callback, 1);
+
+    finishApplicationMaster(
+        FinishApplicationMasterRequest.newInstance(null, null, null),
+        attemptId);
+  }
+
+  @Test(timeout = 5000)
+  public void testReRegister()
+      throws YarnException, IOException, InterruptedException {
+
+    createAndRegisterApplicationMaster(
+        RegisterApplicationMasterRequest.newInstance(null, 0, null), attemptId);
+
+    uam.setShouldReRegisterNext();
+
+    allocateAsync(AllocateRequest.newInstance(0, 0, null, null, null), callback,
+        attemptId);
+
+    // Wait for outstanding async allocate callback
+    waitForCallBackCountAndCheckZeroPending(callback, 1);
+
+    uam.setShouldReRegisterNext();
+
+    finishApplicationMaster(
+        FinishApplicationMasterRequest.newInstance(null, null, null),
+        attemptId);
+  }
+
+  /**
+   * If register is slow, async allocate requests in the meanwhile should not
+   * throw or be dropped.
+   */
+  @Test(timeout = 5000)
+  public void testSlowRegisterCall()
+      throws YarnException, IOException, InterruptedException {
+
+    // Register with wait() in RM in a separate thread
+    Thread registerAMThread = new Thread(new Runnable() {
+      @Override
+      public void run() {
+        try {
+          createAndRegisterApplicationMaster(
+              RegisterApplicationMasterRequest.newInstance(null, 1001, null),
+              attemptId);
+        } catch (Exception e) {
+          LOG.info("Register thread exception", e);
+        }
+      }
+    });
+
+    // Sync obj from mock RM
+    Object syncObj = MockResourceManagerFacade.getSyncObj();
+
+    // Wait for register call in the thread get into RM and then wake us
+    synchronized (syncObj) {
+      LOG.info("Starting register thread");
+      registerAMThread.start();
+      try {
+        LOG.info("Test main starts waiting");
+        syncObj.wait();
+        LOG.info("Test main wait finished");
+      } catch (Exception e) {
+        LOG.info("Test main wait interrupted", e);
+      }
+    }
+
+    // First allocate before register succeeds
+    allocateAsync(AllocateRequest.newInstance(0, 0, null, null, null), callback,
+        attemptId);
+
+    // Notify the register thread
+    synchronized (syncObj) {
+      syncObj.notifyAll();
+    }
+
+    LOG.info("Test main wait for register thread to finish");
+    registerAMThread.join();
+    LOG.info("Register thread finished");
+
+    // Second allocate, normal case
+    allocateAsync(AllocateRequest.newInstance(0, 0, null, null, null), callback,
+        attemptId);
+
+    // Both allocate before should respond
+    waitForCallBackCountAndCheckZeroPending(callback, 2);
+
+    finishApplicationMaster(
+        FinishApplicationMasterRequest.newInstance(null, null, null),
+        attemptId);
+
+    // Allocates after finishAM should be ignored
+    allocateAsync(AllocateRequest.newInstance(0, 0, null, null, null), callback,
+        attemptId);
+    allocateAsync(AllocateRequest.newInstance(0, 0, null, null, null), callback,
+        attemptId);
+
+    Assert.assertEquals(0, callback.requestQueueSize);
+
+    // A short wait just in case the allocates get executed
+    try {
+      Thread.sleep(100);
+    } catch (InterruptedException e) {
+    }
+
+    Assert.assertEquals(2, callback.callBackCount);
+  }
+
+  @Test(expected = Exception.class)
+  public void testAllocateWithoutRegister()
+      throws YarnException, IOException, InterruptedException {
+    allocateAsync(AllocateRequest.newInstance(0, 0, null, null, null), callback,
+        attemptId);
+  }
+
+  @Test(expected = Exception.class)
+  public void testFinishWithoutRegister()
+      throws YarnException, IOException, InterruptedException {
+    finishApplicationMaster(
+        FinishApplicationMasterRequest.newInstance(null, null, null),
+        attemptId);
+  }
+
+  @Test
+  public void testForceKill()
+      throws YarnException, IOException, InterruptedException {
+    createAndRegisterApplicationMaster(
+        RegisterApplicationMasterRequest.newInstance(null, 0, null), attemptId);
+    uam.forceKillApplication();
+
+    try {
+      uam.forceKillApplication();
+      Assert.fail("Should fail because application is already killed");
+    } catch (YarnException t) {
+    }
+  }
+
+  protected UserGroupInformation getUGIWithToken(
+      ApplicationAttemptId appAttemptId) {
+    UserGroupInformation ugi =
+        UserGroupInformation.createRemoteUser(appAttemptId.toString());
+    AMRMTokenIdentifier token = new AMRMTokenIdentifier(appAttemptId, 1);
+    ugi.addTokenIdentifier(token);
+    return ugi;
+  }
+
+  protected RegisterApplicationMasterResponse
+      createAndRegisterApplicationMaster(
+          final RegisterApplicationMasterRequest request,
+          ApplicationAttemptId appAttemptId)
+          throws YarnException, IOException, InterruptedException {
+    return getUGIWithToken(appAttemptId).doAs(
+        new PrivilegedExceptionAction<RegisterApplicationMasterResponse>() {
+          @Override
+          public RegisterApplicationMasterResponse run()
+              throws YarnException, IOException {
+            RegisterApplicationMasterResponse response =
+                uam.createAndRegisterApplicationMaster(request);
+            return response;
+          }
+        });
+  }
+
+  protected void allocateAsync(final AllocateRequest request,
+      final AsyncCallback<AllocateResponse> callBack,
+      ApplicationAttemptId appAttemptId)
+      throws YarnException, IOException, InterruptedException {
+    getUGIWithToken(appAttemptId).doAs(new PrivilegedExceptionAction<Object>() {
+      @Override
+      public Object run() throws YarnException {
+        uam.allocateAsync(request, callBack);
+        return null;
+      }
+    });
+  }
+
+  protected FinishApplicationMasterResponse finishApplicationMaster(
+      final FinishApplicationMasterRequest request,
+      ApplicationAttemptId appAttemptId)
+      throws YarnException, IOException, InterruptedException {
+    return getUGIWithToken(appAttemptId)
+        .doAs(new PrivilegedExceptionAction<FinishApplicationMasterResponse>() {
+          @Override
+          public FinishApplicationMasterResponse run()
+              throws YarnException, IOException {
+            FinishApplicationMasterResponse response =
+                uam.finishApplicationMaster(request);
+            return response;
+          }
+        });
+  }
+
+  protected class CountingCallback implements AsyncCallback<AllocateResponse> {
+    private int callBackCount;
+    private int requestQueueSize;
+
+    @Override
+    public void callback(AllocateResponse response) {
+      synchronized (this) {
+        callBackCount++;
+        requestQueueSize = uam.getRequestQueueSize();
+        this.notifyAll();
+      }
+    }
+  }
+
+  /**
+   * Testable UnmanagedApplicationManager that talks to a mock RM.
+   */
+  public static class TestableUnmanagedApplicationManager
+      extends UnmanagedApplicationManager {
+
+    private MockResourceManagerFacade rmProxy;
+
+    public TestableUnmanagedApplicationManager(Configuration conf,
+        ApplicationId appId, String queueName, String submitter,
+        String appNameSuffix) {
+      super(conf, appId, queueName, submitter, appNameSuffix);
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    protected <T> T createRMProxy(final Class<T> protocol, Configuration config,
+        UserGroupInformation user, Token<AMRMTokenIdentifier> token) {
+      if (rmProxy == null) {
+        rmProxy = new MockResourceManagerFacade(config, 0);
+      }
+      return (T) rmProxy;
+    }
+
+    public void setShouldReRegisterNext() {
+      if (rmProxy != null) {
+        rmProxy.setShouldReRegisterNext();
+      }
+    }
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c73abec4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/DefaultRequestInterceptor.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/DefaultRequestInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/DefaultRequestInterceptor.java
index 22fc8f6..3ba4d20 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/DefaultRequestInterceptor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/DefaultRequestInterceptor.java
@@ -36,7 +36,6 @@ import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest
 import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
-import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.client.ClientRMProxy;
 import org.apache.hadoop.yarn.conf.HAUtil;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
@@ -48,6 +47,7 @@ import org.apache.hadoop.yarn.server.api.ServerRMProxy;
 import org.apache.hadoop.yarn.server.api.protocolrecords.DistributedSchedulingAllocateRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.DistributedSchedulingAllocateResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterDistributedSchedulingAMResponse;
+import org.apache.hadoop.yarn.server.utils.YarnServerSecurityUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -134,7 +134,8 @@ public final class DefaultRequestInterceptor extends
     }
     AllocateResponse allocateResponse = rmClient.allocate(request);
     if (allocateResponse.getAMRMToken() != null) {
-      updateAMRMToken(allocateResponse.getAMRMToken());
+      YarnServerSecurityUtils.updateAMRMToken(allocateResponse.getAMRMToken(),
+          this.user, getConf());
     }
 
     return allocateResponse;
@@ -170,7 +171,9 @@ public final class DefaultRequestInterceptor extends
           ((DistributedSchedulingAMProtocol)rmClient)
               .allocateForDistributedScheduling(request);
       if (allocateResponse.getAllocateResponse().getAMRMToken() != null) {
-        updateAMRMToken(allocateResponse.getAllocateResponse().getAMRMToken());
+        YarnServerSecurityUtils.updateAMRMToken(
+            allocateResponse.getAllocateResponse().getAMRMToken(), this.user,
+            getConf());
       }
       return allocateResponse;
     } else {
@@ -195,18 +198,6 @@ public final class DefaultRequestInterceptor extends
             + "Check if the interceptor pipeline configuration is correct");
   }
 
-  private void updateAMRMToken(Token token) throws IOException {
-    org.apache.hadoop.security.token.Token<AMRMTokenIdentifier> amrmToken =
-        new org.apache.hadoop.security.token.Token<AMRMTokenIdentifier>(
-            token.getIdentifier().array(), token.getPassword().array(),
-            new Text(token.getKind()), new Text(token.getService()));
-    // Preserve the token service sent by the RM when adding the token
-    // to ensure we replace the previous token setup by the RM.
-    // Afterwards we can update the service address for the RPC layer.
-    user.addToken(amrmToken);
-    amrmToken.setService(ClientRMProxy.getAMRMTokenService(getConf()));
-  }
-
   @VisibleForTesting
   public void setRMClient(final ApplicationMasterProtocol rmClient) {
     if (rmClient instanceof DistributedSchedulingAMProtocol) {
@@ -257,19 +248,12 @@ public final class DefaultRequestInterceptor extends
     for (org.apache.hadoop.security.token.Token<? extends TokenIdentifier> token : UserGroupInformation
         .getCurrentUser().getTokens()) {
       if (token.getKind().equals(AMRMTokenIdentifier.KIND_NAME)) {
-        token.setService(getAMRMTokenService(conf));
+        token.setService(ClientRMProxy.getAMRMTokenService(conf));
       }
     }
   }
 
   @InterfaceStability.Unstable
-  public static Text getAMRMTokenService(Configuration conf) {
-    return getTokenService(conf, YarnConfiguration.RM_SCHEDULER_ADDRESS,
-        YarnConfiguration.DEFAULT_RM_SCHEDULER_ADDRESS,
-        YarnConfiguration.DEFAULT_RM_SCHEDULER_PORT);
-  }
-
-  @InterfaceStability.Unstable
   public static Text getTokenService(Configuration conf, String address,
       String defaultAddr, int defaultPort) {
     if (HAUtil.isHAEnabled(conf)) {


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


[22/50] [abbrv] 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/bff50f89
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/bff50f89
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/bff50f89

Branch: refs/heads/YARN-2915
Commit: bff50f896ee17509768d675d6ea281bd982b79f1
Parents: 8b61514
Author: Subru Krishnan <su...@apache.org>
Authored: Tue Nov 1 19:54:18 2016 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Tue Jul 25 16:56:32 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/bff50f89/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/bff50f89/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/bff50f89/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/bff50f89/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/bff50f89/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/bff50f89/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/bff50f89/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/bff50f89/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/bff50f89/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/bff50f89/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


[09/50] [abbrv] hadoop git commit: YARN-5413. Create a proxy chain for ResourceManager Admin API in the Router. (Giovanni Matteo Fumarola via Subru).

Posted by su...@apache.org.
YARN-5413. Create a proxy chain for ResourceManager Admin API 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/f4b45eb6
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/f4b45eb6
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/f4b45eb6

Branch: refs/heads/YARN-2915
Commit: f4b45eb6625c37af298f94a545ee08561c4a119c
Parents: 469a5c9
Author: Subru Krishnan <su...@apache.org>
Authored: Tue May 9 19:19:27 2017 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Tue Jul 25 16:56:32 2017 -0700

----------------------------------------------------------------------
 .../hadoop/yarn/conf/YarnConfiguration.java     |  22 +-
 .../src/main/resources/yarn-default.xml         |  25 +-
 .../hadoop/yarn/util/TestLRUCacheHashMap.java   |   2 +-
 .../yarn/server/MockResourceManagerFacade.java  | 120 +++++-
 .../hadoop/yarn/server/router/Router.java       |  10 +
 .../AbstractClientRequestInterceptor.java       |  11 +-
 .../DefaultClientRequestInterceptor.java        |   2 +-
 .../router/clientrm/RouterClientRMService.java  |  16 +-
 .../AbstractRMAdminRequestInterceptor.java      |  90 ++++
 .../DefaultRMAdminRequestInterceptor.java       | 215 ++++++++++
 .../rmadmin/RMAdminRequestInterceptor.java      |  65 +++
 .../router/rmadmin/RouterRMAdminService.java    | 423 +++++++++++++++++++
 .../server/router/rmadmin/package-info.java     |  20 +
 .../router/clientrm/BaseRouterClientRMTest.java |   2 +-
 .../router/rmadmin/BaseRouterRMAdminTest.java   | 346 +++++++++++++++
 .../rmadmin/MockRMAdminRequestInterceptor.java  |  36 ++
 .../PassThroughRMAdminRequestInterceptor.java   | 148 +++++++
 .../rmadmin/TestRouterRMAdminService.java       | 219 ++++++++++
 18 files changed, 1750 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f4b45eb6/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 cf9c237..1432867 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
@@ -2639,6 +2639,8 @@ public class YarnConfiguration extends Configuration {
 
   public static final String ROUTER_PREFIX = YARN_PREFIX + "router.";
 
+  public static final String ROUTER_BIND_HOST = ROUTER_PREFIX + "bind-host";
+
   public static final String ROUTER_CLIENTRM_PREFIX =
       ROUTER_PREFIX + "clientrm.";
 
@@ -2654,9 +2656,23 @@ public class YarnConfiguration extends Configuration {
       "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;
+  public static final String ROUTER_PIPELINE_CACHE_MAX_SIZE =
+      ROUTER_PREFIX + "pipeline.cache-max-size";
+  public static final int DEFAULT_ROUTER_PIPELINE_CACHE_MAX_SIZE = 25;
+
+  public static final String ROUTER_RMADMIN_PREFIX = ROUTER_PREFIX + "rmadmin.";
+
+  public static final String ROUTER_RMADMIN_ADDRESS =
+      ROUTER_RMADMIN_PREFIX + ".address";
+  public static final int DEFAULT_ROUTER_RMADMIN_PORT = 8052;
+  public static final String DEFAULT_ROUTER_RMADMIN_ADDRESS =
+      "0.0.0.0:" + DEFAULT_ROUTER_RMADMIN_PORT;
+
+  public static final String ROUTER_RMADMIN_INTERCEPTOR_CLASS_PIPELINE =
+      ROUTER_RMADMIN_PREFIX + "interceptor-class.pipeline";
+  public static final String DEFAULT_ROUTER_RMADMIN_INTERCEPTOR_CLASS =
+      "org.apache.hadoop.yarn.server.router.rmadmin."
+          + "DefaultRMAdminRequestInterceptor";
 
   ////////////////////////////////
   // Other Configs

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f4b45eb6/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 94dccd1..8219325 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
@@ -3177,14 +3177,35 @@
 
   <property>
     <description>
-      Size of LRU cache for Router ClientRM Service.
+      Size of LRU cache for Router ClientRM Service and RMAdmin Service.
     </description>
-    <name>yarn.router.clientrm.cache-max-size</name>
+    <name>yarn.router.pipeline.cache-max-size</name>
     <value>25</value>
   </property>
 
   <property>
     <description>
+      The comma separated list of class names that implement the
+      RequestInterceptor interface. This is used by the RouterRMAdminService
+      to create the request processing pipeline for users.
+    </description>
+    <name>yarn.router.rmadmin.interceptor-class.pipeline</name>
+    <value>org.apache.hadoop.yarn.server.router.rmadmin.DefaultRMAdminRequestInterceptor</value>
+  </property>
+
+  <property>
+    <description>
+      The actual address the server will bind to. If this optional address is
+      set, the RPC and webapp servers will bind to this address and the port specified in
+      yarn.router.address and yarn.router.webapp.address, respectively. This is
+      most useful for making Router listen to all interfaces by setting to 0.0.0.0.
+    </description>
+    <name>yarn.router.bind-host</name>
+    <value></value>
+  </property>
+
+  <property>
+    <description>
       Comma-separated list of PlacementRules to determine how applications
       submitted by certain users get mapped to certain queues. Default is
       user-group, which corresponds to UserGroupMappingPlacementRule.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f4b45eb6/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
index 1cbb56c..9d3ec32 100644
--- 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
@@ -24,7 +24,7 @@ import org.junit.Assert;
 import org.junit.Test;
 
 /**
- * Test class to validate the correctness of the LRUCacheHashMap.
+ * Test class to validate the correctness of the {@code LRUCacheHashMap}.
  *
  */
 public class TestLRUCacheHashMap {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f4b45eb6/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
index e302c70..696188b 100644
--- 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
@@ -26,6 +26,7 @@ import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ipc.StandbyException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
@@ -117,6 +118,33 @@ 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.server.api.ResourceManagerAdministrationProtocol;
+import org.apache.hadoop.yarn.server.api.protocolrecords.AddToClusterNodeLabelsRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.AddToClusterNodeLabelsResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.CheckForDecommissioningNodesRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.CheckForDecommissioningNodesResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshAdminAclsRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshAdminAclsResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshClusterMaxPriorityRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshClusterMaxPriorityResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshNodesRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshNodesResourcesRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshNodesResourcesResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshNodesResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshQueuesRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshQueuesResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshServiceAclsRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshServiceAclsResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshSuperUserGroupsConfigurationRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshSuperUserGroupsConfigurationResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshUserToGroupsMappingsRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshUserToGroupsMappingsResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RemoveFromClusterNodeLabelsRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RemoveFromClusterNodeLabelsResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.ReplaceLabelsOnNodeRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.ReplaceLabelsOnNodeResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.UpdateNodeResourceRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.UpdateNodeResourceResponse;
 import org.apache.hadoop.yarn.util.Records;
 import org.junit.Assert;
 import org.slf4j.Logger;
@@ -130,8 +158,8 @@ import com.google.common.base.Strings;
  * implementation is expected by the Router/AMRMProxy unit test cases. So please
  * change the implementation with care.
  */
-public class MockResourceManagerFacade
-    implements ApplicationClientProtocol, ApplicationMasterProtocol {
+public class MockResourceManagerFacade implements ApplicationClientProtocol,
+    ApplicationMasterProtocol, ResourceManagerAdministrationProtocol {
 
   private static final Logger LOG =
       LoggerFactory.getLogger(MockResourceManagerFacade.class);
@@ -508,4 +536,92 @@ public class MockResourceManagerFacade
       throws YarnException, IOException {
     return UpdateApplicationTimeoutsResponse.newInstance();
   }
+
+  @Override
+  public RefreshQueuesResponse refreshQueues(RefreshQueuesRequest request)
+      throws StandbyException, YarnException, IOException {
+    return RefreshQueuesResponse.newInstance();
+  }
+
+  @Override
+  public RefreshNodesResponse refreshNodes(RefreshNodesRequest request)
+      throws StandbyException, YarnException, IOException {
+    return RefreshNodesResponse.newInstance();
+  }
+
+  @Override
+  public RefreshSuperUserGroupsConfigurationResponse refreshSuperUserGroupsConfiguration(
+      RefreshSuperUserGroupsConfigurationRequest request)
+      throws StandbyException, YarnException, IOException {
+    return RefreshSuperUserGroupsConfigurationResponse.newInstance();
+  }
+
+  @Override
+  public RefreshUserToGroupsMappingsResponse refreshUserToGroupsMappings(
+      RefreshUserToGroupsMappingsRequest request)
+      throws StandbyException, YarnException, IOException {
+    return RefreshUserToGroupsMappingsResponse.newInstance();
+  }
+
+  @Override
+  public RefreshAdminAclsResponse refreshAdminAcls(
+      RefreshAdminAclsRequest request) throws YarnException, IOException {
+    return RefreshAdminAclsResponse.newInstance();
+  }
+
+  @Override
+  public RefreshServiceAclsResponse refreshServiceAcls(
+      RefreshServiceAclsRequest request) throws YarnException, IOException {
+    return RefreshServiceAclsResponse.newInstance();
+  }
+
+  @Override
+  public UpdateNodeResourceResponse updateNodeResource(
+      UpdateNodeResourceRequest request) throws YarnException, IOException {
+    return UpdateNodeResourceResponse.newInstance();
+  }
+
+  @Override
+  public RefreshNodesResourcesResponse refreshNodesResources(
+      RefreshNodesResourcesRequest request) throws YarnException, IOException {
+    return RefreshNodesResourcesResponse.newInstance();
+  }
+
+  @Override
+  public AddToClusterNodeLabelsResponse addToClusterNodeLabels(
+      AddToClusterNodeLabelsRequest request) throws YarnException, IOException {
+    return AddToClusterNodeLabelsResponse.newInstance();
+  }
+
+  @Override
+  public RemoveFromClusterNodeLabelsResponse removeFromClusterNodeLabels(
+      RemoveFromClusterNodeLabelsRequest request)
+      throws YarnException, IOException {
+    return RemoveFromClusterNodeLabelsResponse.newInstance();
+  }
+
+  @Override
+  public ReplaceLabelsOnNodeResponse replaceLabelsOnNode(
+      ReplaceLabelsOnNodeRequest request) throws YarnException, IOException {
+    return ReplaceLabelsOnNodeResponse.newInstance();
+  }
+
+  @Override
+  public CheckForDecommissioningNodesResponse checkForDecommissioningNodes(
+      CheckForDecommissioningNodesRequest checkForDecommissioningNodesRequest)
+      throws YarnException, IOException {
+    return CheckForDecommissioningNodesResponse.newInstance(null);
+  }
+
+  @Override
+  public RefreshClusterMaxPriorityResponse refreshClusterMaxPriority(
+      RefreshClusterMaxPriorityRequest request)
+      throws YarnException, IOException {
+    return RefreshClusterMaxPriorityResponse.newInstance();
+  }
+
+  @Override
+  public String[] getGroupsForUser(String user) throws IOException {
+    return new String[0];
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f4b45eb6/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 7cfabf5..d2eee5a 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
@@ -29,6 +29,7 @@ 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.apache.hadoop.yarn.server.router.rmadmin.RouterRMAdminService;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -54,6 +55,7 @@ public class Router extends CompositeService {
   private Configuration conf;
   private AtomicBoolean isStopping = new AtomicBoolean(false);
   private RouterClientRMService clientRMProxyService;
+  private RouterRMAdminService rmAdminProxyService;
 
   /**
    * Priority of the Router shutdown hook.
@@ -71,8 +73,12 @@ public class Router extends CompositeService {
   @Override
   protected void serviceInit(Configuration config) throws Exception {
     this.conf = config;
+    // ClientRM Proxy
     clientRMProxyService = createClientRMProxyService();
     addService(clientRMProxyService);
+    // RMAdmin Proxy
+    rmAdminProxyService = createRMAdminProxyService();
+    addService(rmAdminProxyService);
     super.serviceInit(conf);
   }
 
@@ -107,6 +113,10 @@ public class Router extends CompositeService {
     return new RouterClientRMService();
   }
 
+  protected RouterRMAdminService createRMAdminProxyService() {
+    return new RouterRMAdminService();
+  }
+
   public static void main(String[] argv) {
     Configuration conf = new YarnConfiguration();
     Thread

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f4b45eb6/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
index fc6a118..5980b03 100644
--- 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
@@ -21,8 +21,9 @@ 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.
+ * Implements the {@link ClientRequestInterceptor} interface and provides common
+ * functionality which can can be used and/or extended by other concrete
+ * intercepter classes.
  *
  */
 public abstract class AbstractClientRequestInterceptor
@@ -31,7 +32,7 @@ public abstract class AbstractClientRequestInterceptor
   private ClientRequestInterceptor nextInterceptor;
 
   /**
-   * Sets the {@code RequestInterceptor} in the chain.
+   * Sets the {@link ClientRequestInterceptor} in the chain.
    */
   @Override
   public void setNextInterceptor(ClientRequestInterceptor nextInterceptor) {
@@ -59,7 +60,7 @@ public abstract class AbstractClientRequestInterceptor
   }
 
   /**
-   * Initializes the {@code ClientRequestInterceptor}.
+   * Initializes the {@link ClientRequestInterceptor}.
    */
   @Override
   public void init(String user) {
@@ -69,7 +70,7 @@ public abstract class AbstractClientRequestInterceptor
   }
 
   /**
-   * Disposes the {@code ClientRequestInterceptor}.
+   * Disposes the {@link ClientRequestInterceptor}.
    */
   @Override
   public void shutdown() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f4b45eb6/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
index 12b933b..9e2bfed 100644
--- 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
@@ -91,7 +91,7 @@ import org.slf4j.LoggerFactory;
 import com.google.common.annotations.VisibleForTesting;
 
 /**
- * Extends the AbstractRequestInterceptorClient class and provides an
+ * Extends the {@code AbstractRequestInterceptorClient} class and provides an
  * implementation that simply forwards the client requests to the cluster
  * resource manager.
  *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f4b45eb6/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
index 00016dd..fd2c610 100644
--- 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
@@ -104,10 +104,11 @@ 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
+ * to intercept and inspect {@link ApplicationClientProtocol} messages from
+ * client to the cluster resource manager. It listens
+ * {@link ApplicationClientProtocol} messages from the client and creates a
+ * request intercepting pipeline instance for each client. The pipeline is a
+ * chain of {@link ClientRequestInterceptor} instances that can inspect and
  * modify the request/response as needed. The main difference with
  * AMRMProxyService is the protocol they implement.
  */
@@ -137,13 +138,14 @@ public class RouterClientRMService extends AbstractService
     UserGroupInformation.setConfiguration(conf);
 
     this.listenerEndpoint =
-        conf.getSocketAddr(YarnConfiguration.ROUTER_CLIENTRM_ADDRESS,
+        conf.getSocketAddr(YarnConfiguration.ROUTER_BIND_HOST,
+            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);
+        conf.getInt(YarnConfiguration.ROUTER_PIPELINE_CACHE_MAX_SIZE,
+            YarnConfiguration.DEFAULT_ROUTER_PIPELINE_CACHE_MAX_SIZE);
     this.userPipelineMap = Collections.synchronizedMap(
         new LRUCacheHashMap<String, RequestInterceptorChainWrapper>(
             maxCacheSize, true));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f4b45eb6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/AbstractRMAdminRequestInterceptor.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/rmadmin/AbstractRMAdminRequestInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/AbstractRMAdminRequestInterceptor.java
new file mode 100644
index 0000000..a4972fc
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/AbstractRMAdminRequestInterceptor.java
@@ -0,0 +1,90 @@
+/**
+ * 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.rmadmin;
+
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Implements the {@link RMAdminRequestInterceptor} interface and provides
+ * common functionality which can can be used and/or extended by other concrete
+ * intercepter classes.
+ *
+ */
+public abstract class AbstractRMAdminRequestInterceptor
+    implements RMAdminRequestInterceptor {
+  private Configuration conf;
+  private RMAdminRequestInterceptor nextInterceptor;
+
+  /**
+   * Sets the {@link RMAdminRequestInterceptor} in the chain.
+   */
+  @Override
+  public void setNextInterceptor(RMAdminRequestInterceptor 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 {@link RMAdminRequestInterceptor}.
+   */
+  @Override
+  public void init(String user) {
+    if (this.nextInterceptor != null) {
+      this.nextInterceptor.init(user);
+    }
+  }
+
+  /**
+   * Disposes the {@link RMAdminRequestInterceptor}.
+   */
+  @Override
+  public void shutdown() {
+    if (this.nextInterceptor != null) {
+      this.nextInterceptor.shutdown();
+    }
+  }
+
+  /**
+   * Gets the next {@link RMAdminRequestInterceptor} in the chain.
+   */
+  @Override
+  public RMAdminRequestInterceptor getNextInterceptor() {
+    return this.nextInterceptor;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f4b45eb6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/DefaultRMAdminRequestInterceptor.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/rmadmin/DefaultRMAdminRequestInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/DefaultRMAdminRequestInterceptor.java
new file mode 100644
index 0000000..7e6a1ff
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/DefaultRMAdminRequestInterceptor.java
@@ -0,0 +1,215 @@
+/**
+ * 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.rmadmin;
+
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ipc.StandbyException;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.client.ClientRMProxy;
+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.api.protocolrecords.AddToClusterNodeLabelsRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.AddToClusterNodeLabelsResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.CheckForDecommissioningNodesRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.CheckForDecommissioningNodesResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshAdminAclsRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshAdminAclsResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshClusterMaxPriorityRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshClusterMaxPriorityResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshNodesRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshNodesResourcesRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshNodesResourcesResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshNodesResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshQueuesRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshQueuesResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshServiceAclsRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshServiceAclsResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshSuperUserGroupsConfigurationRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshSuperUserGroupsConfigurationResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshUserToGroupsMappingsRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshUserToGroupsMappingsResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RemoveFromClusterNodeLabelsRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RemoveFromClusterNodeLabelsResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.ReplaceLabelsOnNodeRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.ReplaceLabelsOnNodeResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.UpdateNodeResourceRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.UpdateNodeResourceResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Extends the {@link AbstractRMAdminRequestInterceptor} class and provides an
+ * implementation that simply forwards the client requests to the cluster
+ * resource manager.
+ *
+ */
+public class DefaultRMAdminRequestInterceptor
+    extends AbstractRMAdminRequestInterceptor {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(DefaultRMAdminRequestInterceptor.class);
+  private ResourceManagerAdministrationProtocol rmAdminProxy;
+  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();
+
+      rmAdminProxy = user.doAs(
+          new PrivilegedExceptionAction<ResourceManagerAdministrationProtocol>() {
+            @Override
+            public ResourceManagerAdministrationProtocol run()
+                throws Exception {
+              return ClientRMProxy.createRMProxy(conf,
+                  ResourceManagerAdministrationProtocol.class);
+            }
+          });
+    } catch (IOException e) {
+      String message = "Error while creating Router RMAdmin 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(RMAdminRequestInterceptor next) {
+    throw new YarnRuntimeException("setNextInterceptor is being called on "
+        + "DefaultRMAdminRequestInterceptor, which should be the last one "
+        + "in the chain. Check if the interceptor pipeline configuration "
+        + "is correct");
+  }
+
+  @VisibleForTesting
+  public void setRMAdmin(ResourceManagerAdministrationProtocol rmAdmin) {
+    this.rmAdminProxy = rmAdmin;
+
+  }
+
+  @Override
+  public RefreshQueuesResponse refreshQueues(RefreshQueuesRequest request)
+      throws StandbyException, YarnException, IOException {
+    return rmAdminProxy.refreshQueues(request);
+  }
+
+  @Override
+  public RefreshNodesResponse refreshNodes(RefreshNodesRequest request)
+      throws StandbyException, YarnException, IOException {
+    return rmAdminProxy.refreshNodes(request);
+  }
+
+  @Override
+  public RefreshSuperUserGroupsConfigurationResponse refreshSuperUserGroupsConfiguration(
+      RefreshSuperUserGroupsConfigurationRequest request)
+      throws StandbyException, YarnException, IOException {
+    return rmAdminProxy.refreshSuperUserGroupsConfiguration(request);
+  }
+
+  @Override
+  public RefreshUserToGroupsMappingsResponse refreshUserToGroupsMappings(
+      RefreshUserToGroupsMappingsRequest request)
+      throws StandbyException, YarnException, IOException {
+    return rmAdminProxy.refreshUserToGroupsMappings(request);
+  }
+
+  @Override
+  public RefreshAdminAclsResponse refreshAdminAcls(
+      RefreshAdminAclsRequest request) throws YarnException, IOException {
+    return rmAdminProxy.refreshAdminAcls(request);
+  }
+
+  @Override
+  public RefreshServiceAclsResponse refreshServiceAcls(
+      RefreshServiceAclsRequest request) throws YarnException, IOException {
+    return rmAdminProxy.refreshServiceAcls(request);
+  }
+
+  @Override
+  public UpdateNodeResourceResponse updateNodeResource(
+      UpdateNodeResourceRequest request) throws YarnException, IOException {
+    return rmAdminProxy.updateNodeResource(request);
+  }
+
+  @Override
+  public RefreshNodesResourcesResponse refreshNodesResources(
+      RefreshNodesResourcesRequest request) throws YarnException, IOException {
+    return rmAdminProxy.refreshNodesResources(request);
+  }
+
+  @Override
+  public AddToClusterNodeLabelsResponse addToClusterNodeLabels(
+      AddToClusterNodeLabelsRequest request) throws YarnException, IOException {
+    return rmAdminProxy.addToClusterNodeLabels(request);
+  }
+
+  @Override
+  public RemoveFromClusterNodeLabelsResponse removeFromClusterNodeLabels(
+      RemoveFromClusterNodeLabelsRequest request)
+      throws YarnException, IOException {
+    return rmAdminProxy.removeFromClusterNodeLabels(request);
+  }
+
+  @Override
+  public ReplaceLabelsOnNodeResponse replaceLabelsOnNode(
+      ReplaceLabelsOnNodeRequest request) throws YarnException, IOException {
+    return rmAdminProxy.replaceLabelsOnNode(request);
+  }
+
+  @Override
+  public CheckForDecommissioningNodesResponse checkForDecommissioningNodes(
+      CheckForDecommissioningNodesRequest checkForDecommissioningNodesRequest)
+      throws YarnException, IOException {
+    return rmAdminProxy
+        .checkForDecommissioningNodes(checkForDecommissioningNodesRequest);
+  }
+
+  @Override
+  public RefreshClusterMaxPriorityResponse refreshClusterMaxPriority(
+      RefreshClusterMaxPriorityRequest request)
+      throws YarnException, IOException {
+    return rmAdminProxy.refreshClusterMaxPriority(request);
+  }
+
+  @Override
+  public String[] getGroupsForUser(String userName) throws IOException {
+    return rmAdminProxy.getGroupsForUser(userName);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f4b45eb6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/RMAdminRequestInterceptor.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/rmadmin/RMAdminRequestInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/RMAdminRequestInterceptor.java
new file mode 100644
index 0000000..dc4bda0
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/RMAdminRequestInterceptor.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.rmadmin;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.yarn.server.api.ResourceManagerAdministrationProtocol;
+
+/**
+ * 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 RMAdminRequestInterceptor
+    extends ResourceManagerAdministrationProtocol, 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 RMAdminRequestInterceptor to set in the pipeline
+   */
+  void setNextInterceptor(RMAdminRequestInterceptor nextInterceptor);
+
+  /**
+   * Returns the next intercepter in the chain.
+   *
+   * @return the next intercepter in the chain
+   */
+  RMAdminRequestInterceptor getNextInterceptor();
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f4b45eb6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/RouterRMAdminService.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/rmadmin/RouterRMAdminService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/RouterRMAdminService.java
new file mode 100644
index 0000000..b8b7ad8
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/RouterRMAdminService.java
@@ -0,0 +1,423 @@
+/**
+ * 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.rmadmin;
+
+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.ipc.StandbyException;
+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.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.server.api.ResourceManagerAdministrationProtocol;
+import org.apache.hadoop.yarn.server.api.protocolrecords.AddToClusterNodeLabelsRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.AddToClusterNodeLabelsResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.CheckForDecommissioningNodesRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.CheckForDecommissioningNodesResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshAdminAclsRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshAdminAclsResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshClusterMaxPriorityRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshClusterMaxPriorityResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshNodesRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshNodesResourcesRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshNodesResourcesResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshNodesResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshQueuesRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshQueuesResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshServiceAclsRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshServiceAclsResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshSuperUserGroupsConfigurationRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshSuperUserGroupsConfigurationResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshUserToGroupsMappingsRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshUserToGroupsMappingsResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RemoveFromClusterNodeLabelsRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RemoveFromClusterNodeLabelsResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.ReplaceLabelsOnNodeRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.ReplaceLabelsOnNodeResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.UpdateNodeResourceRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.UpdateNodeResourceResponse;
+import org.apache.hadoop.yarn.util.LRUCacheHashMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * RouterRMAdminService is a service that runs on each router that can be used
+ * to intercept and inspect {@code ResourceManagerAdministrationProtocol}
+ * messages from client to the cluster resource manager. It listens
+ * {@code ResourceManagerAdministrationProtocol} 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 RouterRMAdminService extends AbstractService
+    implements ResourceManagerAdministrationProtocol {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RouterRMAdminService.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 RouterRMAdminService() {
+    super(RouterRMAdminService.class.getName());
+  }
+
+  @Override
+  protected void serviceStart() throws Exception {
+    LOG.info("Starting Router RMAdmin Service");
+    Configuration conf = getConfig();
+    YarnRPC rpc = YarnRPC.create(conf);
+    UserGroupInformation.setConfiguration(conf);
+
+    this.listenerEndpoint =
+        conf.getSocketAddr(YarnConfiguration.ROUTER_BIND_HOST,
+            YarnConfiguration.ROUTER_RMADMIN_ADDRESS,
+            YarnConfiguration.DEFAULT_ROUTER_RMADMIN_ADDRESS,
+            YarnConfiguration.DEFAULT_ROUTER_RMADMIN_PORT);
+
+    int maxCacheSize =
+        conf.getInt(YarnConfiguration.ROUTER_PIPELINE_CACHE_MAX_SIZE,
+            YarnConfiguration.DEFAULT_ROUTER_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_ADMIN_CLIENT_THREAD_COUNT,
+            YarnConfiguration.DEFAULT_RM_ADMIN_CLIENT_THREAD_COUNT);
+
+    this.server = rpc.getServer(ResourceManagerAdministrationProtocol.class,
+        this, listenerEndpoint, serverConf, null, numWorkerThreads);
+
+    this.server.start();
+    LOG.info("Router RMAdminService listening on address: "
+        + this.server.getListenerAddress());
+    super.serviceStart();
+  }
+
+  @Override
+  protected void serviceStop() throws Exception {
+    LOG.info("Stopping Router RMAdminService");
+    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_RMADMIN_INTERCEPTOR_CLASS_PIPELINE,
+            YarnConfiguration.DEFAULT_ROUTER_RMADMIN_INTERCEPTOR_CLASS);
+
+    List<String> interceptorClassNames = new ArrayList<String>();
+    Collection<String> tempList =
+        StringUtils.getStringCollection(configuredInterceptorClassNames);
+    for (String item : tempList) {
+      interceptorClassNames.add(item.trim());
+    }
+
+    return interceptorClassNames;
+  }
+
+  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 RMAdminRequestInterceptor createRequestInterceptorChain() {
+    Configuration conf = getConfig();
+
+    List<String> interceptorClassNames = getInterceptorClassNames(conf);
+
+    RMAdminRequestInterceptor pipeline = null;
+    RMAdminRequestInterceptor current = null;
+    for (String interceptorClassName : interceptorClassNames) {
+      try {
+        Class<?> interceptorClass = conf.getClassByName(interceptorClassName);
+        if (RMAdminRequestInterceptor.class
+            .isAssignableFrom(interceptorClass)) {
+          RMAdminRequestInterceptor interceptorInstance =
+              (RMAdminRequestInterceptor) 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 "
+                  + RMAdminRequestInterceptor.class.getCanonicalName());
+        }
+      } catch (ClassNotFoundException e) {
+        throw new YarnRuntimeException(
+            "Could not instantiate RMAdminRequestInterceptor: "
+                + 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 user.
+   *
+   * @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 the user: {}", user);
+
+    try {
+      RMAdminRequestInterceptor 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 RMAdminRequestInterceptor rootInterceptor;
+
+    /**
+     * Initializes the wrapper with the specified parameters.
+     *
+     * @param interceptor the first interceptor in the pipeline
+     */
+    public synchronized void init(RMAdminRequestInterceptor interceptor) {
+      this.rootInterceptor = interceptor;
+    }
+
+    /**
+     * Gets the root request intercepter.
+     *
+     * @return the root request intercepter
+     */
+    public synchronized RMAdminRequestInterceptor getRootInterceptor() {
+      return rootInterceptor;
+    }
+
+    /**
+     * Shutdown the chain of interceptors when the object is destroyed.
+     */
+    @Override
+    protected void finalize() {
+      rootInterceptor.shutdown();
+    }
+  }
+
+  @Override
+  public String[] getGroupsForUser(String user) throws IOException {
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getGroupsForUser(user);
+  }
+
+  @Override
+  public RefreshQueuesResponse refreshQueues(RefreshQueuesRequest request)
+      throws StandbyException, YarnException, IOException {
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().refreshQueues(request);
+
+  }
+
+  @Override
+  public RefreshNodesResponse refreshNodes(RefreshNodesRequest request)
+      throws StandbyException, YarnException, IOException {
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().refreshNodes(request);
+
+  }
+
+  @Override
+  public RefreshSuperUserGroupsConfigurationResponse refreshSuperUserGroupsConfiguration(
+      RefreshSuperUserGroupsConfigurationRequest request)
+      throws StandbyException, YarnException, IOException {
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor()
+        .refreshSuperUserGroupsConfiguration(request);
+
+  }
+
+  @Override
+  public RefreshUserToGroupsMappingsResponse refreshUserToGroupsMappings(
+      RefreshUserToGroupsMappingsRequest request)
+      throws StandbyException, YarnException, IOException {
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().refreshUserToGroupsMappings(request);
+
+  }
+
+  @Override
+  public RefreshAdminAclsResponse refreshAdminAcls(
+      RefreshAdminAclsRequest request) throws YarnException, IOException {
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().refreshAdminAcls(request);
+
+  }
+
+  @Override
+  public RefreshServiceAclsResponse refreshServiceAcls(
+      RefreshServiceAclsRequest request) throws YarnException, IOException {
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().refreshServiceAcls(request);
+
+  }
+
+  @Override
+  public UpdateNodeResourceResponse updateNodeResource(
+      UpdateNodeResourceRequest request) throws YarnException, IOException {
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().updateNodeResource(request);
+
+  }
+
+  @Override
+  public RefreshNodesResourcesResponse refreshNodesResources(
+      RefreshNodesResourcesRequest request) throws YarnException, IOException {
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().refreshNodesResources(request);
+
+  }
+
+  @Override
+  public AddToClusterNodeLabelsResponse addToClusterNodeLabels(
+      AddToClusterNodeLabelsRequest request) throws YarnException, IOException {
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().addToClusterNodeLabels(request);
+
+  }
+
+  @Override
+  public RemoveFromClusterNodeLabelsResponse removeFromClusterNodeLabels(
+      RemoveFromClusterNodeLabelsRequest request)
+      throws YarnException, IOException {
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().removeFromClusterNodeLabels(request);
+
+  }
+
+  @Override
+  public ReplaceLabelsOnNodeResponse replaceLabelsOnNode(
+      ReplaceLabelsOnNodeRequest request) throws YarnException, IOException {
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().replaceLabelsOnNode(request);
+
+  }
+
+  @Override
+  public CheckForDecommissioningNodesResponse checkForDecommissioningNodes(
+      CheckForDecommissioningNodesRequest checkForDecommissioningNodesRequest)
+      throws YarnException, IOException {
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor()
+        .checkForDecommissioningNodes(checkForDecommissioningNodesRequest);
+  }
+
+  @Override
+  public RefreshClusterMaxPriorityResponse refreshClusterMaxPriority(
+      RefreshClusterMaxPriorityRequest request)
+      throws YarnException, IOException {
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().refreshClusterMaxPriority(request);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f4b45eb6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/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/rmadmin/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/rmadmin/package-info.java
new file mode 100644
index 0000000..98a7ed0
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/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 RM Admin Proxy Service package. **/
+package org.apache.hadoop.yarn.server.router.rmadmin;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f4b45eb6/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
index a283a62..7e15084 100644
--- 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
@@ -133,7 +133,7 @@ public abstract class BaseRouterClientRMTest {
             + "," + mockPassThroughInterceptorClass + ","
             + MockClientRequestInterceptor.class.getName());
 
-    this.conf.setInt(YarnConfiguration.ROUTER_CLIENTRM_PIPELINE_CACHE_MAX_SIZE,
+    this.conf.setInt(YarnConfiguration.ROUTER_PIPELINE_CACHE_MAX_SIZE,
         TEST_MAX_CACHE_SIZE);
 
     this.dispatcher = new AsyncDispatcher();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f4b45eb6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/rmadmin/BaseRouterRMAdminTest.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/rmadmin/BaseRouterRMAdminTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/rmadmin/BaseRouterRMAdminTest.java
new file mode 100644
index 0000000..d3eba61
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/rmadmin/BaseRouterRMAdminTest.java
@@ -0,0 +1,346 @@
+/**
+ * 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.rmadmin;
+
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+import java.util.HashMap;
+import java.util.Set;
+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.records.NodeId;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.event.AsyncDispatcher;
+import org.apache.hadoop.yarn.server.api.protocolrecords.AddToClusterNodeLabelsRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.AddToClusterNodeLabelsResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.CheckForDecommissioningNodesRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.CheckForDecommissioningNodesResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshAdminAclsRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshAdminAclsResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshClusterMaxPriorityRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshClusterMaxPriorityResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshNodesRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshNodesResourcesRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshNodesResourcesResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshNodesResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshQueuesRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshQueuesResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshServiceAclsRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshServiceAclsResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshSuperUserGroupsConfigurationRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshSuperUserGroupsConfigurationResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshUserToGroupsMappingsRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshUserToGroupsMappingsResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RemoveFromClusterNodeLabelsRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RemoveFromClusterNodeLabelsResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.ReplaceLabelsOnNodeRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.ReplaceLabelsOnNodeResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.UpdateNodeResourceRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.UpdateNodeResourceResponse;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+
+/**
+ * Base class for all the RouterRMAdminService test cases. It provides utility
+ * methods that can be used by the concrete test case classes.
+ *
+ */
+public abstract class BaseRouterRMAdminTest {
+
+  /**
+   * The RouterRMAdminService instance that will be used by all the test cases.
+   */
+  private MockRouterRMAdminService rmAdminService;
+  /**
+   * 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 MockRouterRMAdminService getRouterRMAdminService() {
+    Assert.assertNotNull(this.rmAdminService);
+    return this.rmAdminService;
+  }
+
+  @Before
+  public void setUp() {
+    this.conf = new YarnConfiguration();
+    String mockPassThroughInterceptorClass =
+        PassThroughRMAdminRequestInterceptor.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_RMADMIN_INTERCEPTOR_CLASS_PIPELINE,
+        mockPassThroughInterceptorClass + "," + mockPassThroughInterceptorClass
+            + "," + mockPassThroughInterceptorClass + ","
+            + MockRMAdminRequestInterceptor.class.getName());
+
+    this.conf.setInt(YarnConfiguration.ROUTER_PIPELINE_CACHE_MAX_SIZE,
+        TEST_MAX_CACHE_SIZE);
+
+    this.dispatcher = new AsyncDispatcher();
+    this.dispatcher.init(conf);
+    this.dispatcher.start();
+    this.rmAdminService = createAndStartRouterRMAdminService();
+  }
+
+  @After
+  public void tearDown() {
+    if (rmAdminService != null) {
+      rmAdminService.stop();
+      rmAdminService = null;
+    }
+    if (this.dispatcher != null) {
+      this.dispatcher.stop();
+    }
+  }
+
+  protected ExecutorService getThreadPool() {
+    return threadpool;
+  }
+
+  protected MockRouterRMAdminService createAndStartRouterRMAdminService() {
+    MockRouterRMAdminService svc = new MockRouterRMAdminService();
+    svc.init(conf);
+    svc.start();
+    return svc;
+  }
+
+  protected static class MockRouterRMAdminService extends RouterRMAdminService {
+    public MockRouterRMAdminService() {
+      super();
+    }
+  }
+
+  protected RefreshQueuesResponse refreshQueues(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<RefreshQueuesResponse>() {
+          @Override
+          public RefreshQueuesResponse run() throws Exception {
+            RefreshQueuesRequest req = RefreshQueuesRequest.newInstance();
+            RefreshQueuesResponse response =
+                getRouterRMAdminService().refreshQueues(req);
+            return response;
+          }
+        });
+  }
+
+  protected RefreshNodesResponse refreshNodes(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<RefreshNodesResponse>() {
+          @Override
+          public RefreshNodesResponse run() throws Exception {
+            RefreshNodesRequest req = RefreshNodesRequest.newInstance();
+            RefreshNodesResponse response =
+                getRouterRMAdminService().refreshNodes(req);
+            return response;
+          }
+        });
+  }
+
+  protected RefreshSuperUserGroupsConfigurationResponse refreshSuperUserGroupsConfiguration(
+      String user) throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user).doAs(
+        new PrivilegedExceptionAction<RefreshSuperUserGroupsConfigurationResponse>() {
+          @Override
+          public RefreshSuperUserGroupsConfigurationResponse run()
+              throws Exception {
+            RefreshSuperUserGroupsConfigurationRequest req =
+                RefreshSuperUserGroupsConfigurationRequest.newInstance();
+            RefreshSuperUserGroupsConfigurationResponse response =
+                getRouterRMAdminService()
+                    .refreshSuperUserGroupsConfiguration(req);
+            return response;
+          }
+        });
+  }
+
+  protected RefreshUserToGroupsMappingsResponse refreshUserToGroupsMappings(
+      String user) throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user).doAs(
+        new PrivilegedExceptionAction<RefreshUserToGroupsMappingsResponse>() {
+          @Override
+          public RefreshUserToGroupsMappingsResponse run() throws Exception {
+            RefreshUserToGroupsMappingsRequest req =
+                RefreshUserToGroupsMappingsRequest.newInstance();
+            RefreshUserToGroupsMappingsResponse response =
+                getRouterRMAdminService().refreshUserToGroupsMappings(req);
+            return response;
+          }
+        });
+  }
+
+  protected RefreshAdminAclsResponse refreshAdminAcls(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<RefreshAdminAclsResponse>() {
+          @Override
+          public RefreshAdminAclsResponse run() throws Exception {
+            RefreshAdminAclsRequest req = RefreshAdminAclsRequest.newInstance();
+            RefreshAdminAclsResponse response =
+                getRouterRMAdminService().refreshAdminAcls(req);
+            return response;
+          }
+        });
+  }
+
+  protected RefreshServiceAclsResponse refreshServiceAcls(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<RefreshServiceAclsResponse>() {
+          @Override
+          public RefreshServiceAclsResponse run() throws Exception {
+            RefreshServiceAclsRequest req =
+                RefreshServiceAclsRequest.newInstance();
+            RefreshServiceAclsResponse response =
+                getRouterRMAdminService().refreshServiceAcls(req);
+            return response;
+          }
+        });
+  }
+
+  protected UpdateNodeResourceResponse updateNodeResource(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<UpdateNodeResourceResponse>() {
+          @Override
+          public UpdateNodeResourceResponse run() throws Exception {
+            UpdateNodeResourceRequest req =
+                UpdateNodeResourceRequest.newInstance(null);
+            UpdateNodeResourceResponse response =
+                getRouterRMAdminService().updateNodeResource(req);
+            return response;
+          }
+        });
+  }
+
+  protected RefreshNodesResourcesResponse refreshNodesResources(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<RefreshNodesResourcesResponse>() {
+          @Override
+          public RefreshNodesResourcesResponse run() throws Exception {
+            RefreshNodesResourcesRequest req =
+                RefreshNodesResourcesRequest.newInstance();
+            RefreshNodesResourcesResponse response =
+                getRouterRMAdminService().refreshNodesResources(req);
+            return response;
+          }
+        });
+  }
+
+  protected AddToClusterNodeLabelsResponse addToClusterNodeLabels(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<AddToClusterNodeLabelsResponse>() {
+          @Override
+          public AddToClusterNodeLabelsResponse run() throws Exception {
+            AddToClusterNodeLabelsRequest req =
+                AddToClusterNodeLabelsRequest.newInstance(null);
+            AddToClusterNodeLabelsResponse response =
+                getRouterRMAdminService().addToClusterNodeLabels(req);
+            return response;
+          }
+        });
+  }
+
+  protected RemoveFromClusterNodeLabelsResponse removeFromClusterNodeLabels(
+      String user) throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user).doAs(
+        new PrivilegedExceptionAction<RemoveFromClusterNodeLabelsResponse>() {
+          @Override
+          public RemoveFromClusterNodeLabelsResponse run() throws Exception {
+            RemoveFromClusterNodeLabelsRequest req =
+                RemoveFromClusterNodeLabelsRequest.newInstance(null);
+            RemoveFromClusterNodeLabelsResponse response =
+                getRouterRMAdminService().removeFromClusterNodeLabels(req);
+            return response;
+          }
+        });
+  }
+
+  protected ReplaceLabelsOnNodeResponse replaceLabelsOnNode(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<ReplaceLabelsOnNodeResponse>() {
+          @Override
+          public ReplaceLabelsOnNodeResponse run() throws Exception {
+            ReplaceLabelsOnNodeRequest req = ReplaceLabelsOnNodeRequest
+                .newInstance(new HashMap<NodeId, Set<String>>());
+            ReplaceLabelsOnNodeResponse response =
+                getRouterRMAdminService().replaceLabelsOnNode(req);
+            return response;
+          }
+        });
+  }
+
+  protected CheckForDecommissioningNodesResponse checkForDecommissioningNodes(
+      String user) throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user).doAs(
+        new PrivilegedExceptionAction<CheckForDecommissioningNodesResponse>() {
+          @Override
+          public CheckForDecommissioningNodesResponse run() throws Exception {
+            CheckForDecommissioningNodesRequest req =
+                CheckForDecommissioningNodesRequest.newInstance();
+            CheckForDecommissioningNodesResponse response =
+                getRouterRMAdminService().checkForDecommissioningNodes(req);
+            return response;
+          }
+        });
+  }
+
+  protected RefreshClusterMaxPriorityResponse refreshClusterMaxPriority(
+      String user) throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user).doAs(
+        new PrivilegedExceptionAction<RefreshClusterMaxPriorityResponse>() {
+          @Override
+          public RefreshClusterMaxPriorityResponse run() throws Exception {
+            RefreshClusterMaxPriorityRequest req =
+                RefreshClusterMaxPriorityRequest.newInstance();
+            RefreshClusterMaxPriorityResponse response =
+                getRouterRMAdminService().refreshClusterMaxPriority(req);
+            return response;
+          }
+        });
+  }
+
+  protected String[] getGroupsForUser(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<String[]>() {
+          @Override
+          public String[] run() throws Exception {
+            String[] response =
+                getRouterRMAdminService().getGroupsForUser(user);
+            return response;
+          }
+        });
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f4b45eb6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/rmadmin/MockRMAdminRequestInterceptor.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/rmadmin/MockRMAdminRequestInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/rmadmin/MockRMAdminRequestInterceptor.java
new file mode 100644
index 0000000..ab7bdb4
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/rmadmin/MockRMAdminRequestInterceptor.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.rmadmin;
+
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.MockResourceManagerFacade;
+
+/**
+ * This class mocks the RMAmdinRequestInterceptor.
+ */
+public class MockRMAdminRequestInterceptor
+    extends DefaultRMAdminRequestInterceptor {
+
+  public void init(String user) {
+    MockResourceManagerFacade mockRM = new MockResourceManagerFacade(
+        new YarnConfiguration(super.getConf()), 0);
+    super.setRMAdmin(mockRM);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f4b45eb6/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/rmadmin/PassThroughRMAdminRequestInterceptor.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/rmadmin/PassThroughRMAdminRequestInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/rmadmin/PassThroughRMAdminRequestInterceptor.java
new file mode 100644
index 0000000..38dcc3d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/rmadmin/PassThroughRMAdminRequestInterceptor.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
+ *
+ *     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.rmadmin;
+
+import java.io.IOException;
+
+import org.apache.hadoop.ipc.StandbyException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.api.protocolrecords.AddToClusterNodeLabelsRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.AddToClusterNodeLabelsResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.CheckForDecommissioningNodesRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.CheckForDecommissioningNodesResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshAdminAclsRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshAdminAclsResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshClusterMaxPriorityRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshClusterMaxPriorityResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshNodesRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshNodesResourcesRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshNodesResourcesResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshNodesResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshQueuesRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshQueuesResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshServiceAclsRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshServiceAclsResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshSuperUserGroupsConfigurationRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshSuperUserGroupsConfigurationResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshUserToGroupsMappingsRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshUserToGroupsMappingsResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RemoveFromClusterNodeLabelsRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RemoveFromClusterNodeLabelsResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.ReplaceLabelsOnNodeRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.ReplaceLabelsOnNodeResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.UpdateNodeResourceRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.UpdateNodeResourceResponse;
+
+/**
+ * Mock intercepter that does not do anything other than forwarding it to the
+ * next intercepter in the chain.
+ */
+public class PassThroughRMAdminRequestInterceptor
+    extends AbstractRMAdminRequestInterceptor {
+
+  @Override
+  public RefreshQueuesResponse refreshQueues(RefreshQueuesRequest request)
+      throws StandbyException, YarnException, IOException {
+    return getNextInterceptor().refreshQueues(request);
+  }
+
+  @Override
+  public RefreshNodesResponse refreshNodes(RefreshNodesRequest request)
+      throws StandbyException, YarnException, IOException {
+    return getNextInterceptor().refreshNodes(request);
+  }
+
+  @Override
+  public RefreshSuperUserGroupsConfigurationResponse refreshSuperUserGroupsConfiguration(
+      RefreshSuperUserGroupsConfigurationRequest request)
+      throws StandbyException, YarnException, IOException {
+    return getNextInterceptor().refreshSuperUserGroupsConfiguration(request);
+  }
+
+  @Override
+  public RefreshUserToGroupsMappingsResponse refreshUserToGroupsMappings(
+      RefreshUserToGroupsMappingsRequest request)
+      throws StandbyException, YarnException, IOException {
+    return getNextInterceptor().refreshUserToGroupsMappings(request);
+  }
+
+  @Override
+  public RefreshAdminAclsResponse refreshAdminAcls(
+      RefreshAdminAclsRequest request) throws YarnException, IOException {
+    return getNextInterceptor().refreshAdminAcls(request);
+  }
+
+  @Override
+  public RefreshServiceAclsResponse refreshServiceAcls(
+      RefreshServiceAclsRequest request) throws YarnException, IOException {
+    return getNextInterceptor().refreshServiceAcls(request);
+  }
+
+  @Override
+  public UpdateNodeResourceResponse updateNodeResource(
+      UpdateNodeResourceRequest request) throws YarnException, IOException {
+    return getNextInterceptor().updateNodeResource(request);
+  }
+
+  @Override
+  public RefreshNodesResourcesResponse refreshNodesResources(
+      RefreshNodesResourcesRequest request) throws YarnException, IOException {
+    return getNextInterceptor().refreshNodesResources(request);
+  }
+
+  @Override
+  public AddToClusterNodeLabelsResponse addToClusterNodeLabels(
+      AddToClusterNodeLabelsRequest request) throws YarnException, IOException {
+    return getNextInterceptor().addToClusterNodeLabels(request);
+  }
+
+  @Override
+  public RemoveFromClusterNodeLabelsResponse removeFromClusterNodeLabels(
+      RemoveFromClusterNodeLabelsRequest request)
+      throws YarnException, IOException {
+    return getNextInterceptor().removeFromClusterNodeLabels(request);
+  }
+
+  @Override
+  public ReplaceLabelsOnNodeResponse replaceLabelsOnNode(
+      ReplaceLabelsOnNodeRequest request) throws YarnException, IOException {
+    return getNextInterceptor().replaceLabelsOnNode(request);
+  }
+
+  @Override
+  public CheckForDecommissioningNodesResponse checkForDecommissioningNodes(
+      CheckForDecommissioningNodesRequest checkForDecommissioningNodesRequest)
+      throws YarnException, IOException {
+    return getNextInterceptor()
+        .checkForDecommissioningNodes(checkForDecommissioningNodesRequest);
+  }
+
+  @Override
+  public RefreshClusterMaxPriorityResponse refreshClusterMaxPriority(
+      RefreshClusterMaxPriorityRequest request)
+      throws YarnException, IOException {
+    return getNextInterceptor().refreshClusterMaxPriority(request);
+  }
+
+  @Override
+  public String[] getGroupsForUser(String user) throws IOException {
+    return getNextInterceptor().getGroupsForUser(user);
+  }
+
+}


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


[28/50] [abbrv] hadoop git commit: YARN-6511. Federation: transparently spanning application across multiple sub-clusters. (Botong Huang via Subru).

Posted by su...@apache.org.
YARN-6511. Federation: transparently spanning application across multiple sub-clusters. (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/2c10a9be
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/2c10a9be
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/2c10a9be

Branch: refs/heads/YARN-2915
Commit: 2c10a9bee2d42cb8f0937149d5b2ceb38b2ae149
Parents: 376dc4a
Author: Subru Krishnan <su...@apache.org>
Authored: Wed Jun 7 14:45:51 2017 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Tue Jul 25 16:56:32 2017 -0700

----------------------------------------------------------------------
 .../policies/FederationPolicyUtils.java         | 168 +++++
 .../federation/policies/RouterPolicyFacade.java |  21 +-
 .../amrmproxy/FederationInterceptor.java        | 685 ++++++++++++++++++-
 .../amrmproxy/TestFederationInterceptor.java    | 251 +++++++
 .../TestableFederationInterceptor.java          |   6 +
 5 files changed, 1095 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c10a9be/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyUtils.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/FederationPolicyUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyUtils.java
new file mode 100644
index 0000000..37ce942
--- /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/FederationPolicyUtils.java
@@ -0,0 +1,168 @@
+/**
+ * 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 org.apache.hadoop.classification.InterfaceAudience.Private;
+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.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.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyConfiguration;
+import org.apache.hadoop.yarn.server.federation.utils.FederationStateStoreFacade;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Utility class for Federation policy.
+ */
+@Private
+public final class FederationPolicyUtils {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(FederationPolicyUtils.class);
+
+  /** Disable constructor. */
+  private FederationPolicyUtils() {
+  }
+
+  /**
+   * A utilize method to instantiate a policy manager class given the type
+   * (class name) from {@link SubClusterPolicyConfiguration}.
+   *
+   * @param newType class name of the policy manager to create
+   * @return Policy manager
+   * @throws FederationPolicyInitializationException if fails
+   */
+  public 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;
+  }
+
+  /**
+   * Get Federation policy configuration from state store, using default queue
+   * and configuration as fallback.
+   *
+   * @param queue the queue of the application
+   * @param conf the Yarn configuration
+   * @param federationFacade state store facade
+   * @return SubClusterPolicyConfiguration recreated
+   */
+  public static SubClusterPolicyConfiguration loadPolicyConfiguration(
+      String queue, Configuration conf,
+      FederationStateStoreFacade federationFacade) {
+
+    // The facade might cache this request, based on its parameterization
+    SubClusterPolicyConfiguration configuration = null;
+    if (queue != null) {
+      try {
+        configuration = federationFacade.getPolicyConfiguration(queue);
+      } catch (YarnException e) {
+        LOG.warn("Failed to get policy from FederationFacade with queue "
+            + queue + ": " + e.getMessage());
+      }
+    }
+
+    // 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
+    if (configuration == null) {
+      LOG.info("No policy configured for queue {} in StateStore,"
+          + " fallback to default queue", queue);
+      queue = YarnConfiguration.DEFAULT_FEDERATION_POLICY_KEY;
+      try {
+        configuration = federationFacade.getPolicyConfiguration(queue);
+      } 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) {
+      LOG.info("No policy configured for default queue {} in StateStore,"
+          + " fallback to local config", queue);
+
+      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(queue,
+          defaultFederationPolicyManager, defaultPolicyParam);
+    }
+    return configuration;
+  }
+
+  /**
+   * Get AMRMProxy policy from state store, using default queue and
+   * configuration as fallback.
+   *
+   * @param queue the queue of the application
+   * @param oldPolicy the previous policy instance (can be null)
+   * @param conf the Yarn configuration
+   * @param federationFacade state store facade
+   * @param homeSubClusterId home sub-cluster id
+   * @return FederationAMRMProxyPolicy recreated
+   * @throws FederationPolicyInitializationException if fails
+   */
+  public static FederationAMRMProxyPolicy loadAMRMPolicy(String queue,
+      FederationAMRMProxyPolicy oldPolicy, Configuration conf,
+      FederationStateStoreFacade federationFacade,
+      SubClusterId homeSubClusterId)
+      throws FederationPolicyInitializationException {
+
+    // Local policy and its configuration
+    SubClusterPolicyConfiguration configuration =
+        loadPolicyConfiguration(queue, conf, federationFacade);
+
+    // Instantiate the policyManager and get policy
+    FederationPolicyInitializationContext context =
+        new FederationPolicyInitializationContext(configuration,
+            federationFacade.getSubClusterResolver(), federationFacade,
+            homeSubClusterId);
+
+    LOG.info("Creating policy manager of type: " + configuration.getType());
+    FederationPolicyManager federationPolicyManager =
+        instantiatePolicyManager(configuration.getType());
+    // set queue, reinit policy if required (implementation lazily check
+    // content of conf), and cache it
+    federationPolicyManager.setQueue(configuration.getQueue());
+    return federationPolicyManager.getAMRMPolicy(context, oldPolicy);
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c10a9be/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 8c22623..5e31a08 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
@@ -95,7 +95,7 @@ public class RouterPolicyFacade {
         new FederationPolicyInitializationContext(configuration,
             subClusterResolver, federationFacade, homeSubcluster);
     FederationPolicyManager fallbackPolicyManager =
-        instantiatePolicyManager(configuration.getType());
+        FederationPolicyUtils.instantiatePolicyManager(configuration.getType());
     fallbackPolicyManager.setQueue(defaulKey);
 
     // add to the cache the fallback behavior
@@ -209,7 +209,7 @@ public class RouterPolicyFacade {
     FederationRouterPolicy routerPolicy = policyMap.get(queue);
 
     FederationPolicyManager federationPolicyManager =
-        instantiatePolicyManager(newType);
+        FederationPolicyUtils.instantiatePolicyManager(newType);
     // set queue, reinit policy if required (implementation lazily check
     // content of conf), and cache it
     federationPolicyManager.setQueue(queue);
@@ -224,23 +224,6 @@ public class RouterPolicyFacade {
     }
   }
 
-  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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c10a9be/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/FederationInterceptor.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/FederationInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/FederationInterceptor.java
index 5f82d69..ffe47f4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/FederationInterceptor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/FederationInterceptor.java
@@ -24,7 +24,14 @@ import java.util.Collection;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -38,20 +45,35 @@ import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRespo
 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.NMToken;
+import org.apache.hadoop.yarn.api.records.PreemptionContract;
+import org.apache.hadoop.yarn.api.records.PreemptionMessage;
 import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.api.records.StrictPreemptionContract;
 import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.InvalidApplicationMasterRequestException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.server.federation.failover.FederationProxyProviderUtil;
+import org.apache.hadoop.yarn.server.federation.policies.FederationPolicyUtils;
+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.resolver.SubClusterResolver;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.federation.utils.FederationStateStoreFacade;
+import org.apache.hadoop.yarn.server.uam.UnmanagedAMPoolManager;
 import org.apache.hadoop.yarn.server.utils.AMRMClientUtils;
 import org.apache.hadoop.yarn.server.utils.YarnServerSecurityUtils;
+import org.apache.hadoop.yarn.util.AsyncCallback;
+import org.apache.hadoop.yarn.util.resource.Resources;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+
 /**
  * Extends the AbstractRequestInterceptor and provides an implementation for
  * federation of YARN RM and scaling an application across multiple YARN
@@ -70,6 +92,27 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
   private SubClusterId homeSubClusterId;
 
   /**
+   * UAM pool for secondary sub-clusters (ones other than home sub-cluster),
+   * using subClusterId as uamId. One UAM is created per sub-cluster RM except
+   * the home RM.
+   *
+   * Creation and register of UAM in secondary sub-clusters happen on-demand,
+   * when AMRMProxy policy routes resource request to these sub-clusters for the
+   * first time. AM heart beats to them are also handled asynchronously for
+   * performance reasons.
+   */
+  private UnmanagedAMPoolManager uamPool;
+
+  /** Thread pool used for asynchronous operations. */
+  private ExecutorService threadpool;
+
+  /**
+   * Stores the AllocateResponses that are received asynchronously from all the
+   * sub-cluster resource managers except the home RM.
+   */
+  private Map<SubClusterId, List<AllocateResponse>> asyncResponseSink;
+
+  /**
    * Used to keep track of the container Id and the sub cluster RM that created
    * the container, so that we know which sub-cluster to forward later requests
    * about existing containers to.
@@ -89,7 +132,17 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
    */
   private RegisterApplicationMasterResponse amRegistrationResponse;
 
-  /** The proxy ugi used to talk to home RM. */
+  private FederationStateStoreFacade federationFacade;
+
+  private SubClusterResolver subClusterResolver;
+
+  /** The policy used to split requests among sub-clusters. */
+  private FederationAMRMProxyPolicy policyInterpreter;
+
+  /**
+   * The proxy ugi used to talk to home RM, loaded with the up-to-date AMRMToken
+   * issued by home RM.
+   */
   private UserGroupInformation appOwner;
 
   /**
@@ -97,6 +150,10 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
    */
   public FederationInterceptor() {
     this.containerIdToSubClusterIdMap = new ConcurrentHashMap<>();
+    this.asyncResponseSink = new ConcurrentHashMap<>();
+    this.threadpool = Executors.newCachedThreadPool();
+    this.uamPool = createUnmanagedAMPoolManager(this.threadpool);
+    this.amRegistrationRequest = null;
     this.amRegistrationResponse = null;
   }
 
@@ -126,6 +183,15 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
     this.homeSubClusterId =
         SubClusterId.newInstance(YarnConfiguration.getClusterId(conf));
     this.homeRM = createHomeRMProxy(appContext);
+
+    this.federationFacade = FederationStateStoreFacade.getInstance();
+    this.subClusterResolver = this.federationFacade.getSubClusterResolver();
+
+    // AMRMProxyPolicy will be initialized in registerApplicationMaster
+    this.policyInterpreter = null;
+
+    this.uamPool.init(conf);
+    this.uamPool.start();
   }
 
   /**
@@ -202,7 +268,7 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
     if (queue == null) {
       LOG.warn("Received null queue for application "
           + getApplicationContext().getApplicationAttemptId().getApplicationId()
-          + " from home subcluster. Will use default queue name "
+          + " from home sub-cluster. Will use default queue name "
           + YarnConfiguration.DEFAULT_QUEUE_NAME
           + " for getting AMRMProxyPolicy");
     } else {
@@ -211,6 +277,14 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
           + " belongs to queue " + queue);
     }
 
+    // Initialize the AMRMProxyPolicy
+    try {
+      this.policyInterpreter =
+          FederationPolicyUtils.loadAMRMPolicy(queue, this.policyInterpreter,
+              getConf(), this.federationFacade, this.homeSubClusterId);
+    } catch (FederationPolicyInitializationException e) {
+      throw new YarnRuntimeException(e);
+    }
     return this.amRegistrationResponse;
   }
 
@@ -221,6 +295,8 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
   @Override
   public AllocateResponse allocate(AllocateRequest request)
       throws YarnException {
+    Preconditions.checkArgument(this.policyInterpreter != null,
+        "Allocate should be called after registerApplicationMaster");
 
     try {
       // Split the heart beat request into multiple requests, one for each
@@ -228,12 +304,28 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
       Map<SubClusterId, AllocateRequest> requests =
           splitAllocateRequest(request);
 
+      // Send the requests to the secondary sub-cluster resource managers.
+      // These secondary requests are send asynchronously and the responses will
+      // be collected and merged with the home response. In addition, it also
+      // return the newly registered Unmanaged AMs.
+      Registrations newRegistrations =
+          sendRequestsToSecondaryResourceManagers(requests);
+
       // Send the request to the home RM and get the response
       AllocateResponse homeResponse = AMRMClientUtils.allocateWithReRegister(
           requests.get(this.homeSubClusterId), this.homeRM,
           this.amRegistrationRequest,
           getApplicationContext().getApplicationAttemptId());
 
+      // Notify policy of home response
+      try {
+        this.policyInterpreter.notifyOfResponse(this.homeSubClusterId,
+            homeResponse);
+      } catch (YarnException e) {
+        LOG.warn("notifyOfResponse for policy failed for home sub-cluster "
+            + this.homeSubClusterId, e);
+      }
+
       // If the resource manager sent us a new token, add to the current user
       if (homeResponse.getAMRMToken() != null) {
         LOG.debug("Received new AMRMToken");
@@ -244,6 +336,13 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
       // Merge the responses from home and secondary sub-cluster RMs
       homeResponse = mergeAllocateResponses(homeResponse);
 
+      // Merge the containers and NMTokens from the new registrations into
+      // the homeResponse.
+      if (!isNullOrEmpty(newRegistrations.getSuccessfulRegistrations())) {
+        homeResponse = mergeRegistrationResponses(homeResponse,
+            newRegistrations.getSuccessfulRegistrations());
+      }
+
       // return the final response to the application master.
       return homeResponse;
     } catch (IOException ex) {
@@ -261,10 +360,83 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
       FinishApplicationMasterRequest request)
       throws YarnException, IOException {
 
+    // TODO: consider adding batchFinishApplicationMaster in UAMPoolManager
+    boolean failedToUnRegister = false;
+    ExecutorCompletionService<FinishApplicationMasterResponseInfo> compSvc =
+        null;
+
+    // Application master is completing operation. Send the finish
+    // application master request to all the registered sub-cluster resource
+    // managers in parallel, wait for the responses and aggregate the results.
+    Set<String> subClusterIds = this.uamPool.getAllUAMIds();
+    if (subClusterIds.size() > 0) {
+      final FinishApplicationMasterRequest finishRequest = request;
+      compSvc =
+          new ExecutorCompletionService<FinishApplicationMasterResponseInfo>(
+              this.threadpool);
+
+      LOG.info("Sending finish application request to {} sub-cluster RMs",
+          subClusterIds.size());
+      for (final String subClusterId : subClusterIds) {
+        compSvc.submit(new Callable<FinishApplicationMasterResponseInfo>() {
+          @Override
+          public FinishApplicationMasterResponseInfo call() throws Exception {
+            LOG.info("Sending finish application request to RM {}",
+                subClusterId);
+            FinishApplicationMasterResponse uamResponse = null;
+            try {
+              uamResponse =
+                  uamPool.finishApplicationMaster(subClusterId, finishRequest);
+            } catch (Throwable e) {
+              LOG.warn("Failed to finish unmanaged application master: "
+                  + "RM address: " + subClusterId + " ApplicationId: "
+                  + getApplicationContext().getApplicationAttemptId(), e);
+            }
+            return new FinishApplicationMasterResponseInfo(uamResponse,
+                subClusterId);
+          }
+        });
+      }
+    }
+
+    // While the finish application request is being processed
+    // asynchronously by other sub-cluster resource managers, send the same
+    // request to the home resource manager on this thread.
     FinishApplicationMasterResponse homeResponse =
         AMRMClientUtils.finishAMWithReRegister(request, this.homeRM,
             this.amRegistrationRequest,
             getApplicationContext().getApplicationAttemptId());
+
+    if (subClusterIds.size() > 0) {
+      // Wait for other sub-cluster resource managers to return the
+      // response and merge it with the home response
+      LOG.info(
+          "Waiting for finish application response from {} sub-cluster RMs",
+          subClusterIds.size());
+      for (int i = 0; i < subClusterIds.size(); ++i) {
+        try {
+          Future<FinishApplicationMasterResponseInfo> future = compSvc.take();
+          FinishApplicationMasterResponseInfo uamResponse = future.get();
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Received finish application response from RM: "
+                + uamResponse.getSubClusterId());
+          }
+          if (uamResponse.getResponse() == null
+              || !uamResponse.getResponse().getIsUnregistered()) {
+            failedToUnRegister = true;
+          }
+        } catch (Throwable e) {
+          failedToUnRegister = true;
+          LOG.warn("Failed to finish unmanaged application master: "
+              + " ApplicationId: "
+              + getApplicationContext().getApplicationAttemptId(), e);
+        }
+      }
+    }
+
+    if (failedToUnRegister) {
+      homeResponse.setIsUnregistered(false);
+    }
     return homeResponse;
   }
 
@@ -281,10 +453,33 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
    */
   @Override
   public void shutdown() {
+    if (this.uamPool != null) {
+      this.uamPool.stop();
+    }
+    if (threadpool != null) {
+      try {
+        threadpool.shutdown();
+      } catch (Throwable ex) {
+      }
+      threadpool = null;
+    }
     super.shutdown();
   }
 
   /**
+   * Create the UAM pool manager for secondary sub-clsuters. For unit test to
+   * override.
+   *
+   * @param threadPool the thread pool to use
+   * @return the UAM pool manager instance
+   */
+  @VisibleForTesting
+  protected UnmanagedAMPoolManager createUnmanagedAMPoolManager(
+      ExecutorService threadPool) {
+    return new UnmanagedAMPoolManager(threadPool);
+  }
+
+  /**
    * Returns instance of the ApplicationMasterProtocol proxy class that is used
    * to connect to the Home resource manager.
    *
@@ -302,6 +497,23 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
     }
   }
 
+  private SubClusterId getSubClusterForNode(String nodeName) {
+    SubClusterId subClusterId = null;
+    try {
+      subClusterId = this.subClusterResolver.getSubClusterForNode(nodeName);
+    } catch (YarnException e) {
+      LOG.error("Failed to resolve sub-cluster for node " + nodeName
+          + ", skipping this node", e);
+      return null;
+    }
+    if (subClusterId == null) {
+      LOG.error("Failed to resolve sub-cluster for node {}, skipping this node",
+          nodeName);
+      return null;
+    }
+    return subClusterId;
+  }
+
   /**
    * In federation, the heart beat request needs to be sent to all the sub
    * clusters from which the AM has requested containers. This method splits the
@@ -317,20 +529,39 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
     findOrCreateAllocateRequestForSubCluster(this.homeSubClusterId, request,
         requestMap);
 
+    // Create heart beat request instances for all other already registered
+    // sub-cluster resource managers
+    Set<String> subClusterIds = this.uamPool.getAllUAMIds();
+    for (String subClusterId : subClusterIds) {
+      findOrCreateAllocateRequestForSubCluster(
+          SubClusterId.newInstance(subClusterId), request, requestMap);
+    }
+
     if (!isNullOrEmpty(request.getAskList())) {
-      AllocateRequest newRequest = findOrCreateAllocateRequestForSubCluster(
-          this.homeSubClusterId, request, requestMap);
-      newRequest.getAskList().addAll(request.getAskList());
+      // Ask the federation policy interpreter to split the ask list for
+      // sending it to all the sub-cluster resource managers.
+      Map<SubClusterId, List<ResourceRequest>> asks =
+          splitResourceRequests(request.getAskList());
+
+      // Add the askLists to the corresponding sub-cluster requests.
+      for (Entry<SubClusterId, List<ResourceRequest>> entry : asks.entrySet()) {
+        AllocateRequest newRequest = findOrCreateAllocateRequestForSubCluster(
+            entry.getKey(), request, requestMap);
+        newRequest.getAskList().addAll(entry.getValue());
+      }
     }
 
     if (request.getResourceBlacklistRequest() != null && !isNullOrEmpty(
         request.getResourceBlacklistRequest().getBlacklistAdditions())) {
       for (String resourceName : request.getResourceBlacklistRequest()
           .getBlacklistAdditions()) {
-        AllocateRequest newRequest = findOrCreateAllocateRequestForSubCluster(
-            this.homeSubClusterId, request, requestMap);
-        newRequest.getResourceBlacklistRequest().getBlacklistAdditions()
-            .add(resourceName);
+        SubClusterId subClusterId = getSubClusterForNode(resourceName);
+        if (subClusterId != null) {
+          AllocateRequest newRequest = findOrCreateAllocateRequestForSubCluster(
+              subClusterId, request, requestMap);
+          newRequest.getResourceBlacklistRequest().getBlacklistAdditions()
+              .add(resourceName);
+        }
       }
     }
 
@@ -338,10 +569,13 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
         request.getResourceBlacklistRequest().getBlacklistRemovals())) {
       for (String resourceName : request.getResourceBlacklistRequest()
           .getBlacklistRemovals()) {
-        AllocateRequest newRequest = findOrCreateAllocateRequestForSubCluster(
-            this.homeSubClusterId, request, requestMap);
-        newRequest.getResourceBlacklistRequest().getBlacklistRemovals()
-            .add(resourceName);
+        SubClusterId subClusterId = getSubClusterForNode(resourceName);
+        if (subClusterId != null) {
+          AllocateRequest newRequest = findOrCreateAllocateRequestForSubCluster(
+              subClusterId, request, requestMap);
+          newRequest.getResourceBlacklistRequest().getBlacklistRemovals()
+              .add(resourceName);
+        }
       }
     }
 
@@ -371,6 +605,174 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
   }
 
   /**
+   * This methods sends the specified AllocateRequests to the appropriate
+   * sub-cluster resource managers.
+   *
+   * @param requests contains the heart beat requests to send to the resource
+   *          manager keyed by the resource manager address
+   * @return the registration responses from the newly added sub-cluster
+   *         resource managers
+   * @throws YarnException
+   * @throws IOException
+   */
+  private Registrations sendRequestsToSecondaryResourceManagers(
+      Map<SubClusterId, AllocateRequest> requests)
+      throws YarnException, IOException {
+
+    // Create new UAM instances for the sub-cluster that we have not seen
+    // before
+    Registrations registrations = registerWithNewSubClusters(requests.keySet());
+
+    // Now that all the registrations are done, send the allocation request
+    // to the sub-cluster RMs using the Unmanaged application masters
+    // asynchronously and don't wait for the response. The responses will
+    // arrive asynchronously and will be added to the response sink. These
+    // responses will be sent to the application master in some future heart
+    // beat response.
+    for (Entry<SubClusterId, AllocateRequest> entry : requests.entrySet()) {
+      final SubClusterId subClusterId = entry.getKey();
+
+      if (subClusterId.equals(this.homeSubClusterId)) {
+        // Skip the request for the home sub-cluster resource manager.
+        // It will be handled separately in the allocate() method
+        continue;
+      }
+
+      if (!this.uamPool.hasUAMId(subClusterId.getId())) {
+        // TODO: This means that the registration for this sub-cluster RM
+        // failed. For now, we ignore the resource requests and continue
+        // but we need to fix this and handle this situation. One way would
+        // be to send the request to another RM by consulting the policy.
+        LOG.warn("Unmanaged AM registration not found for sub-cluster {}",
+            subClusterId);
+        continue;
+      }
+
+      this.uamPool.allocateAsync(subClusterId.getId(), entry.getValue(),
+          new AsyncCallback<AllocateResponse>() {
+            @Override
+            public void callback(AllocateResponse response) {
+              synchronized (asyncResponseSink) {
+                List<AllocateResponse> responses = null;
+                if (asyncResponseSink.containsKey(subClusterId)) {
+                  responses = asyncResponseSink.get(subClusterId);
+                } else {
+                  responses = new ArrayList<>();
+                  asyncResponseSink.put(subClusterId, responses);
+                }
+                responses.add(response);
+              }
+
+              // Notify policy of secondary sub-cluster responses
+              try {
+                policyInterpreter.notifyOfResponse(subClusterId, response);
+              } catch (YarnException e) {
+                LOG.warn(
+                    "notifyOfResponse for policy failed for home sub-cluster "
+                        + subClusterId,
+                    e);
+              }
+            }
+          });
+    }
+
+    return registrations;
+  }
+
+  /**
+   * This method ensures that Unmanaged AMs are created for each of the
+   * specified sub-cluster specified in the input and registers with the
+   * corresponding resource managers.
+   */
+  private Registrations registerWithNewSubClusters(
+      Set<SubClusterId> subClusterSet) throws IOException {
+
+    List<SubClusterId> failedRegistrations = new ArrayList<>();
+    Map<SubClusterId, RegisterApplicationMasterResponse>
+        successfulRegistrations = new HashMap<>();
+
+    // Check to see if there are any new sub-clusters in this request
+    // list and create and register Unmanaged AM instance for the new ones
+    List<String> newSubClusters = new ArrayList<>();
+    for (SubClusterId subClusterId : subClusterSet) {
+      if (!subClusterId.equals(this.homeSubClusterId)
+          && !this.uamPool.hasUAMId(subClusterId.getId())) {
+        newSubClusters.add(subClusterId.getId());
+      }
+    }
+
+    if (newSubClusters.size() > 0) {
+      final RegisterApplicationMasterRequest registerRequest =
+          this.amRegistrationRequest;
+      final AMRMProxyApplicationContext appContext = getApplicationContext();
+      ExecutorCompletionService<RegisterApplicationMasterResponseInfo>
+          completionService = new ExecutorCompletionService<>(threadpool);
+
+      for (final String subClusterId : newSubClusters) {
+        completionService
+            .submit(new Callable<RegisterApplicationMasterResponseInfo>() {
+              @Override
+              public RegisterApplicationMasterResponseInfo call()
+                  throws Exception {
+
+                // Create a config loaded with federation on and subclusterId
+                // for each UAM
+                YarnConfiguration config = new YarnConfiguration(getConf());
+                FederationProxyProviderUtil.updateConfForFederation(config,
+                    subClusterId);
+
+                RegisterApplicationMasterResponse uamResponse = null;
+                try {
+                  // For appNameSuffix, use subClusterId of the home sub-cluster
+                  uamResponse = uamPool.createAndRegisterNewUAM(subClusterId,
+                      registerRequest, config,
+                      appContext.getApplicationAttemptId().getApplicationId(),
+                      amRegistrationResponse.getQueue(), appContext.getUser(),
+                      homeSubClusterId.toString());
+                } catch (Throwable e) {
+                  LOG.error("Failed to register application master: "
+                      + subClusterId + " Application: "
+                      + appContext.getApplicationAttemptId(), e);
+                }
+                return new RegisterApplicationMasterResponseInfo(uamResponse,
+                    SubClusterId.newInstance(subClusterId));
+              }
+            });
+      }
+
+      // Wait for other sub-cluster resource managers to return the
+      // response and add it to the Map for returning to the caller
+      for (int i = 0; i < newSubClusters.size(); ++i) {
+        try {
+          Future<RegisterApplicationMasterResponseInfo> future =
+              completionService.take();
+          RegisterApplicationMasterResponseInfo uamResponse = future.get();
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Received register application response from RM: "
+                + uamResponse.getSubClusterId());
+          }
+
+          if (uamResponse.getResponse() == null) {
+            failedRegistrations.add(uamResponse.getSubClusterId());
+          } else {
+            LOG.info("Successfully registered unmanaged application master: "
+                + uamResponse.getSubClusterId() + " ApplicationId: "
+                + getApplicationContext().getApplicationAttemptId());
+            successfulRegistrations.put(uamResponse.getSubClusterId(),
+                uamResponse.getResponse());
+          }
+        } catch (Exception e) {
+          LOG.warn("Failed to register unmanaged application master: "
+              + " ApplicationId: "
+              + getApplicationContext().getApplicationAttemptId(), e);
+        }
+      }
+    }
+
+    return new Registrations(successfulRegistrations, failedRegistrations);
+  }
+
+  /**
    * Merges the responses from other sub-clusters that we received
    * asynchronously with the specified home cluster response and keeps track of
    * the containers received from each sub-cluster resource managers.
@@ -388,6 +790,24 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
     cacheAllocatedContainers(homeResponse.getAllocatedContainers(),
         this.homeSubClusterId);
 
+    synchronized (this.asyncResponseSink) {
+      for (Entry<SubClusterId, List<AllocateResponse>> entry : asyncResponseSink
+          .entrySet()) {
+        SubClusterId subClusterId = entry.getKey();
+        List<AllocateResponse> responses = entry.getValue();
+        if (responses.size() > 0) {
+          for (AllocateResponse response : responses) {
+            removeFinishedContainersFromCache(
+                response.getCompletedContainersStatuses());
+            cacheAllocatedContainers(response.getAllocatedContainers(),
+                subClusterId);
+            mergeAllocateResponse(homeResponse, response, subClusterId);
+          }
+          responses.clear();
+        }
+      }
+    }
+
     return homeResponse;
   }
 
@@ -405,6 +825,130 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
   }
 
   /**
+   * Helper method for merging the responses from the secondary sub cluster RMs
+   * with the home response to return to the AM.
+   */
+  private AllocateResponse mergeRegistrationResponses(
+      AllocateResponse homeResponse,
+      Map<SubClusterId, RegisterApplicationMasterResponse> registrations) {
+
+    for (Entry<SubClusterId, RegisterApplicationMasterResponse> entry :
+        registrations.entrySet()) {
+      RegisterApplicationMasterResponse registration = entry.getValue();
+
+      if (!isNullOrEmpty(registration.getContainersFromPreviousAttempts())) {
+        List<Container> tempContainers = homeResponse.getAllocatedContainers();
+        if (!isNullOrEmpty(tempContainers)) {
+          tempContainers
+              .addAll(registration.getContainersFromPreviousAttempts());
+          homeResponse.setAllocatedContainers(tempContainers);
+        } else {
+          homeResponse.setAllocatedContainers(
+              registration.getContainersFromPreviousAttempts());
+        }
+        cacheAllocatedContainers(
+            registration.getContainersFromPreviousAttempts(), entry.getKey());
+      }
+
+      if (!isNullOrEmpty(registration.getNMTokensFromPreviousAttempts())) {
+        List<NMToken> tempTokens = homeResponse.getNMTokens();
+        if (!isNullOrEmpty(tempTokens)) {
+          tempTokens.addAll(registration.getNMTokensFromPreviousAttempts());
+          homeResponse.setNMTokens(tempTokens);
+        } else {
+          homeResponse
+              .setNMTokens(registration.getNMTokensFromPreviousAttempts());
+        }
+      }
+    }
+
+    return homeResponse;
+  }
+
+  private void mergeAllocateResponse(AllocateResponse homeResponse,
+      AllocateResponse otherResponse, SubClusterId otherRMAddress) {
+
+    if (!isNullOrEmpty(otherResponse.getAllocatedContainers())) {
+      if (!isNullOrEmpty(homeResponse.getAllocatedContainers())) {
+        homeResponse.getAllocatedContainers()
+            .addAll(otherResponse.getAllocatedContainers());
+      } else {
+        homeResponse
+            .setAllocatedContainers(otherResponse.getAllocatedContainers());
+      }
+    }
+
+    if (otherResponse.getAvailableResources() != null) {
+      if (homeResponse.getAvailableResources() != null) {
+        homeResponse.setAvailableResources(
+            Resources.add(homeResponse.getAvailableResources(),
+                otherResponse.getAvailableResources()));
+      } else {
+        homeResponse
+            .setAvailableResources(otherResponse.getAvailableResources());
+      }
+    }
+
+    if (!isNullOrEmpty(otherResponse.getCompletedContainersStatuses())) {
+      if (!isNullOrEmpty(homeResponse.getCompletedContainersStatuses())) {
+        homeResponse.getCompletedContainersStatuses()
+            .addAll(otherResponse.getCompletedContainersStatuses());
+      } else {
+        homeResponse.setCompletedContainersStatuses(
+            otherResponse.getCompletedContainersStatuses());
+      }
+    }
+
+    if (!isNullOrEmpty(otherResponse.getUpdatedNodes())) {
+      if (!isNullOrEmpty(homeResponse.getUpdatedNodes())) {
+        homeResponse.getUpdatedNodes().addAll(otherResponse.getUpdatedNodes());
+      } else {
+        homeResponse.setUpdatedNodes(otherResponse.getUpdatedNodes());
+      }
+    }
+
+    if (!isNullOrEmpty(otherResponse.getNMTokens())) {
+      if (!isNullOrEmpty(homeResponse.getNMTokens())) {
+        homeResponse.getNMTokens().addAll(otherResponse.getNMTokens());
+      } else {
+        homeResponse.setNMTokens(otherResponse.getNMTokens());
+      }
+    }
+
+    PreemptionMessage homePreempMessage = homeResponse.getPreemptionMessage();
+    PreemptionMessage otherPreempMessage = otherResponse.getPreemptionMessage();
+
+    if (homePreempMessage == null && otherPreempMessage != null) {
+      homeResponse.setPreemptionMessage(otherPreempMessage);
+    }
+
+    if (homePreempMessage != null && otherPreempMessage != null) {
+      PreemptionContract par1 = homePreempMessage.getContract();
+      PreemptionContract par2 = otherPreempMessage.getContract();
+
+      if (par1 == null && par2 != null) {
+        homePreempMessage.setContract(par2);
+      }
+
+      if (par1 != null && par2 != null) {
+        par1.getResourceRequest().addAll(par2.getResourceRequest());
+        par2.getContainers().addAll(par2.getContainers());
+      }
+
+      StrictPreemptionContract spar1 = homePreempMessage.getStrictContract();
+      StrictPreemptionContract spar2 = otherPreempMessage.getStrictContract();
+
+      if (spar1 == null && spar2 != null) {
+        homePreempMessage.setStrictContract(spar2);
+      }
+
+      if (spar1 != null && spar2 != null) {
+        spar1.getContainers().addAll(spar2.getContainers());
+      }
+    }
+  }
+
+  /**
    * Add allocated containers to cache mapping.
    */
   private void cacheAllocatedContainers(List<Container> containers,
@@ -418,10 +962,10 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
           // container allocation more than once. Just move on in this case.
           LOG.warn(
               "Duplicate containerID: {} found in the allocated containers"
-                  + " from same subcluster: {}, so ignoring.",
+                  + " from same sub-cluster: {}, so ignoring.",
               container.getId(), subClusterId);
         } else {
-          // The same container allocation from different subclusters,
+          // The same container allocation from different sub-clusters,
           // something is wrong.
           // TODO: YARN-6667 if some subcluster RM is configured wrong, we
           // should not fail the entire heartbeat.
@@ -432,7 +976,7 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
                   + " ApplicationId: "
                   + getApplicationContext().getApplicationAttemptId()
                   + " From RM: " + subClusterId
-                  + " . Previous container was from subcluster: "
+                  + " . Previous container was from sub-cluster: "
                   + existingSubClusterId);
         }
       }
@@ -498,7 +1042,102 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
   }
 
   /**
-   * Utility method to check if the specified Collection is null or empty
+   * Splits the specified request to send it to different sub clusters. The
+   * splitting algorithm is very simple. If the request does not have a node
+   * preference, the policy decides the sub cluster. If the request has a node
+   * preference and if locality is required, then it is sent to the sub cluster
+   * that contains the requested node. If node preference is specified and
+   * locality is not required, then the policy decides the sub cluster.
+   *
+   * @param askList the ask list to split
+   * @return the split asks
+   * @throws YarnException if split fails
+   */
+  protected Map<SubClusterId, List<ResourceRequest>> splitResourceRequests(
+      List<ResourceRequest> askList) throws YarnException {
+    return this.policyInterpreter.splitResourceRequests(askList);
+  }
+
+  @VisibleForTesting
+  public int getUnmanagedAMPoolSize() {
+    return this.uamPool.getAllUAMIds().size();
+  }
+
+  /**
+   * Private structure for encapsulating SubClusterId and
+   * RegisterApplicationMasterResponse instances.
+   */
+  private static class RegisterApplicationMasterResponseInfo {
+    private RegisterApplicationMasterResponse response;
+    private SubClusterId subClusterId;
+
+    RegisterApplicationMasterResponseInfo(
+        RegisterApplicationMasterResponse response, SubClusterId subClusterId) {
+      this.response = response;
+      this.subClusterId = subClusterId;
+    }
+
+    public RegisterApplicationMasterResponse getResponse() {
+      return response;
+    }
+
+    public SubClusterId getSubClusterId() {
+      return subClusterId;
+    }
+  }
+
+  /**
+   * Private structure for encapsulating SubClusterId and
+   * FinishApplicationMasterResponse instances.
+   */
+  private static class FinishApplicationMasterResponseInfo {
+    private FinishApplicationMasterResponse response;
+    private String subClusterId;
+
+    FinishApplicationMasterResponseInfo(
+        FinishApplicationMasterResponse response, String subClusterId) {
+      this.response = response;
+      this.subClusterId = subClusterId;
+    }
+
+    public FinishApplicationMasterResponse getResponse() {
+      return response;
+    }
+
+    public String getSubClusterId() {
+      return subClusterId;
+    }
+  }
+
+  /**
+   * Private structure for encapsulating successful and failed application
+   * master registration responses.
+   */
+  private static class Registrations {
+    private Map<SubClusterId, RegisterApplicationMasterResponse>
+        successfulRegistrations;
+    private List<SubClusterId> failedRegistrations;
+
+    Registrations(
+        Map<SubClusterId, RegisterApplicationMasterResponse>
+            successfulRegistrations,
+        List<SubClusterId> failedRegistrations) {
+      this.successfulRegistrations = successfulRegistrations;
+      this.failedRegistrations = failedRegistrations;
+    }
+
+    public Map<SubClusterId, RegisterApplicationMasterResponse>
+        getSuccessfulRegistrations() {
+      return this.successfulRegistrations;
+    }
+
+    public List<SubClusterId> getFailedRegistrations() {
+      return this.failedRegistrations;
+    }
+  }
+
+  /**
+   * Utility method to check if the specified Collection is null or empty.
    *
    * @param c the collection object
    * @param <T> element type of the collection
@@ -507,4 +1146,16 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
   public static <T> boolean isNullOrEmpty(Collection<T> c) {
     return (c == null || c.size() == 0);
   }
+
+  /**
+   * Utility method to check if the specified Collection is null or empty.
+   *
+   * @param c the map object
+   * @param <T1> key type of the map
+   * @param <T2> value type of the map
+   * @return whether is it is null or empty
+   */
+  public static <T1, T2> boolean isNullOrEmpty(Map<T1, T2> c) {
+    return (c == null || c.size() == 0);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c10a9be/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestFederationInterceptor.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/TestFederationInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestFederationInterceptor.java
index 3b564f0..4e15323 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestFederationInterceptor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestFederationInterceptor.java
@@ -19,13 +19,31 @@
 package org.apache.hadoop.yarn.server.nodemanager.amrmproxy;
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
 
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+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.InvalidApplicationMasterRequestException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.federation.policies.manager.UniformBroadcastPolicyManager;
+import org.apache.hadoop.yarn.server.federation.store.impl.MemoryFederationStateStore;
+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.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.util.Records;
 import org.junit.Assert;
 import org.junit.Test;
@@ -45,6 +63,7 @@ public class TestFederationInterceptor extends BaseAMRMProxyTest {
   public static final String HOME_SC_ID = "SC-home";
 
   private TestableFederationInterceptor interceptor;
+  private MemoryFederationStateStore stateStore;
 
   private int testAppId;
   private ApplicationAttemptId attemptId;
@@ -54,6 +73,11 @@ public class TestFederationInterceptor extends BaseAMRMProxyTest {
     super.setUp();
     interceptor = new TestableFederationInterceptor();
 
+    stateStore = new MemoryFederationStateStore();
+    stateStore.init(getConf());
+    FederationStateStoreFacade.getInstance().reinitialize(stateStore,
+        getConf());
+
     testAppId = 1;
     attemptId = getApplicationAttemptId(testAppId);
     interceptor.init(new AMRMProxyApplicationContextImpl(null, getConf(),
@@ -82,11 +106,238 @@ public class TestFederationInterceptor extends BaseAMRMProxyTest {
         mockPassThroughInterceptorClass + "," + mockPassThroughInterceptorClass
             + "," + TestableFederationInterceptor.class.getName());
 
+    conf.set(YarnConfiguration.FEDERATION_POLICY_MANAGER,
+        UniformBroadcastPolicyManager.class.getName());
+
     conf.set(YarnConfiguration.RM_CLUSTER_ID, HOME_SC_ID);
 
+    // Disable StateStoreFacade cache
+    conf.setInt(YarnConfiguration.FEDERATION_CACHE_TIME_TO_LIVE_SECS, 0);
+
     return conf;
   }
 
+  private void registerSubCluster(SubClusterId subClusterId)
+      throws YarnException {
+    stateStore
+        .registerSubCluster(SubClusterRegisterRequest.newInstance(SubClusterInfo
+            .newInstance(subClusterId, "1.2.3.4:1", "1.2.3.4:2", "1.2.3.4:3",
+                "1.2.3.4:4", SubClusterState.SC_RUNNING, 0, "capacity")));
+  }
+
+  private void deRegisterSubCluster(SubClusterId subClusterId)
+      throws YarnException {
+    stateStore.deregisterSubCluster(SubClusterDeregisterRequest
+        .newInstance(subClusterId, SubClusterState.SC_UNREGISTERED));
+  }
+
+  private List<Container> getContainersAndAssert(int numberOfResourceRequests,
+      int numberOfAllocationExcepted) throws Exception {
+    AllocateRequest allocateRequest = Records.newRecord(AllocateRequest.class);
+    allocateRequest.setResponseId(1);
+
+    List<Container> containers =
+        new ArrayList<Container>(numberOfResourceRequests);
+    List<ResourceRequest> askList =
+        new ArrayList<ResourceRequest>(numberOfResourceRequests);
+    for (int id = 0; id < numberOfResourceRequests; id++) {
+      askList.add(createResourceRequest("test-node-" + Integer.toString(id),
+          6000, 2, id % 5, 1));
+    }
+
+    allocateRequest.setAskList(askList);
+
+    AllocateResponse allocateResponse = interceptor.allocate(allocateRequest);
+    Assert.assertNotNull("allocate() returned null response", allocateResponse);
+
+    containers.addAll(allocateResponse.getAllocatedContainers());
+    LOG.info("Number of allocated containers in the original request: "
+        + Integer.toString(allocateResponse.getAllocatedContainers().size()));
+
+    // Send max 10 heart beats to receive all the containers. If not, we will
+    // fail the test
+    int numHeartbeat = 0;
+    while (containers.size() < numberOfAllocationExcepted
+        && numHeartbeat++ < 10) {
+      allocateResponse =
+          interceptor.allocate(Records.newRecord(AllocateRequest.class));
+      Assert.assertNotNull("allocate() returned null response",
+          allocateResponse);
+
+      containers.addAll(allocateResponse.getAllocatedContainers());
+
+      LOG.info("Number of allocated containers in this request: "
+          + Integer.toString(allocateResponse.getAllocatedContainers().size()));
+      LOG.info("Total number of allocated containers: "
+          + Integer.toString(containers.size()));
+      Thread.sleep(10);
+    }
+    Assert.assertEquals(numberOfAllocationExcepted, containers.size());
+    return containers;
+  }
+
+  private void releaseContainersAndAssert(List<Container> containers)
+      throws Exception {
+    Assert.assertTrue(containers.size() > 0);
+    AllocateRequest allocateRequest = Records.newRecord(AllocateRequest.class);
+    allocateRequest.setResponseId(1);
+
+    List<ContainerId> relList = new ArrayList<ContainerId>(containers.size());
+    for (Container container : containers) {
+      relList.add(container.getId());
+    }
+
+    allocateRequest.setReleaseList(relList);
+
+    AllocateResponse allocateResponse = interceptor.allocate(allocateRequest);
+    Assert.assertNotNull(allocateResponse);
+
+    // The way the mock resource manager is setup, it will return the containers
+    // that were released in the allocated containers. The release request will
+    // be split and handled by the corresponding UAM. The release containers
+    // returned by the mock resource managers will be aggregated and returned
+    // back to us and we can check if total request size and returned size are
+    // the same
+    List<Container> containersForReleasedContainerIds =
+        new ArrayList<Container>();
+    containersForReleasedContainerIds
+        .addAll(allocateResponse.getAllocatedContainers());
+    LOG.info("Number of containers received in the original request: "
+        + Integer.toString(allocateResponse.getAllocatedContainers().size()));
+
+    // Send max 10 heart beats to receive all the containers. If not, we will
+    // fail the test
+    int numHeartbeat = 0;
+    while (containersForReleasedContainerIds.size() < relList.size()
+        && numHeartbeat++ < 10) {
+      allocateResponse =
+          interceptor.allocate(Records.newRecord(AllocateRequest.class));
+      Assert.assertNotNull(allocateResponse);
+      containersForReleasedContainerIds
+          .addAll(allocateResponse.getAllocatedContainers());
+
+      LOG.info("Number of containers received in this request: "
+          + Integer.toString(allocateResponse.getAllocatedContainers().size()));
+      LOG.info("Total number of containers received: "
+          + Integer.toString(containersForReleasedContainerIds.size()));
+      Thread.sleep(10);
+    }
+
+    Assert.assertEquals(relList.size(),
+        containersForReleasedContainerIds.size());
+  }
+
+  @Test
+  public void testMultipleSubClusters() throws Exception {
+
+    // Register the application
+    RegisterApplicationMasterRequest registerReq =
+        Records.newRecord(RegisterApplicationMasterRequest.class);
+    registerReq.setHost(Integer.toString(testAppId));
+    registerReq.setRpcPort(testAppId);
+    registerReq.setTrackingUrl("");
+
+    RegisterApplicationMasterResponse registerResponse =
+        interceptor.registerApplicationMaster(registerReq);
+    Assert.assertNotNull(registerResponse);
+
+    Assert.assertEquals(0, interceptor.getUnmanagedAMPoolSize());
+
+    // Allocate the first batch of containers, with sc1 and sc2 active
+    registerSubCluster(SubClusterId.newInstance("SC-1"));
+    registerSubCluster(SubClusterId.newInstance("SC-2"));
+
+    int numberOfContainers = 3;
+    List<Container> containers =
+        getContainersAndAssert(numberOfContainers, numberOfContainers * 2);
+    Assert.assertEquals(2, interceptor.getUnmanagedAMPoolSize());
+
+    // Allocate the second batch of containers, with sc1 and sc3 active
+    deRegisterSubCluster(SubClusterId.newInstance("SC-2"));
+    registerSubCluster(SubClusterId.newInstance("SC-3"));
+
+    numberOfContainers = 1;
+    containers.addAll(
+        getContainersAndAssert(numberOfContainers, numberOfContainers * 2));
+    Assert.assertEquals(3, interceptor.getUnmanagedAMPoolSize());
+
+    // Allocate the third batch of containers with only in home sub-cluster
+    // active
+    deRegisterSubCluster(SubClusterId.newInstance("SC-1"));
+    deRegisterSubCluster(SubClusterId.newInstance("SC-3"));
+    registerSubCluster(SubClusterId.newInstance(HOME_SC_ID));
+
+    numberOfContainers = 2;
+    containers.addAll(
+        getContainersAndAssert(numberOfContainers, numberOfContainers * 1));
+    Assert.assertEquals(3, interceptor.getUnmanagedAMPoolSize());
+
+    // Release all containers
+    releaseContainersAndAssert(containers);
+
+    // Finish the application
+    FinishApplicationMasterRequest finishReq =
+        Records.newRecord(FinishApplicationMasterRequest.class);
+    finishReq.setDiagnostics("");
+    finishReq.setTrackingUrl("");
+    finishReq.setFinalApplicationStatus(FinalApplicationStatus.SUCCEEDED);
+
+    FinishApplicationMasterResponse finshResponse =
+        interceptor.finishApplicationMaster(finishReq);
+    Assert.assertNotNull(finshResponse);
+    Assert.assertEquals(true, finshResponse.getIsUnregistered());
+  }
+
+  /*
+   * Test re-register when RM fails over.
+   */
+  @Test
+  public void testReregister() throws Exception {
+
+    // Register the application
+    RegisterApplicationMasterRequest registerReq =
+        Records.newRecord(RegisterApplicationMasterRequest.class);
+    registerReq.setHost(Integer.toString(testAppId));
+    registerReq.setRpcPort(testAppId);
+    registerReq.setTrackingUrl("");
+
+    RegisterApplicationMasterResponse registerResponse =
+        interceptor.registerApplicationMaster(registerReq);
+    Assert.assertNotNull(registerResponse);
+
+    Assert.assertEquals(0, interceptor.getUnmanagedAMPoolSize());
+
+    // Allocate the first batch of containers
+    registerSubCluster(SubClusterId.newInstance("SC-1"));
+    registerSubCluster(SubClusterId.newInstance(HOME_SC_ID));
+
+    interceptor.setShouldReRegisterNext();
+
+    int numberOfContainers = 3;
+    List<Container> containers =
+        getContainersAndAssert(numberOfContainers, numberOfContainers * 2);
+    Assert.assertEquals(1, interceptor.getUnmanagedAMPoolSize());
+
+    interceptor.setShouldReRegisterNext();
+
+    // Release all containers
+    releaseContainersAndAssert(containers);
+
+    interceptor.setShouldReRegisterNext();
+
+    // Finish the application
+    FinishApplicationMasterRequest finishReq =
+        Records.newRecord(FinishApplicationMasterRequest.class);
+    finishReq.setDiagnostics("");
+    finishReq.setTrackingUrl("");
+    finishReq.setFinalApplicationStatus(FinalApplicationStatus.SUCCEEDED);
+
+    FinishApplicationMasterResponse finshResponse =
+        interceptor.finishApplicationMaster(finishReq);
+    Assert.assertNotNull(finshResponse);
+    Assert.assertEquals(true, finshResponse.getIsUnregistered());
+  }
+
   @Test
   public void testRequestInterceptorChainCreation() throws Exception {
     RequestInterceptor root =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c10a9be/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestableFederationInterceptor.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/TestableFederationInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestableFederationInterceptor.java
index 0ca7488..d4b8735 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestableFederationInterceptor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestableFederationInterceptor.java
@@ -45,6 +45,12 @@ public class TestableFederationInterceptor extends FederationInterceptor {
   private MockResourceManagerFacade mockRm;
 
   @Override
+  protected UnmanagedAMPoolManager createUnmanagedAMPoolManager(
+      ExecutorService threadPool) {
+    return new TestableUnmanagedAMPoolManager(threadPool);
+  }
+
+  @Override
   protected ApplicationMasterProtocol createHomeRMProxy(
       AMRMProxyApplicationContext appContext) {
     synchronized (this) {


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


[32/50] [abbrv] hadoop git commit: YARN-6815. FederationStateStoreFacade return behavior should be consistent irrespective of whether caching is enabled or not.

Posted by su...@apache.org.
YARN-6815. FederationStateStoreFacade return behavior should be consistent irrespective of whether caching is enabled or not.


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

Branch: refs/heads/YARN-2915
Commit: 6cd635f14eeabd18aebbc48d4e52f2f0199de53c
Parents: f20d855
Author: Subru Krishnan <su...@apache.org>
Authored: Thu Jul 13 18:51:06 2017 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Tue Jul 25 16:56:32 2017 -0700

----------------------------------------------------------------------
 .../federation/policies/RouterPolicyFacade.java | 53 +++++++++++---------
 .../store/FederationMembershipStateStore.java   |  7 +--
 .../federation/store/FederationPolicyStore.java |  3 +-
 .../store/impl/MemoryFederationStateStore.java  |  9 ++--
 .../store/impl/SQLFederationStateStore.java     | 11 ++--
 .../utils/FederationStateStoreFacade.java       | 20 +++++---
 .../impl/FederationStateStoreBaseTest.java      | 21 +++-----
 .../utils/FederationStateStoreTestUtil.java     |  1 +
 .../utils/TestFederationStateStoreFacade.java   | 16 +++++-
 9 files changed, 84 insertions(+), 57 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6cd635f1/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 52c2905..bbf08e0 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
@@ -23,13 +23,10 @@ import java.util.List;
 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.conf.Configuration;
 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.manager.FederationPolicyManager;
@@ -38,6 +35,8 @@ 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 org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -47,8 +46,8 @@ import com.google.common.annotations.VisibleForTesting;
  */
 public class RouterPolicyFacade {
 
-  private static final Log LOG =
-      LogFactory.getLog(LocalityMulticastAMRMProxyPolicy.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RouterPolicyFacade.class);
 
   private final SubClusterResolver subClusterResolver;
   private final FederationStateStoreFacade federationFacade;
@@ -68,10 +67,10 @@ public class RouterPolicyFacade {
     this.globalPolicyMap = new ConcurrentHashMap<>();
 
     // load default behavior from store if possible
-    String defaulKey = YarnConfiguration.DEFAULT_FEDERATION_POLICY_KEY;
+    String defaultKey = YarnConfiguration.DEFAULT_FEDERATION_POLICY_KEY;
     SubClusterPolicyConfiguration configuration = null;
     try {
-      configuration = federationFacade.getPolicyConfiguration(defaulKey);
+      configuration = federationFacade.getPolicyConfiguration(defaultKey);
     } catch (YarnException e) {
       LOG.warn("No fallback behavior defined in store, defaulting to XML "
           + "configuration fallback behavior.");
@@ -88,7 +87,7 @@ public class RouterPolicyFacade {
       ByteBuffer defaultPolicyParam = ByteBuffer
           .wrap(defaultPolicyParamString.getBytes(StandardCharsets.UTF_8));
 
-      configuration = SubClusterPolicyConfiguration.newInstance(defaulKey,
+      configuration = SubClusterPolicyConfiguration.newInstance(defaultKey,
           defaultFederationPolicyManager, defaultPolicyParam);
     }
 
@@ -98,12 +97,12 @@ public class RouterPolicyFacade {
             subClusterResolver, federationFacade, homeSubcluster);
     FederationPolicyManager fallbackPolicyManager =
         FederationPolicyUtils.instantiatePolicyManager(configuration.getType());
-    fallbackPolicyManager.setQueue(defaulKey);
+    fallbackPolicyManager.setQueue(defaultKey);
 
     // add to the cache the fallback behavior
-    globalConfMap.put(defaulKey,
+    globalConfMap.put(defaultKey,
         fallbackContext.getSubClusterPolicyConfiguration());
-    globalPolicyMap.put(defaulKey,
+    globalPolicyMap.put(defaultKey,
         fallbackPolicyManager.getRouterPolicy(fallbackContext, null));
 
   }
@@ -155,29 +154,37 @@ public class RouterPolicyFacade {
     try {
       configuration = federationFacade.getPolicyConfiguration(queue);
     } catch (YarnException e) {
-      LOG.debug(e);
+      String errMsg = "There is no policy configured for the queue: " + queue
+          + ", falling back to defaults.";
+      LOG.warn(errMsg, 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);
+      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);
+      queue = YarnConfiguration.DEFAULT_FEDERATION_POLICY_KEY;
+      try {
+        configuration = federationFacade.getPolicyConfiguration(queue);
       } 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);
+        String errMsg = "Cannot retrieve policy configured for the queue: "
+            + queue + ", falling back to defaults.";
+        LOG.warn(errMsg, e);
+
       }
     }
 
+    // the fallback is not configure via store, but via XML, using
+    // previously loaded configuration.
+    if (configuration == null) {
+      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)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6cd635f1/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 7778d5f..49ec3bf 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
@@ -70,7 +70,7 @@ public interface FederationMembershipStateStore {
    */
   SubClusterDeregisterResponse deregisterSubCluster(
       SubClusterDeregisterRequest subClusterDeregisterRequest)
-          throws YarnException;
+      throws YarnException;
 
   /**
    * Periodic heartbeat from a <code>ResourceManager</code> participating in
@@ -86,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
@@ -94,7 +94,8 @@ public interface FederationMembershipStateStore {
    * endpoint and current capabilities as represented by {@code SubClusterInfo}.
    *
    * @param subClusterRequest the subcluster whose information is required
-   * @return the {@code SubClusterInfo}
+   * @return the {@code SubClusterInfo}, or {@code null} if there is no mapping
+   *         for the subcluster
    * @throws YarnException if the request is invalid/fails
    */
   GetSubClusterInfoResponse getSubCluster(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6cd635f1/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
index 9d9bd9b..b0e03a6 100644
--- 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
@@ -45,7 +45,8 @@ public interface FederationPolicyStore {
    *
    * @param request the queue whose {@code SubClusterPolicyConfiguration} is
    *          required
-   * @return the {@code SubClusterPolicyConfiguration} for the specified queue
+   * @return the {@code SubClusterPolicyConfiguration} for the specified queue,
+   *         or {@code null} if there is no mapping for the queue
    * @throws YarnException if the request is invalid/fails
    */
   GetSubClusterPolicyConfigurationResponse getPolicyConfiguration(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6cd635f1/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 fbdb7bf..7c06256 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
@@ -165,9 +165,8 @@ public class MemoryFederationStateStore implements FederationStateStore {
     FederationMembershipStateStoreInputValidator.validate(request);
     SubClusterId subClusterId = request.getSubClusterId();
     if (!membership.containsKey(subClusterId)) {
-      String errMsg =
-          "SubCluster " + subClusterId.toString() + " does not exist";
-      FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+      LOG.warn("The queried SubCluster: {} does not exist.", subClusterId);
+      return null;
     }
 
     return GetSubClusterInfoResponse.newInstance(membership.get(subClusterId));
@@ -274,8 +273,8 @@ public class MemoryFederationStateStore implements FederationStateStore {
     FederationPolicyStoreInputValidator.validate(request);
     String queue = request.getQueue();
     if (!policies.containsKey(queue)) {
-      String errMsg = "Policy for queue " + queue + " does not exist";
-      FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+      LOG.warn("Policy for queue: {} does not exist.", queue);
+      return null;
     }
 
     return GetSubClusterPolicyConfigurationResponse

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6cd635f1/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
index a849c6a..63d8e42 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/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
@@ -385,6 +385,12 @@ public class SQLFederationStateStore implements FederationStateStore {
       String rmAdminAddress = cstmt.getString(4);
       String webAppAddress = cstmt.getString(5);
 
+      // first check if the subCluster exists
+      if((amRMAddress == null) || (clientRMAddress == null)) {
+        LOG.warn("The queried SubCluster: {} does not exist.", subClusterId);
+        return null;
+      }
+
       Timestamp heartBeatTimeStamp = cstmt.getTimestamp(6, utcCalendar);
       long lastHeartBeat =
           heartBeatTimeStamp != null ? heartBeatTimeStamp.getTime() : 0;
@@ -788,9 +794,8 @@ public class SQLFederationStateStore implements FederationStateStore {
               + subClusterPolicyConfiguration.toString());
         }
       } else {
-        String errMsg =
-            "Policy for queue " + request.getQueue() + " does not exist";
-        FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+        LOG.warn("Policy for queue: {} does not exist.", request.getQueue());
+        return null;
       }
 
     } catch (SQLException e) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6cd635f1/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 5693342..389c769 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
@@ -55,6 +55,7 @@ import org.apache.hadoop.yarn.server.federation.store.records.ApplicationHomeSub
 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.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;
@@ -221,7 +222,8 @@ public final class FederationStateStoreFacade {
    * Returns the {@link SubClusterInfo} for the specified {@link SubClusterId}.
    *
    * @param subClusterId the identifier of the sub-cluster
-   * @return the sub cluster information
+   * @return the sub cluster information, or
+   *         {@code null} if there is no mapping for the subClusterId
    * @throws YarnException if the call to the state store is unsuccessful
    */
   public SubClusterInfo getSubCluster(final SubClusterId subClusterId)
@@ -229,9 +231,13 @@ public final class FederationStateStoreFacade {
     if (isCachingEnabled()) {
       return getSubClusters(false).get(subClusterId);
     } else {
-      return stateStore
-          .getSubCluster(GetSubClusterInfoRequest.newInstance(subClusterId))
-          .getSubClusterInfo();
+      GetSubClusterInfoResponse response = stateStore
+          .getSubCluster(GetSubClusterInfoRequest.newInstance(subClusterId));
+      if (response == null) {
+        return null;
+      } else {
+        return response.getSubClusterInfo();
+      }
     }
   }
 
@@ -282,7 +288,8 @@ public final class FederationStateStoreFacade {
    * Returns the {@link SubClusterPolicyConfiguration} for the specified queue.
    *
    * @param queue the queue whose policy is required
-   * @return the corresponding configured policy
+   * @return the corresponding configured policy, or {@code null} if there is no
+   *         mapping for the queue
    * @throws YarnException if the call to the state store is unsuccessful
    */
   public SubClusterPolicyConfiguration getPolicyConfiguration(
@@ -295,8 +302,7 @@ public final class FederationStateStoreFacade {
           stateStore.getPolicyConfiguration(
               GetSubClusterPolicyConfigurationRequest.newInstance(queue));
       if (response == null) {
-        throw new YarnException("The stateStore returned a null for "
-            + "GetSubClusterPolicyConfigurationResponse for queue " + queue);
+        return null;
       } else {
         return response.getPolicyConfiguration();
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6cd635f1/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 db04592..15cc0f0 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
@@ -38,6 +38,7 @@ import org.apache.hadoop.yarn.server.federation.store.records.GetApplicationHome
 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;
@@ -157,13 +158,8 @@ public abstract class FederationStateStoreBaseTest {
     GetSubClusterInfoRequest request =
         GetSubClusterInfoRequest.newInstance(subClusterId);
 
-    try {
-      stateStore.getSubCluster(request).getSubClusterInfo();
-      Assert.fail();
-    } catch (FederationStateStoreException e) {
-      Assert.assertTrue(
-          e.getMessage().startsWith("SubCluster SC does not exist"));
-    }
+    GetSubClusterInfoResponse response = stateStore.getSubCluster(request);
+    Assert.assertNull(response);
   }
 
   @Test
@@ -473,13 +469,10 @@ public abstract class FederationStateStoreBaseTest {
 
     GetSubClusterPolicyConfigurationRequest request =
         GetSubClusterPolicyConfigurationRequest.newInstance("Queue");
-    try {
-      stateStore.getPolicyConfiguration(request);
-      Assert.fail();
-    } catch (FederationStateStoreException e) {
-      Assert.assertTrue(
-          e.getMessage().startsWith("Policy for queue Queue does not exist"));
-    }
+
+    GetSubClusterPolicyConfigurationResponse response =
+        stateStore.getPolicyConfiguration(request);
+    Assert.assertNull(response);
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6cd635f1/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 423bf86..5d4c8d5 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
@@ -51,6 +51,7 @@ public class FederationStateStoreTestUtil {
   public static final String SC_PREFIX = "SC-";
   public static final String Q_PREFIX = "queue-";
   public static final String POLICY_PREFIX = "policy-";
+  public static final String INVALID = "dummy";
 
   private FederationStateStore stateStore;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6cd635f1/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 d46bef0..6328122 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
@@ -47,9 +47,10 @@ import org.junit.runners.Parameterized.Parameters;
 public class TestFederationStateStoreFacade {
 
   @Parameters
+  @SuppressWarnings({"NoWhitespaceAfter"})
   public static Collection<Boolean[]> getParameters() {
     return Arrays
-        .asList(new Boolean[][] {{Boolean.FALSE }, {Boolean.TRUE } });
+        .asList(new Boolean[][] { { Boolean.FALSE }, { Boolean.TRUE } });
   }
 
   private final long clusterTs = System.currentTimeMillis();
@@ -99,6 +100,13 @@ public class TestFederationStateStoreFacade {
   }
 
   @Test
+  public void testInvalidGetSubCluster() throws YarnException {
+    SubClusterId subClusterId =
+        SubClusterId.newInstance(FederationStateStoreTestUtil.INVALID);
+    Assert.assertNull(facade.getSubCluster(subClusterId));
+  }
+
+  @Test
   public void testGetSubClusterFlushCache() throws YarnException {
     for (int i = 0; i < numSubClusters; i++) {
       SubClusterId subClusterId =
@@ -128,6 +136,12 @@ public class TestFederationStateStoreFacade {
   }
 
   @Test
+  public void testInvalidGetPolicyConfiguration() throws YarnException {
+    Assert.assertNull(
+        facade.getPolicyConfiguration(FederationStateStoreTestUtil.INVALID));
+  }
+
+  @Test
   public void testGetPoliciesConfigurations() throws YarnException {
     Map<String, SubClusterPolicyConfiguration> queuePolicies =
         facade.getPoliciesConfigurations();


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


[35/50] [abbrv] 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/c1990752
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/c1990752
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/c1990752

Branch: refs/heads/YARN-2915
Commit: c199075257903b91a8006e10300ecbbc556c4651
Parents: 5627b4e
Author: Carlo Curino <cu...@apache.org>
Authored: Tue Apr 25 15:14:02 2017 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Tue Jul 25 16:56:32 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/c1990752/LICENSE.txt
----------------------------------------------------------------------
diff --git a/LICENSE.txt b/LICENSE.txt
index 5391fd5..af3175a 100644
--- a/LICENSE.txt
+++ b/LICENSE.txt
@@ -698,6 +698,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/c1990752/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 5395a10..15bd1fa 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>
@@ -1284,6 +1286,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/c1990752/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 987f8cf..7bcb123 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
@@ -2614,6 +2614,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/c1990752/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/c1990752/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 6cf41e7..c9f6d79 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
@@ -110,6 +110,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/c1990752/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/c1990752/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/c1990752/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/c1990752/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/c1990752/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/c1990752/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/c1990752/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/c1990752/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/c1990752/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


[24/50] [abbrv] 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/9657be49
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/9657be49
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/9657be49

Branch: refs/heads/YARN-2915
Commit: 9657be491acac08b6467edfb93d21cf6ee36a391
Parents: 13a87bf
Author: Subru Krishnan <su...@apache.org>
Authored: Tue Nov 22 18:30:40 2016 -0800
Committer: Subru Krishnan <su...@apache.org>
Committed: Tue Jul 25 16:56:32 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/9657be49/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/9657be49/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


[27/50] [abbrv] hadoop git commit: YARN-3659. Federation: routing client invocations transparently to multiple RMs. (Giovanni Matteo Fumarola via Subru).

Posted by su...@apache.org.
YARN-3659. Federation: routing client invocations transparently to multiple RMs. (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/9edf7208
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/9edf7208
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/9edf7208

Branch: refs/heads/YARN-2915
Commit: 9edf7208333bc98a274d447bd932b6df785efc2d
Parents: a4f9fdf
Author: Subru Krishnan <su...@apache.org>
Authored: Mon Jun 26 13:27:26 2017 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Tue Jul 25 16:56:32 2017 -0700

----------------------------------------------------------------------
 .../hadoop/yarn/conf/YarnConfiguration.java     |   8 +
 .../yarn/conf/TestYarnConfigurationFields.java  |   5 +
 .../hadoop/yarn/conf/TestYarnConfiguration.java |   1 -
 ...ionPolicyInitializationContextValidator.java |   2 +-
 .../policies/FederationPolicyUtils.java         |  35 +
 .../federation/policies/RouterPolicyFacade.java |   3 +-
 .../policies/router/HashBasedRouterPolicy.java  |   5 +
 .../policies/router/LoadBasedRouterPolicy.java  |   5 +
 .../policies/router/PriorityRouterPolicy.java   |   5 +
 .../router/UniformRandomRouterPolicy.java       |   4 +
 .../router/WeightedRandomRouterPolicy.java      |   5 +
 .../yarn/server/MockResourceManagerFacade.java  | 165 ++++-
 .../policies/router/BaseRouterPoliciesTest.java |  28 +
 .../utils/FederationStateStoreTestUtil.java     |  15 +-
 .../hadoop-yarn-server-router/pom.xml           |   5 +
 .../yarn/server/router/RouterServerUtil.java    |  63 ++
 .../AbstractClientRequestInterceptor.java       |  41 +-
 .../DefaultClientRequestInterceptor.java        |  31 +-
 .../clientrm/FederationClientInterceptor.java   | 684 +++++++++++++++++++
 .../router/clientrm/BaseRouterClientRMTest.java |  22 +-
 .../TestFederationClientInterceptor.java        | 403 +++++++++++
 .../TestFederationClientInterceptorRetry.java   | 295 ++++++++
 .../TestableFederationClientInterceptor.java    |  75 ++
 23 files changed, 1866 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9edf7208/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 1432867..0c0dd11 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
@@ -2674,6 +2674,14 @@ public class YarnConfiguration extends Configuration {
       "org.apache.hadoop.yarn.server.router.rmadmin."
           + "DefaultRMAdminRequestInterceptor";
 
+  /**
+   * The number of retries for GetNewApplication and SubmitApplication in
+   * {@code FederationClientInterceptor}.
+   */
+  public static final String ROUTER_CLIENTRM_SUBMIT_RETRY =
+      ROUTER_PREFIX + "submit.retry";
+  public static final int DEFAULT_ROUTER_CLIENTRM_SUBMIT_RETRY = 3;
+
   ////////////////////////////////
   // Other Configs
   ////////////////////////////////

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9edf7208/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 c3cb78d..a439cd5 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
@@ -136,6 +136,11 @@ public class TestYarnConfigurationFields extends TestConfigurationFieldsBase {
     configurationPrefixToSkipCompare
         .add(YarnConfiguration.NM_CPU_RESOURCE_ENABLED);
 
+    // Ignore all Router Federation variables
+
+    configurationPrefixToSkipCompare
+        .add(YarnConfiguration.ROUTER_CLIENTRM_SUBMIT_RETRY);
+
     // Set by container-executor.cfg
     configurationPrefixToSkipCompare.add(YarnConfiguration.NM_USER_HOME_DIR);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9edf7208/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfiguration.java
index 7389423..a053fdb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfiguration.java
@@ -24,7 +24,6 @@ import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
 import org.junit.Test;
 
 import java.net.InetSocketAddress;
-import java.net.SocketAddress;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9edf7208/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 3c44e7e..da63bc1 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
@@ -51,7 +51,7 @@ public final class FederationPolicyInitializationContextValidator {
 
     if (policyContext.getFederationSubclusterResolver() == null) {
       throw new FederationPolicyInitializationException(
-          "The FederationStateStoreFacase provided is null. Cannot"
+          "The FederationSubclusterResolver provided is null. Cannot"
               + " reinitalize successfully.");
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9edf7208/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyUtils.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/FederationPolicyUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyUtils.java
index 37ce942..97e4848 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/policies/FederationPolicyUtils.java
@@ -19,12 +19,14 @@ package org.apache.hadoop.yarn.server.federation.policies;
 
 import java.nio.ByteBuffer;
 import java.nio.charset.StandardCharsets;
+import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 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.amrmproxy.FederationAMRMProxyPolicy;
+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.store.records.SubClusterId;
@@ -41,6 +43,9 @@ public final class FederationPolicyUtils {
   private static final Logger LOG =
       LoggerFactory.getLogger(FederationPolicyUtils.class);
 
+  public static final String NO_ACTIVE_SUBCLUSTER_AVAILABLE =
+      "No active SubCluster available to submit the request.";
+
   /** Disable constructor. */
   private FederationPolicyUtils() {
   }
@@ -165,4 +170,34 @@ public final class FederationPolicyUtils {
     return federationPolicyManager.getAMRMPolicy(context, oldPolicy);
   }
 
+  /**
+   * Validate if there is any active subcluster that is not blacklisted, it will
+   * throw an exception if there are no usable subclusters.
+   *
+   * @param activeSubClusters the list of subClusters as identified by
+   *          {@link SubClusterId} currently active.
+   * @param blackListSubClusters the list of subClusters as identified by
+   *          {@link SubClusterId} to blackList from the selection of the home
+   *          subCluster.
+   * @throws FederationPolicyException if there are no usable subclusters.
+   */
+  public static void validateSubClusterAvailability(
+      List<SubClusterId> activeSubClusters,
+      List<SubClusterId> blackListSubClusters)
+      throws FederationPolicyException {
+    if (activeSubClusters != null && !activeSubClusters.isEmpty()) {
+      if (blackListSubClusters == null) {
+        return;
+      }
+      for (SubClusterId scId : activeSubClusters) {
+        if (!blackListSubClusters.contains(scId)) {
+          // There is at least one active subcluster
+          return;
+        }
+      }
+    }
+    throw new FederationPolicyException(
+        FederationPolicyUtils.NO_ACTIVE_SUBCLUSTER_AVAILABLE);
+  }
+
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9edf7208/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 44c1b10..52c2905 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
@@ -25,6 +25,7 @@ import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
@@ -56,7 +57,7 @@ public class RouterPolicyFacade {
   @VisibleForTesting
   Map<String, FederationRouterPolicy> globalPolicyMap;
 
-  public RouterPolicyFacade(YarnConfiguration conf,
+  public RouterPolicyFacade(Configuration conf,
       FederationStateStoreFacade facade, SubClusterResolver resolver,
       SubClusterId homeSubcluster)
       throws FederationPolicyInitializationException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9edf7208/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
index 257a9fe..cc11880 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/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
@@ -26,6 +26,7 @@ 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.FederationPolicyUtils;
 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;
@@ -76,6 +77,10 @@ public class HashBasedRouterPolicy extends AbstractRouterPolicy {
     Map<SubClusterId, SubClusterInfo> activeSubclusters =
         getActiveSubclusters();
 
+    FederationPolicyUtils.validateSubClusterAvailability(
+        new ArrayList<SubClusterId>(activeSubclusters.keySet()),
+        blackListSubClusters);
+
     if (blackListSubClusters != null) {
 
       // Remove from the active SubClusters from StateStore the blacklisted ones

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9edf7208/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 c124001..06e445b 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,12 +17,14 @@
 
 package org.apache.hadoop.yarn.server.federation.policies.router;
 
+import java.util.ArrayList;
 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.FederationPolicyUtils;
 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;
@@ -72,6 +74,9 @@ public class LoadBasedRouterPolicy extends AbstractRouterPolicy {
     Map<SubClusterId, SubClusterInfo> activeSubclusters =
         getActiveSubclusters();
 
+    FederationPolicyUtils.validateSubClusterAvailability(
+        new ArrayList<SubClusterId>(activeSubclusters.keySet()), blacklist);
+
     Map<SubClusterIdInfo, Float> weights =
         getPolicyInfo().getRouterPolicyWeights();
     SubClusterIdInfo chosen = null;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9edf7208/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 59f8767..a1f7666 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,11 +17,13 @@
 
 package org.apache.hadoop.yarn.server.federation.policies.router;
 
+import java.util.ArrayList;
 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.FederationPolicyUtils;
 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;
@@ -44,6 +46,9 @@ public class PriorityRouterPolicy extends AbstractRouterPolicy {
     Map<SubClusterId, SubClusterInfo> activeSubclusters =
         getActiveSubclusters();
 
+    FederationPolicyUtils.validateSubClusterAvailability(
+        new ArrayList<SubClusterId>(activeSubclusters.keySet()), blacklist);
+
     // This finds the sub-cluster with the highest weight among the
     // currently active ones.
     Map<SubClusterIdInfo, Float> weights =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9edf7208/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 bc729b7..7a8be91 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
@@ -26,6 +26,7 @@ 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.FederationPolicyUtils;
 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;
@@ -86,6 +87,9 @@ public class UniformRandomRouterPolicy extends AbstractRouterPolicy {
 
     List<SubClusterId> list = new ArrayList<>(activeSubclusters.keySet());
 
+    FederationPolicyUtils.validateSubClusterAvailability(list,
+        blackListSubClusters);
+
     if (blackListSubClusters != null) {
 
       // Remove from the active SubClusters from StateStore the blacklisted ones

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9edf7208/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 7f230a7..aec7576 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,12 +18,14 @@
 
 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.FederationPolicyUtils;
 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;
@@ -51,6 +53,9 @@ public class WeightedRandomRouterPolicy extends AbstractRouterPolicy {
     Map<SubClusterId, SubClusterInfo> activeSubclusters =
         getActiveSubclusters();
 
+    FederationPolicyUtils.validateSubClusterAvailability(
+        new ArrayList<SubClusterId>(activeSubclusters.keySet()), blacklist);
+
     // 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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9edf7208/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
index 65c12c6..68c55ac 100644
--- 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
@@ -19,6 +19,7 @@
 package org.apache.hadoop.yarn.server;
 
 import java.io.IOException;
+import java.net.ConnectException;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -154,6 +155,7 @@ 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;
 
 /**
@@ -175,6 +177,13 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
       new HashMap<ContainerId, Container>();
   private AtomicInteger containerIndex = new AtomicInteger(0);
   private Configuration conf;
+  private int subClusterId;
+  final private AtomicInteger applicationCounter = new AtomicInteger(0);
+
+  // True if the Mock RM is running, false otherwise.
+  // This property allows us to write tests for specific scenario as Yarn RM
+  // down e.g. network issue, failover.
+  private boolean isRunning;
 
   private boolean shouldReRegisterNext = false;
 
@@ -187,14 +196,25 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
 
   public MockResourceManagerFacade(Configuration conf,
       int startContainerIndex) {
+    this(conf, startContainerIndex, 0, true);
+  }
+
+  public MockResourceManagerFacade(Configuration conf, int startContainerIndex,
+      int subClusterId, boolean isRunning) {
     this.conf = conf;
     this.containerIndex.set(startContainerIndex);
+    this.subClusterId = subClusterId;
+    this.isRunning = isRunning;
   }
 
   public void setShouldReRegisterNext() {
     shouldReRegisterNext = true;
   }
 
+  public void setRunningMode(boolean mode) {
+    this.isRunning = mode;
+  }
+
   private static String getAppIdentifier() throws IOException {
     AMRMTokenIdentifier result = null;
     UserGroupInformation remoteUgi = UserGroupInformation.getCurrentUser();
@@ -208,10 +228,19 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
     return result != null ? result.getApplicationAttemptId().toString() : "";
   }
 
+  private void validateRunning() throws ConnectException {
+    if (!isRunning) {
+      throw new ConnectException("RM is stopped");
+    }
+  }
+
   @Override
   public RegisterApplicationMasterResponse registerApplicationMaster(
       RegisterApplicationMasterRequest request)
       throws YarnException, IOException {
+
+    validateRunning();
+
     String amrmToken = getAppIdentifier();
     LOG.info("Registering application attempt: " + amrmToken);
 
@@ -248,6 +277,9 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
   public FinishApplicationMasterResponse finishApplicationMaster(
       FinishApplicationMasterRequest request)
       throws YarnException, IOException {
+
+    validateRunning();
+
     String amrmToken = getAppIdentifier();
     LOG.info("Finishing application attempt: " + amrmToken);
 
@@ -284,6 +316,9 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
   @Override
   public AllocateResponse allocate(AllocateRequest request)
       throws YarnException, IOException {
+
+    validateRunning();
+
     if (request.getAskList() != null && request.getAskList().size() > 0
         && request.getReleaseList() != null
         && request.getReleaseList().size() > 0) {
@@ -391,6 +426,8 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
   public GetApplicationReportResponse getApplicationReport(
       GetApplicationReportRequest request) throws YarnException, IOException {
 
+    validateRunning();
+
     GetApplicationReportResponse response =
         Records.newRecord(GetApplicationReportResponse.class);
     ApplicationReport report = Records.newRecord(ApplicationReport.class);
@@ -407,6 +444,8 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
       GetApplicationAttemptReportRequest request)
       throws YarnException, IOException {
 
+    validateRunning();
+
     GetApplicationAttemptReportResponse response =
         Records.newRecord(GetApplicationAttemptReportResponse.class);
     ApplicationAttemptReport report =
@@ -420,12 +459,19 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
   @Override
   public GetNewApplicationResponse getNewApplication(
       GetNewApplicationRequest request) throws YarnException, IOException {
-    return GetNewApplicationResponse.newInstance(null, null, null);
+
+    validateRunning();
+
+    return GetNewApplicationResponse.newInstance(ApplicationId.newInstance(
+        subClusterId, applicationCounter.incrementAndGet()), null, null);
   }
 
   @Override
   public SubmitApplicationResponse submitApplication(
       SubmitApplicationRequest request) throws YarnException, IOException {
+
+    validateRunning();
+
     ApplicationId appId = null;
     if (request.getApplicationSubmissionContext() != null) {
       appId = request.getApplicationSubmissionContext().getApplicationId();
@@ -438,6 +484,9 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
   @Override
   public KillApplicationResponse forceKillApplication(
       KillApplicationRequest request) throws YarnException, IOException {
+
+    validateRunning();
+
     ApplicationId appId = null;
     if (request.getApplicationId() != null) {
       appId = request.getApplicationId();
@@ -453,48 +502,72 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
   @Override
   public GetClusterMetricsResponse getClusterMetrics(
       GetClusterMetricsRequest request) throws YarnException, IOException {
+
+    validateRunning();
+
     return GetClusterMetricsResponse.newInstance(null);
   }
 
   @Override
   public GetApplicationsResponse getApplications(GetApplicationsRequest request)
       throws YarnException, IOException {
+
+    validateRunning();
+
     return GetApplicationsResponse.newInstance(null);
   }
 
   @Override
   public GetClusterNodesResponse getClusterNodes(GetClusterNodesRequest request)
       throws YarnException, IOException {
+
+    validateRunning();
+
     return GetClusterNodesResponse.newInstance(null);
   }
 
   @Override
   public GetQueueInfoResponse getQueueInfo(GetQueueInfoRequest request)
       throws YarnException, IOException {
+
+    validateRunning();
+
     return GetQueueInfoResponse.newInstance(null);
   }
 
   @Override
   public GetQueueUserAclsInfoResponse getQueueUserAcls(
       GetQueueUserAclsInfoRequest request) throws YarnException, IOException {
+
+    validateRunning();
+
     return GetQueueUserAclsInfoResponse.newInstance(null);
   }
 
   @Override
   public GetDelegationTokenResponse getDelegationToken(
       GetDelegationTokenRequest request) throws YarnException, IOException {
+
+    validateRunning();
+
     return GetDelegationTokenResponse.newInstance(null);
   }
 
   @Override
   public RenewDelegationTokenResponse renewDelegationToken(
       RenewDelegationTokenRequest request) throws YarnException, IOException {
+
+    validateRunning();
+
     return RenewDelegationTokenResponse.newInstance(0);
   }
 
   @Override
   public CancelDelegationTokenResponse cancelDelegationToken(
       CancelDelegationTokenRequest request) throws YarnException, IOException {
+
+    validateRunning();
+
     return CancelDelegationTokenResponse.newInstance();
   }
 
@@ -502,36 +575,54 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
   public MoveApplicationAcrossQueuesResponse moveApplicationAcrossQueues(
       MoveApplicationAcrossQueuesRequest request)
       throws YarnException, IOException {
+
+    validateRunning();
+
     return MoveApplicationAcrossQueuesResponse.newInstance();
   }
 
   @Override
   public GetApplicationAttemptsResponse getApplicationAttempts(
       GetApplicationAttemptsRequest request) throws YarnException, IOException {
+
+    validateRunning();
+
     return GetApplicationAttemptsResponse.newInstance(null);
   }
 
   @Override
   public GetContainerReportResponse getContainerReport(
       GetContainerReportRequest request) throws YarnException, IOException {
+
+    validateRunning();
+
     return GetContainerReportResponse.newInstance(null);
   }
 
   @Override
   public GetContainersResponse getContainers(GetContainersRequest request)
       throws YarnException, IOException {
+
+    validateRunning();
+
     return GetContainersResponse.newInstance(null);
   }
 
   @Override
   public ReservationSubmissionResponse submitReservation(
       ReservationSubmissionRequest request) throws YarnException, IOException {
+
+    validateRunning();
+
     return ReservationSubmissionResponse.newInstance();
   }
 
   @Override
   public ReservationListResponse listReservations(
       ReservationListRequest request) throws YarnException, IOException {
+
+    validateRunning();
+
     return ReservationListResponse
         .newInstance(new ArrayList<ReservationAllocationState>());
   }
@@ -539,18 +630,27 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
   @Override
   public ReservationUpdateResponse updateReservation(
       ReservationUpdateRequest request) throws YarnException, IOException {
+
+    validateRunning();
+
     return ReservationUpdateResponse.newInstance();
   }
 
   @Override
   public ReservationDeleteResponse deleteReservation(
       ReservationDeleteRequest request) throws YarnException, IOException {
+
+    validateRunning();
+
     return ReservationDeleteResponse.newInstance();
   }
 
   @Override
   public GetNodesToLabelsResponse getNodeToLabels(
       GetNodesToLabelsRequest request) throws YarnException, IOException {
+
+    validateRunning();
+
     return GetNodesToLabelsResponse
         .newInstance(new HashMap<NodeId, Set<String>>());
   }
@@ -558,18 +658,27 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
   @Override
   public GetClusterNodeLabelsResponse getClusterNodeLabels(
       GetClusterNodeLabelsRequest request) throws YarnException, IOException {
+
+    validateRunning();
+
     return GetClusterNodeLabelsResponse.newInstance(new ArrayList<NodeLabel>());
   }
 
   @Override
   public GetLabelsToNodesResponse getLabelsToNodes(
       GetLabelsToNodesRequest request) throws YarnException, IOException {
+
+    validateRunning();
+
     return GetLabelsToNodesResponse.newInstance(null);
   }
 
   @Override
   public GetNewReservationResponse getNewReservation(
       GetNewReservationRequest request) throws YarnException, IOException {
+
+    validateRunning();
+
     return GetNewReservationResponse
         .newInstance(ReservationId.newInstance(0, 0));
   }
@@ -577,6 +686,9 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
   @Override
   public FailApplicationAttemptResponse failApplicationAttempt(
       FailApplicationAttemptRequest request) throws YarnException, IOException {
+
+    validateRunning();
+
     return FailApplicationAttemptResponse.newInstance();
   }
 
@@ -584,12 +696,18 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
   public UpdateApplicationPriorityResponse updateApplicationPriority(
       UpdateApplicationPriorityRequest request)
       throws YarnException, IOException {
+
+    validateRunning();
+
     return UpdateApplicationPriorityResponse.newInstance(null);
   }
 
   @Override
   public SignalContainerResponse signalToContainer(
       SignalContainerRequest request) throws YarnException, IOException {
+
+    validateRunning();
+
     return new SignalContainerResponsePBImpl();
   }
 
@@ -597,18 +715,27 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
   public UpdateApplicationTimeoutsResponse updateApplicationTimeouts(
       UpdateApplicationTimeoutsRequest request)
       throws YarnException, IOException {
+
+    validateRunning();
+
     return UpdateApplicationTimeoutsResponse.newInstance();
   }
 
   @Override
   public RefreshQueuesResponse refreshQueues(RefreshQueuesRequest request)
       throws StandbyException, YarnException, IOException {
+
+    validateRunning();
+
     return RefreshQueuesResponse.newInstance();
   }
 
   @Override
   public RefreshNodesResponse refreshNodes(RefreshNodesRequest request)
       throws StandbyException, YarnException, IOException {
+
+    validateRunning();
+
     return RefreshNodesResponse.newInstance();
   }
 
@@ -616,6 +743,9 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
   public RefreshSuperUserGroupsConfigurationResponse refreshSuperUserGroupsConfiguration(
       RefreshSuperUserGroupsConfigurationRequest request)
       throws StandbyException, YarnException, IOException {
+
+    validateRunning();
+
     return RefreshSuperUserGroupsConfigurationResponse.newInstance();
   }
 
@@ -623,36 +753,54 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
   public RefreshUserToGroupsMappingsResponse refreshUserToGroupsMappings(
       RefreshUserToGroupsMappingsRequest request)
       throws StandbyException, YarnException, IOException {
+
+    validateRunning();
+
     return RefreshUserToGroupsMappingsResponse.newInstance();
   }
 
   @Override
   public RefreshAdminAclsResponse refreshAdminAcls(
       RefreshAdminAclsRequest request) throws YarnException, IOException {
+
+    validateRunning();
+
     return RefreshAdminAclsResponse.newInstance();
   }
 
   @Override
   public RefreshServiceAclsResponse refreshServiceAcls(
       RefreshServiceAclsRequest request) throws YarnException, IOException {
+
+    validateRunning();
+
     return RefreshServiceAclsResponse.newInstance();
   }
 
   @Override
   public UpdateNodeResourceResponse updateNodeResource(
       UpdateNodeResourceRequest request) throws YarnException, IOException {
+
+    validateRunning();
+
     return UpdateNodeResourceResponse.newInstance();
   }
 
   @Override
   public RefreshNodesResourcesResponse refreshNodesResources(
       RefreshNodesResourcesRequest request) throws YarnException, IOException {
+
+    validateRunning();
+
     return RefreshNodesResourcesResponse.newInstance();
   }
 
   @Override
   public AddToClusterNodeLabelsResponse addToClusterNodeLabels(
       AddToClusterNodeLabelsRequest request) throws YarnException, IOException {
+
+    validateRunning();
+
     return AddToClusterNodeLabelsResponse.newInstance();
   }
 
@@ -660,12 +808,18 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
   public RemoveFromClusterNodeLabelsResponse removeFromClusterNodeLabels(
       RemoveFromClusterNodeLabelsRequest request)
       throws YarnException, IOException {
+
+    validateRunning();
+
     return RemoveFromClusterNodeLabelsResponse.newInstance();
   }
 
   @Override
   public ReplaceLabelsOnNodeResponse replaceLabelsOnNode(
       ReplaceLabelsOnNodeRequest request) throws YarnException, IOException {
+
+    validateRunning();
+
     return ReplaceLabelsOnNodeResponse.newInstance();
   }
 
@@ -673,6 +827,9 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
   public CheckForDecommissioningNodesResponse checkForDecommissioningNodes(
       CheckForDecommissioningNodesRequest checkForDecommissioningNodesRequest)
       throws YarnException, IOException {
+
+    validateRunning();
+
     return CheckForDecommissioningNodesResponse.newInstance(null);
   }
 
@@ -680,11 +837,17 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
   public RefreshClusterMaxPriorityResponse refreshClusterMaxPriority(
       RefreshClusterMaxPriorityRequest request)
       throws YarnException, IOException {
+
+    validateRunning();
+
     return RefreshClusterMaxPriorityResponse.newInstance();
   }
 
   @Override
   public String[] getGroupsForUser(String user) throws IOException {
+
+    validateRunning();
+
     return new String[0];
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9edf7208/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
index c7a7767..d09ba75 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/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
@@ -26,6 +26,7 @@ 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.BaseFederationPoliciesTest;
+import org.apache.hadoop.yarn.server.federation.policies.FederationPolicyUtils;
 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;
@@ -87,4 +88,31 @@ public abstract class BaseRouterPoliciesTest
       Assert.assertEquals(removed, chosen);
     }
   }
+
+  /**
+   * This test validates the correctness of blacklist logic in case the cluster
+   * has no active subclusters.
+   */
+  @Test
+  public void testAllBlacklistSubcluster() throws YarnException {
+    FederationRouterPolicy localPolicy = (FederationRouterPolicy) getPolicy();
+    ApplicationSubmissionContext applicationSubmissionContext =
+        ApplicationSubmissionContext.newInstance(null, null, null, null, null,
+            false, false, 0, Resources.none(), null, false, null, null);
+    Map<SubClusterId, SubClusterInfo> activeSubClusters =
+        getActiveSubclusters();
+    if (activeSubClusters != null && activeSubClusters.size() > 1
+        && !(localPolicy instanceof RejectRouterPolicy)) {
+      List<SubClusterId> blacklistSubclusters =
+          new ArrayList<SubClusterId>(activeSubClusters.keySet());
+      try {
+        localPolicy.getHomeSubcluster(applicationSubmissionContext,
+            blacklistSubclusters);
+        Assert.fail();
+      } catch (YarnException e) {
+        Assert.assertTrue(e.getMessage()
+            .equals(FederationPolicyUtils.NO_ACTIVE_SUBCLUSTER_AVAILABLE));
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9edf7208/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 649a61b..423bf86 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
@@ -33,6 +33,7 @@ import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterPolic
 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.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.SubClusterPolicyConfiguration;
@@ -69,7 +70,7 @@ public class FederationStateStoreTestUtil {
         SubClusterState.SC_RUNNING, CLOCK.getTime(), "capability");
   }
 
-  private void registerSubCluster(SubClusterId subClusterId)
+  public void registerSubCluster(SubClusterId subClusterId)
       throws YarnException {
 
     SubClusterInfo subClusterInfo = createSubClusterInfo(subClusterId);
@@ -164,4 +165,16 @@ public class FederationStateStoreTestUtil {
     return result.getPolicyConfiguration();
   }
 
+  public void deregisterAllSubClusters() throws YarnException {
+    for (SubClusterId sc : getAllSubClusterIds(true)) {
+      deRegisterSubCluster(sc);
+    }
+  }
+
+  private void deRegisterSubCluster(SubClusterId subClusterId)
+      throws YarnException {
+    stateStore.deregisterSubCluster(SubClusterDeregisterRequest
+        .newInstance(subClusterId, SubClusterState.SC_UNREGISTERED));
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9edf7208/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 f9169e1..f27b2b2 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
@@ -58,6 +58,11 @@
     <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-yarn-server-common</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-server-common</artifactId>
       <type>test-jar</type>
       <scope>test</scope>
     </dependency>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9edf7208/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/RouterServerUtil.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/RouterServerUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/RouterServerUtil.java
new file mode 100644
index 0000000..cc96da6
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/RouterServerUtil.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.router;
+
+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.exceptions.YarnException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Common utility methods used by the Router server.
+ *
+ */
+@Private
+@Unstable
+public final class RouterServerUtil {
+
+  /** Disable constructor. */
+  private RouterServerUtil() {
+  }
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(RouterServerUtil.class);
+
+  /**
+   * Throws an exception due to an error.
+   *
+   * @param errMsg the error message
+   * @param t the throwable raised in the called class.
+   * @throws YarnException on failure
+   */
+  @Public
+  @Unstable
+  public static void logAndThrowException(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);
+    }
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9edf7208/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
index 5980b03..01ba3bd 100644
--- 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
@@ -18,7 +18,13 @@
 
 package org.apache.hadoop.yarn.server.router.clientrm;
 
+import java.io.IOException;
+
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Implements the {@link ClientRequestInterceptor} interface and provides common
@@ -28,9 +34,16 @@ import org.apache.hadoop.conf.Configuration;
  */
 public abstract class AbstractClientRequestInterceptor
     implements ClientRequestInterceptor {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(AbstractClientRequestInterceptor.class);
+
   private Configuration conf;
   private ClientRequestInterceptor nextInterceptor;
 
+  @SuppressWarnings("checkstyle:visibilitymodifier")
+  protected UserGroupInformation user = null;
+
   /**
    * Sets the {@link ClientRequestInterceptor} in the chain.
    */
@@ -63,9 +76,10 @@ public abstract class AbstractClientRequestInterceptor
    * Initializes the {@link ClientRequestInterceptor}.
    */
   @Override
-  public void init(String user) {
+  public void init(String userName) {
+    setupUser(userName);
     if (this.nextInterceptor != null) {
-      this.nextInterceptor.init(user);
+      this.nextInterceptor.init(userName);
     }
   }
 
@@ -87,4 +101,27 @@ public abstract class AbstractClientRequestInterceptor
     return this.nextInterceptor;
   }
 
+  private void setupUser(String 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());
+      }
+    } 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);
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9edf7208/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
index 9e2bfed..71de6b4 100644
--- 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
@@ -22,7 +22,6 @@ 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;
@@ -85,8 +84,6 @@ import org.apache.hadoop.yarn.api.protocolrecords.UpdateApplicationTimeoutsRespo
 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;
 
@@ -98,27 +95,14 @@ import com.google.common.annotations.VisibleForTesting;
  */
 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();
 
+    final Configuration conf = this.getConf();
+    try {
       clientRMProxy =
           user.doAs(new PrivilegedExceptionAction<ApplicationClientProtocol>() {
             @Override
@@ -127,16 +111,9 @@ public class DefaultClientRequestInterceptor
                   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);
+      throw new YarnRuntimeException(
+          "Unable to create the interface to reach the YarnRM", e);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9edf7208/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/FederationClientInterceptor.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/FederationClientInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/FederationClientInterceptor.java
new file mode 100644
index 0000000..ecf53ac
--- /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/FederationClientInterceptor.java
@@ -0,0 +1,684 @@
+/**
+ * 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.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.commons.lang.NotImplementedException;
+import org.apache.hadoop.conf.Configuration;
+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.api.records.ApplicationId;
+import org.apache.hadoop.yarn.client.ClientRMProxy;
+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.policies.FederationPolicyUtils;
+import org.apache.hadoop.yarn.server.federation.policies.RouterPolicyFacade;
+import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
+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.utils.FederationStateStoreFacade;
+import org.apache.hadoop.yarn.server.router.RouterServerUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Extends the {@code AbstractRequestInterceptorClient} class and provides an
+ * implementation for federation of YARN RM and scaling an application across
+ * multiple YARN SubClusters. All the federation specific implementation is
+ * encapsulated in this class. This is always the last intercepter in the chain.
+ */
+public class FederationClientInterceptor
+    extends AbstractClientRequestInterceptor {
+
+  /*
+   * TODO YARN-6740 Federation Router (hiding multiple RMs for
+   * ApplicationClientProtocol) phase 2.
+   *
+   * The current implementation finalized the main 4 calls (getNewApplication,
+   * submitApplication, forceKillApplication and getApplicationReport). Those
+   * allow us to execute applications E2E.
+   */
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(FederationClientInterceptor.class);
+
+  private int numSubmitRetries;
+  private Map<SubClusterId, ApplicationClientProtocol> clientRMProxies;
+  private FederationStateStoreFacade federationFacade;
+  private Random rand;
+  private RouterPolicyFacade policyFacade;
+
+  @Override
+  public void init(String userName) {
+    super.init(userName);
+
+    federationFacade = FederationStateStoreFacade.getInstance();
+    rand = new Random(System.currentTimeMillis());
+
+    final Configuration conf = this.getConf();
+
+    try {
+      policyFacade = new RouterPolicyFacade(conf, federationFacade,
+          this.federationFacade.getSubClusterResolver(), null);
+    } catch (FederationPolicyInitializationException e) {
+      LOG.error(e.getMessage());
+    }
+
+    numSubmitRetries =
+        conf.getInt(YarnConfiguration.ROUTER_CLIENTRM_SUBMIT_RETRY,
+            YarnConfiguration.DEFAULT_ROUTER_CLIENTRM_SUBMIT_RETRY);
+
+    clientRMProxies =
+        new ConcurrentHashMap<SubClusterId, ApplicationClientProtocol>();
+
+  }
+
+  @Override
+  public void setNextInterceptor(ClientRequestInterceptor next) {
+    throw new YarnRuntimeException("setNextInterceptor is being called on "
+        + "FederationClientRequestInterceptor, which should be the last one "
+        + "in the chain. Check if the interceptor pipeline configuration "
+        + "is correct");
+  }
+
+  @VisibleForTesting
+  protected ApplicationClientProtocol getClientRMProxyForSubCluster(
+      SubClusterId subClusterId) throws YarnException {
+
+    if (clientRMProxies.containsKey(subClusterId)) {
+      return clientRMProxies.get(subClusterId);
+    }
+
+    ApplicationClientProtocol clientRMProxy = null;
+    try {
+      clientRMProxy =
+          user.doAs(new PrivilegedExceptionAction<ApplicationClientProtocol>() {
+            @Override
+            public ApplicationClientProtocol run() throws Exception {
+              return ClientRMProxy.createRMProxy(getConf(),
+                  ApplicationClientProtocol.class);
+            }
+          });
+    } catch (Exception e) {
+      RouterServerUtil.logAndThrowException(
+          "Unable to create the interface to reach the SubCluster "
+              + subClusterId,
+          e);
+    }
+
+    clientRMProxies.put(subClusterId, clientRMProxy);
+    return clientRMProxy;
+  }
+
+  private SubClusterId getRandomActiveSubCluster(
+      Map<SubClusterId, SubClusterInfo> activeSubclusters)
+      throws YarnException {
+
+    if (activeSubclusters == null || activeSubclusters.size() < 1) {
+      RouterServerUtil.logAndThrowException(
+          FederationPolicyUtils.NO_ACTIVE_SUBCLUSTER_AVAILABLE, null);
+    }
+    List<SubClusterId> list = new ArrayList<>(activeSubclusters.keySet());
+
+    return list.get(rand.nextInt(list.size()));
+  }
+
+  /**
+   * Yarn Router forwards every getNewApplication requests to any RM. During
+   * this operation there will be no communication with the State Store. The
+   * Router will forward the requests to any SubCluster. The Router will retry
+   * to submit the request on #numSubmitRetries different SubClusters. The
+   * SubClusters are randomly chosen from the active ones.
+   *
+   * Possible failures and behaviors:
+   *
+   * Client: identical behavior as {@code ClientRMService}.
+   *
+   * Router: the Client will timeout and resubmit.
+   *
+   * ResourceManager: the Router will timeout and contacts another RM.
+   *
+   * StateStore: not in the execution.
+   */
+  @Override
+  public GetNewApplicationResponse getNewApplication(
+      GetNewApplicationRequest request) throws YarnException, IOException {
+    Map<SubClusterId, SubClusterInfo> subClustersActive =
+        federationFacade.getSubClusters(true);
+
+    for (int i = 0; i < numSubmitRetries; ++i) {
+      SubClusterId subClusterId = getRandomActiveSubCluster(subClustersActive);
+      LOG.debug(
+          "getNewApplication try #" + i + " on SubCluster " + subClusterId);
+      ApplicationClientProtocol clientRMProxy =
+          getClientRMProxyForSubCluster(subClusterId);
+      GetNewApplicationResponse response = null;
+      try {
+        response = clientRMProxy.getNewApplication(request);
+      } catch (Exception e) {
+        LOG.warn("Unable to create a new ApplicationId in SubCluster "
+            + subClusterId.getId(), e);
+      }
+
+      if (response != null) {
+        return response;
+      } else {
+        // Empty response from the ResourceManager.
+        // Blacklist this subcluster for this request.
+        subClustersActive.remove(subClusterId);
+      }
+
+    }
+
+    String errMsg = "Fail to create a new application.";
+    LOG.error(errMsg);
+    throw new YarnException(errMsg);
+  }
+
+  /**
+   * Today, in YARN there are no checks of any applicationId submitted.
+   *
+   * Base scenarios:
+   *
+   * The Client submits an application to the Router. • The Router selects one
+   * SubCluster to forward the request. • The Router inserts a tuple into
+   * StateStore with the selected SubCluster (e.g. SC1) and the appId. • The
+   * State Store replies with the selected SubCluster (e.g. SC1). • The Router
+   * submits the request to the selected SubCluster.
+   *
+   * In case of State Store failure:
+   *
+   * The client submits an application to the Router. • The Router selects one
+   * SubCluster to forward the request. • The Router inserts a tuple into State
+   * Store with the selected SubCluster (e.g. SC1) and the appId. • Due to the
+   * State Store down the Router times out and it will retry depending on the
+   * FederationFacade settings. • The Router replies to the client with an error
+   * message.
+   *
+   * If State Store fails after inserting the tuple: identical behavior as
+   * {@code ClientRMService}.
+   *
+   * In case of Router failure:
+   *
+   * Scenario 1 – Crash before submission to the ResourceManager
+   *
+   * The Client submits an application to the Router. • The Router selects one
+   * SubCluster to forward the request. • The Router inserts a tuple into State
+   * Store with the selected SubCluster (e.g. SC1) and the appId. • The Router
+   * crashes. • The Client timeouts and resubmits the application. • The Router
+   * selects one SubCluster to forward the request. • The Router inserts a tuple
+   * into State Store with the selected SubCluster (e.g. SC2) and the appId. •
+   * Because the tuple is already inserted in the State Store, it returns the
+   * previous selected SubCluster (e.g. SC1). • The Router submits the request
+   * to the selected SubCluster (e.g. SC1).
+   *
+   * Scenario 2 – Crash after submission to the ResourceManager
+   *
+   * • The Client submits an application to the Router. • The Router selects one
+   * SubCluster to forward the request. • The Router inserts a tuple into State
+   * Store with the selected SubCluster (e.g. SC1) and the appId. • The Router
+   * submits the request to the selected SubCluster. • The Router crashes. • The
+   * Client timeouts and resubmit the application. • The Router selects one
+   * SubCluster to forward the request. • The Router inserts a tuple into State
+   * Store with the selected SubCluster (e.g. SC2) and the appId. • The State
+   * Store replies with the selected SubCluster (e.g. SC1). • The Router submits
+   * the request to the selected SubCluster (e.g. SC1). When a client re-submits
+   * the same application to the same RM, it does not raise an exception and
+   * replies with operation successful message.
+   *
+   * In case of Client failure: identical behavior as {@code ClientRMService}.
+   *
+   * In case of ResourceManager failure:
+   *
+   * The Client submits an application to the Router. • The Router selects one
+   * SubCluster to forward the request. • The Router inserts a tuple into State
+   * Store with the selected SubCluster (e.g. SC1) and the appId. • The Router
+   * submits the request to the selected SubCluster. • The entire SubCluster is
+   * down – all the RMs in HA or the master RM is not reachable. • The Router
+   * times out. • The Router selects a new SubCluster to forward the request. •
+   * The Router update a tuple into State Store with the selected SubCluster
+   * (e.g. SC2) and the appId. • The State Store replies with OK answer. • The
+   * Router submits the request to the selected SubCluster (e.g. SC2).
+   */
+  @Override
+  public SubmitApplicationResponse submitApplication(
+      SubmitApplicationRequest request) throws YarnException, IOException {
+    if (request == null || request.getApplicationSubmissionContext() == null
+        || request.getApplicationSubmissionContext()
+            .getApplicationId() == null) {
+      RouterServerUtil
+          .logAndThrowException("Missing submitApplication request or "
+              + "applicationSubmissionContex information.", null);
+    }
+
+    ApplicationId applicationId =
+        request.getApplicationSubmissionContext().getApplicationId();
+
+    List<SubClusterId> blacklist = new ArrayList<SubClusterId>();
+
+    for (int i = 0; i < numSubmitRetries; ++i) {
+
+      SubClusterId subClusterId = policyFacade.getHomeSubcluster(
+          request.getApplicationSubmissionContext(), blacklist);
+      LOG.info("submitApplication appId" + applicationId + " try #" + i
+          + " on SubCluster " + subClusterId);
+
+      ApplicationHomeSubCluster appHomeSubCluster =
+          ApplicationHomeSubCluster.newInstance(applicationId, subClusterId);
+
+      if (i == 0) {
+        try {
+          // persist the mapping of applicationId and the subClusterId which has
+          // been selected as its home
+          subClusterId =
+              federationFacade.addApplicationHomeSubCluster(appHomeSubCluster);
+        } catch (YarnException e) {
+          String message = "Unable to insert the ApplicationId " + applicationId
+              + " into the FederationStateStore";
+          RouterServerUtil.logAndThrowException(message, e);
+        }
+      } else {
+        try {
+          // update the mapping of applicationId and the home subClusterId to
+          // the new subClusterId we have selected
+          federationFacade.updateApplicationHomeSubCluster(appHomeSubCluster);
+        } catch (YarnException e) {
+          String message = "Unable to update the ApplicationId " + applicationId
+              + " into the FederationStateStore";
+          SubClusterId subClusterIdInStateStore =
+              federationFacade.getApplicationHomeSubCluster(applicationId);
+          if (subClusterId == subClusterIdInStateStore) {
+            LOG.info("Application " + applicationId
+                + " already submitted on SubCluster " + subClusterId);
+          } else {
+            RouterServerUtil.logAndThrowException(message, e);
+          }
+        }
+      }
+
+      ApplicationClientProtocol clientRMProxy =
+          getClientRMProxyForSubCluster(subClusterId);
+
+      SubmitApplicationResponse response = null;
+      try {
+        response = clientRMProxy.submitApplication(request);
+      } catch (Exception e) {
+        LOG.warn("Unable to submit the application " + applicationId
+            + "to SubCluster " + subClusterId.getId(), e);
+      }
+
+      if (response != null) {
+        LOG.info("Application "
+            + request.getApplicationSubmissionContext().getApplicationName()
+            + " with appId " + applicationId + " submitted on " + subClusterId);
+        return response;
+      } else {
+        // Empty response from the ResourceManager.
+        // Blacklist this subcluster for this request.
+        blacklist.add(subClusterId);
+      }
+    }
+
+    String errMsg = "Application "
+        + request.getApplicationSubmissionContext().getApplicationName()
+        + " with appId " + applicationId + " failed to be submitted.";
+    LOG.error(errMsg);
+    throw new YarnException(errMsg);
+  }
+
+  /**
+   * The Yarn Router will forward to the respective Yarn RM in which the AM is
+   * running.
+   *
+   * Possible failures and behaviors:
+   *
+   * Client: identical behavior as {@code ClientRMService}.
+   *
+   * Router: the Client will timeout and resubmit the request.
+   *
+   * ResourceManager: the Router will timeout and the call will fail.
+   *
+   * State Store: the Router will timeout and it will retry depending on the
+   * FederationFacade settings - if the failure happened before the select
+   * operation.
+   */
+  @Override
+  public KillApplicationResponse forceKillApplication(
+      KillApplicationRequest request) throws YarnException, IOException {
+
+    if (request == null || request.getApplicationId() == null) {
+      RouterServerUtil.logAndThrowException(
+          "Missing forceKillApplication request or ApplicationId.", null);
+    }
+    ApplicationId applicationId = request.getApplicationId();
+    SubClusterId subClusterId = null;
+
+    try {
+      subClusterId = federationFacade
+          .getApplicationHomeSubCluster(request.getApplicationId());
+    } catch (YarnException e) {
+      RouterServerUtil.logAndThrowException("Application " + applicationId
+          + " does not exist in FederationStateStore", e);
+    }
+
+    ApplicationClientProtocol clientRMProxy =
+        getClientRMProxyForSubCluster(subClusterId);
+
+    KillApplicationResponse response = null;
+    try {
+      LOG.info("forceKillApplication " + applicationId + " on SubCluster "
+          + subClusterId);
+      response = clientRMProxy.forceKillApplication(request);
+    } catch (Exception e) {
+      LOG.error("Unable to kill the application report for "
+          + request.getApplicationId() + "to SubCluster "
+          + subClusterId.getId(), e);
+      throw e;
+    }
+
+    if (response == null) {
+      LOG.error("No response when attempting to kill the application "
+          + applicationId + " to SubCluster " + subClusterId.getId());
+    }
+
+    return response;
+  }
+
+  /**
+   * The Yarn Router will forward to the respective Yarn RM in which the AM is
+   * running.
+   *
+   * Possible failure:
+   *
+   * Client: identical behavior as {@code ClientRMService}.
+   *
+   * Router: the Client will timeout and resubmit the request.
+   *
+   * ResourceManager: the Router will timeout and the call will fail.
+   *
+   * State Store: the Router will timeout and it will retry depending on the
+   * FederationFacade settings - if the failure happened before the select
+   * operation.
+   */
+  @Override
+  public GetApplicationReportResponse getApplicationReport(
+      GetApplicationReportRequest request) throws YarnException, IOException {
+
+    if (request == null || request.getApplicationId() == null) {
+      RouterServerUtil.logAndThrowException(
+          "Missing getApplicationReport request or applicationId information.",
+          null);
+    }
+
+    SubClusterId subClusterId = null;
+
+    try {
+      subClusterId = federationFacade
+          .getApplicationHomeSubCluster(request.getApplicationId());
+    } catch (YarnException e) {
+      RouterServerUtil
+          .logAndThrowException("Application " + request.getApplicationId()
+              + " does not exist in FederationStateStore", e);
+    }
+
+    ApplicationClientProtocol clientRMProxy =
+        getClientRMProxyForSubCluster(subClusterId);
+
+    GetApplicationReportResponse response = null;
+    try {
+      response = clientRMProxy.getApplicationReport(request);
+    } catch (Exception e) {
+      LOG.error("Unable to get the application report for "
+          + request.getApplicationId() + "to SubCluster "
+          + subClusterId.getId(), e);
+      throw e;
+    }
+
+    if (response == null) {
+      LOG.error("No response when attempting to retrieve the report of "
+          + "the application " + request.getApplicationId() + " to SubCluster "
+          + subClusterId.getId());
+    }
+
+    return response;
+  }
+
+  @Override
+  public GetApplicationsResponse getApplications(GetApplicationsRequest request)
+      throws YarnException, IOException {
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public GetClusterMetricsResponse getClusterMetrics(
+      GetClusterMetricsRequest 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 MoveApplicationAcrossQueuesResponse moveApplicationAcrossQueues(
+      MoveApplicationAcrossQueuesRequest 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 GetLabelsToNodesResponse getLabelsToNodes(
+      GetLabelsToNodesRequest request) throws YarnException, IOException {
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public GetClusterNodeLabelsResponse getClusterNodeLabels(
+      GetClusterNodeLabelsRequest request) throws YarnException, IOException {
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public GetApplicationAttemptReportResponse getApplicationAttemptReport(
+      GetApplicationAttemptReportRequest 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 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 FailApplicationAttemptResponse failApplicationAttempt(
+      FailApplicationAttemptRequest request) throws YarnException, IOException {
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public UpdateApplicationPriorityResponse updateApplicationPriority(
+      UpdateApplicationPriorityRequest request)
+      throws YarnException, IOException {
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public SignalContainerResponse signalToContainer(
+      SignalContainerRequest 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/9edf7208/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
index 7e15084..7fc4719 100644
--- 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
@@ -119,29 +119,41 @@ public abstract class BaseRouterClientRMTest {
     return this.clientrmService;
   }
 
-  @Before
-  public void setUp() {
-    this.conf = new YarnConfiguration();
+  protected YarnConfiguration createConfiguration() {
+    YarnConfiguration config = 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,
+    config.set(YarnConfiguration.ROUTER_CLIENTRM_INTERCEPTOR_CLASS_PIPELINE,
         mockPassThroughInterceptorClass + "," + mockPassThroughInterceptorClass
             + "," + mockPassThroughInterceptorClass + ","
             + MockClientRequestInterceptor.class.getName());
 
-    this.conf.setInt(YarnConfiguration.ROUTER_PIPELINE_CACHE_MAX_SIZE,
+    config.setInt(YarnConfiguration.ROUTER_PIPELINE_CACHE_MAX_SIZE,
         TEST_MAX_CACHE_SIZE);
+    return config;
+  }
 
+  @Before
+  public void setUp() {
+    this.conf = createConfiguration();
     this.dispatcher = new AsyncDispatcher();
     this.dispatcher.init(conf);
     this.dispatcher.start();
     this.clientrmService = createAndStartRouterClientRMService();
   }
 
+  public void setUpConfig() {
+    this.conf = createConfiguration();
+  }
+
+  protected Configuration getConf() {
+    return this.conf;
+  }
+
   @After
   public void tearDown() {
     if (clientrmService != null) {


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


[17/50] [abbrv] 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/8b615145/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/8b615145/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/8b615145/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/8b615145/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/8b615145/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/8b615145/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/8b615145/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/8b615145/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/8b615145/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/8b615145/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/8b615145/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/8b615145/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/8b615145/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/8b615145/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/8b615145/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


[45/50] [abbrv] 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/469a5c93/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/469a5c93/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/469a5c93/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/469a5c93/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/469a5c93/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/469a5c93/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/469a5c93/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


[18/50] [abbrv] 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/8b615145
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/8b615145
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/8b615145

Branch: refs/heads/YARN-2915
Commit: 8b6151456b2ae8ae68d2447c00d2d208c5fd6335
Parents: a359ab0
Author: Subru Krishnan <su...@apache.org>
Authored: Thu Oct 13 17:59:13 2016 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Tue Jul 25 16:56:32 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/8b615145/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/8b615145/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/8b615145/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/8b615145/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/8b615145/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/8b615145/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/8b615145/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/8b615145/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/8b615145/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/8b615145/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/8b615145/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/8b615145/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/8b615145/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/8b615145/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/8b615145/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/8b615145/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/8b615145/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


[50/50] [abbrv] hadoop git commit: Bumping up hadoop-yarn-server-router module to 3.0.0-beta1 post rebase.

Posted by su...@apache.org.
Bumping up hadoop-yarn-server-router module to 3.0.0-beta1 post rebase.


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

Branch: refs/heads/YARN-2915
Commit: d4cb180052bedb6c9bacc3f2845293e061222ac9
Parents: 1c49b10
Author: Subru Krishnan <su...@apache.org>
Authored: Tue Jul 25 16:58:43 2017 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Tue Jul 25 16:58:43 2017 -0700

----------------------------------------------------------------------
 .../hadoop-yarn-server/hadoop-yarn-server-router/pom.xml           | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4cb1800/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 78e5e59..4eea9a6 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
@@ -19,7 +19,7 @@
   <parent>
     <artifactId>hadoop-yarn-server</artifactId>
     <groupId>org.apache.hadoop</groupId>
-    <version>3.0.0-alpha4-SNAPSHOT</version>
+    <version>3.0.0-beta1-SNAPSHOT</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
   <groupId>org.apache.hadoop</groupId>


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


[49/50] [abbrv] hadoop git commit: YARN-6484. Documenting the YARN Federation feature. (Carlo Curino via Subru).

Posted by su...@apache.org.
YARN-6484. Documenting the YARN Federation feature. (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/09c5c5fb
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/09c5c5fb
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/09c5c5fb

Branch: refs/heads/YARN-2915
Commit: 09c5c5fb725d174bfd104d3b55ab2b23b501f08b
Parents: d06bbbf
Author: Subru Krishnan <su...@apache.org>
Authored: Tue May 30 18:20:32 2017 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Tue Jul 25 16:56:32 2017 -0700

----------------------------------------------------------------------
 hadoop-project/src/site/site.xml                |   1 +
 .../src/site/markdown/Federation.md             | 232 +++++++++++++++++++
 .../hadoop-yarn-site/src/site/markdown/YARN.md  |   5 +-
 .../resources/images/amrmproxy_architecture.png | Bin 0 -> 35010 bytes
 .../images/federation_architecture.png          | Bin 0 -> 46964 bytes
 .../images/federation_sequence_diagram.png      | Bin 0 -> 47910 bytes
 6 files changed, 237 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/09c5c5fb/hadoop-project/src/site/site.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/src/site/site.xml b/hadoop-project/src/site/site.xml
index ae3aef5..e7af227 100644
--- a/hadoop-project/src/site/site.xml
+++ b/hadoop-project/src/site/site.xml
@@ -138,6 +138,7 @@
       <item name="Reservation System" href="hadoop-yarn/hadoop-yarn-site/ReservationSystem.html"/>
       <item name="Graceful Decommission" href="hadoop-yarn/hadoop-yarn-site/GracefulDecommission.html"/>
       <item name="Opportunistic Containers" href="hadoop-yarn/hadoop-yarn-site/OpportunisticContainers.html"/>
+      <item name="YARN Federation" href="hadoop-yarn/hadoop-yarn-site/Federation.html"/>
     </menu>
 
     <menu name="YARN REST APIs" inherit="top">

http://git-wip-us.apache.org/repos/asf/hadoop/blob/09c5c5fb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/Federation.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/Federation.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/Federation.md
new file mode 100644
index 0000000..c50ba76
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/Federation.md
@@ -0,0 +1,232 @@
+<!---
+  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.
+-->
+
+Hadoop: YARN Federation
+=======================
+
+<!-- MACRO{toc|fromDepth=0|toDepth=3} -->
+
+Purpose
+-------
+YARN is known to scale to thousands of nodes. The scalability of [YARN](./YARN.html) is determined by the Resource Manager, and is proportional to number of nodes, active applications, active containers, and frequency of heartbeat (of both nodes and applications). Lowering heartbeat can provide scalability increase, but is detrimental to utilization (see old Hadoop 1.x experience).
+This document described a federation-based approach to scale a single YARN cluster to tens of thousands of nodes, by federating multiple YARN sub-clusters.  The proposed approach is to divide a large (10-100k nodes) cluster into smaller units called sub-clusters, each with its own YARN RM and compute nodes. The federation system will stitch these sub-clusters together and make them appear as one large YARN cluster to the applications.
+The applications running in this federated environment will see a single massive YARN cluster and will be able to schedule tasks on any node of the federated cluster. Under the hood, the federation system will negotiate with sub-clusters resource managers and provide resources to the application. The goal is to allow an individual job to “span” sub-clusters seamlessly.
+
+This design is structurally scalable, as we bound the number of nodes each RM is responsible for, and appropriate policies, will try to ensure that the majority of applications will reside within a single sub-cluster, thus the number of applications each RM will see is also bounded. This means we could almost linearly scale, by simply adding sub-clusters (as very little coordination is needed across them).
+This architecture can provide very tight enforcement of scheduling invariants within each sub-cluster (simply inherits from YARN), while continuous rebalancing across subcluster will enforce (less strictly) that these properties are also respected at a global level (e.g., if a sub-cluster loses a large number of nodes, we could re-map queues to other sub-clusters to ensure users running on the impaired sub-cluster are not unfairly affected).
+
+Federation is designed as a “layer” atop of existing YARN codebase, with limited changes in the core YARN mechanisms.
+
+Assumptions:
+* We assume reasonably good connectivity across sub-clusters (e.g., we are not looking to federate across DC yet, though future investigations of this are not excluded).
+* We rely on HDFS federation (or equivalently scalable DFS solutions) to take care of scalability of the store side.
+
+
+Architecture
+------------
+OSS YARN has been known to scale up to about few thousand nodes. The proposed architecture leverages the notion of federating a number of such smaller YARN clusters, referred to as sub-clusters, into a larger federated YARN cluster comprising of tens of thousands of nodes.
+The applications running in this federated environment see a unified large YARN cluster and will be able to schedule tasks on any nodes in the cluster. Under the hood, the federation system will negotiate with sub-clusters RMs and provide resources to the application.  The logical architecture in Figure 1 shows the main components that comprise the federated cluster, which are described below.
+
+![YARN Federation Architecture | width=800](./images/federation_architecture.png)
+
+###YARN Sub-cluster
+A sub-cluster is a YARN cluster with up to few thousands nodes. The exact size of the sub-cluster will be determined considering ease of deployment/maintenance, alignment
+with network or availability zones and general best practices.
+
+The sub-cluster YARN RM will run with work-preserving high-availability turned-on, i.e., we should be able to tolerate YARN RM, NM failures with minimal disruption.
+If the entire sub-cluster is compromised, external mechanisms will ensure that jobs are resubmitted in a separate sub-cluster (this could eventually be included in the federation design).
+
+Sub-cluster is also the scalability unit in a federated environment. We can scale out the federated environment by adding one or more sub-clusters.
+
+*Note*: by design each sub-cluster is a fully functional YARN RM, and its contribution to the federation can be set to be only a fraction of its overall capacity,
+i.e. a sub-cluster can have a “partial” commitment to the federation, while retaining the ability to give out part of its capacity in a completely local way.
+
+###Router
+YARN applications are submitted to one of the Routers, which in turn applies a routing policy (obtained from the Policy Store), queries the State Store for the sub-cluster
+URL and redirects the application submission request to the appropriate sub-cluster RM. We call the sub-cluster where the job is started the “home sub-cluster”, and we call
+“secondary sub-clusters” all other sub-cluster a job is spanning on.
+The Router exposes the ApplicationClientProtocol to the outside world, transparently hiding the presence of multiple RMs. To achieve this the Router also persists the mapping
+between the application and its home sub-cluster into the State Store. This allows Routers to be soft-state while supporting user requests cheaply, as any Router can recover
+this application to home sub-cluster mapping and direct requests to the right RM without broadcasting them. For performance caching and session stickiness might be advisable.
+
+###AMRMProxy
+The AMRMProxy is a key component to allow the application to scale and run across sub-clusters. The AMRMProxy runs on all the NM machines and acts as a proxy to the
+YARN RM for the AMs by implementing the ApplicationMasterProtocol. Applications will not be allowed to communicate with the sub-cluster RMs directly. They are forced
+by the system to connect only to the AMRMProxy endpoint, which would provide transparent access to multiple YARN RMs (by dynamically routing/splitting/merging the communications).
+At any one time, a job can span across one home sub-cluster and multiple secondary sub-clusters, but the policies operating in the AMRMProxy try to limit the footprint of each job
+to minimize overhead on the scheduling infrastructure (more in section on scalability/load). The interceptor chain architecture of the ARMMProxy is showing in figure.
+
+![Architecture of the AMRMProxy interceptor chain | width=800](./images/amrmproxy_architecture.png)
+
+*Role of AMRMProxy*
+1. Protect the sub-cluster YARN RMs from misbehaving AMs. The AMRMProxy can prevent DDOS attacks by throttling/killing AMs that are asking too many resources.
+2. Mask the multiple YARN RMs in the cluster, and can transparently allow the AM to span across sub-clusters. All container allocations are done by the YARN RM framework that consists of the AMRMProxy fronting the home and other sub-cluster RMs.
+3. Intercepts all the requests, thus it can enforce application quotas, which would not be enforceable by sub-cluster RM (as each only see a fraction of the AM requests).
+4. The AMRMProxy can enforce load-balancing / overflow policies.
+
+###Global Policy Generator
+Global Policy Generator overlooks the entire federation and ensures that the system is configured and tuned properly all the time.
+A key design point is that the cluster availability does not depends on an always-on GPG. The GPG operates continuously but out-of-band from all cluster operations,
+and provide us with a unique vantage point, that allows to enforce global invariants, affect load balancing, trigger draining of sub-clusters that will undergo maintenance, etc.
+More precisely the GPG will update user capacity allocation-to-subcluster mappings, and more rarely change the policies that run in Routers, AMRMProxy (and possible RMs).
+
+In case the GPG is not-available, cluster operations will continue as of the last time the GPG published policies, and while a long-term unavailability might mean some
+of the desirable properties of balance, optimal cluster utilization and global invariants might drift away, compute and access to data will not be compromised.
+
+*NOTE*: In the current implementation the GPG is a manual tuning process, simply exposed via a CLI (YARN-3657).
+
+
+###Federation State-Store
+The Federation State defines the additional state that needs to be maintained to loosely couple multiple individual sub-clusters into a single large federated cluster. This includes the following information:
+
+####Sub-cluster Membership
+The member YARN RMs continuously heartbeat to the state store to keep alive and publish their current capability/load information.  This information is used by the
+Global Policy Generator (GPG) to make proper policy decisions. Also this information can be used by routers to select the best home sub-cluster.  This mechanism allows
+us to dynamically grow/shrink the “cluster fleet” by adding or removing sub-clusters.  This also allows for easy maintenance of each sub-cluster. This is new functionality
+that needs to be added to the YARN RM but the mechanisms are well understood as it’s similar to individual YARN RM HA.
+
+####Application’s Home Sub-cluster
+The sub-cluster on which the Application Master (AM) runs is called the Application’s “home sub-cluster”. The AM is not limited to resources from the home sub-cluster
+but can also request resources from other sub-clusters, referred to as secondary sub-clusters.
+The federated environment will be configured and tuned periodically such that when an AM is placed on a sub-cluster, it should be able to find most of the resources
+on the home sub-cluster. Only in certain cases it should need to ask for resources from other sub-clusters.
+
+###Federation Policy Store
+The federation Policy Store is a logically separate store (while it might be backed
+by the same physical component), which contains information about how applications and
+resource requests are routed to different sub-clusters. The current implementation provides
+several policies, ranging from random/hashing/roundrobin/priority to more sophisticated
+ones which account for sub-cluster load, and request locality needs.
+
+
+Running Applications across Sub-Clusters
+----------------------------------------
+
+When an application is submitted, the system will determine the most appropriate sub-cluster to run the application,
+which we call as the application’s home sub-cluster. All the communications from the AM to the RM will be proxied via
+the AMRMProxy running locally on the AM machine.
+AMRMProxy exposes the same ApplicationMasterService protocol endpoint as the YARN RM. The AM can request containers
+using the locality information exposed by the storage layer. In ideal case, the application will be placed on a sub-cluster
+where all the resources and data required by the application will be available, but if it does need containers on nodes in
+other sub-clusters, AMRMProxy will negotiate with the RMs of those sub-clusters transparently and provide the resources to
+the application, thereby enabling the application to view the entire federated environment as one massive YARN cluster.
+AMRMProxy, Global Policy Generator (GPG) and Router work together to make this happen seamlessly.
+
+![Federation Sequence Diagram | width=800](./images/federation_sequence_diagram.png)
+
+
+The figure shows a sequence diagram for the following job execution flow:
+1. The Router receives an application submission request that is complaint to the YARN Application Client Protocol.
+2. The router interrogates a routing table / policy to choose the “home RM” for the job (the policy configuration is received from the state-store on heartbeat).
+3. The router queries the membership state to determine the endpoint of the home RM.
+4. The router then redirects the application submission request to the home RM.
+5. The router updates the application state with the home sub-cluster identifier.
+6. Once the application is submitted to the home RM, the stock YARN flow is triggered, i.e. the application is added to the scheduler queue and its AM started in the home sub-cluster, on the first NodeManager that has available resources.
+    a. During this process, the AM environment is modified by indicating that the address of the AMRMProxy as the YARN RM to talk to.
+    b. The security tokens are also modified by the NM when launching the AM, so that the AM can only talk with the AMRMProxy. Any future communication from AM to the YARN RM is mediated by the AMRMProxy.
+7. The AM will then request containers using the locality information exposed by HDFS.
+8. Based on a policy the AMRMProxy can impersonate the AM on other sub-clusters, by submitting an Unmanaged AM, and by forwarding the AM heartbeats to relevant sub-clusters.
+9. The AMRMProxy will use both locality information and a pluggable policy configured in the state-store to decide whether to forward the resource requests received by the AM to the Home RM or to one (or more) Secondary RMs. In Figure 1, we show the case in which the AMRMProxy decides to forward the request to the secondary RM.
+10. The secondary RM will provide the AMRMProxy with valid container tokens to start a new container on some node in its sub-cluster. This mechanism ensures that each sub-cluster uses its own security tokens and avoids the need for a cluster wide shared secret to create tokens.
+11. The AMRMProxy forwards the allocation response back to the AM.
+12. The AM starts the container on the target NodeManager (on sub-cluster 2) using the standard YARN protocols.
+
+
+
+
+Configuration
+-------------
+
+  To configure the `YARN` to use the `Federation`, set the following property in the **conf/yarn-site.xml**:
+
+###EVERYWHERE:
+
+These are common configurations that should appear in the **conf/yarn-site.xml** at each machine in the federation.
+
+
+| Property | Example | Description |
+|:---- |:---- |
+|`yarn.federation.enabled` | `true` | Whether federation is enabled or not |
+|`yarn.federation.state-store.class` | `org.apache.hadoop.yarn.server.federation.store.impl.SQLFederationStateStore` | The type of state-store to use. |
+|`yarn.federation.state-store.sql.url` | `jdbc:sqlserver://<host>:<port>;database` | For SQLFederationStateStore the name of the DB where the state is stored. |
+|`yarn.federation.state-store.sql.jdbc-class` | `com.microsoft.sqlserver.jdbc.SQLServerDataSource` | For SQLFederationStateStore the jdbc class to use. |
+|`yarn.federation.state-store.sql.username` | `<dbuser>` | For SQLFederationStateStore the username for the DB connection. |
+|`yarn.federation.state-store.sql.password` | `<dbpass>` | For SQLFederationStateStore the password for the DB connection. |
+|`yarn.resourcemanager.cluster-id` | `<unique-subcluster-id>` | The unique subcluster identifier for this RM (same as the one used for HA). |
+
+
+Optional:
+
+| Property | Example | Description |
+|:---- |:---- |
+|`yarn.federation.failover.enabled` | `true` | Whether should retry considering RM failover within each subcluster. |
+|`yarn.federation.blacklist-subclusters` | `<subcluster-id>` | A list of black-listed sub-clusters, useful to disable a sub-cluster |
+|`yarn.federation.policy-manager` | `org.apache.hadoop.yarn.server.federation.policies.manager.WeightedLocalityPolicyManager` | The choice of policy manager determines how Applications and ResourceRequests are routed through the system. |
+|`yarn.federation.policy-manager-params` | `<binary>` | The payload that configures the policy. In our example a set of weights for router and amrmproxy policies. This is typically generated by serializing a policymanager that has been configured programmatically, or by populating the state-store with the .json serialized form of it. |
+|`yarn.federation.subcluster-resolver.class` | `org.apache.hadoop.yarn.server.federation.resolver.DefaultSubClusterResolverImpl` | The class used to resolve which subcluster a node belongs to, and which subcluster(s) a rack belongs to. |
+| `yarn.federation.machine-list` | `node1,subcluster1,rack1\n node2 , subcluster2, RACK1\n noDE3,subcluster3, rack2\n node4, subcluster3, rack2\n` | a list of Nodes, Sub-clusters, Rack, used by the `DefaultSubClusterResolverImpl` |
+
+###ON RMs:
+
+These are extra configurations that should appear in the **conf/yarn-site.xml** at each ResourceManager.
+
+| Property | Example | Description |
+|:---- |:---- |
+|`yarn.resourcemanager.epoch` | `<unique-epoch>` | The seed value for the epoch. This is used to guarantee uniqueness of container-IDs generate by different RMs. It must therefore be unique among sub-clusters and `well-spaced` to allow for failures which increment epoch. Increments of 1000 allow for a large number of sub-clusters and
+ practically ensure near-zero chance of collisions (a clash will only happen if a container is still alive for 1000 restarts of one RM, while the next RM never restarted, and an app requests more containers). |
+
+Optional:
+
+| Property | Example | Description |
+|:---- |:---- |
+|`yarn.federation.state-store.heartbeat-interval-secs` | `60` | The rate at which RMs report their membership to the federation to the central state-store. |
+
+
+###ON ROUTER:
+
+These are extra configurations that should appear in the **conf/yarn-site.xml** at each Router.
+
+| Property | Example | Description |
+|:---- |:---- |
+|`yarn.router.bind-host` | `0.0.0.0` | Host IP to bind the router to.  The actual address the server will bind to. If this optional address is set, the RPC and webapp servers will bind to this address and the port specified in yarn.router.*.address respectively. This is most useful for making Router listen to all interfaces by setting to 0.0.0.0. |
+
+Optional:
+
+| Property | Example | Description |
+|:---- |:---- |
+|`yarn.router.hostname` | `0.0.0.0` | Router host name.
+|`yarn.router.clientrm.address` | `0.0.0.0:8032` | Router client address. |
+|`yarn.router.webapp.address` | `0.0.0.0:80` | Webapp address at the router. |
+|`yarn.router.admin.address` | `0.0.0.0:8033` | Admin address at the router. |
+|`yarn.router.webapp.https.address` | `0.0.0.0:443` | Secure webapp address at the router. |
+|`yarn.router.submit.retry` | `3` | The number of retries in the router before we give up. |
+|`yarn.federation.statestore.max-connections` | `10` | This is the maximum number of parallel connections each Router makes to the state-store. |
+|`yarn.federation.cache-ttl.secs` | `60` | The Router caches informations, and this is the time to leave before the cache is invalidated. |
+
+
+###ON NMs:
+
+These are extra configurations that should appear in the **conf/yarn-site.xml** at each NodeManager.
+
+
+| Property | Example | Description |
+|:---- |:---- |
+|`yarn.nodemanager.amrmproxy.interceptor-class.pipeline` | `org.apache.hadoop.yarn.server.nodemanager.amrmproxy.FederationInterceptor` | A coma-separated list of interceptors to be run at the amrmproxy. For federation the last step in the pipeline should be the FederationInterceptor. |
+
+Optional:
+
+| Property | Example | Description |
+|:---- |:---- |
+|`yarn.federation.statestore.max-connections` | `1` | The maximum number of parallel connections from each AMRMProxy to the state-store. This value is typically lower than the router one, since we have many AMRMProxy that could burn-through many DB connections quickly. |
+|`yarn.federation.cache-ttl.secs` | `300` | The time to leave for the AMRMProxy cache. Typically larger than at the router, as the number of AMRMProxy is large, and we want to limit the load to the centralized state-store. |
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/09c5c5fb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/YARN.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/YARN.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/YARN.md
index 433c737..598ee6f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/YARN.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/YARN.md
@@ -33,4 +33,7 @@ The ApplicationsManager is responsible for accepting job-submissions, negotiatin
 
 MapReduce in hadoop-2.x maintains **API compatibility** with previous stable release (hadoop-1.x). This means that all MapReduce jobs should still run unchanged on top of YARN with just a recompile.
 
-YARN also supports the notion of **resource reservation** via the [ReservationSystem](./ReservationSystem.html), a component that allows users to specify a profile of resources over-time and temporal constraints (e.g., deadlines), and reserve resources to ensure the predictable execution of important jobs.The *ReservationSystem* tracks resources over-time, performs admission control for reservations, and dynamically instruct the underlying scheduler to ensure that the reservation is fullfilled.
+YARN supports the notion of **resource reservation** via the [ReservationSystem](./ReservationSystem.html), a component that allows users to specify a profile of resources over-time and temporal constraints (e.g., deadlines), and reserve resources to ensure the predictable execution of important jobs.The *ReservationSystem* tracks resources over-time, performs admission control for reservations, and dynamically instruct the underlying scheduler to ensure that the reservation is fullfilled.
+
+In order to scale YARN beyond few thousands nodes, YARN supports the notion of **Federation** via the [YARN Federation](.Federation.html) feature. Federation allows to transparently wire together multiple yarn (sub-)clusters, and
+ make them appear as a single massive cluster. This can be used to achieve larger scale, and/or to allow multiple independent clusters to be used together for very large jobs, or for tenants who have capacity across all of them.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/09c5c5fb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/resources/images/amrmproxy_architecture.png
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/resources/images/amrmproxy_architecture.png b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/resources/images/amrmproxy_architecture.png
new file mode 100644
index 0000000..8740ada
Binary files /dev/null and b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/resources/images/amrmproxy_architecture.png differ

http://git-wip-us.apache.org/repos/asf/hadoop/blob/09c5c5fb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/resources/images/federation_architecture.png
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/resources/images/federation_architecture.png b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/resources/images/federation_architecture.png
new file mode 100644
index 0000000..b2ed72f
Binary files /dev/null and b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/resources/images/federation_architecture.png differ

http://git-wip-us.apache.org/repos/asf/hadoop/blob/09c5c5fb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/resources/images/federation_sequence_diagram.png
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/resources/images/federation_sequence_diagram.png b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/resources/images/federation_sequence_diagram.png
new file mode 100644
index 0000000..32812e0
Binary files /dev/null and b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/resources/images/federation_sequence_diagram.png differ


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


[15/50] [abbrv] 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/8f2cae39
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/8f2cae39
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/8f2cae39

Branch: refs/heads/YARN-2915
Commit: 8f2cae393573023a0022f72d82962e58b2811b41
Parents: c62772f
Author: Subru Krishnan <su...@apache.org>
Authored: Thu Sep 1 13:55:54 2016 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Tue Jul 25 16:56:32 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/8f2cae39/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/8f2cae39/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


[33/50] [abbrv] hadoop git commit: YARN-6691. Update YARN daemon startup/shutdown scripts to include Router service. (Giovanni Matteo Fumarola via asuresh).

Posted by su...@apache.org.
YARN-6691. Update YARN daemon startup/shutdown scripts to include Router service. (Giovanni Matteo Fumarola via asuresh).


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

Branch: refs/heads/YARN-2915
Commit: 474c0a649143eff019a9342cfd513ef93d41ad8f
Parents: 2c10a9b
Author: Arun Suresh <as...@apache.org>
Authored: Mon Jun 19 10:52:23 2017 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Tue Jul 25 16:56:32 2017 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/hadoop-yarn/bin/yarn         |  5 +++++
 hadoop-yarn-project/hadoop-yarn/bin/yarn.cmd     | 13 ++++++++++++-
 hadoop-yarn-project/hadoop-yarn/conf/yarn-env.sh | 12 ++++++++++++
 3 files changed, 29 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/474c0a64/hadoop-yarn-project/hadoop-yarn/bin/yarn
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/bin/yarn b/hadoop-yarn-project/hadoop-yarn/bin/yarn
index cf6457b..a37d183 100755
--- a/hadoop-yarn-project/hadoop-yarn/bin/yarn
+++ b/hadoop-yarn-project/hadoop-yarn/bin/yarn
@@ -46,6 +46,7 @@ function hadoop_usage
   hadoop_add_subcommand "queue" "prints queue information"
   hadoop_add_subcommand "resourcemanager" "run the ResourceManager"
   hadoop_add_subcommand "rmadmin" "admin tools"
+  hadoop_add_subcommand "router" "run the Router daemon"
   hadoop_add_subcommand "scmadmin" "SharedCacheManager admin tools"
   hadoop_add_subcommand "sharedcachemanager" "run the SharedCacheManager daemon"
   hadoop_add_subcommand "timelinereader" "run the timeline reader server"
@@ -137,6 +138,10 @@ function yarncmd_case
     rmadmin)
       HADOOP_CLASSNAME='org.apache.hadoop.yarn.client.cli.RMAdminCLI'
     ;;
+    router)
+      HADOOP_SUBCMD_SUPPORTDAEMONIZATION="true"
+      HADOOP_CLASSNAME='org.apache.hadoop.yarn.server.router.Router'
+    ;;
     scmadmin)
       HADOOP_CLASSNAME='org.apache.hadoop.yarn.client.SCMAdmin'
     ;;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/474c0a64/hadoop-yarn-project/hadoop-yarn/bin/yarn.cmd
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/bin/yarn.cmd b/hadoop-yarn-project/hadoop-yarn/bin/yarn.cmd
index ca879f5..690badf 100644
--- a/hadoop-yarn-project/hadoop-yarn/bin/yarn.cmd
+++ b/hadoop-yarn-project/hadoop-yarn/bin/yarn.cmd
@@ -130,6 +130,10 @@ if "%1" == "--loglevel" (
     set CLASSPATH=%CLASSPATH%;%HADOOP_YARN_HOME%\yarn-server\yarn-server-applicationhistoryservice\target\classes
   )
 
+  if exist %HADOOP_YARN_HOME%\yarn-server\yarn-server-router\target\classes (
+    set CLASSPATH=%CLASSPATH%;%HADOOP_YARN_HOME%\yarn-server\yarn-server-router\target\classes
+  )
+
   if exist %HADOOP_YARN_HOME%\build\test\classes (
     set CLASSPATH=%CLASSPATH%;%HADOOP_YARN_HOME%\build\test\classes
   )
@@ -151,7 +155,7 @@ if "%1" == "--loglevel" (
 
   set yarncommands=resourcemanager nodemanager proxyserver rmadmin version jar ^
      application applicationattempt container node queue logs daemonlog historyserver ^
-     timelineserver timelinereader classpath
+     timelineserver timelinereader router classpath
   for %%i in ( %yarncommands% ) do (
     if %yarn-command% == %%i set yarncommand=true
   )
@@ -248,6 +252,12 @@ goto :eof
   set YARN_OPTS=%YARN_OPTS% %YARN_TIMELINEREADER_OPTS%
   goto :eof
 
+:router
+  set CLASSPATH=%CLASSPATH%;%YARN_CONF_DIR%\router-config\log4j.properties
+  set CLASS=org.apache.hadoop.yarn.server.router.Router
+  set YARN_OPTS=%YARN_OPTS% %HADOOP_ROUTER_OPTS%
+  goto :eof
+
 :nodemanager
   set CLASSPATH=%CLASSPATH%;%YARN_CONF_DIR%\nm-config\log4j.properties
   set CLASS=org.apache.hadoop.yarn.server.nodemanager.NodeManager
@@ -317,6 +327,7 @@ goto :eof
   @echo        where COMMAND is one of:
   @echo   resourcemanager      run the ResourceManager
   @echo   nodemanager          run a nodemanager on each slave
+  @echo   router               run the Router daemon
   @echo   timelineserver       run the timeline server
   @echo   timelinereader       run the timeline reader server
   @echo   rmadmin              admin tools

http://git-wip-us.apache.org/repos/asf/hadoop/blob/474c0a64/hadoop-yarn-project/hadoop-yarn/conf/yarn-env.sh
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/conf/yarn-env.sh b/hadoop-yarn-project/hadoop-yarn/conf/yarn-env.sh
index d003adb..be42298 100644
--- a/hadoop-yarn-project/hadoop-yarn/conf/yarn-env.sh
+++ b/hadoop-yarn-project/hadoop-yarn/conf/yarn-env.sh
@@ -136,3 +136,15 @@
 # See ResourceManager for some examples
 #
 #export YARN_SHAREDCACHEMANAGER_OPTS=
+
+###
+# Router specific parameters
+###
+
+# Specify the JVM options to be used when starting the Router.
+# These options will be appended to the options specified as HADOOP_OPTS
+# and therefore may override any similar flags set in HADOOP_OPTS
+#
+# See ResourceManager for some examples
+#
+#export YARN_ROUTER_OPTS=


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


[25/50] [abbrv] 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/b6a4fb6d
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/b6a4fb6d
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/b6a4fb6d

Branch: refs/heads/YARN-2915
Commit: b6a4fb6d3cd6f796ea78a8289e31b768bb32cdc3
Parents: 9657be4
Author: Subru Krishnan <su...@apache.org>
Authored: Tue Nov 22 18:37:30 2016 -0800
Committer: Subru Krishnan <su...@apache.org>
Committed: Tue Jul 25 16:56:32 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/b6a4fb6d/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/b6a4fb6d/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/b6a4fb6d/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/b6a4fb6d/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/b6a4fb6d/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/b6a4fb6d/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


[40/50] [abbrv] 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/13a87bf6
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/13a87bf6
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/13a87bf6

Branch: refs/heads/YARN-2915
Commit: 13a87bf626aaa3cf35397528fd1c1066ca5bc760
Parents: 6eced35
Author: Subru Krishnan <su...@apache.org>
Authored: Tue Nov 22 15:02:22 2016 -0800
Committer: Subru Krishnan <su...@apache.org>
Committed: Tue Jul 25 16:56:32 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/13a87bf6/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 33bde54..18c3444 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
@@ -2600,7 +2600,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/13a87bf6/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/13a87bf6/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/13a87bf6/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/13a87bf6/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/13a87bf6/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/13a87bf6/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/13a87bf6/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/13a87bf6/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/13a87bf6/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/13a87bf6/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/13a87bf6/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/13a87bf6/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/13a87bf6/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/13a87bf6/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/13a87bf6/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/13a87bf6/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/13a87bf6/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/13a87bf6/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/13a87bf6/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/13a87bf6/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/13a87bf6/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/13a87bf6/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/13a87bf6/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


[06/50] [abbrv] 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/dc374d3b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/dc374d3b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/dc374d3b

Branch: refs/heads/YARN-2915
Commit: dc374d3bdf258a10d942fc11d79e52401875e9df
Parents: 02559e8
Author: Subru Krishnan <su...@apache.org>
Authored: Mon Aug 15 14:47:02 2016 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Tue Jul 25 16:56:31 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/dc374d3b/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/dc374d3b/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/dc374d3b/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/dc374d3b/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/dc374d3b/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/dc374d3b/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


[19/50] [abbrv] hadoop git commit: YARN-5531. UnmanagedAM pool manager for federating application across clusters. (Botong Huang via Subru).

Posted by su...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/c73abec4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.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/ApplicationMasterService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java
index 76a1640..7638207 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java
@@ -211,15 +211,13 @@ public class ApplicationMasterService extends AbstractService implements
     synchronized (lock) {
       AllocateResponse lastResponse = lock.getAllocateResponse();
       if (hasApplicationMasterRegistered(applicationAttemptId)) {
-        String message =
-            "Application Master is already registered : "
-                + appID;
+        String message = AMRMClientUtils.APP_ALREADY_REGISTERED_MESSAGE + appID;
         LOG.warn(message);
         RMAuditLogger.logFailure(
-            this.rmContext.getRMApps()
-                .get(appID).getUser(),
-            AuditConstants.REGISTER_AM, "", "ApplicationMasterService", message,
-            appID, applicationAttemptId);
+          this.rmContext.getRMApps()
+            .get(appID).getUser(),
+          AuditConstants.REGISTER_AM, "", "ApplicationMasterService", message,
+          appID, applicationAttemptId);
         throw new InvalidApplicationMasterRequestException(message);
       }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c73abec4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.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/TestApplicationMasterLauncher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.java
index 9e84010..8e14176 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.java
@@ -73,6 +73,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.ApplicationMaste
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
+import org.apache.hadoop.yarn.server.utils.AMRMClientUtils;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.log4j.Level;
 import org.apache.log4j.LogManager;
@@ -338,9 +339,8 @@ public class TestApplicationMasterLauncher {
       am.registerAppAttempt(false);
       Assert.fail();
     } catch (Exception e) {
-      Assert.assertEquals("Application Master is already registered : "
-          + attempt.getAppAttemptId().getApplicationId(),
-        e.getMessage());
+      Assert.assertEquals(AMRMClientUtils.APP_ALREADY_REGISTERED_MESSAGE
+          + attempt.getAppAttemptId().getApplicationId(), e.getMessage());
     }
 
     // Simulate an AM that was disconnected and app attempt was removed


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


[38/50] [abbrv] hadoop git commit: YARN-6821. Move FederationStateStore SQL DDL files from test resource to sbin.

Posted by su...@apache.org.
YARN-6821. Move FederationStateStore SQL DDL files from test resource to sbin.


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

Branch: refs/heads/YARN-2915
Commit: 3ec835b4f188105573bd2f950a9241740b8a91bc
Parents: 6cd635f
Author: Subru Krishnan <su...@apache.org>
Authored: Thu Jul 13 18:53:21 2017 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Tue Jul 25 16:56:32 2017 -0700

----------------------------------------------------------------------
 .../resources/assemblies/hadoop-yarn-dist.xml   |   1 +
 .../FederationStateStoreStoreProcs.sql          | 511 +++++++++++++++++++
 .../SQLServer/FederationStateStoreTables.sql    | 122 +++++
 .../FederationStateStoreStoreProcs.sql          | 511 -------------------
 .../SQLServer/FederationStateStoreTables.sql    | 122 -----
 5 files changed, 634 insertions(+), 633 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3ec835b4/hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-dist.xml
----------------------------------------------------------------------
diff --git a/hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-dist.xml b/hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-dist.xml
index 74ce9bc..289061f 100644
--- a/hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-dist.xml
+++ b/hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-dist.xml
@@ -61,6 +61,7 @@
         <include>stop-yarn.sh</include>
         <include>start-yarn.cmd</include>
         <include>stop-yarn.cmd</include>
+        <include>FederationStateStore**/**</include>
       </includes>
       <fileMode>0755</fileMode>
     </fileSet>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3ec835b4/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/SQLServer/FederationStateStoreStoreProcs.sql
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/SQLServer/FederationStateStoreStoreProcs.sql b/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/SQLServer/FederationStateStoreStoreProcs.sql
new file mode 100644
index 0000000..66d6f0e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/bin/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/3ec835b4/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/SQLServer/FederationStateStoreTables.sql
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/SQLServer/FederationStateStoreTables.sql b/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/SQLServer/FederationStateStoreTables.sql
new file mode 100644
index 0000000..a97385b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/bin/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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3ec835b4/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
deleted file mode 100644
index 66d6f0e..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/resources/FederationStateStore/SQLServer/FederationStateStoreStoreProcs.sql
+++ /dev/null
@@ -1,511 +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.
- */
-
-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/3ec835b4/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
deleted file mode 100644
index a97385b..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/resources/FederationStateStore/SQLServer/FederationStateStoreTables.sql
+++ /dev/null
@@ -1,122 +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.
- */
-
-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


[21/50] [abbrv] 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/6eced352
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/6eced352
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/6eced352

Branch: refs/heads/YARN-2915
Commit: 6eced352c1d24caa82c65e4fef8be12c88d52a5e
Parents: bff50f8
Author: Subru Krishnan <su...@apache.org>
Authored: Wed Nov 16 19:39:25 2016 -0800
Committer: Subru Krishnan <su...@apache.org>
Committed: Tue Jul 25 16:56:32 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/6eced352/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/6eced352/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 5dcb993..33bde54 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
@@ -2594,6 +2594,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/6eced352/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/6eced352/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/6eced352/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/6eced352/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/6eced352/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/6eced352/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/6eced352/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/6eced352/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/6eced352/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