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 cu...@apache.org on 2017/09/22 00:58:22 UTC

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

Repository: hadoop
Updated Branches:
  refs/heads/branch-2 fc6d9612c -> d11be2dca


YARN-3671. Integrate Federation services with ResourceManager. Contributed by Subru Krishnan

(cherry picked from commit 8573c286e27623155c715a632d56a68f23523c72)
(cherry picked from commit 94a24567d65469091c8b521987dc2003d0327159)


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

Branch: refs/heads/branch-2
Commit: ea2ecdb796297573ee66727be15e41c0350408fa
Parents: 615c912
Author: Jian He <ji...@apache.org>
Authored: Tue Aug 30 12:20:52 2016 +0800
Committer: Carlo Curino <cu...@apache.org>
Committed: Thu Sep 21 16:23:33 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 |  27 ++
 .../FederationStateStoreHeartbeat.java          | 108 +++++++
 .../federation/FederationStateStoreService.java | 304 +++++++++++++++++++
 .../federation/package-info.java                |  17 ++
 .../webapp/dao/ClusterMetricsInfo.java          |   5 +-
 .../TestFederationRMStateStoreService.java      | 170 +++++++++++
 12 files changed, 649 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea2ecdb7/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 eec0cd0..c9db167 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
@@ -2515,9 +2515,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";
 
@@ -2530,6 +2527,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/ea2ecdb7/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 2d02c30..38e2668 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
@@ -74,9 +74,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/ea2ecdb7/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/ea2ecdb7/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/ea2ecdb7/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/ea2ecdb7/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/ea2ecdb7/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 e665893..2eb69e0 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
@@ -60,6 +60,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.SystemMetricsPublisher;
 import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingEditPolicy;
 import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingMonitor;
@@ -165,6 +166,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;
@@ -464,6 +466,10 @@ public class ResourceManager extends CompositeService implements Recoverable {
     return new RMApplicationHistoryWriter();
   }
 
+  private FederationStateStoreService createFederationStateStoreService() {
+    return new FederationStateStoreService(rmContext);
+  }
+
   protected SystemMetricsPublisher createSystemMetricsPublisher() {
     return new SystemMetricsPublisher(); 
   }
@@ -670,6 +676,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);
@@ -977,6 +997,7 @@ public class ResourceManager extends CompositeService implements Recoverable {
       builder.withAttribute(WebAppProxy.FETCHER_ATTRIBUTE, fetcher);
       String[] proxyParts = proxyHostAndPort.split(":");
       builder.withAttribute(WebAppProxy.PROXY_HOST_ATTRIBUTE, proxyParts[0]);
+    }
 
     }
     webApp = builder.start(new RMWebApp(this));
@@ -1226,6 +1247,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/ea2ecdb7/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/ea2ecdb7/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/ea2ecdb7/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/ea2ecdb7/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/ea2ecdb7/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


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

Posted by cu...@apache.org.
YARN-6484. Documenting the YARN Federation feature. (Carlo Curino via Subru).

(cherry picked from commit 749ca890e6e62671b4df69d020396ec93edadf16)
(cherry picked from commit d6c29b2f5b953a0b747887cfc9cf4290f0402b1e)


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

Branch: refs/heads/branch-2
Commit: 7dd6caff9e25c470937010f7e8d7a6e3aaac66d1
Parents: b9fdc24
Author: Subru Krishnan <su...@apache.org>
Authored: Tue May 30 18:20:32 2017 -0700
Committer: Carlo Curino <cu...@apache.org>
Committed: Thu Sep 21 16:49:54 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/7dd6caff/hadoop-project/src/site/site.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/src/site/site.xml b/hadoop-project/src/site/site.xml
index 7fb7723..8f21fdd 100644
--- a/hadoop-project/src/site/site.xml
+++ b/hadoop-project/src/site/site.xml
@@ -137,6 +137,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/7dd6caff/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/7dd6caff/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/7dd6caff/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/7dd6caff/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/7dd6caff/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


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

Posted by cu...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/3c5dfa0f/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/3c5dfa0f/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/3c5dfa0f/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/3c5dfa0f/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/3c5dfa0f/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/3c5dfa0f/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/3c5dfa0f/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/3c5dfa0f/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


[46/50] [abbrv] hadoop git commit: YARN-5412. Create a proxy chain for ResourceManager REST API in the Router. (Contributed by Giovanni Matteo Fumarola via curino)

Posted by cu...@apache.org.
YARN-5412. Create a proxy chain for ResourceManager REST API in the Router. (Contributed by Giovanni Matteo Fumarola via curino)

(cherry picked from commit b6240b92abf453affc5fd64e1eedf2d29842aa75)
(cherry picked from commit acda6b96a4e92e432bd1d97fa14004a11e70387e)


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

Branch: refs/heads/branch-2
Commit: bfd967d33866d7a3067f0b7cd107d5d45e4adf6e
Parents: 049f7c8
Author: Carlo Curino <cu...@apache.org>
Authored: Thu Jul 27 14:34:45 2017 -0700
Committer: Carlo Curino <cu...@apache.org>
Committed: Thu Sep 21 17:13:28 2017 -0700

----------------------------------------------------------------------
 .../hadoop/yarn/conf/YarnConfiguration.java     |   24 +
 .../hadoop/yarn/webapp/util/WebAppUtils.java    |   14 +
 .../src/main/resources/yarn-default.xml         |   30 +
 .../resourcemanager/webapp/RMWSConsts.java      |   15 +
 .../resourcemanager/webapp/RMWebAppUtil.java    |   29 +
 .../webapp/RMWebServiceProtocol.java            |  133 +-
 .../resourcemanager/webapp/RMWebServices.java   |    4 +-
 .../webapp/dao/AppAttemptInfo.java              |    5 +-
 .../TestFederationRMStateStoreService.java      |    9 +-
 .../hadoop-yarn-server-router/pom.xml           |   34 +-
 .../hadoop/yarn/server/router/Router.java       |   35 +
 .../webapp/AbstractRESTRequestInterceptor.java  |   89 ++
 .../webapp/DefaultRequestInterceptorREST.java   |  496 +++++++
 .../yarn/server/router/webapp/HTTPMethods.java  |   34 +
 .../router/webapp/RESTRequestInterceptor.java   |  125 ++
 .../yarn/server/router/webapp/RouterWebApp.java |   48 +
 .../router/webapp/RouterWebServiceUtil.java     |  227 +++
 .../server/router/webapp/RouterWebServices.java |  876 ++++++++++++
 .../yarn/server/router/webapp/package-info.java |   20 +
 .../webapp/BaseRouterWebServicesTest.java       |  601 ++++++++
 .../yarn/server/router/webapp/JavaProcess.java  |   52 +
 .../webapp/MockRESTRequestInterceptor.java      |  340 +++++
 .../PassThroughRESTRequestInterceptor.java      |  339 +++++
 .../router/webapp/TestRouterWebServices.java    |  269 ++++
 .../webapp/TestRouterWebServicesREST.java       | 1298 ++++++++++++++++++
 .../src/test/resources/capacity-scheduler.xml   |  111 ++
 .../src/test/resources/log4j.properties         |   19 +
 .../src/test/resources/yarn-site.xml            |   30 +
 28 files changed, 5237 insertions(+), 69 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/bfd967d3/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 7adfdf1..34374cf 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
@@ -2629,6 +2629,30 @@ public class YarnConfiguration extends Configuration {
       ROUTER_PREFIX + "submit.retry";
   public static final int DEFAULT_ROUTER_CLIENTRM_SUBMIT_RETRY = 3;
 
+  public static final String ROUTER_WEBAPP_PREFIX = ROUTER_PREFIX + "webapp.";
+
+  /** The address of the Router web application. */
+  public static final String ROUTER_WEBAPP_ADDRESS =
+      ROUTER_WEBAPP_PREFIX + "address";
+
+  public static final int DEFAULT_ROUTER_WEBAPP_PORT = 8089;
+  public static final String DEFAULT_ROUTER_WEBAPP_ADDRESS =
+      "0.0.0.0:" + DEFAULT_ROUTER_WEBAPP_PORT;
+
+  /** The https address of the Router web application. */
+  public static final String ROUTER_WEBAPP_HTTPS_ADDRESS =
+      ROUTER_WEBAPP_PREFIX + "https.address";
+
+  public static final int DEFAULT_ROUTER_WEBAPP_HTTPS_PORT = 8091;
+  public static final String DEFAULT_ROUTER_WEBAPP_HTTPS_ADDRESS =
+      "0.0.0.0:" + DEFAULT_ROUTER_WEBAPP_HTTPS_PORT;
+
+  public static final String ROUTER_WEBAPP_INTERCEPTOR_CLASS_PIPELINE =
+      ROUTER_WEBAPP_PREFIX + "interceptor-class.pipeline";
+  public static final String DEFAULT_ROUTER_WEBAPP_INTERCEPTOR_CLASS =
+      "org.apache.hadoop.yarn.server.router.webapp."
+          + "DefaultRequestInterceptorREST";
+
   ////////////////////////////////
   // Other Configs
   ////////////////////////////////

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bfd967d3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/util/WebAppUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/util/WebAppUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/util/WebAppUtils.java
index a32b2be..d62a810 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/util/WebAppUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/util/WebAppUtils.java
@@ -129,6 +129,20 @@ public class WebAppUtils {
     return getRMWebAppURLWithoutScheme(conf, false);
   }
 
+  public static String getRouterWebAppURLWithScheme(Configuration conf) {
+    return getHttpSchemePrefix(conf) + getRouterWebAppURLWithoutScheme(conf);
+  }
+
+  public static String getRouterWebAppURLWithoutScheme(Configuration conf) {
+    if (YarnConfiguration.useHttps(conf)) {
+      return conf.get(YarnConfiguration.ROUTER_WEBAPP_HTTPS_ADDRESS,
+          YarnConfiguration.DEFAULT_ROUTER_WEBAPP_HTTPS_ADDRESS);
+    } else {
+      return conf.get(YarnConfiguration.ROUTER_WEBAPP_ADDRESS,
+          YarnConfiguration.DEFAULT_ROUTER_WEBAPP_ADDRESS);
+    }
+  }
+
   public static List<String> getProxyHostsAndPortsForAmFilter(
       Configuration conf) {
     List<String> addrs = new ArrayList<String>();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bfd967d3/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 998e4cb..122b824 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
@@ -3153,4 +3153,34 @@
     <value></value>
   </property>
 
+  <property>
+    <description>
+      The comma separated list of class names that implement the
+      RequestInterceptor interface. This is used by the RouterWebServices
+      to create the request processing pipeline for users.
+    </description>
+    <name>yarn.router.webapp.interceptor-class.pipeline</name>
+    <value>org.apache.hadoop.yarn.server.router.webapp.DefaultRequestInterceptorREST</value>
+  </property>
+
+  <property>
+    <description>
+      The http address of the Router web application.
+      If only a host is provided as the value,
+      the webapp will be served on a random port.
+    </description>
+    <name>yarn.router.webapp.address</name>
+    <value>0.0.0.0:8089</value>
+  </property>
+
+  <property>
+    <description>
+      The https address of the Router web application.
+      If only a host is provided as the value,
+      the webapp will be served on a random port.
+    </description>
+    <name> yarn.router.webapp.https.address</name>
+    <value>0.0.0.0:8091</value>
+  </property>
+
 </configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bfd967d3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWSConsts.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/RMWSConsts.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWSConsts.java
index 23d4bb1..5a945da 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWSConsts.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWSConsts.java
@@ -168,6 +168,12 @@ public final class RMWSConsts {
    */
   public static final String APPS_TIMEOUT = "/apps/{appid}/timeout";
 
+  /**
+   * Path for {@code RouterWebServices#getContainer}.
+   */
+  public static final String GET_CONTAINER =
+      "/apps/{appid}/appattempts/{appattemptid}/containers/{containerid}";
+
   // ----------------QueryParams for RMWebServiceProtocol----------------
 
   public static final String TIME = "time";
@@ -194,6 +200,15 @@ public final class RMWSConsts {
   public static final String END_TIME = "end-time";
   public static final String INCLUDE_RESOURCE = "include-resource-allocations";
   public static final String TYPE = "type";
+  public static final String CONTAINERID = "containerid";
+  public static final String APPATTEMPTS = "appattempts";
+  public static final String TIMEOUTS = "timeouts";
+  public static final String PRIORITY = "priority";
+  public static final String TIMEOUT = "timeout";
+  public static final String ATTEMPTS = "appattempts";
+  public static final String GET_LABELS = "get-labels";
+  public static final String DESELECTS = "deSelects";
+  public static final String CONTAINERS = "containers";
 
   private RMWSConsts() {
     // not called

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bfd967d3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebAppUtil.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/RMWebAppUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebAppUtil.java
index f529dc2..ce05456 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebAppUtil.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebAppUtil.java
@@ -20,10 +20,13 @@ package org.apache.hadoop.yarn.server.resourcemanager.webapp;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.security.Principal;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.Map;
 
+import javax.servlet.http.HttpServletRequest;
+
 import org.apache.commons.codec.binary.Base64;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -341,4 +344,30 @@ public final class RMWebAppUtil {
         logAggregationContextInfo.getLogAggregationPolicyClassName(),
         logAggregationContextInfo.getLogAggregationPolicyParameters());
   }
+
+ /**
+   * Helper method to retrieve the UserGroupInformation from the
+   * HttpServletRequest.
+   *
+   * @param hsr the servlet request
+   * @param usePrincipal true if we need to use the principal user, remote
+   *          otherwise.
+   * @return the user group information of the caller.
+   **/
+  public static UserGroupInformation getCallerUserGroupInformation(
+      HttpServletRequest hsr, boolean usePrincipal) {
+
+    String remoteUser = hsr.getRemoteUser();
+    if (usePrincipal) {
+      Principal princ = hsr.getUserPrincipal();
+      remoteUser = princ == null ? null : princ.getName();
+    }
+
+    UserGroupInformation callerUGI = null;
+    if (remoteUser != null) {
+      callerUGI = UserGroupInformation.createRemoteUser(remoteUser);
+    }
+
+    return callerUGI;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bfd967d3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServiceProtocol.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/RMWebServiceProtocol.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServiceProtocol.java
index 250cb95..062ca4c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServiceProtocol.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServiceProtocol.java
@@ -108,7 +108,7 @@ public interface RMWebServiceProtocol {
    * This method dumps the scheduler logs for the time got in input, and it is
    * reachable by using {@link RMWSConsts#SCHEDULER_LOGS}.
    *
-   * @param time the period of time
+   * @param time the period of time. It is a FormParam.
    * @param hsr the servlet request
    * @return the result of the operation
    * @throws IOException when it cannot create dump log file
@@ -121,7 +121,7 @@ public interface RMWebServiceProtocol {
    * reachable by using {@link RMWSConsts#NODES}.
    *
    * @see ApplicationClientProtocol#getClusterNodes
-   * @param states the states we want to filter
+   * @param states the states we want to filter. It is a QueryParam.
    * @return all nodes in the cluster. If the states param is given, returns all
    *         nodes that are in the comma-separated list of states
    */
@@ -131,7 +131,8 @@ public interface RMWebServiceProtocol {
    * This method retrieves a specific node information, and it is reachable by
    * using {@link RMWSConsts#NODES_NODEID}.
    *
-   * @param nodeId the node we want to retrieve the information
+   * @param nodeId the node we want to retrieve the information. It is a
+   *          PathParam.
    * @return the information about the node in input
    */
   NodeInfo getNode(String nodeId);
@@ -142,19 +143,25 @@ public interface RMWebServiceProtocol {
    *
    * @see ApplicationClientProtocol#getApplications
    * @param hsr the servlet request
-   * @param stateQuery right now the stateQuery is deprecated
-   * @param statesQuery filter the result by states
-   * @param finalStatusQuery filter the result by final states
-   * @param userQuery filter the result by user
-   * @param queueQuery filter the result by queue
-   * @param count set a limit of the result
-   * @param startedBegin filter the result by started begin time
-   * @param startedEnd filter the result by started end time
-   * @param finishBegin filter the result by finish begin time
-   * @param finishEnd filter the result by finish end time
-   * @param applicationTypes filter the result by types
-   * @param applicationTags filter the result by tags
-   * @param unselectedFields De-selected params to avoid from report
+   * @param stateQuery right now the stateQuery is deprecated. It is a
+   *          QueryParam.
+   * @param statesQuery filter the result by states. It is a QueryParam.
+   * @param finalStatusQuery filter the result by final states. It is a
+   *          QueryParam.
+   * @param userQuery filter the result by user. It is a QueryParam.
+   * @param queueQuery filter the result by queue. It is a QueryParam.
+   * @param count set a limit of the result. It is a QueryParam.
+   * @param startedBegin filter the result by started begin time. It is a
+   *          QueryParam.
+   * @param startedEnd filter the result by started end time. It is a
+   *          QueryParam.
+   * @param finishBegin filter the result by finish begin time. It is a
+   *          QueryParam.
+   * @param finishEnd filter the result by finish end time. It is a QueryParam.
+   * @param applicationTypes filter the result by types. It is a QueryParam.
+   * @param applicationTags filter the result by tags. It is a QueryParam.
+   * @param unselectedFields De-selected params to avoid from report. It is a
+   *          QueryParam.
    * @return all apps in the cluster
    */
   @SuppressWarnings("checkstyle:parameternumber")
@@ -169,7 +176,8 @@ public interface RMWebServiceProtocol {
    * reachable by using {@link RMWSConsts#SCHEDULER_ACTIVITIES}.
    *
    * @param hsr the servlet request
-   * @param nodeId the node we want to retrieve the activities
+   * @param nodeId the node we want to retrieve the activities. It is a
+   *          QueryParam.
    * @return all the activities in the specific node
    */
   ActivitiesInfo getActivities(HttpServletRequest hsr, String nodeId);
@@ -180,8 +188,10 @@ public interface RMWebServiceProtocol {
    * {@link RMWSConsts#SCHEDULER_APP_ACTIVITIES}.
    *
    * @param hsr the servlet request
-   * @param appId the applicationId we want to retrieve the activities
-   * @param time for how long we want to retrieve the activities
+   * @param appId the applicationId we want to retrieve the activities. It is a
+   *          QueryParam.
+   * @param time for how long we want to retrieve the activities. It is a
+   *          QueryParam.
    * @return all the activities about a specific app for a specific time
    */
   AppActivitiesInfo getAppActivities(HttpServletRequest hsr, String appId,
@@ -192,8 +202,8 @@ public interface RMWebServiceProtocol {
    * reachable by using {@link RMWSConsts#APP_STATISTICS}.
    *
    * @param hsr the servlet request
-   * @param stateQueries filter the result by states
-   * @param typeQueries filter the result by type names
+   * @param stateQueries filter the result by states. It is a QueryParam.
+   * @param typeQueries filter the result by type names. It is a QueryParam.
    * @return the application's statistics for specific states and types
    */
   ApplicationStatisticsInfo getAppStatistics(HttpServletRequest hsr,
@@ -205,8 +215,10 @@ public interface RMWebServiceProtocol {
    *
    * @see ApplicationClientProtocol#getApplicationReport
    * @param hsr the servlet request
-   * @param appId the Id of the application we want the report
-   * @param unselectedFields De-selected params to avoid from report
+   * @param appId the Id of the application we want the report. It is a
+   *          PathParam.
+   * @param unselectedFields De-selected param list to avoid from report. It is
+   *          a QueryParam.
    * @return the app report for a specific application
    */
   AppInfo getApp(HttpServletRequest hsr, String appId,
@@ -217,7 +229,8 @@ public interface RMWebServiceProtocol {
    * using {@link RMWSConsts#APPS_APPID_STATE}.
    *
    * @param hsr the servlet request
-   * @param appId the Id of the application we want the state
+   * @param appId the Id of the application we want the state. It is a
+   *          PathParam.
    * @return the state for a specific application
    * @throws AuthorizationException if the user is not authorized
    */
@@ -228,9 +241,10 @@ public interface RMWebServiceProtocol {
    * This method updates the state of the app in input, and it is reachable by
    * using {@link RMWSConsts#APPS_APPID_STATE}.
    *
-   * @param targetState the target state for the app
+   * @param targetState the target state for the app. It is a content param.
    * @param hsr the servlet request
-   * @param appId the Id of the application we want to update the state
+   * @param appId the Id of the application we want to update the state. It is a
+   *          PathParam.
    * @return Response containing the status code
    * @throws AuthorizationException if the user is not authorized to invoke this
    *           method
@@ -259,7 +273,7 @@ public interface RMWebServiceProtocol {
    * cluster, and it is reachable by using {@link RMWSConsts#LABEL_MAPPINGS}.
    *
    * @see ApplicationClientProtocol#getLabelsToNodes
-   * @param labels filter the result by node labels
+   * @param labels filter the result by node labels. It is a QueryParam.
    * @return all the nodes within multiple node labels
    * @throws IOException if an IOException happened
    */
@@ -270,7 +284,7 @@ public interface RMWebServiceProtocol {
    * reachable by using {@link RMWSConsts#REPLACE_NODE_TO_LABELS}.
    *
    * @see ResourceManagerAdministrationProtocol#replaceLabelsOnNode
-   * @param newNodeToLabels the list of new labels
+   * @param newNodeToLabels the list of new labels. It is a content param.
    * @param hsr the servlet request
    * @return Response containing the status code
    * @throws Exception if an exception happened
@@ -283,9 +297,10 @@ public interface RMWebServiceProtocol {
    * reachable by using {@link RMWSConsts#NODES_NODEID_REPLACE_LABELS}.
    *
    * @see ResourceManagerAdministrationProtocol#replaceLabelsOnNode
-   * @param newNodeLabelsName the list of new labels
+   * @param newNodeLabelsName the list of new labels. It is a QueryParam.
    * @param hsr the servlet request
-   * @param nodeId the node we want to replace the node labels
+   * @param nodeId the node we want to replace the node labels. It is a
+   *          PathParam.
    * @return Response containing the status code
    * @throws Exception if an exception happened
    */
@@ -309,7 +324,7 @@ public interface RMWebServiceProtocol {
    * reachable by using {@link RMWSConsts#ADD_NODE_LABELS}.
    *
    * @see ResourceManagerAdministrationProtocol#addToClusterNodeLabels
-   * @param newNodeLabels the node labels to add
+   * @param newNodeLabels the node labels to add. It is a content param.
    * @param hsr the servlet request
    * @return Response containing the status code
    * @throws Exception in case of bad request
@@ -322,7 +337,7 @@ public interface RMWebServiceProtocol {
    * reachable by using {@link RMWSConsts#REMOVE_NODE_LABELS}.
    *
    * @see ResourceManagerAdministrationProtocol#removeFromClusterNodeLabels
-   * @param oldNodeLabels the node labels to remove
+   * @param oldNodeLabels the node labels to remove. It is a QueryParam.
    * @param hsr the servlet request
    * @return Response containing the status code
    * @throws Exception in case of bad request
@@ -335,7 +350,8 @@ public interface RMWebServiceProtocol {
    * reachable by using {@link RMWSConsts#NODES_NODEID_GETLABELS}.
    *
    * @param hsr the servlet request
-   * @param nodeId the node we want to get all the node labels
+   * @param nodeId the node we want to get all the node labels. It is a
+   *          PathParam.
    * @return all the labels for a specific node.
    * @throws IOException if an IOException happened
    */
@@ -347,7 +363,7 @@ public interface RMWebServiceProtocol {
    * by using {@link RMWSConsts#APPS_APPID_PRIORITY}.
    *
    * @param hsr the servlet request
-   * @param appId the app we want to get the priority
+   * @param appId the app we want to get the priority. It is a PathParam.
    * @return the priority for a specific application
    * @throws AuthorizationException in case of the user is not authorized
    */
@@ -358,9 +374,11 @@ public interface RMWebServiceProtocol {
    * This method updates the priority for a specific application, and it is
    * reachable by using {@link RMWSConsts#APPS_APPID_PRIORITY}.
    *
-   * @param targetPriority the priority we want to set for the app
+   * @param targetPriority the priority we want to set for the app. It is a
+   *          content param.
    * @param hsr the servlet request
-   * @param appId the application we want to update its priority
+   * @param appId the application we want to update its priority. It is a
+   *          PathParam.
    * @return Response containing the status code
    * @throws AuthorizationException if the user is not authenticated
    * @throws YarnException if the target is null
@@ -376,7 +394,8 @@ public interface RMWebServiceProtocol {
    * using {@link RMWSConsts#APPS_APPID_QUEUE}.
    *
    * @param hsr the servlet request
-   * @param appId the application we want to retrieve its queue
+   * @param appId the application we want to retrieve its queue. It is a
+   *          PathParam.
    * @return the Queue for a specific application.
    * @throws AuthorizationException if the user is not authenticated
    */
@@ -387,9 +406,10 @@ public interface RMWebServiceProtocol {
    * This method updates the queue for a specific application, and it is
    * reachable by using {@link RMWSConsts#APPS_APPID_QUEUE}.
    *
-   * @param targetQueue the queue we want to set
+   * @param targetQueue the queue we want to set. It is a content param.
    * @param hsr the servlet request
-   * @param appId the application we want to change its queue
+   * @param appId the application we want to change its queue. It is a
+   *          PathParam.
    * @return Response containing the status code
    * @throws AuthorizationException if the user is not authenticated
    * @throws YarnException if the app is not found
@@ -424,7 +444,7 @@ public interface RMWebServiceProtocol {
    * @see ApplicationClientProtocol#submitApplication
    *
    * @param newApp structure containing information to construct the
-   *          ApplicationSubmissionContext
+   *          ApplicationSubmissionContext. It is a content param.
    * @param hsr the servlet request
    * @return Response containing the status code
    * @throws AuthorizationException if the user is not authorized to invoke this
@@ -441,7 +461,7 @@ public interface RMWebServiceProtocol {
    * by using {@link RMWSConsts#DELEGATION_TOKEN}.
    *
    * @see ApplicationBaseProtocol#getDelegationToken
-   * @param tokenData the token to delegate
+   * @param tokenData the token to delegate. It is a content param.
    * @param hsr the servlet request
    * @return Response containing the status code
    * @throws AuthorizationException if Kerberos auth failed
@@ -508,7 +528,7 @@ public interface RMWebServiceProtocol {
    * @see ApplicationClientProtocol#submitReservation
    *
    * @param resContext provides information to construct the
-   *          ReservationSubmissionRequest
+   *          ReservationSubmissionRequest. It is a content param.
    * @param hsr the servlet request
    * @return Response containing the status code
    * @throws AuthorizationException if the user is not authorized to invoke this
@@ -527,7 +547,7 @@ public interface RMWebServiceProtocol {
    * @see ApplicationClientProtocol#updateReservation
    *
    * @param resContext provides information to construct the
-   *          ReservationUpdateRequest
+   *          ReservationUpdateRequest. It is a content param.
    * @param hsr the servlet request
    * @return Response containing the status code
    * @throws AuthorizationException if the user is not authorized to invoke this
@@ -546,7 +566,7 @@ public interface RMWebServiceProtocol {
    * @see ApplicationClientProtocol#deleteReservation
    *
    * @param resContext provides information to construct the
-   *          ReservationDeleteRequest
+   *          ReservationDeleteRequest. It is a content param.
    * @param hsr the servlet request
    * @return Response containing the status code
    * @throws AuthorizationException when the user group information cannot be
@@ -566,12 +586,13 @@ public interface RMWebServiceProtocol {
    * reachable by using {@link RMWSConsts#RESERVATION_LIST}.
    *
    * @see ApplicationClientProtocol#listReservations
-   * @param queue filter the result by queue
-   * @param reservationId filter the result by reservationId
-   * @param startTime filter the result by start time
-   * @param endTime filter the result by end time
+   * @param queue filter the result by queue. It is a QueryParam.
+   * @param reservationId filter the result by reservationId. It is a
+   *          QueryParam.
+   * @param startTime filter the result by start time. It is a QueryParam.
+   * @param endTime filter the result by end time. It is a QueryParam.
    * @param includeResourceAllocations true if the resource allocation should be
-   *          in the result, false otherwise
+   *          in the result, false otherwise. It is a QueryParam.
    * @param hsr the servlet request
    * @return Response containing the status code
    * @throws Exception in case of bad request
@@ -586,8 +607,8 @@ public interface RMWebServiceProtocol {
    * {@link RMWSConsts#APPS_TIMEOUTS_TYPE}.
    *
    * @param hsr the servlet request
-   * @param appId the application we want to get the timeout
-   * @param type the type of the timeouts
+   * @param appId the application we want to get the timeout. It is a PathParam.
+   * @param type the type of the timeouts. It is a PathParam.
    * @return the timeout for a specific application with a specific type.
    * @throws AuthorizationException if the user is not authorized
    */
@@ -599,7 +620,8 @@ public interface RMWebServiceProtocol {
    * reachable by using {@link RMWSConsts#APPS_TIMEOUTS}.
    *
    * @param hsr the servlet request
-   * @param appId the application we want to get the timeouts
+   * @param appId the application we want to get the timeouts. It is a
+   *          PathParam.
    * @return the timeouts for a specific application
    * @throws AuthorizationException if the user is not authorized
    */
@@ -611,9 +633,9 @@ public interface RMWebServiceProtocol {
    * reachable by using {@link RMWSConsts#APPS_TIMEOUT}.
    *
    * @see ApplicationClientProtocol#updateApplicationTimeouts
-   * @param appTimeout the appTimeoutInfo
+   * @param appTimeout the appTimeoutInfo. It is a content param.
    * @param hsr the servlet request
-   * @param appId the application we want to update
+   * @param appId the application we want to update. It is a PathParam.
    * @return Response containing the status code
    * @throws AuthorizationException if the user is not authorized to invoke this
    *           method
@@ -631,7 +653,8 @@ public interface RMWebServiceProtocol {
    *
    * @see ApplicationBaseProtocol#getApplicationAttempts
    * @param hsr the servlet request
-   * @param appId the application we want to get the attempts
+   * @param appId the application we want to get the attempts. It is a
+   *          PathParam.
    * @return all the attempts info for a specific application
    */
   AppAttemptsInfo getAppAttempts(HttpServletRequest hsr, String appId);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bfd967d3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.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/RMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
index b8bd9fb..166a4c9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
@@ -435,7 +435,7 @@ public class RMWebServices extends WebServices implements RMWebServiceProtocol {
       @QueryParam(RMWSConsts.FINISHED_TIME_END) String finishEnd,
       @QueryParam(RMWSConsts.APPLICATION_TYPES) Set<String> applicationTypes,
       @QueryParam(RMWSConsts.APPLICATION_TAGS) Set<String> applicationTags,
-      @QueryParam("deSelects") Set<String> unselectedFields) {
+      @QueryParam(RMWSConsts.DESELECTS) Set<String> unselectedFields) {
     boolean checkCount = false;
     boolean checkStart = false;
     boolean checkEnd = false;
@@ -818,7 +818,7 @@ public class RMWebServices extends WebServices implements RMWebServiceProtocol {
   @Override
   public AppInfo getApp(@Context HttpServletRequest hsr,
       @PathParam(RMWSConsts.APPID) String appId,
-      @QueryParam("deSelects") Set<String> unselectedFields) {
+      @QueryParam(RMWSConsts.DESELECTS) Set<String> unselectedFields) {
     init();
     ApplicationId id = WebAppUtils.parseApplicationId(recordFactory, appId);
     RMApp app = rm.getRMContext().getRMApps().get(id);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bfd967d3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppAttemptInfo.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/AppAttemptInfo.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/AppAttemptInfo.java
index 55bf999..82a946e 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/AppAttemptInfo.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/AppAttemptInfo.java
@@ -27,7 +27,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
-import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
 
 @XmlRootElement(name = "appAttempt")
@@ -106,4 +105,8 @@ public class AppAttemptInfo {
   public String getLogsLink() {
     return this.logsLink;
   }
+
+  public String getAppAttemptId() {
+    return this.appAttemptId;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bfd967d3/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 d92a793..e5e156d 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
@@ -29,6 +29,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.records.GetSubClusterInfoRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterInfoResponse;
 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;
@@ -86,12 +87,8 @@ public class TestFederationRMStateStoreService {
     // 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"));
-    }
+    GetSubClusterInfoResponse response = stateStore.getSubCluster(request);
+    Assert.assertNull(response);
 
     // Validate if sub-cluster is registered
     rm.start();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bfd967d3/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 4eea9a6..e8b4d56 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,6 +50,25 @@
 
     <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>
+
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+
+    <!-- 'mvn dependency:analyze' fails to detect use of this dependency -->
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-common</artifactId>
       <type>test-jar</type>
       <scope>test</scope>
@@ -57,21 +76,26 @@
 
     <dependency>
       <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-yarn-server-common</artifactId>
+      <artifactId>hadoop-yarn-server-resourcemanager</artifactId>
     </dependency>
 
     <dependency>
       <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-yarn-server-common</artifactId>
-      <type>test-jar</type>
+      <artifactId>hadoop-yarn-server-nodemanager</artifactId>
       <scope>test</scope>
     </dependency>
 
     <dependency>
-      <groupId>junit</groupId>
-      <artifactId>junit</artifactId>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
       <scope>test</scope>
     </dependency>
+
+    <dependency>
+      <groupId>com.google.inject</groupId>
+      <artifactId>guice</artifactId>
+    </dependency>
+
   </dependencies>
 
   <build>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bfd967d3/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 d2eee5a..121e534 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
@@ -21,6 +21,7 @@ package org.apache.hadoop.yarn.server.router;
 import java.io.IOException;
 import java.util.concurrent.atomic.AtomicBoolean;
 
+import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.service.CompositeService;
 import org.apache.hadoop.util.ShutdownHookManager;
@@ -28,11 +29,19 @@ 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.resourcemanager.webapp.RMWebAppUtil;
 import org.apache.hadoop.yarn.server.router.clientrm.RouterClientRMService;
 import org.apache.hadoop.yarn.server.router.rmadmin.RouterRMAdminService;
+import org.apache.hadoop.yarn.server.router.webapp.RouterWebApp;
+import org.apache.hadoop.yarn.webapp.WebApp;
+import org.apache.hadoop.yarn.webapp.WebApps;
+import org.apache.hadoop.yarn.webapp.WebApps.Builder;
+import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
  * 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
@@ -56,6 +65,9 @@ public class Router extends CompositeService {
   private AtomicBoolean isStopping = new AtomicBoolean(false);
   private RouterClientRMService clientRMProxyService;
   private RouterRMAdminService rmAdminProxyService;
+  private WebApp webApp;
+  @VisibleForTesting
+  protected String webAppAddress;
 
   /**
    * Priority of the Router shutdown hook.
@@ -79,6 +91,10 @@ public class Router extends CompositeService {
     // RMAdmin Proxy
     rmAdminProxyService = createRMAdminProxyService();
     addService(rmAdminProxyService);
+    // WebService
+    webAppAddress = WebAppUtils.getWebAppBindURL(this.conf,
+        YarnConfiguration.ROUTER_BIND_HOST,
+        WebAppUtils.getRouterWebAppURLWithoutScheme(this.conf));
     super.serviceInit(conf);
   }
 
@@ -89,11 +105,15 @@ public class Router extends CompositeService {
     } catch (IOException e) {
       throw new YarnRuntimeException("Failed Router login", e);
     }
+    startWepApp();
     super.serviceStart();
   }
 
   @Override
   protected void serviceStop() throws Exception {
+    if (webApp != null) {
+      webApp.stop();
+    }
     if (isStopping.getAndSet(true)) {
       return;
     }
@@ -117,6 +137,21 @@ public class Router extends CompositeService {
     return new RouterRMAdminService();
   }
 
+  @Private
+  public WebApp getWebapp() {
+    return this.webApp;
+  }
+
+  @VisibleForTesting
+  public void startWepApp() {
+
+    RMWebAppUtil.setupSecurityAndFilters(conf, null);
+
+    Builder<Object> builder =
+        WebApps.$for("cluster", null, null, "ws").with(conf).at(webAppAddress);
+    webApp = builder.start(new RouterWebApp(this));
+  }
+
   public static void main(String[] argv) {
     Configuration conf = new YarnConfiguration();
     Thread

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bfd967d3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/AbstractRESTRequestInterceptor.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/webapp/AbstractRESTRequestInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/AbstractRESTRequestInterceptor.java
new file mode 100644
index 0000000..a2d78a4
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/AbstractRESTRequestInterceptor.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.webapp;
+
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Extends the RequestInterceptor class and provides common functionality which
+ * can be used and/or extended by other concrete intercepter classes.
+ */
+public abstract class AbstractRESTRequestInterceptor
+    implements RESTRequestInterceptor {
+
+  private Configuration conf;
+  private RESTRequestInterceptor nextInterceptor;
+
+  /**
+   * Sets the {@link RESTRequestInterceptor} in the chain.
+   */
+  @Override
+  public void setNextInterceptor(RESTRequestInterceptor 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 RESTRequestInterceptor}.
+   */
+  @Override
+  public void init(String user) {
+    if (this.nextInterceptor != null) {
+      this.nextInterceptor.init(user);
+    }
+  }
+
+  /**
+   * Disposes the {@link RESTRequestInterceptor}.
+   */
+  @Override
+  public void shutdown() {
+    if (this.nextInterceptor != null) {
+      this.nextInterceptor.shutdown();
+    }
+  }
+
+  /**
+   * Gets the next {@link RESTRequestInterceptor} in the chain.
+   */
+  @Override
+  public RESTRequestInterceptor getNextInterceptor() {
+    return this.nextInterceptor;
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bfd967d3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/DefaultRequestInterceptorREST.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/webapp/DefaultRequestInterceptorREST.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/DefaultRequestInterceptorREST.java
new file mode 100644
index 0000000..aa8e3eb
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/DefaultRequestInterceptorREST.java
@@ -0,0 +1,496 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, 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.webapp;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import javax.ws.rs.core.Response;
+
+import org.apache.hadoop.security.authorize.AuthorizationException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ActivitiesInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppActivitiesInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppAttemptsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppPriority;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppQueue;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppState;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppTimeoutInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppTimeoutsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ApplicationStatisticsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ApplicationSubmissionContextInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ClusterInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ClusterMetricsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.DelegationToken;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.LabelsToNodesInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeLabelsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeToLabelsEntryList;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeToLabelsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodesInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationDeleteRequestInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationSubmissionRequestInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationUpdateRequestInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.SchedulerTypeInfo;
+import org.apache.hadoop.yarn.server.webapp.dao.AppAttemptInfo;
+import org.apache.hadoop.yarn.server.webapp.dao.ContainerInfo;
+import org.apache.hadoop.yarn.server.webapp.dao.ContainersInfo;
+import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
+
+/**
+ * Extends the AbstractRequestInterceptorClient class and provides an
+ * implementation that simply forwards the client requests to the resource
+ * manager.
+ */
+public final class DefaultRequestInterceptorREST
+    extends AbstractRESTRequestInterceptor {
+
+  private String webAppAddress;
+
+  @Override
+  public void init(String user) {
+    webAppAddress = WebAppUtils.getRMWebAppURLWithScheme(getConf());
+  }
+
+  @Override
+  public ClusterInfo get() {
+    return getClusterInfo();
+  }
+
+  @Override
+  public ClusterInfo getClusterInfo() {
+    return RouterWebServiceUtil.genericForward(webAppAddress, null,
+        ClusterInfo.class, HTTPMethods.GET,
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.INFO, null, null);
+  }
+
+  @Override
+  public ClusterMetricsInfo getClusterMetricsInfo() {
+    return RouterWebServiceUtil.genericForward(webAppAddress, null,
+        ClusterMetricsInfo.class, HTTPMethods.GET,
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.METRICS, null, null);
+  }
+
+  @Override
+  public SchedulerTypeInfo getSchedulerInfo() {
+    return RouterWebServiceUtil.genericForward(webAppAddress, null,
+        SchedulerTypeInfo.class, HTTPMethods.GET,
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.SCHEDULER, null, null);
+  }
+
+  @Override
+  public String dumpSchedulerLogs(String time, HttpServletRequest hsr)
+      throws IOException {
+    // time is specified inside hsr
+    return RouterWebServiceUtil.genericForward(webAppAddress, null,
+        String.class, HTTPMethods.GET,
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.SCHEDULER_LOGS, null, null);
+  }
+
+  @Override
+  public NodesInfo getNodes(String states) {
+    // states will be part of additionalParam
+    Map<String, String[]> additionalParam = new HashMap<String, String[]>();
+    additionalParam.put(RMWSConsts.STATES, new String[] {states});
+    return RouterWebServiceUtil.genericForward(webAppAddress, null,
+        NodesInfo.class, HTTPMethods.GET,
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.NODES, null,
+        additionalParam);
+  }
+
+  @Override
+  public NodeInfo getNode(String nodeId) {
+    return RouterWebServiceUtil.genericForward(webAppAddress, null,
+        NodeInfo.class, HTTPMethods.GET,
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.NODES + "/" + nodeId, null,
+        null);
+  }
+
+  @Override
+  public AppsInfo getApps(HttpServletRequest hsr, String stateQuery,
+      Set<String> statesQuery, String finalStatusQuery, String userQuery,
+      String queueQuery, String count, String startedBegin, String startedEnd,
+      String finishBegin, String finishEnd, Set<String> applicationTypes,
+      Set<String> applicationTags, Set<String> unselectedFields) {
+    // all the params are specified inside hsr
+    return RouterWebServiceUtil.genericForward(webAppAddress, hsr,
+        AppsInfo.class, HTTPMethods.GET,
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS, null, null);
+  }
+
+  @Override
+  public ActivitiesInfo getActivities(HttpServletRequest hsr, String nodeId) {
+    // nodeId is specified inside hsr
+    return RouterWebServiceUtil.genericForward(webAppAddress, hsr,
+        ActivitiesInfo.class, HTTPMethods.GET,
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.SCHEDULER_ACTIVITIES, null,
+        null);
+  }
+
+  @Override
+  public AppActivitiesInfo getAppActivities(HttpServletRequest hsr,
+      String appId, String time) {
+    // time and appId are specified inside hsr
+    return RouterWebServiceUtil.genericForward(webAppAddress, hsr,
+        AppActivitiesInfo.class, HTTPMethods.GET,
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.SCHEDULER_APP_ACTIVITIES,
+        null, null);
+  }
+
+  @Override
+  public ApplicationStatisticsInfo getAppStatistics(HttpServletRequest hsr,
+      Set<String> stateQueries, Set<String> typeQueries) {
+    // stateQueries and typeQueries are specified inside hsr
+    return RouterWebServiceUtil.genericForward(webAppAddress, hsr,
+        ApplicationStatisticsInfo.class, HTTPMethods.GET,
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APP_STATISTICS, null, null);
+  }
+
+  @Override
+  public AppInfo getApp(HttpServletRequest hsr, String appId,
+      Set<String> unselectedFields) {
+    // unselectedFields is specified inside hsr
+    return RouterWebServiceUtil.genericForward(webAppAddress, hsr,
+        AppInfo.class, HTTPMethods.GET,
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS + "/" + appId, null,
+        null);
+  }
+
+  @Override
+  public AppState getAppState(HttpServletRequest hsr, String appId)
+      throws AuthorizationException {
+    return RouterWebServiceUtil.genericForward(webAppAddress, hsr,
+        AppState.class, HTTPMethods.GET, RMWSConsts.RM_WEB_SERVICE_PATH
+            + RMWSConsts.APPS + "/" + appId + "/" + RMWSConsts.STATE,
+        null, null);
+  }
+
+  @Override
+  public Response updateAppState(AppState targetState, HttpServletRequest hsr,
+      String appId) throws AuthorizationException, YarnException,
+      InterruptedException, IOException {
+    return RouterWebServiceUtil.genericForward(webAppAddress, hsr,
+        Response.class, HTTPMethods.PUT, RMWSConsts.RM_WEB_SERVICE_PATH
+            + RMWSConsts.APPS + "/" + appId + "/" + RMWSConsts.STATE,
+        targetState, null);
+  }
+
+  @Override
+  public NodeToLabelsInfo getNodeToLabels(HttpServletRequest hsr)
+      throws IOException {
+    return RouterWebServiceUtil.genericForward(webAppAddress, hsr,
+        NodeToLabelsInfo.class, HTTPMethods.GET,
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.GET_NODE_TO_LABELS, null,
+        null);
+  }
+
+  @Override
+  public LabelsToNodesInfo getLabelsToNodes(Set<String> labels)
+      throws IOException {
+    // labels will be part of additionalParam
+    Map<String, String[]> additionalParam = new HashMap<String, String[]>();
+    additionalParam.put(RMWSConsts.LABELS,
+        labels.toArray(new String[labels.size()]));
+    return RouterWebServiceUtil.genericForward(webAppAddress, null,
+        LabelsToNodesInfo.class, HTTPMethods.GET,
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.LABEL_MAPPINGS, null,
+        additionalParam);
+  }
+
+  @Override
+  public Response replaceLabelsOnNodes(NodeToLabelsEntryList newNodeToLabels,
+      HttpServletRequest hsr) throws IOException {
+    return RouterWebServiceUtil.genericForward(webAppAddress, hsr,
+        Response.class, HTTPMethods.POST,
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.REPLACE_NODE_TO_LABELS,
+        newNodeToLabels, null);
+  }
+
+  @Override
+  public Response replaceLabelsOnNode(Set<String> newNodeLabelsName,
+      HttpServletRequest hsr, String nodeId) throws Exception {
+    // newNodeLabelsName is specified inside hsr
+    return RouterWebServiceUtil
+        .genericForward(webAppAddress, hsr,
+            Response.class, HTTPMethods.POST, RMWSConsts.RM_WEB_SERVICE_PATH
+                + RMWSConsts.NODES + "/" + nodeId + "/replace-labels",
+            null, null);
+  }
+
+  @Override
+  public NodeLabelsInfo getClusterNodeLabels(HttpServletRequest hsr)
+      throws IOException {
+    return RouterWebServiceUtil.genericForward(webAppAddress, hsr,
+        NodeLabelsInfo.class, HTTPMethods.GET,
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.GET_NODE_LABELS, null,
+        null);
+  }
+
+  @Override
+  public Response addToClusterNodeLabels(NodeLabelsInfo newNodeLabels,
+      HttpServletRequest hsr) throws Exception {
+    return RouterWebServiceUtil.genericForward(webAppAddress, hsr,
+        Response.class, HTTPMethods.POST,
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.ADD_NODE_LABELS,
+        newNodeLabels, null);
+  }
+
+  @Override
+  public Response removeFromCluserNodeLabels(Set<String> oldNodeLabels,
+      HttpServletRequest hsr) throws Exception {
+    // oldNodeLabels is specified inside hsr
+    return RouterWebServiceUtil.genericForward(webAppAddress, hsr,
+        Response.class, HTTPMethods.POST,
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.REMOVE_NODE_LABELS, null,
+        null);
+  }
+
+  @Override
+  public NodeLabelsInfo getLabelsOnNode(HttpServletRequest hsr, String nodeId)
+      throws IOException {
+    return RouterWebServiceUtil.genericForward(webAppAddress, hsr,
+        NodeLabelsInfo.class, HTTPMethods.GET, RMWSConsts.RM_WEB_SERVICE_PATH
+            + RMWSConsts.NODES + "/" + nodeId + "/get-labels",
+        null, null);
+  }
+
+  @Override
+  public AppPriority getAppPriority(HttpServletRequest hsr, String appId)
+      throws AuthorizationException {
+    return RouterWebServiceUtil.genericForward(webAppAddress, hsr,
+        AppPriority.class, HTTPMethods.GET, RMWSConsts.RM_WEB_SERVICE_PATH
+            + RMWSConsts.APPS + "/" + appId + "/" + RMWSConsts.PRIORITY,
+        null, null);
+  }
+
+  @Override
+  public Response updateApplicationPriority(AppPriority targetPriority,
+      HttpServletRequest hsr, String appId) throws AuthorizationException,
+      YarnException, InterruptedException, IOException {
+    return RouterWebServiceUtil.genericForward(webAppAddress, hsr,
+        Response.class, HTTPMethods.PUT, RMWSConsts.RM_WEB_SERVICE_PATH
+            + RMWSConsts.APPS + "/" + appId + "/" + RMWSConsts.PRIORITY,
+        targetPriority, null);
+  }
+
+  @Override
+  public AppQueue getAppQueue(HttpServletRequest hsr, String appId)
+      throws AuthorizationException {
+    return RouterWebServiceUtil.genericForward(webAppAddress, hsr,
+        AppQueue.class, HTTPMethods.GET, RMWSConsts.RM_WEB_SERVICE_PATH
+            + RMWSConsts.APPS + "/" + appId + "/" + RMWSConsts.QUEUE,
+        null, null);
+  }
+
+  @Override
+  public Response updateAppQueue(AppQueue targetQueue, HttpServletRequest hsr,
+      String appId) throws AuthorizationException, YarnException,
+      InterruptedException, IOException {
+    return RouterWebServiceUtil.genericForward(webAppAddress, hsr,
+        Response.class, HTTPMethods.PUT, RMWSConsts.RM_WEB_SERVICE_PATH
+            + RMWSConsts.APPS + "/" + appId + "/" + RMWSConsts.QUEUE,
+        targetQueue, null);
+  }
+
+  @Override
+  public Response createNewApplication(HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException {
+    return RouterWebServiceUtil.genericForward(webAppAddress, hsr,
+        Response.class, HTTPMethods.POST,
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS_NEW_APPLICATION, null,
+        null);
+  }
+
+  @Override
+  public Response submitApplication(ApplicationSubmissionContextInfo newApp,
+      HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException {
+    return RouterWebServiceUtil.genericForward(webAppAddress, hsr,
+        Response.class, HTTPMethods.POST,
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS, newApp, null);
+  }
+
+  @Override
+  public Response postDelegationToken(DelegationToken tokenData,
+      HttpServletRequest hsr) throws AuthorizationException, IOException,
+      InterruptedException, Exception {
+    return RouterWebServiceUtil.genericForward(webAppAddress, hsr,
+        Response.class, HTTPMethods.POST,
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.DELEGATION_TOKEN, tokenData,
+        null);
+  }
+
+  @Override
+  public Response postDelegationTokenExpiration(HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException,
+      Exception {
+    return RouterWebServiceUtil.genericForward(webAppAddress, hsr,
+        Response.class, HTTPMethods.POST,
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.DELEGATION_TOKEN_EXPIRATION,
+        null, null);
+  }
+
+  @Override
+  public Response cancelDelegationToken(HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException,
+      Exception {
+    return RouterWebServiceUtil.genericForward(webAppAddress, hsr,
+        Response.class, HTTPMethods.DELETE,
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.DELEGATION_TOKEN, null,
+        null);
+  }
+
+  @Override
+  public Response createNewReservation(HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException {
+    return RouterWebServiceUtil.genericForward(webAppAddress, hsr,
+        Response.class, HTTPMethods.POST,
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.RESERVATION_NEW, null,
+        null);
+  }
+
+  @Override
+  public Response submitReservation(ReservationSubmissionRequestInfo resContext,
+      HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException {
+    return RouterWebServiceUtil.genericForward(webAppAddress, hsr,
+        Response.class, HTTPMethods.POST,
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.RESERVATION_SUBMIT,
+        resContext, null);
+  }
+
+  @Override
+  public Response updateReservation(ReservationUpdateRequestInfo resContext,
+      HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException {
+    return RouterWebServiceUtil.genericForward(webAppAddress, hsr,
+        Response.class, HTTPMethods.POST,
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.RESERVATION_UPDATE,
+        resContext, null);
+  }
+
+  @Override
+  public Response deleteReservation(ReservationDeleteRequestInfo resContext,
+      HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException {
+    return RouterWebServiceUtil.genericForward(webAppAddress, hsr,
+        Response.class, HTTPMethods.POST,
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.RESERVATION_DELETE,
+        resContext, null);
+  }
+
+  @Override
+  public Response listReservation(String queue, String reservationId,
+      long startTime, long endTime, boolean includeResourceAllocations,
+      HttpServletRequest hsr) throws Exception {
+    // queue, reservationId, startTime, endTime, includeResourceAllocations are
+    // specified inside hsr
+    return RouterWebServiceUtil.genericForward(webAppAddress, hsr,
+        Response.class, HTTPMethods.GET,
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.RESERVATION_LIST, null,
+        null);
+  }
+
+  @Override
+  public AppTimeoutInfo getAppTimeout(HttpServletRequest hsr, String appId,
+      String type) throws AuthorizationException {
+    return RouterWebServiceUtil
+        .genericForward(webAppAddress, hsr, AppTimeoutInfo.class,
+            HTTPMethods.GET, RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS
+                + "/" + appId + "/" + RMWSConsts.TIMEOUTS + "/" + type,
+            null, null);
+  }
+
+  @Override
+  public AppTimeoutsInfo getAppTimeouts(HttpServletRequest hsr, String appId)
+      throws AuthorizationException {
+    return RouterWebServiceUtil.genericForward(webAppAddress, hsr,
+        AppTimeoutsInfo.class, HTTPMethods.GET, RMWSConsts.RM_WEB_SERVICE_PATH
+            + RMWSConsts.APPS + "/" + appId + "/" + RMWSConsts.TIMEOUTS,
+        null, null);
+  }
+
+  @Override
+  public Response updateApplicationTimeout(AppTimeoutInfo appTimeout,
+      HttpServletRequest hsr, String appId) throws AuthorizationException,
+      YarnException, InterruptedException, IOException {
+    return RouterWebServiceUtil.genericForward(webAppAddress, hsr,
+        Response.class, HTTPMethods.PUT, RMWSConsts.RM_WEB_SERVICE_PATH
+            + RMWSConsts.APPS + "/" + appId + "/" + RMWSConsts.TIMEOUT,
+        appTimeout, null);
+  }
+
+  @Override
+  public AppAttemptsInfo getAppAttempts(HttpServletRequest hsr, String appId) {
+    return RouterWebServiceUtil.genericForward(webAppAddress, hsr,
+        AppAttemptsInfo.class, HTTPMethods.GET, RMWSConsts.RM_WEB_SERVICE_PATH
+            + RMWSConsts.APPS + "/" + appId + "/" + RMWSConsts.APPATTEMPTS,
+        null, null);
+  }
+
+  @Override
+  public AppAttemptInfo getAppAttempt(HttpServletRequest req,
+      HttpServletResponse res, String appId, String appAttemptId) {
+    return RouterWebServiceUtil.genericForward(webAppAddress, req,
+        AppAttemptInfo.class,
+        HTTPMethods.GET, RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS + "/"
+            + appId + "/" + RMWSConsts.APPATTEMPTS + "/" + appAttemptId,
+        null, null);
+  }
+
+  @Override
+  public ContainersInfo getContainers(HttpServletRequest req,
+      HttpServletResponse res, String appId, String appAttemptId) {
+    return RouterWebServiceUtil.genericForward(webAppAddress, req,
+        ContainersInfo.class, HTTPMethods.GET,
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS + "/" + appId + "/"
+            + RMWSConsts.APPATTEMPTS + "/" + appAttemptId + "/"
+            + RMWSConsts.CONTAINERS,
+        null, null);
+  }
+
+  @Override
+  public ContainerInfo getContainer(HttpServletRequest req,
+      HttpServletResponse res, String appId, String appAttemptId,
+      String containerId) {
+    return RouterWebServiceUtil.genericForward(webAppAddress, req,
+        ContainerInfo.class, HTTPMethods.GET,
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS + "/" + appId + "/"
+            + RMWSConsts.APPATTEMPTS + "/" + appAttemptId + "/"
+            + RMWSConsts.CONTAINERS + "/" + containerId,
+        null, null);
+  }
+
+  @Override
+  public void setNextInterceptor(RESTRequestInterceptor next) {
+    throw new YarnRuntimeException("setNextInterceptor is being called on "
+        + "DefaultRequestInterceptorREST, which should be the last one "
+        + "in the chain. Check if the interceptor pipeline configuration "
+        + "is correct");
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bfd967d3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/HTTPMethods.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/webapp/HTTPMethods.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/HTTPMethods.java
new file mode 100644
index 0000000..45056ca
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/HTTPMethods.java
@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, 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.webapp;
+
+/**
+ * HTTP verbs.
+ **/
+public enum HTTPMethods {
+
+  /* to retrieve resource representation/information */
+  GET,
+  /* to update existing resource */
+  PUT,
+  /* to delete resources */
+  DELETE,
+  /* to create new subordinate resources */
+  POST
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bfd967d3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/RESTRequestInterceptor.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/webapp/RESTRequestInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/RESTRequestInterceptor.java
new file mode 100644
index 0000000..06f39b5
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/RESTRequestInterceptor.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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, 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.webapp;
+
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWebServiceProtocol;
+import org.apache.hadoop.yarn.server.webapp.WebServices;
+import org.apache.hadoop.yarn.server.webapp.dao.AppAttemptInfo;
+import org.apache.hadoop.yarn.server.webapp.dao.ContainerInfo;
+import org.apache.hadoop.yarn.server.webapp.dao.ContainersInfo;
+
+/**
+ * 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 server.
+ *
+ * This class includes 4 methods getAppAttempts, getAppAttempt, getContainers
+ * and getContainer that belong to {@link WebServices}. They are in this class
+ * to make sure that RouterWebServices implements the same REST methods of
+ * {@code RMWebServices}.
+ */
+public interface RESTRequestInterceptor
+    extends RMWebServiceProtocol, 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 RESTRequestInterceptor to set in the pipeline
+   */
+  void setNextInterceptor(RESTRequestInterceptor nextInterceptor);
+
+  /**
+   * Returns the next intercepter in the chain.
+   *
+   * @return the next intercepter in the chain
+   */
+  RESTRequestInterceptor getNextInterceptor();
+
+  /**
+   *
+   * @see WebServices#getAppAttempt(HttpServletRequest, HttpServletResponse,
+   *      String, String)
+   * @param req the servlet request
+   * @param res the servlet response
+   * @param appId the application we want to get the appAttempt. It is a
+   *          PathParam.
+   * @param appAttemptId the AppAttempt we want to get the info. It is a
+   *          PathParam.
+   * @return AppAttemptInfo of the specific AppAttempt
+   */
+  AppAttemptInfo getAppAttempt(HttpServletRequest req, HttpServletResponse res,
+      String appId, String appAttemptId);
+
+  /**
+   *
+   * @see WebServices#getContainers(HttpServletRequest, HttpServletResponse,
+   *      String, String)
+   * @param req the servlet request
+   * @param res the servlet response
+   * @param appId the application we want to get the containers info. It is a
+   *          PathParam.
+   * @param appAttemptId the AppAttempt we want to get the info. It is a
+   *          PathParam.
+   * @return ContainersInfo of all the containers that belong to the specific
+   *         AppAttempt
+   */
+  ContainersInfo getContainers(HttpServletRequest req, HttpServletResponse res,
+      String appId, String appAttemptId);
+
+  /**
+   *
+   * @see WebServices#getContainer(HttpServletRequest, HttpServletResponse,
+   *      String, String, String)
+   * @param req the servlet request
+   * @param res the servlet response
+   * @param appId the application we want to get the containers info. It is a
+   *          PathParam.
+   * @param appAttemptId the AppAttempt we want to get the info. It is a
+   *          PathParam.
+   * @param containerId the container we want to get the info. It is a
+   *          PathParam.
+   * @return ContainerInfo of the specific ContainerId
+   */
+  ContainerInfo getContainer(HttpServletRequest req, HttpServletResponse res,
+      String appId, String appAttemptId, String containerId);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bfd967d3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/RouterWebApp.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/webapp/RouterWebApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/RouterWebApp.java
new file mode 100644
index 0000000..5436bad
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/RouterWebApp.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.router.webapp;
+
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.JAXBContextResolver;
+import org.apache.hadoop.yarn.server.router.Router;
+import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
+import org.apache.hadoop.yarn.webapp.WebApp;
+import org.apache.hadoop.yarn.webapp.YarnWebParams;
+
+/**
+ * The Router webapp.
+ */
+public class RouterWebApp extends WebApp implements YarnWebParams {
+  private Router router;
+
+  public RouterWebApp(Router router) {
+    this.router = router;
+  }
+
+  @Override
+  public void setup() {
+    bind(JAXBContextResolver.class);
+    bind(RouterWebServices.class);
+    bind(GenericExceptionHandler.class);
+    bind(RouterWebApp.class).toInstance(this);
+
+    if (router != null) {
+      bind(Router.class).toInstance(router);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bfd967d3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/RouterWebServiceUtil.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/webapp/RouterWebServiceUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/RouterWebServiceUtil.java
new file mode 100644
index 0000000..18618ee
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/RouterWebServiceUtil.java
@@ -0,0 +1,227 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, 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.webapp;
+
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import javax.servlet.http.HttpServletRequest;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.MultivaluedMap;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.Response.ResponseBuilder;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWebAppUtil;
+import org.apache.hadoop.yarn.webapp.BadRequestException;
+import org.apache.hadoop.yarn.webapp.ForbiddenException;
+import org.apache.hadoop.yarn.webapp.NotFoundException;
+
+import com.sun.jersey.api.ConflictException;
+import com.sun.jersey.api.client.Client;
+import com.sun.jersey.api.client.ClientResponse;
+import com.sun.jersey.api.client.WebResource;
+import com.sun.jersey.api.client.WebResource.Builder;
+import com.sun.jersey.core.util.MultivaluedMapImpl;
+
+/**
+ * The Router webservice util class.
+ */
+public final class RouterWebServiceUtil {
+
+  private static String user = "YarnRouter";
+
+  private static final Log LOG =
+      LogFactory.getLog(RouterWebServiceUtil.class.getName());
+
+  /** Disable constructor. */
+  private RouterWebServiceUtil() {
+  }
+
+  /**
+   * Creates and performs a REST call to a specific WebService.
+   *
+   * @param webApp the address of the remote webap
+   * @param hsr the servlet request
+   * @param returnType the return type of the REST call
+   * @param <T> Type of return object.
+   * @param method the HTTP method of the REST call
+   * @param targetPath additional path to add to the webapp address
+   * @param formParam the form parameters as input for a specific REST call
+   * @param additionalParam the query parameters as input for a specific REST
+   *          call in case the call has no servlet request
+   * @return the retrieved entity from the REST call
+   */
+  protected static <T> T genericForward(String webApp, HttpServletRequest hsr,
+      final Class<T> returnType, HTTPMethods method, String targetPath,
+      Object formParam, Map<String, String[]> additionalParam) {
+
+    UserGroupInformation callerUGI = null;
+
+    if (hsr != null) {
+      callerUGI = RMWebAppUtil.getCallerUserGroupInformation(hsr, true);
+    } else {
+      // user not required
+      callerUGI = UserGroupInformation.createRemoteUser(user);
+
+    }
+
+    if (callerUGI == null) {
+      LOG.error("Unable to obtain user name, user not authenticated");
+      return null;
+    }
+
+    try {
+      return callerUGI.doAs(new PrivilegedExceptionAction<T>() {
+        @SuppressWarnings("unchecked")
+        @Override
+        public T run() {
+
+          Map<String, String[]> paramMap = null;
+
+          // We can have hsr or additionalParam. There are no case with both.
+          if (hsr != null) {
+            paramMap = hsr.getParameterMap();
+          } else if (additionalParam != null) {
+            paramMap = additionalParam;
+          }
+
+          ClientResponse response = RouterWebServiceUtil.invokeRMWebService(
+              webApp, targetPath, method,
+              (hsr == null) ? null : hsr.getPathInfo(), paramMap, formParam);
+          if (Response.class.equals(returnType)) {
+            return (T) RouterWebServiceUtil.clientResponseToResponse(response);
+          }
+          // YARN RM can answer with Status.OK or it throws an exception
+          if (response.getStatus() == 200) {
+            return response.getEntity(returnType);
+          }
+          RouterWebServiceUtil.retrieveException(response);
+          return null;
+        }
+      });
+    } catch (InterruptedException e) {
+      return null;
+    } catch (IOException e) {
+      return null;
+    }
+  }
+
+  /**
+   * Performs an invocation of a REST call on a remote RMWebService.
+   *
+   * @param additionalParam
+   */
+  private static ClientResponse invokeRMWebService(String webApp, String path,
+      HTTPMethods method, String additionalPath,
+      Map<String, String[]> queryParams, Object formParam) {
+    Client client = Client.create();
+
+    WebResource webResource = client.resource(webApp).path(path);
+
+    if (additionalPath != null && !additionalPath.isEmpty()) {
+      webResource = webResource.path(additionalPath);
+    }
+
+    if (queryParams != null && !queryParams.isEmpty()) {
+      MultivaluedMap<String, String> paramMap = new MultivaluedMapImpl();
+
+      for (Entry<String, String[]> param : queryParams.entrySet()) {
+        String[] values = param.getValue();
+        for (int i = 0; i < values.length; i++) {
+          paramMap.add(param.getKey(), values[i]);
+        }
+      }
+      webResource = webResource.queryParams(paramMap);
+    }
+
+    // I can forward the call in JSON or XML since the Router will convert it
+    // again in Object before send it back to the client
+    Builder builder = null;
+    if (formParam != null) {
+      builder = webResource.entity(formParam, MediaType.APPLICATION_XML);
+      builder = builder.accept(MediaType.APPLICATION_XML);
+    } else {
+      builder = webResource.accept(MediaType.APPLICATION_XML);
+    }
+
+    ClientResponse response = null;
+
+    switch (method) {
+    case DELETE:
+      response = builder.delete(ClientResponse.class);
+      break;
+    case GET:
+      response = builder.get(ClientResponse.class);
+      break;
+    case POST:
+      response = builder.post(ClientResponse.class);
+      break;
+    case PUT:
+      response = builder.put(ClientResponse.class);
+      break;
+    default:
+      break;
+    }
+
+    return response;
+  }
+
+  public static Response clientResponseToResponse(ClientResponse r) {
+    if (r == null) {
+      return null;
+    }
+    // copy the status code
+    ResponseBuilder rb = Response.status(r.getStatus());
+    // copy all the headers
+    for (Entry<String, List<String>> entry : r.getHeaders().entrySet()) {
+      for (String value : entry.getValue()) {
+        rb.header(entry.getKey(), value);
+      }
+    }
+    // copy the entity
+    rb.entity(r.getEntityInputStream());
+    // return the response
+    return rb.build();
+  }
+
+  public static void retrieveException(ClientResponse response) {
+    String serverErrorMsg = response.getEntity(String.class);
+    int status = response.getStatus();
+    if (status == 400) {
+      throw new BadRequestException(serverErrorMsg);
+    }
+    if (status == 403) {
+      throw new ForbiddenException(serverErrorMsg);
+    }
+    if (status == 404) {
+      throw new NotFoundException(serverErrorMsg);
+    }
+    if (status == 409) {
+      throw new ConflictException(serverErrorMsg);
+    }
+
+  }
+
+}
\ 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


[19/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 cu...@apache.org.
YARN-6370. Properly handle rack requests for non-active subclusters in LocalityMulticastAMRMProxyPolicy. (Contributed by Botong Huang via curino).

(cherry picked from commit ce419881c32b178c48c3a01b5a15e4e3a3e750f5)
(cherry picked from commit 1c64e1709b627846f29dc2fe6d637f074de8b403)


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

Branch: refs/heads/branch-2
Commit: b40bdafe31168fc60f4a6d3a060e7d16097b14ff
Parents: 61c07e4
Author: Carlo Curino <cu...@apache.org>
Authored: Wed Mar 22 13:53:47 2017 -0700
Committer: Carlo Curino <cu...@apache.org>
Committed: Thu Sep 21 16:25:10 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/b40bdafe/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/b40bdafe/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


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

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

(cherry picked from commit f8208fe0b536f29aa65af71d20c3b3e3765679fd)
(cherry picked from commit 81472778d7ef013ea6b2714bc734bc6fc3ca32fa)


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

Branch: refs/heads/branch-2
Commit: 23c42408ba2722f8298a921614a5d2080ef8fe7b
Parents: aac8755
Author: Subru Krishnan <su...@apache.org>
Authored: Wed Sep 7 17:33:34 2016 -0700
Committer: Carlo Curino <cu...@apache.org>
Committed: Thu Sep 21 16:23:41 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/23c42408/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/23c42408/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/23c42408/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/23c42408/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/23c42408/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/23c42408/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/23c42408/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/23c42408/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/23c42408/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/23c42408/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/23c42408/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/23c42408/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/23c42408/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/23c42408/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/23c42408/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/23c42408/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/23c42408/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


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

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

(cherry picked from commit 326a2e6bde1cf266ecc7d3b513cdaac6abcebbe4)
(cherry picked from commit e1da8f0667589dd660e6fcd776cc87f1b8ef6db9)


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

Branch: refs/heads/branch-2
Commit: b9dcf9283e29281e55a65b59bd74c21dc6806296
Parents: b40bdaf
Author: Subru Krishnan <su...@apache.org>
Authored: Wed Apr 5 15:02:00 2017 -0700
Committer: Carlo Curino <cu...@apache.org>
Committed: Thu Sep 21 16:25:10 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/b9dcf928/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/b9dcf928/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/b9dcf928/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/b9dcf928/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/b9dcf928/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/b9dcf928/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/b9dcf928/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/b9dcf928/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/b9dcf928/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/b9dcf928/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/b9dcf928/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/b9dcf928/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/b9dcf928/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/b9dcf928/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/b9dcf928/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/b9dcf928/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/b9dcf928/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/b9dcf928/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


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

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

(cherry picked from commit 4846069061b6baa06da3b524b9e36567dd368388)
(cherry picked from commit 80e1904000923819c98ae68456cb0452c255d5f3)


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

Branch: refs/heads/branch-2
Commit: dc0a2e6c593851342d6faf48a25bb07d4c3550cd
Parents: 3c5dfa0
Author: Subru Krishnan <su...@apache.org>
Authored: Wed May 3 18:26:15 2017 -0700
Committer: Carlo Curino <cu...@apache.org>
Committed: Thu Sep 21 16:43:08 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  | 513 +++++++++++++++++
 .../hadoop-yarn-server-nodemanager/pom.xml      |   7 +
 .../amrmproxy/MockRequestInterceptor.java       |  14 +-
 .../amrmproxy/MockResourceManagerFacade.java    | 513 -----------------
 .../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, 2962 insertions(+), 547 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/dc0a2e6c/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 152dcc7..d61dc4b 100755
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -273,6 +273,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/dc0a2e6c/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 6fbf1a6..a3b53d6 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
@@ -2584,6 +2584,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/dc0a2e6c/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/dc0a2e6c/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 074a95d..5d53e14 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
@@ -3114,4 +3114,22 @@
     <value>false</value>
   </property>
 
+  <property>
+    <description>
+      The comma separated list of class names that implement the
+      RequestInterceptor interface. This is used by the RouterClientRMService
+      to create the request processing pipeline for users.
+    </description>
+    <name>yarn.router.clientrm.interceptor-class.pipeline</name>
+    <value>org.apache.hadoop.yarn.server.router.clientrm.DefaultClientRequestInterceptor</value>
+  </property>
+
+  <property>
+    <description>
+      Size of LRU cache for Router ClientRM Service.
+    </description>
+    <name>yarn.router.clientrm.cache-max-size</name>
+    <value>25</value>
+  </property>
+
 </configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dc0a2e6c/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/dc0a2e6c/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 2d78885..7722f4f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml
@@ -215,6 +215,17 @@
           </excludes>
         </configuration>
       </plugin>
+      <plugin>
+        <artifactId>maven-jar-plugin</artifactId>
+        <executions>
+          <execution>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+            <phase>test-compile</phase>
+          </execution>
+        </executions>
+      </plugin>
     </plugins>
   </build>
 </project>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dc0a2e6c/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..499a5cb
--- /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,513 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, 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.mortbay.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.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 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/dc0a2e6c/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 13a336e..a1be7dc 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
@@ -173,6 +173,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/dc0a2e6c/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/dc0a2e6c/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 499a5cb..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,513 +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.mortbay.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.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 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/dc0a2e6c/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/dc0a2e6c/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/dc0a2e6c/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/dc0a2e6c/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


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

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

(cherry picked from commit 5c486961cd3a175b122ef86275c99b72964f2c50)
(cherry picked from commit 8623644f4599f51d34ba79c4c1453b3997205d8f)


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

Branch: refs/heads/branch-2
Commit: 6191fac914d35db18e0ef7a4364abe07a156c7e2
Parents: 2dca88b
Author: Carlo Curino <cu...@apache.org>
Authored: Tue Feb 28 17:04:20 2017 -0800
Committer: Carlo Curino <cu...@apache.org>
Committed: Thu Sep 21 16:25:08 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/6191fac9/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/6191fac9/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/6191fac9/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/6191fac9/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/6191fac9/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


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

Posted by cu...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/dc0a2e6c/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/dc0a2e6c/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/dc0a2e6c/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/dc0a2e6c/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/dc0a2e6c/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/dc0a2e6c/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/dc0a2e6c/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


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

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

(cherry picked from commit bd9c7b1bec3293fe338553c0d5c3612e6176fb26)
(cherry picked from commit f317e0cb48c16e072aa259a0a7ed71a05274bb80)


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

Branch: refs/heads/branch-2
Commit: e4832bef6d38894703bc3f0e6e6a9496801eeb84
Parents: 9476d86
Author: Subru Krishnan <su...@apache.org>
Authored: Fri May 26 17:10:03 2017 -0700
Committer: Carlo Curino <cu...@apache.org>
Committed: Thu Sep 21 16:47:50 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/e4832bef/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


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

Posted by cu...@apache.org.
YARN-5531. UnmanagedAM pool manager for federating application across clusters. (Botong Huang via Subru).

(cherry picked from commit 73bb2102ce4b82b3a3bed91319f7c8f067ddc3e8)
(cherry picked from commit 859aa1f9d621d07693825e610bdc0149f7a2770a)


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

Branch: refs/heads/branch-2
Commit: 9476d86ce869b51fc7524ae58dd53862bc2d7d72
Parents: 7f00f93
Author: Subru Krishnan <su...@apache.org>
Authored: Fri May 26 16:23:38 2017 -0700
Committer: Carlo Curino <cu...@apache.org>
Committed: Thu Sep 21 16:47:43 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  |  10 +-
 .../uam/TestUnmanagedApplicationManager.java    | 335 ++++++++++
 .../amrmproxy/DefaultRequestInterceptor.java    |  30 +-
 .../ApplicationMasterService.java               |  12 +-
 .../TestApplicationMasterLauncher.java          |   6 +-
 12 files changed, 1590 insertions(+), 118 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9476d86c/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/9476d86c/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/9476d86c/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/9476d86c/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/9476d86c/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/9476d86c/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/9476d86c/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/9476d86c/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 bda41d4..c4a4002 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
@@ -116,7 +116,10 @@ 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.mortbay.log.Log;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Strings;
 
 /**
  * Mock Resource Manager facade implementation that exposes all the methods
@@ -165,8 +168,9 @@ public class MockResourceManagerFacade
     Log.info("Registering application attempt: " + amrmToken);
 
     synchronized (applicationContainerIdMap) {
-      Assert.assertFalse("The application id is already registered: "
-          + amrmToken, applicationContainerIdMap.containsKey(amrmToken));
+      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>());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9476d86c/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/9476d86c/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)) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9476d86c/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 147ba34..aa4d620 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
@@ -213,15 +213,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/9476d86c/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 93ab8d0..1603c2d 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
@@ -75,6 +75,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;
@@ -346,9 +347,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-6807. Adding required missing configs to Federation configuration guide based on e2e testing. (Tanuj Nayak via Subru).

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

(cherry picked from commit b4ac9d1b63dd4031eee8b17d2462087721050b9a)
(cherry picked from commit f427e4201f969dd4d1e086c24a48a247d10cdc2f)


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

Branch: refs/heads/branch-2
Commit: d3fb1711a44c63966d4b8ec99511c1b2b1521217
Parents: 169037c
Author: Subru Krishnan <su...@apache.org>
Authored: Thu Jul 13 18:44:32 2017 -0700
Committer: Carlo Curino <cu...@apache.org>
Committed: Thu Sep 21 16:55:39 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/d3fb1711/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


[18/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 cu...@apache.org.
YARN-6247. Share a single instance of SubClusterResolver instead of instantiating one per AM. (Botong Huang via Subru)

(cherry picked from commit 51aeb2ce0c599176aca9466a939c3ad55df30036)
(cherry picked from commit 86b2bec56e28a2d1ece53ab5a452860fd0444268)


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

Branch: refs/heads/branch-2
Commit: 61c07e4f3ff23d130eae487ea7d058746bb1166a
Parents: 6191fac
Author: Subru Krishnan <su...@apache.org>
Authored: Thu Mar 2 18:54:53 2017 -0800
Committer: Carlo Curino <cu...@apache.org>
Committed: Thu Sep 21 16:25:09 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/61c07e4f/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 906d632..fffef1c 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
@@ -2541,6 +2541,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/61c07e4f/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 bc9e853..074a95d 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
@@ -2632,6 +2632,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/61c07e4f/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/61c07e4f/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/61c07e4f/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


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

Posted by cu...@apache.org.
YARN-3666. Federation Intercepting and propagating AM- home RM communications. (Botong Huang via Subru).

(cherry picked from commit 2399eb8200609246cb623c74450ca4a2032063cc)
(cherry picked from commit bed1832c934fe4ba44efdcdc49fce06457dc3d4f)


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

Branch: refs/heads/branch-2
Commit: 7ede8c1a53e6c2fc30dfc2923929890049bb62a9
Parents: 7dd6caf
Author: Subru Krishnan <su...@apache.org>
Authored: Wed May 31 13:21:09 2017 -0700
Committer: Carlo Curino <cu...@apache.org>
Committed: Thu Sep 21 16:49:54 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/7ede8c1a/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 4c4298d..73f1038 100644
--- a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
+++ b/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
@@ -567,4 +567,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/7ede8c1a/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/7ede8c1a/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 cf8db07..937ede5 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/7ede8c1a/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/7ede8c1a/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


[44/50] [abbrv] hadoop git commit: YARN-5412. Create a proxy chain for ResourceManager REST API in the Router. (Contributed by Giovanni Matteo Fumarola via curino)

Posted by cu...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/bfd967d3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/PassThroughRESTRequestInterceptor.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/webapp/PassThroughRESTRequestInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/PassThroughRESTRequestInterceptor.java
new file mode 100644
index 0000000..ea985a2
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/PassThroughRESTRequestInterceptor.java
@@ -0,0 +1,339 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, 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.webapp;
+
+import java.io.IOException;
+import java.util.Set;
+
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import javax.ws.rs.core.Response;
+
+import org.apache.hadoop.security.authorize.AuthorizationException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ActivitiesInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppActivitiesInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppAttemptsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppPriority;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppQueue;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppState;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppTimeoutInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppTimeoutsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ApplicationStatisticsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ApplicationSubmissionContextInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ClusterInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ClusterMetricsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.DelegationToken;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.LabelsToNodesInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeLabelsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeToLabelsEntryList;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeToLabelsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodesInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationDeleteRequestInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationSubmissionRequestInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationUpdateRequestInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.SchedulerTypeInfo;
+import org.apache.hadoop.yarn.server.webapp.dao.AppAttemptInfo;
+import org.apache.hadoop.yarn.server.webapp.dao.ContainerInfo;
+import org.apache.hadoop.yarn.server.webapp.dao.ContainersInfo;
+
+/**
+ * Mock intercepter that does not do anything other than forwarding it to the
+ * next intercepter in the chain.
+ */
+public class PassThroughRESTRequestInterceptor
+    extends AbstractRESTRequestInterceptor {
+
+  @Override
+  public AppAttemptsInfo getAppAttempts(HttpServletRequest hsr, String appId) {
+    return getNextInterceptor().getAppAttempts(hsr, appId);
+  }
+
+  @Override
+  public AppAttemptInfo getAppAttempt(HttpServletRequest req,
+      HttpServletResponse res, String appId, String appAttemptId) {
+    return getNextInterceptor().getAppAttempt(req, res, appId, appAttemptId);
+  }
+
+  @Override
+  public ContainersInfo getContainers(HttpServletRequest req,
+      HttpServletResponse res, String appId, String appAttemptId) {
+    return getNextInterceptor().getContainers(req, res, appId, appAttemptId);
+  }
+
+  @Override
+  public ContainerInfo getContainer(HttpServletRequest req,
+      HttpServletResponse res, String appId, String appAttemptId,
+      String containerId) {
+    return getNextInterceptor().getContainer(req, res, appId, appAttemptId,
+        containerId);
+  }
+
+  @Override
+  public ClusterInfo get() {
+    return getNextInterceptor().get();
+  }
+
+  @Override
+  public ClusterInfo getClusterInfo() {
+    return getNextInterceptor().getClusterInfo();
+  }
+
+  @Override
+  public ClusterMetricsInfo getClusterMetricsInfo() {
+    return getNextInterceptor().getClusterMetricsInfo();
+  }
+
+  @Override
+  public SchedulerTypeInfo getSchedulerInfo() {
+    return getNextInterceptor().getSchedulerInfo();
+  }
+
+  @Override
+  public String dumpSchedulerLogs(String time, HttpServletRequest hsr)
+      throws IOException {
+    return getNextInterceptor().dumpSchedulerLogs(time, hsr);
+  }
+
+  @Override
+  public NodesInfo getNodes(String states) {
+    return getNextInterceptor().getNodes(states);
+  }
+
+  @Override
+  public NodeInfo getNode(String nodeId) {
+    return getNextInterceptor().getNode(nodeId);
+  }
+
+  @Override
+  public AppsInfo getApps(HttpServletRequest hsr, String stateQuery,
+      Set<String> statesQuery, String finalStatusQuery, String userQuery,
+      String queueQuery, String count, String startedBegin, String startedEnd,
+      String finishBegin, String finishEnd, Set<String> applicationTypes,
+      Set<String> applicationTags, Set<String> unselectedFields) {
+    return getNextInterceptor().getApps(hsr, stateQuery, statesQuery,
+        finalStatusQuery, userQuery, queueQuery, count, startedBegin,
+        startedEnd, finishBegin, finishEnd, applicationTypes, applicationTags,
+        unselectedFields);
+  }
+
+  @Override
+  public ActivitiesInfo getActivities(HttpServletRequest hsr, String nodeId) {
+    return getNextInterceptor().getActivities(hsr, nodeId);
+  }
+
+  @Override
+  public AppActivitiesInfo getAppActivities(HttpServletRequest hsr,
+      String appId, String time) {
+    return getNextInterceptor().getAppActivities(hsr, appId, time);
+  }
+
+  @Override
+  public ApplicationStatisticsInfo getAppStatistics(HttpServletRequest hsr,
+      Set<String> stateQueries, Set<String> typeQueries) {
+    return getNextInterceptor().getAppStatistics(hsr, stateQueries,
+        typeQueries);
+  }
+
+  @Override
+  public AppInfo getApp(HttpServletRequest hsr, String appId,
+      Set<String> unselectedFields) {
+    return getNextInterceptor().getApp(hsr, appId, unselectedFields);
+  }
+
+  @Override
+  public AppState getAppState(HttpServletRequest hsr, String appId)
+      throws AuthorizationException {
+    return getNextInterceptor().getAppState(hsr, appId);
+  }
+
+  @Override
+  public Response updateAppState(AppState targetState, HttpServletRequest hsr,
+      String appId) throws AuthorizationException, YarnException,
+      InterruptedException, IOException {
+    return getNextInterceptor().updateAppState(targetState, hsr, appId);
+  }
+
+  @Override
+  public NodeToLabelsInfo getNodeToLabels(HttpServletRequest hsr)
+      throws IOException {
+    return getNextInterceptor().getNodeToLabels(hsr);
+  }
+
+  @Override
+  public LabelsToNodesInfo getLabelsToNodes(Set<String> labels)
+      throws IOException {
+    return getNextInterceptor().getLabelsToNodes(labels);
+  }
+
+  @Override
+  public Response replaceLabelsOnNodes(NodeToLabelsEntryList newNodeToLabels,
+      HttpServletRequest hsr) throws Exception {
+    return getNextInterceptor().replaceLabelsOnNodes(newNodeToLabels, hsr);
+  }
+
+  @Override
+  public Response replaceLabelsOnNode(Set<String> newNodeLabelsName,
+      HttpServletRequest hsr, String nodeId) throws Exception {
+    return getNextInterceptor().replaceLabelsOnNode(newNodeLabelsName, hsr,
+        nodeId);
+  }
+
+  @Override
+  public NodeLabelsInfo getClusterNodeLabels(HttpServletRequest hsr)
+      throws IOException {
+    return getNextInterceptor().getClusterNodeLabels(hsr);
+  }
+
+  @Override
+  public Response addToClusterNodeLabels(NodeLabelsInfo newNodeLabels,
+      HttpServletRequest hsr) throws Exception {
+    return getNextInterceptor().addToClusterNodeLabels(newNodeLabels, hsr);
+  }
+
+  @Override
+  public Response removeFromCluserNodeLabels(Set<String> oldNodeLabels,
+      HttpServletRequest hsr) throws Exception {
+    return getNextInterceptor().removeFromCluserNodeLabels(oldNodeLabels, hsr);
+  }
+
+  @Override
+  public NodeLabelsInfo getLabelsOnNode(HttpServletRequest hsr, String nodeId)
+      throws IOException {
+    return getNextInterceptor().getLabelsOnNode(hsr, nodeId);
+  }
+
+  @Override
+  public AppPriority getAppPriority(HttpServletRequest hsr, String appId)
+      throws AuthorizationException {
+    return getNextInterceptor().getAppPriority(hsr, appId);
+  }
+
+  @Override
+  public Response updateApplicationPriority(AppPriority targetPriority,
+      HttpServletRequest hsr, String appId) throws AuthorizationException,
+      YarnException, InterruptedException, IOException {
+    return getNextInterceptor().updateApplicationPriority(targetPriority, hsr,
+        appId);
+  }
+
+  @Override
+  public AppQueue getAppQueue(HttpServletRequest hsr, String appId)
+      throws AuthorizationException {
+    return getNextInterceptor().getAppQueue(hsr, appId);
+  }
+
+  @Override
+  public Response updateAppQueue(AppQueue targetQueue, HttpServletRequest hsr,
+      String appId) throws AuthorizationException, YarnException,
+      InterruptedException, IOException {
+    return getNextInterceptor().updateAppQueue(targetQueue, hsr, appId);
+  }
+
+  @Override
+  public Response createNewApplication(HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException {
+    return getNextInterceptor().createNewApplication(hsr);
+  }
+
+  @Override
+  public Response submitApplication(ApplicationSubmissionContextInfo newApp,
+      HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException {
+    return getNextInterceptor().submitApplication(newApp, hsr);
+  }
+
+  @Override
+  public Response postDelegationToken(DelegationToken tokenData,
+      HttpServletRequest hsr) throws AuthorizationException, IOException,
+      InterruptedException, Exception {
+    return getNextInterceptor().postDelegationToken(tokenData, hsr);
+  }
+
+  @Override
+  public Response postDelegationTokenExpiration(HttpServletRequest hsr)
+      throws AuthorizationException, IOException, Exception {
+    return getNextInterceptor().postDelegationTokenExpiration(hsr);
+  }
+
+  @Override
+  public Response cancelDelegationToken(HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException,
+      Exception {
+    return getNextInterceptor().cancelDelegationToken(hsr);
+  }
+
+  @Override
+  public Response createNewReservation(HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException {
+    return getNextInterceptor().createNewReservation(hsr);
+  }
+
+  @Override
+  public Response submitReservation(ReservationSubmissionRequestInfo resContext,
+      HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException {
+    return getNextInterceptor().submitReservation(resContext, hsr);
+  }
+
+  @Override
+  public Response updateReservation(ReservationUpdateRequestInfo resContext,
+      HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException {
+    return getNextInterceptor().updateReservation(resContext, hsr);
+  }
+
+  @Override
+  public Response deleteReservation(ReservationDeleteRequestInfo resContext,
+      HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException {
+    return getNextInterceptor().deleteReservation(resContext, hsr);
+  }
+
+  @Override
+  public Response listReservation(String queue, String reservationId,
+      long startTime, long endTime, boolean includeResourceAllocations,
+      HttpServletRequest hsr) throws Exception {
+    return getNextInterceptor().listReservation(queue, reservationId, startTime,
+        endTime, includeResourceAllocations, hsr);
+  }
+
+  @Override
+  public AppTimeoutInfo getAppTimeout(HttpServletRequest hsr, String appId,
+      String type) throws AuthorizationException {
+    return getNextInterceptor().getAppTimeout(hsr, appId, type);
+  }
+
+  @Override
+  public AppTimeoutsInfo getAppTimeouts(HttpServletRequest hsr, String appId)
+      throws AuthorizationException {
+    return getNextInterceptor().getAppTimeouts(hsr, appId);
+  }
+
+  @Override
+  public Response updateApplicationTimeout(AppTimeoutInfo appTimeout,
+      HttpServletRequest hsr, String appId) throws AuthorizationException,
+      YarnException, InterruptedException, IOException {
+    return getNextInterceptor().updateApplicationTimeout(appTimeout, hsr,
+        appId);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bfd967d3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestRouterWebServices.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/webapp/TestRouterWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestRouterWebServices.java
new file mode 100644
index 0000000..c96575c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestRouterWebServices.java
@@ -0,0 +1,269 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, 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.webapp;
+
+import java.io.IOException;
+import java.util.Map;
+
+import javax.ws.rs.core.Response;
+
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ActivitiesInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppActivitiesInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppAttemptsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppPriority;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppQueue;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppState;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppTimeoutInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppTimeoutsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ApplicationStatisticsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ClusterInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ClusterMetricsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.LabelsToNodesInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeLabelsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeToLabelsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodesInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.SchedulerTypeInfo;
+import org.apache.hadoop.yarn.server.router.webapp.RouterWebServices.RequestInterceptorChainWrapper;
+import org.apache.hadoop.yarn.server.webapp.dao.AppAttemptInfo;
+import org.apache.hadoop.yarn.server.webapp.dao.ContainerInfo;
+import org.apache.hadoop.yarn.server.webapp.dao.ContainersInfo;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test class to validate the WebService interceptor model inside the Router.
+ */
+public class TestRouterWebServices extends BaseRouterWebServicesTest {
+
+  private String user = "test1";
+
+  /**
+   * Test that all requests get forwarded to the last interceptor in the chain
+   * get back the responses.
+   */
+  @Test
+  public void testRouterWebServicesE2E() throws Exception {
+
+    ClusterInfo clusterInfo = get(user);
+    Assert.assertNotNull(clusterInfo);
+
+    ClusterInfo clusterInfo2 = getClusterInfo(user);
+    Assert.assertNotNull(clusterInfo2);
+
+    ClusterMetricsInfo clusterMetricsInfo = getClusterMetricsInfo(user);
+    Assert.assertNotNull(clusterMetricsInfo);
+
+    SchedulerTypeInfo schedulerTypeInfo = getSchedulerInfo(user);
+    Assert.assertNotNull(schedulerTypeInfo);
+
+    String dumpResult = dumpSchedulerLogs(user);
+    Assert.assertNotNull(dumpResult);
+
+    NodesInfo nodesInfo = getNodes(user);
+    Assert.assertNotNull(nodesInfo);
+
+    NodeInfo nodeInfo = getNode(user);
+    Assert.assertNotNull(nodeInfo);
+
+    AppsInfo appsInfo = getApps(user);
+    Assert.assertNotNull(appsInfo);
+
+    ActivitiesInfo activitiesInfo = getActivities(user);
+    Assert.assertNotNull(activitiesInfo);
+
+    AppActivitiesInfo appActiviesInfo = getAppActivities(user);
+    Assert.assertNotNull(appActiviesInfo);
+
+    ApplicationStatisticsInfo applicationStatisticsInfo =
+        getAppStatistics(user);
+    Assert.assertNotNull(applicationStatisticsInfo);
+
+    AppInfo appInfo = getApp(user);
+    Assert.assertNotNull(appInfo);
+
+    AppState appState = getAppState(user);
+    Assert.assertNotNull(appState);
+
+    Response response = updateAppState(user);
+    Assert.assertNotNull(response);
+
+    NodeToLabelsInfo nodeToLabelsInfo = getNodeToLabels(user);
+    Assert.assertNotNull(nodeToLabelsInfo);
+
+    LabelsToNodesInfo labelsToNodesInfo = getLabelsToNodes(user);
+    Assert.assertNotNull(labelsToNodesInfo);
+
+    Response response2 = replaceLabelsOnNodes(user);
+    Assert.assertNotNull(response2);
+
+    Response response3 = replaceLabelsOnNode(user);
+    Assert.assertNotNull(response3);
+
+    NodeLabelsInfo nodeLabelsInfo = getClusterNodeLabels(user);
+    Assert.assertNotNull(nodeLabelsInfo);
+
+    Response response4 = addToClusterNodeLabels(user);
+    Assert.assertNotNull(response4);
+
+    Response response5 = removeFromCluserNodeLabels(user);
+    Assert.assertNotNull(response5);
+
+    NodeLabelsInfo nodeLabelsInfo2 = getLabelsOnNode(user);
+    Assert.assertNotNull(nodeLabelsInfo2);
+
+    AppPriority appPriority = getAppPriority(user);
+    Assert.assertNotNull(appPriority);
+
+    Response response6 = updateApplicationPriority(user);
+    Assert.assertNotNull(response6);
+
+    AppQueue appQueue = getAppQueue(user);
+    Assert.assertNotNull(appQueue);
+
+    Response response7 = updateAppQueue(user);
+    Assert.assertNotNull(response7);
+
+    Response response8 = createNewApplication(user);
+    Assert.assertNotNull(response8);
+
+    Response response9 = submitApplication(user);
+    Assert.assertNotNull(response9);
+
+    Response response10 = postDelegationToken(user);
+    Assert.assertNotNull(response10);
+
+    Response response11 = postDelegationTokenExpiration(user);
+    Assert.assertNotNull(response11);
+
+    Response response12 = cancelDelegationToken(user);
+    Assert.assertNotNull(response12);
+
+    Response response13 = createNewReservation(user);
+    Assert.assertNotNull(response13);
+
+    Response response14 = submitReservation(user);
+    Assert.assertNotNull(response14);
+
+    Response response15 = updateReservation(user);
+    Assert.assertNotNull(response15);
+
+    Response response16 = deleteReservation(user);
+    Assert.assertNotNull(response16);
+
+    Response response17 = listReservation(user);
+    Assert.assertNotNull(response17);
+
+    AppTimeoutInfo appTimeoutInfo = getAppTimeout(user);
+    Assert.assertNotNull(appTimeoutInfo);
+
+    AppTimeoutsInfo appTimeoutsInfo = getAppTimeouts(user);
+    Assert.assertNotNull(appTimeoutsInfo);
+
+    Response response18 = updateApplicationTimeout(user);
+    Assert.assertNotNull(response18);
+
+    AppAttemptsInfo appAttemptsInfo = getAppAttempts(user);
+    Assert.assertNotNull(appAttemptsInfo);
+
+    AppAttemptInfo appAttemptInfo = getAppAttempt(user);
+    Assert.assertNotNull(appAttemptInfo);
+
+    ContainersInfo containersInfo = getContainers(user);
+    Assert.assertNotNull(containersInfo);
+
+    ContainerInfo containerInfo = getContainer(user);
+    Assert.assertNotNull(containerInfo);
+  }
+
+  /**
+   * Tests if the pipeline is created properly.
+   */
+  @Test
+  public void testRequestInterceptorChainCreation() throws Exception {
+    RESTRequestInterceptor root =
+        super.getRouterWebServices().createRequestInterceptorChain();
+    int index = 0;
+    while (root != null) {
+      // The current pipeline is:
+      // PassThroughRESTRequestInterceptor - index = 0
+      // PassThroughRESTRequestInterceptor - index = 1
+      // PassThroughRESTRequestInterceptor - index = 2
+      // MockRESTRequestInterceptor - 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(PassThroughRESTRequestInterceptor.class.getName(),
+            root.getClass().getName());
+        break;
+      case 3:
+        Assert.assertEquals(MockRESTRequestInterceptor.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 different chains for users are generated, and LRU cache is
+   * working as expected.
+   */
+  @Test
+  public void testUsersChainMapWithLRUCache()
+      throws YarnException, IOException, InterruptedException {
+    getInterceptorChain("test1");
+    getInterceptorChain("test2");
+    getInterceptorChain("test3");
+    getInterceptorChain("test4");
+    getInterceptorChain("test5");
+    getInterceptorChain("test6");
+    getInterceptorChain("test7");
+    getInterceptorChain("test8");
+
+    Map<String, RequestInterceptorChainWrapper> pipelines =
+        getRouterWebServices().getPipelines();
+    Assert.assertEquals(8, pipelines.size());
+
+    getInterceptorChain("test9");
+    getInterceptorChain("test10");
+    getInterceptorChain("test1");
+    getInterceptorChain("test11");
+
+    // The cache max size is defined in TEST_MAX_CACHE_SIZE
+    Assert.assertEquals(10, pipelines.size());
+
+    RequestInterceptorChainWrapper chain = pipelines.get("test1");
+    Assert.assertNotNull("test1 should not be evicted", chain);
+
+    chain = pipelines.get("test2");
+    Assert.assertNull("test2 should have been evicted", chain);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bfd967d3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestRouterWebServicesREST.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/webapp/TestRouterWebServicesREST.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestRouterWebServicesREST.java
new file mode 100644
index 0000000..d7b1a0f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestRouterWebServicesREST.java
@@ -0,0 +1,1298 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, 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.webapp;
+
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+import java.util.List;
+
+import javax.ws.rs.core.MediaType;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.records.NodeLabel;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.nodemanager.NodeManager;
+import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWebServiceProtocol;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ActivitiesInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppActivitiesInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppAttemptInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppAttemptsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppPriority;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppQueue;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppState;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppTimeoutInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppTimeoutsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ApplicationStatisticsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ApplicationSubmissionContextInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ClusterInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ClusterMetricsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.LabelsToNodesInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NewApplication;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NewReservation;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeLabelsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeToLabelsEntryList;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeToLabelsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodesInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationDeleteRequestInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationSubmissionRequestInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationUpdateRequestInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.SchedulerTypeInfo;
+import org.apache.hadoop.yarn.server.router.Router;
+import org.apache.hadoop.yarn.server.webapp.WebServices;
+import org.apache.hadoop.yarn.server.webapp.dao.AppsInfo;
+import org.apache.hadoop.yarn.server.webapp.dao.ContainersInfo;
+import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
+import org.codehaus.jettison.json.JSONException;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.sun.jersey.api.client.Client;
+import com.sun.jersey.api.client.ClientHandlerException;
+import com.sun.jersey.api.client.ClientResponse;
+import com.sun.jersey.api.client.WebResource;
+import com.sun.jersey.api.client.WebResource.Builder;
+
+import net.jcip.annotations.NotThreadSafe;
+
+/**
+ * This test validate E2E the correctness of the RouterWebServices. It starts
+ * Router, RM and NM in 3 different processes to avoid servlet conflicts. Each
+ * test creates a REST call to Router and validate that the operation complete
+ * successfully.
+ */
+@NotThreadSafe
+public class TestRouterWebServicesREST {
+
+  private static String userName = "test";
+
+  private static JavaProcess rm;
+  private static JavaProcess nm;
+  private static JavaProcess router;
+
+  private static Configuration conf;
+
+  private static final int STATUS_OK = 200;
+  private static final int STATUS_ACCEPTED = 202;
+  private static final int STATUS_BADREQUEST = 400;
+  private static final int STATUS_ERROR = 500;
+
+  /**
+   * Wait until the webservice is up and running.
+   */
+  private static void waitWebAppRunning(String address, String path) {
+    while (true) {
+      Client clientToRouter = Client.create();
+      WebResource toRouter = clientToRouter.resource(address).path(path);
+      try {
+        ClientResponse response = toRouter.accept(MediaType.APPLICATION_JSON)
+            .get(ClientResponse.class);
+        if (response.getStatus() == STATUS_OK) {
+          // process is up and running
+          return;
+        }
+      } catch (ClientHandlerException e) {
+        // process is not up and running
+        continue;
+      }
+    }
+  }
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    conf = new YarnConfiguration();
+    rm = new JavaProcess(ResourceManager.class);
+    router = new JavaProcess(Router.class);
+    nm = new JavaProcess(NodeManager.class);
+
+    // The tests cannot start if all the service are not up and running.
+    waitWebAppRunning(WebAppUtils.getRMWebAppURLWithScheme(conf),
+        RMWSConsts.RM_WEB_SERVICE_PATH);
+
+    waitWebAppRunning(WebAppUtils.getRouterWebAppURLWithScheme(conf),
+        RMWSConsts.RM_WEB_SERVICE_PATH);
+
+    waitWebAppRunning("http://" + WebAppUtils.getNMWebAppURLWithoutScheme(conf),
+        "/ws/v1/node");
+  }
+
+  @AfterClass
+  public static void stop() throws Exception {
+    nm.stop();
+    router.stop();
+    rm.stop();
+  }
+
+  /**
+   * Performs 2 GET calls one to RM and the one to Router. In positive case, it
+   * returns the 2 answers in a list.
+   */
+  private static <T> List<T> performGetCalls(String path, Class<T> returnType,
+      String queryName, String queryValue)
+      throws IOException, InterruptedException {
+    Client clientToRouter = Client.create();
+    WebResource toRouter = clientToRouter
+        .resource(WebAppUtils.getRouterWebAppURLWithScheme(conf)).path(path);
+
+    Client clientToRM = Client.create();
+    WebResource toRM = clientToRM
+        .resource(WebAppUtils.getRMWebAppURLWithScheme(conf)).path(path);
+
+    Builder toRouterBuilder;
+    Builder toRMBuilder;
+
+    if (queryValue != null && queryName != null) {
+      toRouterBuilder = toRouter.queryParam(queryName, queryValue)
+          .accept(MediaType.APPLICATION_XML);
+      toRMBuilder = toRM.queryParam(queryName, queryValue)
+          .accept(MediaType.APPLICATION_XML);
+    } else {
+      toRouterBuilder = toRouter.accept(MediaType.APPLICATION_XML);
+      toRMBuilder = toRM.accept(MediaType.APPLICATION_XML);
+    }
+
+    return UserGroupInformation.createRemoteUser(userName)
+        .doAs(new PrivilegedExceptionAction<List<T>>() {
+          @Override
+          public List<T> run() throws Exception {
+            ClientResponse response = toRouterBuilder.get(ClientResponse.class);
+            ClientResponse response2 = toRMBuilder.get(ClientResponse.class);
+            if (response.getStatus() == STATUS_OK
+                && response2.getStatus() == STATUS_OK) {
+              List<T> responses = new ArrayList<T>();
+              responses.add(response.getEntity(returnType));
+              responses.add(response2.getEntity(returnType));
+              return responses;
+            } else {
+              Assert.fail();
+            }
+            return null;
+          }
+        });
+  }
+
+  /**
+   * Performs a POST/PUT/DELETE call to Router and returns the ClientResponse.
+   */
+  private static ClientResponse performCall(String webAddress, String queryKey,
+      String queryValue, Object context, HTTPMethods method)
+      throws IOException, InterruptedException {
+
+    return UserGroupInformation.createRemoteUser(userName)
+        .doAs(new PrivilegedExceptionAction<ClientResponse>() {
+          @Override
+          public ClientResponse run() throws Exception {
+            Client clientToRouter = Client.create();
+            WebResource toRouter = clientToRouter
+                .resource(WebAppUtils.getRouterWebAppURLWithScheme(conf))
+                .path(webAddress);
+
+            WebResource toRouterWR;
+            if (queryKey != null && queryValue != null) {
+              toRouterWR = toRouter.queryParam(queryKey, queryValue);
+            } else {
+              toRouterWR = toRouter;
+            }
+
+            Builder builder = null;
+            if (context != null) {
+              builder = toRouterWR.entity(context, MediaType.APPLICATION_JSON);
+              builder = builder.accept(MediaType.APPLICATION_JSON);
+            } else {
+              builder = toRouter.accept(MediaType.APPLICATION_JSON);
+            }
+
+            ClientResponse response = null;
+
+            switch (method) {
+            case DELETE:
+              response = builder.delete(ClientResponse.class);
+              break;
+            case POST:
+              response = builder.post(ClientResponse.class);
+              break;
+            case PUT:
+              response = builder.put(ClientResponse.class);
+              break;
+            default:
+              break;
+            }
+
+            return response;
+          }
+        });
+  }
+
+  /**
+   * This test validates the correctness of {@link RMWebServiceProtocol#get()}
+   * inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testInfoXML() throws JSONException, Exception {
+
+    List<ClusterInfo> responses = performGetCalls(
+        RMWSConsts.RM_WEB_SERVICE_PATH, ClusterInfo.class, null, null);
+
+    ClusterInfo routerResponse = responses.get(0);
+    ClusterInfo rmResponse = responses.get(1);
+
+    Assert.assertNotNull(routerResponse);
+    Assert.assertNotNull(rmResponse);
+
+    Assert.assertEquals(rmResponse.getRMVersion(),
+        routerResponse.getRMVersion());
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#getClusterInfo()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testClusterInfoXML() throws JSONException, Exception {
+
+    List<ClusterInfo> responses =
+        performGetCalls(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.INFO,
+            ClusterInfo.class, null, null);
+
+    ClusterInfo routerResponse = responses.get(0);
+    ClusterInfo rmResponse = responses.get(1);
+
+    Assert.assertNotNull(routerResponse);
+    Assert.assertNotNull(rmResponse);
+
+    Assert.assertEquals(rmResponse.getRMVersion(),
+        routerResponse.getRMVersion());
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#getClusterMetricsInfo()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testMetricsInfoXML() throws JSONException, Exception {
+
+    List<ClusterMetricsInfo> responses =
+        performGetCalls(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.METRICS,
+            ClusterMetricsInfo.class, null, null);
+
+    ClusterMetricsInfo routerResponse = responses.get(0);
+    ClusterMetricsInfo rmResponse = responses.get(1);
+
+    Assert.assertNotNull(routerResponse);
+    Assert.assertNotNull(rmResponse);
+
+    Assert.assertEquals(rmResponse.getActiveNodes(),
+        routerResponse.getActiveNodes());
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#getSchedulerInfo()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testSchedulerInfoXML() throws JSONException, Exception {
+
+    List<SchedulerTypeInfo> responses =
+        performGetCalls(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.SCHEDULER,
+            SchedulerTypeInfo.class, null, null);
+
+    SchedulerTypeInfo routerResponse = responses.get(0);
+    SchedulerTypeInfo rmResponse = responses.get(1);
+
+    Assert.assertNotNull(routerResponse);
+    Assert.assertNotNull(rmResponse);
+
+    Assert.assertEquals(rmResponse.getSchedulerInfo().getSchedulerType(),
+        routerResponse.getSchedulerInfo().getSchedulerType());
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#getNodes()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testNodesXML() throws JSONException, Exception {
+
+    List<NodesInfo> responses =
+        performGetCalls(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.NODES,
+            NodesInfo.class, RMWSConsts.STATES, "LOST");
+
+    NodesInfo routerResponse = responses.get(0);
+    NodesInfo rmResponse = responses.get(1);
+
+    Assert.assertNotNull(routerResponse);
+    Assert.assertNotNull(rmResponse);
+
+    Assert.assertEquals(rmResponse.getNodes().size(),
+        routerResponse.getNodes().size());
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#getNode()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testNodeXML() throws JSONException, Exception {
+
+    List<NodeInfo> responses = performGetCalls(
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.NODES + "/" + getNodeId(),
+        NodeInfo.class, null, null);
+
+    NodeInfo routerResponse = responses.get(0);
+    NodeInfo rmResponse = responses.get(1);
+
+    Assert.assertNotNull(routerResponse);
+    Assert.assertNotNull(rmResponse);
+
+    Assert.assertEquals(rmResponse.getVersion(), routerResponse.getVersion());
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#getActivities()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testActiviesXML() throws JSONException, Exception {
+
+    List<ActivitiesInfo> responses = performGetCalls(
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.SCHEDULER_ACTIVITIES,
+        ActivitiesInfo.class, null, null);
+
+    ActivitiesInfo routerResponse = responses.get(0);
+    ActivitiesInfo rmResponse = responses.get(1);
+
+    Assert.assertNotNull(routerResponse);
+    Assert.assertNotNull(rmResponse);
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#getAppActivities()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testAppActivitiesXML() throws JSONException, Exception {
+
+    String appId = submitApplication();
+
+    List<AppActivitiesInfo> responses = performGetCalls(
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.SCHEDULER_APP_ACTIVITIES,
+        AppActivitiesInfo.class, RMWSConsts.APP_ID, appId);
+
+    AppActivitiesInfo routerResponse = responses.get(0);
+    AppActivitiesInfo rmResponse = responses.get(1);
+
+    Assert.assertNotNull(routerResponse);
+    Assert.assertNotNull(rmResponse);
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#getAppStatistics()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testAppStatisticsXML() throws JSONException, Exception {
+
+    submitApplication();
+
+    List<ApplicationStatisticsInfo> responses = performGetCalls(
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APP_STATISTICS,
+        ApplicationStatisticsInfo.class, RMWSConsts.STATES, "RUNNING");
+
+    ApplicationStatisticsInfo routerResponse = responses.get(0);
+    ApplicationStatisticsInfo rmResponse = responses.get(1);
+
+    Assert.assertNotNull(routerResponse);
+    Assert.assertNotNull(rmResponse);
+
+    Assert.assertEquals(rmResponse.getStatItems().size(),
+        routerResponse.getStatItems().size());
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#dumpSchedulerLogs()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testDumpSchedulerLogsXML() throws JSONException, Exception {
+
+    // Test with a wrong HTTP method
+    ClientResponse badResponse =
+        performCall(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.SCHEDULER_LOGS,
+            null, null, null, HTTPMethods.PUT);
+
+    Assert.assertEquals(STATUS_ERROR, badResponse.getStatus());
+
+    // Test with the correct HTTP method
+
+    ClientResponse response =
+        performCall(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.SCHEDULER_LOGS,
+            RMWSConsts.TIME, "1", null, HTTPMethods.POST);
+
+    if (response.getStatus() == STATUS_BADREQUEST) {
+      String ci = response.getEntity(String.class);
+      Assert.assertNotNull(ci);
+    } else {
+      Assert.fail();
+    }
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#createNewApplication()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testNewApplicationXML() throws JSONException, Exception {
+
+    // Test with a wrong HTTP method
+    ClientResponse badResponse = performCall(
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS_NEW_APPLICATION, null,
+        null, null, HTTPMethods.PUT);
+
+    Assert.assertEquals(STATUS_ERROR, badResponse.getStatus());
+
+    // Test with the correct HTTP method
+
+    ClientResponse response = performCall(
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS_NEW_APPLICATION, null,
+        null, null, HTTPMethods.POST);
+
+    if (response.getStatus() == STATUS_OK) {
+      NewApplication ci = response.getEntity(NewApplication.class);
+      Assert.assertNotNull(ci);
+    } else {
+      Assert.fail();
+    }
+
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#submitApplication()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testSubmitApplicationXML() throws JSONException, Exception {
+
+    // Test with a wrong HTTP method
+    ClientResponse badResponse =
+        performCall(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS, null,
+            null, null, HTTPMethods.PUT);
+
+    Assert.assertEquals(STATUS_ERROR, badResponse.getStatus());
+
+    // Test with the correct HTTP method
+
+    ApplicationSubmissionContextInfo context =
+        new ApplicationSubmissionContextInfo();
+    context.setApplicationId(getNewApplicationId().getApplicationId());
+
+    ClientResponse response =
+        performCall(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS, null,
+            null, context, HTTPMethods.POST);
+
+    if (response.getStatus() == STATUS_ACCEPTED) {
+      String ci = response.getEntity(String.class);
+      Assert.assertNotNull(ci);
+    } else {
+      Assert.fail();
+    }
+
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#getApps()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testAppsXML() throws JSONException, Exception {
+
+    submitApplication();
+
+    List<AppsInfo> responses =
+        performGetCalls(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS,
+            AppsInfo.class, null, null);
+
+    AppsInfo routerResponse = responses.get(0);
+    AppsInfo rmResponse = responses.get(1);
+
+    Assert.assertNotNull(routerResponse);
+    Assert.assertNotNull(rmResponse);
+
+    Assert.assertEquals(rmResponse.getApps().size(),
+        rmResponse.getApps().size());
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#getApp()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testAppXML() throws JSONException, Exception {
+
+    String appId = submitApplication();
+
+    List<AppInfo> responses = performGetCalls(
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS + "/" + appId,
+        AppInfo.class, null, null);
+
+    AppInfo routerResponse = responses.get(0);
+    AppInfo rmResponse = responses.get(1);
+
+    Assert.assertNotNull(routerResponse);
+    Assert.assertNotNull(rmResponse);
+
+    Assert.assertEquals(rmResponse.getAMHostHttpAddress(),
+        rmResponse.getAMHostHttpAddress());
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#getAppAttempts()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testAppAttemptXML() throws JSONException, Exception {
+
+    String appId = submitApplication();
+
+    List<AppAttemptsInfo> responses = performGetCalls(
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS + "/" + appId + "/"
+            + RMWSConsts.ATTEMPTS,
+        AppAttemptsInfo.class, null, null);
+
+    AppAttemptsInfo routerResponse = responses.get(0);
+    AppAttemptsInfo rmResponse = responses.get(1);
+
+    Assert.assertNotNull(routerResponse);
+    Assert.assertNotNull(rmResponse);
+
+    Assert.assertEquals(rmResponse.getAttempts().size(),
+        rmResponse.getAttempts().size());
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#getAppState()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testAppStateXML() throws JSONException, Exception {
+
+    String appId = submitApplication();
+
+    List<AppState> responses =
+        performGetCalls(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS + "/"
+            + appId + "/" + RMWSConsts.STATE, AppState.class, null, null);
+
+    AppState routerResponse = responses.get(0);
+    AppState rmResponse = responses.get(1);
+
+    Assert.assertNotNull(routerResponse);
+    Assert.assertNotNull(rmResponse);
+
+    Assert.assertEquals(rmResponse.getState(), rmResponse.getState());
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#updateAppState()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testUpdateAppStateXML() throws JSONException, Exception {
+
+    String appId = submitApplication();
+
+    // Test with a wrong HTTP method
+    ClientResponse badResponse = performCall(RMWSConsts.RM_WEB_SERVICE_PATH
+        + RMWSConsts.APPS + "/" + appId + "/" + RMWSConsts.STATE, null, null,
+        null, HTTPMethods.POST);
+
+    Assert.assertEquals(STATUS_ERROR, badResponse.getStatus());
+
+    // Test with the correct HTTP method
+
+    AppState appState = new AppState("KILLED");
+
+    ClientResponse response = performCall(RMWSConsts.RM_WEB_SERVICE_PATH
+        + RMWSConsts.APPS + "/" + appId + "/" + RMWSConsts.STATE, null, null,
+        appState, HTTPMethods.PUT);
+
+    if (response.getStatus() == STATUS_ACCEPTED) {
+      AppState ci = response.getEntity(AppState.class);
+      Assert.assertNotNull(ci);
+    } else {
+      Assert.fail();
+    }
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#getAppPriority()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testAppPriorityXML() throws JSONException, Exception {
+
+    String appId = submitApplication();
+
+    List<AppPriority> responses =
+        performGetCalls(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS + "/"
+            + appId + "/" + RMWSConsts.PRIORITY, AppPriority.class, null, null);
+
+    AppPriority routerResponse = responses.get(0);
+    AppPriority rmResponse = responses.get(1);
+
+    Assert.assertNotNull(routerResponse);
+    Assert.assertNotNull(rmResponse);
+
+    Assert.assertEquals(rmResponse.getPriority(), rmResponse.getPriority());
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#updateApplicationPriority()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testUpdateAppPriorityXML() throws JSONException, Exception {
+
+    String appId = submitApplication();
+
+    // Test with a wrong HTTP method
+    ClientResponse badResponse = performCall(RMWSConsts.RM_WEB_SERVICE_PATH
+        + RMWSConsts.APPS + "/" + appId + "/" + RMWSConsts.PRIORITY, null, null,
+        null, HTTPMethods.POST);
+
+    Assert.assertEquals(STATUS_ERROR, badResponse.getStatus());
+
+    // Test with the correct HTTP method
+
+    AppPriority appPriority = new AppPriority(1);
+
+    ClientResponse response =
+        performCall(
+            RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS + "/" + appId + "/"
+                + RMWSConsts.PRIORITY,
+            null, null, appPriority, HTTPMethods.PUT);
+
+    if (response.getStatus() == STATUS_OK) {
+      AppPriority ci = response.getEntity(AppPriority.class);
+      Assert.assertNotNull(ci);
+    } else {
+      Assert.fail();
+    }
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#getAppQueue()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testAppQueueXML() throws JSONException, Exception {
+
+    String appId = submitApplication();
+
+    List<AppQueue> responses =
+        performGetCalls(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS + "/"
+            + appId + "/" + RMWSConsts.QUEUE, AppQueue.class, null, null);
+
+    AppQueue routerResponse = responses.get(0);
+    AppQueue rmResponse = responses.get(1);
+
+    Assert.assertNotNull(routerResponse);
+    Assert.assertNotNull(rmResponse);
+
+    Assert.assertEquals(rmResponse.getQueue(), rmResponse.getQueue());
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#updateAppQueue()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testUpdateAppQueueXML() throws JSONException, Exception {
+
+    String appId = submitApplication();
+
+    // Test with a wrong HTTP method
+    ClientResponse badResponse = performCall(RMWSConsts.RM_WEB_SERVICE_PATH
+        + RMWSConsts.APPS + "/" + appId + "/" + RMWSConsts.QUEUE, null, null,
+        null, HTTPMethods.POST);
+
+    Assert.assertEquals(STATUS_ERROR, badResponse.getStatus());
+
+    // Test with the correct HTTP method
+
+    AppQueue appQueue = new AppQueue("default");
+
+    ClientResponse response = performCall(RMWSConsts.RM_WEB_SERVICE_PATH
+        + RMWSConsts.APPS + "/" + appId + "/" + RMWSConsts.QUEUE, null, null,
+        appQueue, HTTPMethods.PUT);
+
+    if (response.getStatus() == STATUS_OK) {
+      AppQueue ci = response.getEntity(AppQueue.class);
+      Assert.assertNotNull(ci);
+    } else {
+      Assert.fail();
+    }
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#getAppTimeouts()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testAppTimeoutsXML() throws JSONException, Exception {
+
+    String appId = submitApplication();
+
+    List<AppTimeoutsInfo> responses = performGetCalls(
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS + "/" + appId + "/"
+            + RMWSConsts.TIMEOUTS,
+        AppTimeoutsInfo.class, null, null);
+
+    AppTimeoutsInfo routerResponse = responses.get(0);
+    AppTimeoutsInfo rmResponse = responses.get(1);
+
+    Assert.assertNotNull(routerResponse);
+    Assert.assertNotNull(rmResponse);
+
+    Assert.assertEquals(rmResponse.getAppTimeouts().size(),
+        rmResponse.getAppTimeouts().size());
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#getAppTimeout()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testAppTimeoutXML() throws JSONException, Exception {
+
+    String appId = submitApplication();
+
+    List<AppTimeoutInfo> responses = performGetCalls(
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS + "/" + appId + "/"
+            + RMWSConsts.TIMEOUTS + "/" + "LIFETIME",
+        AppTimeoutInfo.class, null, null);
+
+    AppTimeoutInfo routerResponse = responses.get(0);
+    AppTimeoutInfo rmResponse = responses.get(1);
+
+    Assert.assertNotNull(routerResponse);
+    Assert.assertNotNull(rmResponse);
+
+    Assert.assertEquals(rmResponse.getExpireTime(), rmResponse.getExpireTime());
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#updateApplicationTimeout()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testUpdateAppTimeoutsXML() throws JSONException, Exception {
+
+    String appId = submitApplication();
+
+    // Test with a wrong HTTP method
+    ClientResponse badResponse = performCall(RMWSConsts.RM_WEB_SERVICE_PATH
+        + RMWSConsts.APPS + "/" + appId + "/" + RMWSConsts.TIMEOUT, null, null,
+        null, HTTPMethods.POST);
+
+    Assert.assertEquals(STATUS_ERROR, badResponse.getStatus());
+
+    // Test with the correct HTTP method
+
+    // Create a bad request
+    AppTimeoutInfo appTimeoutInfo = new AppTimeoutInfo();
+
+    ClientResponse response =
+        performCall(
+            RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS + "/" + appId + "/"
+                + RMWSConsts.TIMEOUT,
+            null, null, appTimeoutInfo, HTTPMethods.PUT);
+
+    if (response.getStatus() == STATUS_BADREQUEST) {
+      String ci = response.getEntity(String.class);
+      Assert.assertNotNull(ci);
+    } else {
+      Assert.fail();
+    }
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#createNewReservation()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testNewReservationXML() throws JSONException, Exception {
+
+    // Test with a wrong HTTP method
+    ClientResponse badResponse =
+        performCall(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.RESERVATION_NEW,
+            null, null, null, HTTPMethods.PUT);
+
+    Assert.assertEquals(STATUS_ERROR, badResponse.getStatus());
+
+    // Test with the correct HTTP method
+
+    ClientResponse response =
+        performCall(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.RESERVATION_NEW,
+            null, null, null, HTTPMethods.POST);
+
+    if (response.getStatus() == STATUS_OK) {
+      NewReservation ci = response.getEntity(NewReservation.class);
+      Assert.assertNotNull(ci);
+    } else {
+      Assert.fail();
+    }
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#submitReservation()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testSubmitReservationXML() throws JSONException, Exception {
+
+    // Test with a wrong HTTP method
+    ClientResponse badResponse = performCall(
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.RESERVATION_SUBMIT, null,
+        null, null, HTTPMethods.PUT);
+
+    Assert.assertEquals(STATUS_ERROR, badResponse.getStatus());
+
+    // Test with the correct HTTP method
+
+    ReservationSubmissionRequestInfo context =
+        new ReservationSubmissionRequestInfo();
+    context.setReservationId(getNewReservationId().getReservationId());
+    // ReservationDefinition is null
+
+    ClientResponse response = performCall(
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.RESERVATION_SUBMIT, null,
+        null, context, HTTPMethods.POST);
+
+    if (response.getStatus() == STATUS_BADREQUEST) {
+      String ci = response.getEntity(String.class);
+      Assert.assertNotNull(ci);
+    } else {
+      Assert.fail();
+    }
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#updateReservation()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testUpdateReservationXML() throws JSONException, Exception {
+
+    // Test with a wrong HTTP method
+    ClientResponse badResponse = performCall(
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.RESERVATION_UPDATE, null,
+        null, null, HTTPMethods.PUT);
+
+    Assert.assertEquals(STATUS_ERROR, badResponse.getStatus());
+
+    // Test with the correct HTTP method
+
+    String reservationId = getNewReservationId().getReservationId();
+    ReservationUpdateRequestInfo context = new ReservationUpdateRequestInfo();
+    context.setReservationId(reservationId);
+
+    ClientResponse response = performCall(
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.RESERVATION_UPDATE, null,
+        null, context, HTTPMethods.POST);
+
+    if (response.getStatus() == STATUS_BADREQUEST) {
+      String ci = response.getEntity(String.class);
+      Assert.assertNotNull(ci);
+    } else {
+      Assert.fail();
+    }
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#deleteReservation()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testDeleteReservationXML() throws JSONException, Exception {
+
+    // Test with a wrong HTTP method
+    ClientResponse badResponse = performCall(
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.RESERVATION_DELETE, null,
+        null, null, HTTPMethods.PUT);
+
+    Assert.assertEquals(STATUS_ERROR, badResponse.getStatus());
+
+    // Test with the correct HTTP method
+
+    String reservationId = getNewReservationId().getReservationId();
+    ReservationDeleteRequestInfo context = new ReservationDeleteRequestInfo();
+    context.setReservationId(reservationId);
+
+    ClientResponse response = performCall(
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.RESERVATION_DELETE, null,
+        null, context, HTTPMethods.POST);
+
+    if (response.getStatus() == STATUS_BADREQUEST) {
+      String ci = response.getEntity(String.class);
+      Assert.assertNotNull(ci);
+    } else {
+      Assert.fail();
+    }
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#getNodeToLabels()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testGetNodeToLabelsXML() throws JSONException, Exception {
+
+    List<NodeToLabelsInfo> responses = performGetCalls(
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.GET_NODE_TO_LABELS,
+        NodeToLabelsInfo.class, null, null);
+
+    NodeToLabelsInfo routerResponse = responses.get(0);
+    NodeToLabelsInfo rmResponse = responses.get(1);
+
+    Assert.assertNotNull(routerResponse);
+    Assert.assertNotNull(rmResponse);
+
+    Assert.assertEquals(rmResponse.getNodeToLabels().size(),
+        rmResponse.getNodeToLabels().size());
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#getClusterNodeLabels()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testGetClusterNodeLabelsXML() throws JSONException, Exception {
+
+    List<NodeLabelsInfo> responses = performGetCalls(
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.GET_NODE_LABELS,
+        NodeLabelsInfo.class, null, null);
+
+    NodeLabelsInfo routerResponse = responses.get(0);
+    NodeLabelsInfo rmResponse = responses.get(1);
+
+    Assert.assertNotNull(routerResponse);
+    Assert.assertNotNull(rmResponse);
+
+    Assert.assertEquals(rmResponse.getNodeLabels().size(),
+        rmResponse.getNodeLabels().size());
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#getLabelsOnNode()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testGetLabelsOnNodeXML() throws JSONException, Exception {
+
+    List<NodeLabelsInfo> responses =
+        performGetCalls(
+            RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.NODES + "/"
+                + getNodeId() + "/" + RMWSConsts.GET_LABELS,
+            NodeLabelsInfo.class, null, null);
+
+    NodeLabelsInfo routerResponse = responses.get(0);
+    NodeLabelsInfo rmResponse = responses.get(1);
+
+    Assert.assertNotNull(routerResponse);
+    Assert.assertNotNull(rmResponse);
+
+    Assert.assertEquals(rmResponse.getNodeLabels().size(),
+        rmResponse.getNodeLabels().size());
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#getLabelsToNodes()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testGetLabelsMappingXML() throws JSONException, Exception {
+
+    List<LabelsToNodesInfo> responses = performGetCalls(
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.LABEL_MAPPINGS,
+        LabelsToNodesInfo.class, null, null);
+
+    LabelsToNodesInfo routerResponse = responses.get(0);
+    LabelsToNodesInfo rmResponse = responses.get(1);
+
+    Assert.assertNotNull(routerResponse);
+    Assert.assertNotNull(rmResponse);
+
+    Assert.assertEquals(rmResponse.getLabelsToNodes().size(),
+        rmResponse.getLabelsToNodes().size());
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#addToClusterNodeLabels()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testAddToClusterNodeLabelsXML() throws JSONException, Exception {
+
+    // Test with a wrong HTTP method
+    ClientResponse badResponse =
+        performCall(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.ADD_NODE_LABELS,
+            null, null, null, HTTPMethods.PUT);
+
+    Assert.assertEquals(STATUS_ERROR, badResponse.getStatus());
+
+    // Test with the correct HTTP method
+
+    List<NodeLabel> nodeLabels = new ArrayList<NodeLabel>();
+    nodeLabels.add(NodeLabel.newInstance("default"));
+    NodeLabelsInfo context = new NodeLabelsInfo(nodeLabels);
+
+    ClientResponse response =
+        performCall(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.ADD_NODE_LABELS,
+            null, null, context, HTTPMethods.POST);
+
+    if (response.getStatus() == STATUS_OK) {
+      String ci = response.getEntity(String.class);
+      Assert.assertNotNull(ci);
+    } else {
+      Assert.fail();
+    }
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#removeFromCluserNodeLabels()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testRemoveFromCluserNodeLabelsXML()
+      throws JSONException, Exception {
+
+    // Test with a wrong HTTP method
+    ClientResponse badResponse = performCall(
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.REMOVE_NODE_LABELS, null,
+        null, null, HTTPMethods.PUT);
+
+    Assert.assertEquals(STATUS_ERROR, badResponse.getStatus());
+
+    // Test with the correct HTTP method
+
+    addNodeLabel();
+
+    ClientResponse response = performCall(
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.REMOVE_NODE_LABELS,
+        RMWSConsts.LABELS, "default", null, HTTPMethods.POST);
+
+    if (response.getStatus() == STATUS_OK) {
+      String ci = response.getEntity(String.class);
+      Assert.assertNotNull(ci);
+    } else {
+      Assert.fail();
+    }
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#replaceLabelsOnNodes()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testReplaceLabelsOnNodesXML() throws JSONException, Exception {
+
+    // Test with a wrong HTTP method
+    ClientResponse badResponse = performCall(
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.REPLACE_NODE_TO_LABELS,
+        null, null, null, HTTPMethods.PUT);
+
+    Assert.assertEquals(STATUS_ERROR, badResponse.getStatus());
+
+    // Test with the correct HTTP method
+
+    addNodeLabel();
+
+    NodeToLabelsEntryList context = new NodeToLabelsEntryList();
+
+    ClientResponse response = performCall(
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.REPLACE_NODE_TO_LABELS,
+        null, null, context, HTTPMethods.POST);
+
+    if (response.getStatus() == STATUS_OK) {
+      String ci = response.getEntity(String.class);
+      Assert.assertNotNull(ci);
+    } else {
+      Assert.fail();
+    }
+  }
+
+  /**
+   * This test validates the correctness of
+   * {@link RMWebServiceProtocol#replaceLabelsOnNode()} inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testReplaceLabelsOnNodeXML() throws JSONException, Exception {
+
+    // Test with a wrong HTTP method
+    ClientResponse badResponse =
+        performCall(
+            RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.NODES + "/"
+                + getNodeId() + "/replace-labels",
+            null, null, null, HTTPMethods.PUT);
+
+    Assert.assertEquals(STATUS_ERROR, badResponse.getStatus());
+
+    // Test with the correct HTTP method
+
+    addNodeLabel();
+
+    ClientResponse response = performCall(
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.NODES + "/" + getNodeId()
+            + "/replace-labels",
+        RMWSConsts.LABELS, "default", null, HTTPMethods.POST);
+
+    if (response.getStatus() == STATUS_OK) {
+      String ci = response.getEntity(String.class);
+      Assert.assertNotNull(ci);
+    } else {
+      Assert.fail();
+    }
+  }
+
+  /**
+   * This test validates the correctness of {@link WebServices#getAppAttempt}
+   * inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testGetAppAttemptXML() throws JSONException, Exception {
+
+    String appId = submitApplication();
+
+    List<AppAttemptInfo> responses = performGetCalls(
+        RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS + "/" + appId + "/"
+            + RMWSConsts.APPATTEMPTS + "/" + getAppAttempt(appId),
+        AppAttemptInfo.class, null, null);
+
+    AppAttemptInfo routerResponse = responses.get(0);
+    AppAttemptInfo rmResponse = responses.get(1);
+
+    Assert.assertNotNull(routerResponse);
+    Assert.assertNotNull(rmResponse);
+
+    Assert.assertEquals(rmResponse.getAppAttemptId(),
+        rmResponse.getAppAttemptId());
+  }
+
+  /**
+   * This test validates the correctness of {@link WebServices#getContainers}
+   * inside Router.
+   */
+  @Test(timeout = 1000)
+  public void testGetContainersXML() throws JSONException, Exception {
+
+    String appId = submitApplication();
+
+    List<ContainersInfo> responses =
+        performGetCalls(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS + "/"
+            + appId + "/" + RMWSConsts.APPATTEMPTS + "/" + getAppAttempt(appId)
+            + "/" + RMWSConsts.CONTAINERS, ContainersInfo.class, null, null);
+
+    ContainersInfo routerResponse = responses.get(0);
+    ContainersInfo rmResponse = responses.get(1);
+
+    Assert.assertNotNull(routerResponse);
+    Assert.assertNotNull(rmResponse);
+
+    Assert.assertEquals(rmResponse.getContainers().size(),
+        rmResponse.getContainers().size());
+  }
+
+  private String getNodeId() {
+    Client clientToRM = Client.create();
+    WebResource toRM =
+        clientToRM.resource(WebAppUtils.getRMWebAppURLWithScheme(conf))
+            .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.NODES);
+    ClientResponse response =
+        toRM.accept(MediaType.APPLICATION_XML).get(ClientResponse.class);
+    NodesInfo ci = response.getEntity(NodesInfo.class);
+    return ci.getNodes().get(0).getNodeId();
+  }
+
+  private NewApplication getNewApplicationId() {
+    Client clientToRM = Client.create();
+    WebResource toRM =
+        clientToRM.resource(WebAppUtils.getRMWebAppURLWithScheme(conf)).path(
+            RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS_NEW_APPLICATION);
+    ClientResponse response =
+        toRM.accept(MediaType.APPLICATION_XML).post(ClientResponse.class);
+    return response.getEntity(NewApplication.class);
+  }
+
+  private String submitApplication() {
+    ApplicationSubmissionContextInfo context =
+        new ApplicationSubmissionContextInfo();
+    String appId = getNewApplicationId().getApplicationId();
+    context.setApplicationId(appId);
+
+    Client clientToRouter = Client.create();
+    WebResource toRM =
+        clientToRouter.resource(WebAppUtils.getRMWebAppURLWithScheme(conf))
+            .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS);
+    toRM.entity(context, MediaType.APPLICATION_XML)
+        .accept(MediaType.APPLICATION_XML).post(ClientResponse.class);
+    return appId;
+  }
+
+  private NewReservation getNewReservationId() {
+    Client clientToRM = Client.create();
+    WebResource toRM =
+        clientToRM.resource(WebAppUtils.getRMWebAppURLWithScheme(conf))
+            .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.RESERVATION_NEW);
+    ClientResponse response =
+        toRM.accept(MediaType.APPLICATION_XML).post(ClientResponse.class);
+    return response.getEntity(NewReservation.class);
+  }
+
+  private String addNodeLabel() {
+    Client clientToRM = Client.create();
+    WebResource toRM =
+        clientToRM.resource(WebAppUtils.getRMWebAppURLWithScheme(conf))
+            .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.ADD_NODE_LABELS);
+    List<NodeLabel> nodeLabels = new ArrayList<NodeLabel>();
+    nodeLabels.add(NodeLabel.newInstance("default"));
+    NodeLabelsInfo context = new NodeLabelsInfo(nodeLabels);
+    ClientResponse response = toRM.entity(context, MediaType.APPLICATION_XML)
+        .accept(MediaType.APPLICATION_XML).post(ClientResponse.class);
+    return response.getEntity(String.class);
+  }
+
+  private String getAppAttempt(String appId) {
+    Client clientToRM = Client.create();
+    WebResource toRM =
+        clientToRM.resource(WebAppUtils.getRMWebAppURLWithScheme(conf))
+            .path(RMWSConsts.RM_WEB_SERVICE_PATH + RMWSConsts.APPS + "/" + appId
+                + "/" + RMWSConsts.ATTEMPTS);
+    ClientResponse response =
+        toRM.accept(MediaType.APPLICATION_XML).get(ClientResponse.class);
+    AppAttemptsInfo ci = response.getEntity(AppAttemptsInfo.class);
+    return ci.getAttempts().get(0).getAppAttemptId();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bfd967d3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/resources/capacity-scheduler.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/resources/capacity-scheduler.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/resources/capacity-scheduler.xml
new file mode 100644
index 0000000..90c5eeb
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/resources/capacity-scheduler.xml
@@ -0,0 +1,111 @@
+<!--
+  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.
+-->
+<configuration>
+
+  <property>
+    <name>yarn.scheduler.capacity.maximum-applications</name>
+    <value>10000</value>
+    <description>
+      Maximum number of applications that can be pending and running.
+    </description>
+  </property>
+
+  <property>
+    <name>yarn.scheduler.capacity.maximum-am-resource-percent</name>
+    <value>0.1</value>
+    <description>
+      Maximum percent of resources in the cluster which can be used to run
+      application masters i.e. controls number of concurrent running
+      applications.
+    </description>
+  </property>
+
+  <property>
+    <name>yarn.scheduler.capacity.resource-calculator</name>
+    <value>org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator</value>
+    <description>
+      The ResourceCalculator implementation to be used to compare
+      Resources in the scheduler.
+      The default i.e. DefaultResourceCalculator only uses Memory while
+      DominantResourceCalculator uses dominant-resource to compare
+      multi-dimensional resources such as Memory, CPU etc.
+    </description>
+  </property>
+
+  <property>
+    <name>yarn.scheduler.capacity.root.queues</name>
+    <value>default</value>
+    <description>
+      The queues at the this level (root is the root queue).
+    </description>
+  </property>
+
+  <property>
+    <name>yarn.scheduler.capacity.root.default.capacity</name>
+    <value>100</value>
+    <description>Default queue target capacity.</description>
+  </property>
+
+  <property>
+    <name>yarn.scheduler.capacity.root.default.user-limit-factor</name>
+    <value>1</value>
+    <description>
+      Default queue user limit a percentage from 0.0 to 1.0.
+    </description>
+  </property>
+
+  <property>
+    <name>yarn.scheduler.capacity.root.default.maximum-capacity</name>
+    <value>100</value>
+    <description>
+      The maximum capacity of the default queue.
+    </description>
+  </property>
+
+  <property>
+    <name>yarn.scheduler.capacity.root.default.state</name>
+    <value>RUNNING</value>
+    <description>
+      The state of the default queue. State can be one of RUNNING or STOPPED.
+    </description>
+  </property>
+
+  <property>
+    <name>yarn.scheduler.capacity.root.default.acl_submit_applications</name>
+    <value>*</value>
+    <description>
+      The ACL of who can submit jobs to the default queue.
+    </description>
+  </property>
+
+  <property>
+    <name>yarn.scheduler.capacity.root.default.acl_administer_queue</name>
+    <value>*</value>
+    <description>
+      The ACL of who can administer jobs on the default queue.
+    </description>
+  </property>
+
+  <property>
+    <name>yarn.scheduler.capacity.node-locality-delay</name>
+    <value>-1</value>
+    <description>
+      Number of missed scheduling opportunities after which the CapacityScheduler
+      attempts to schedule rack-local containers.
+      Typically this should be set to number of racks in the cluster, this
+      feature is disabled by default, set to -1.
+    </description>
+  </property>
+
+</configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bfd967d3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/resources/log4j.properties b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/resources/log4j.properties
new file mode 100644
index 0000000..81a3f6a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/resources/log4j.properties
@@ -0,0 +1,19 @@
+#   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.
+
+# log4j configuration used during build and unit tests
+
+log4j.rootLogger=info,stdout
+log4j.threshold=ALL
+log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p [%t] %c{2} (%F:%M(%L)) - %m%n

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bfd967d3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/resources/yarn-site.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/resources/yarn-site.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/resources/yarn-site.xml
new file mode 100644
index 0000000..f3e0de3
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/resources/yarn-site.xml
@@ -0,0 +1,30 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
+  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.
+-->
+
+<configuration>
+  <property>
+    <name>yarn.resourcemanager.reservation-system.enable</name>
+    <value>true</value>
+  </property>
+  <property>
+    <name>yarn.node-labels.enabled</name>
+    <value>true</value>
+  </property>
+  <property>
+    <name>yarn.resourcemanager.webapp.address</name>
+    <value>0.0.0.0:8080</value>
+  </property>
+</configuration>


---------------------------------------------------------------------
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-5676. Add a HashBasedRouterPolicy, and small policies and test refactoring. (Carlo Curino via Subru).

Posted by cu...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/d32ffa9e/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/d32ffa9e/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/d32ffa9e/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/d32ffa9e/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/d32ffa9e/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/d32ffa9e/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/d32ffa9e/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/d32ffa9e/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/d32ffa9e/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/d32ffa9e/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/d32ffa9e/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/d32ffa9e/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/d32ffa9e/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/d32ffa9e/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


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

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

(cherry picked from commit ac1ba2a30427263f4a2eed2018515b2f759fa591)
(cherry picked from commit 1882bc10fbb8937e947d1de82170945c31574a3f)


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

Branch: refs/heads/branch-2
Commit: 8409fef0d9395e61c99e02c75b6e154afadbdc50
Parents: ea2ecdb
Author: Subru Krishnan <su...@apache.org>
Authored: Thu Sep 1 13:55:54 2016 -0700
Committer: Carlo Curino <cu...@apache.org>
Committed: Thu Sep 21 16:23:39 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/8409fef0/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/8409fef0/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


[11/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 cu...@apache.org.
YARN-5905. Update the RM webapp host that is reported as part of Federation membership to current primary RM's IP.

(cherry picked from commit e0f0e7afed51aac4ac9093ac362f6ce64601884d)
(cherry picked from commit 78032e51d90a42a3600ad942eecf1c3f787759d7)


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

Branch: refs/heads/branch-2
Commit: e933a1790c9d6be0ae1e9b8489e4e80bdb96bb93
Parents: d32ffa9
Author: Subru Krishnan <su...@apache.org>
Authored: Tue Nov 22 18:30:40 2016 -0800
Committer: Carlo Curino <cu...@apache.org>
Committed: Thu Sep 21 16:23:45 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/e933a179/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/e933a179/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


[43/50] [abbrv] hadoop git commit: YARN-6866. Minor clean-up and fixes in anticipation of YARN-2915 merge with trunk. (Botong Huang via Subru).

Posted by cu...@apache.org.
YARN-6866. Minor clean-up and fixes in anticipation of YARN-2915 merge with trunk. (Botong Huang via Subru).

(cherry picked from commit 40453879ec860819b080ccc82454480a436c0adc)
(cherry picked from commit 86f05c33125af3b435d6793551680ffa35047aa7)


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

Branch: refs/heads/branch-2
Commit: 049f7c84a1dae5576012c69043bb98d073c85368
Parents: 66980c0
Author: Subru Krishnan <su...@apache.org>
Authored: Tue Jul 25 20:22:45 2017 -0700
Committer: Carlo Curino <cu...@apache.org>
Committed: Thu Sep 21 16:55:43 2017 -0700

----------------------------------------------------------------------
 hadoop-project/pom.xml                                 |  6 +++---
 .../org/apache/hadoop/yarn/conf/YarnConfiguration.java |  4 ++--
 .../hadoop/yarn/conf/TestYarnConfigurationFields.java  |  4 ++++
 .../router/clientrm/FederationClientInterceptor.java   | 13 +++----------
 .../hadoop-yarn-site/src/site/markdown/Federation.md   |  8 ++++----
 5 files changed, 16 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/049f7c84/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index d61dc4b..6b24a75 100755
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -89,9 +89,9 @@
     <apacheds.version>2.0.0-M15</apacheds.version>
 
     <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>
+    <ehcache.version>3.3.1</ehcache.version>
+    <hikari.version>2.4.12</hikari.version>
+    <mssql.version>6.2.1.jre7</mssql.version>
 
     <!-- define the Java language version used by the compiler -->
     <javac.version>1.7</javac.version>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/049f7c84/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 3600709..7adfdf1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
@@ -2592,7 +2592,7 @@ public class YarnConfiguration extends Configuration {
       ROUTER_PREFIX + "clientrm.";
 
   public static final String ROUTER_CLIENTRM_ADDRESS =
-      ROUTER_CLIENTRM_PREFIX + ".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;
@@ -2610,7 +2610,7 @@ public class YarnConfiguration extends Configuration {
   public static final String ROUTER_RMADMIN_PREFIX = ROUTER_PREFIX + "rmadmin.";
 
   public static final String ROUTER_RMADMIN_ADDRESS =
-      ROUTER_RMADMIN_PREFIX + ".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;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/049f7c84/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 910df9e..56fb578 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
@@ -79,6 +79,10 @@ public class TestYarnConfigurationFields extends TestConfigurationFieldsBase {
         .add(YarnConfiguration.FEDERATION_STATESTORE_HEARTBEAT_INTERVAL_SECS);
     configurationPropsToSkipCompare
         .add(YarnConfiguration.RM_EPOCH);
+    configurationPropsToSkipCompare
+        .add(YarnConfiguration.ROUTER_CLIENTRM_ADDRESS);
+    configurationPropsToSkipCompare
+        .add(YarnConfiguration.ROUTER_RMADMIN_ADDRESS);
 
     // Federation policies configs to be ignored
     configurationPropsToSkipCompare

http://git-wip-us.apache.org/repos/asf/hadoop/blob/049f7c84/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
index ecf53ac..7268ebd 100644
--- 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
@@ -19,7 +19,6 @@
 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;
@@ -88,10 +87,10 @@ import org.apache.hadoop.yarn.api.protocolrecords.UpdateApplicationPriorityRespo
 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.failover.FederationProxyProviderUtil;
 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;
@@ -175,14 +174,8 @@ public class FederationClientInterceptor
 
     ApplicationClientProtocol clientRMProxy = null;
     try {
-      clientRMProxy =
-          user.doAs(new PrivilegedExceptionAction<ApplicationClientProtocol>() {
-            @Override
-            public ApplicationClientProtocol run() throws Exception {
-              return ClientRMProxy.createRMProxy(getConf(),
-                  ApplicationClientProtocol.class);
-            }
-          });
+      clientRMProxy = FederationProxyProviderUtil.createRMProxy(getConf(),
+          ApplicationClientProtocol.class, subClusterId, user);
     } catch (Exception e) {
       RouterServerUtil.logAndThrowException(
           "Unable to create the interface to reach the SubCluster "

http://git-wip-us.apache.org/repos/asf/hadoop/blob/049f7c84/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 79225b4..61cb77f 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
@@ -209,9 +209,9 @@ 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.clientrm.address` | `0.0.0.0:8050` | 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.admin.address` | `0.0.0.0:8052` | 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. |
@@ -227,7 +227,7 @@ These are extra configurations that should appear in the **conf/yarn-site.xml**
 |:---- |:---- |
 | `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.|
+| `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:
 
@@ -242,7 +242,7 @@ Currently, the only supported implementation of the state-store is Microsoft 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:
+In order to submit jobs to a Federation cluster one must create a seperate set of configs for the client from which jobs will be submitted. In these, the **conf/yarn-site.xml** should have the following additional configurations:
 
 | Property | Example | Description |
 |:--- |:--- |


---------------------------------------------------------------------
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-6666. Fix unit test failure in TestRouterClientRMService. (Botong Huang via Subru).

Posted by cu...@apache.org.
YARN-6666. Fix unit test failure in TestRouterClientRMService. (Botong Huang via Subru).

(cherry picked from commit e750907d0a7e2fb4b33a7c876eaa4d9122a1deea)
(cherry picked from commit 42949a6e0108ed1d92a34a5ebb547691bbf16929)


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

Branch: refs/heads/branch-2
Commit: b9fdc24e43ba61ebb4b276259e6d2c13da746c04
Parents: e4832be
Author: Subru Krishnan <su...@apache.org>
Authored: Tue May 30 13:37:37 2017 -0700
Committer: Carlo Curino <cu...@apache.org>
Committed: Thu Sep 21 16:49:48 2017 -0700

----------------------------------------------------------------------
 .../yarn/server/MockResourceManagerFacade.java  | 435 +++++++++++++------
 1 file changed, 311 insertions(+), 124 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b9fdc24e/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 c4a4002..4bdff64 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
@@ -16,19 +16,22 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.yarn.server.nodemanager.amrmproxy;
+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.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.ipc.StandbyException;
 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;
@@ -93,8 +96,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.UpdateApplicationPriorityReque
 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.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;
@@ -106,33 +108,66 @@ 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.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;
+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.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;
 
 /**
  * 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.
+ * 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);
 
+  private HashSet<ApplicationId> applicationMap = new HashSet<>();
   private HashMap<String, List<ContainerId>> applicationContainerIdMap =
       new HashMap<String, List<ContainerId>>();
   private HashMap<ContainerId, Container> allocatedContainerMap =
@@ -140,12 +175,25 @@ public class MockResourceManagerFacade
   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();
@@ -156,50 +204,71 @@ public class MockResourceManagerFacade
         break;
       }
     }
-    return result != null ? result.getApplicationAttemptId().toString()
-        : "";
+    return result != null ? result.getApplicationAttemptId().toString() : "";
   }
 
   @Override
   public RegisterApplicationMasterResponse registerApplicationMaster(
-      RegisterApplicationMasterRequest request) throws YarnException,
-      IOException {
+      RegisterApplicationMasterRequest request)
+      throws YarnException, IOException {
     String amrmToken = getAppIdentifier();
-    Log.info("Registering application attempt: " + amrmToken);
+    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>());
+      applicationContainerIdMap.put(amrmToken, new ArrayList<ContainerId>());
     }
-
-    return RegisterApplicationMasterResponse.newInstance(null, null, null,
-        null, null, request.getHost(), null);
+    return RegisterApplicationMasterResponse.newInstance(null, null, null, null,
+        null, request.getHost(), null);
   }
 
   @Override
   public FinishApplicationMasterResponse finishApplicationMaster(
-      FinishApplicationMasterRequest request) throws YarnException,
-      IOException {
+      FinishApplicationMasterRequest request)
+      throws YarnException, IOException {
     String amrmToken = getAppIdentifier();
-    Log.info("Finishing application attempt: " + amrmToken);
+    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, applicationContainerIdMap.containsKey(amrmToken));
+      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);
+    return FinishApplicationMasterResponse.newInstance(
+        request.getFinalApplicationStatus() == FinalApplicationStatus.SUCCEEDED
+            ? true : false);
   }
 
   protected ApplicationId getApplicationId(int id) {
@@ -222,14 +291,20 @@ public class MockResourceManagerFacade
     }
 
     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) {
       for (ResourceRequest rr : request.getAskList()) {
         for (int i = 0; i < rr.getNumContainers(); i++) {
-          ContainerId containerId =
-              ContainerId.newInstance(getApplicationAttemptId(1),
-                  containerIndex.incrementAndGet());
+          ContainerId containerId = ContainerId.newInstance(
+              getApplicationAttemptId(1), containerIndex.incrementAndGet());
           Container container = Records.newRecord(Container.class);
           container.setId(containerId);
           container.setPriority(rr.getPriority());
@@ -237,9 +312,8 @@ public class MockResourceManagerFacade
           // 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);
+              NodeId.newInstance(!Strings.isNullOrEmpty(rr.getResourceName())
+                  ? rr.getResourceName() : "dummy", 1000);
           container.setNodeId(nodeId);
           container.setResource(rr.getCapability());
           containerList.add(container);
@@ -251,8 +325,7 @@ public class MockResourceManagerFacade
                 "The application id is Not registered before allocate(): "
                     + amrmToken,
                 applicationContainerIdMap.containsKey(amrmToken));
-            List<ContainerId> ids =
-                applicationContainerIdMap.get(amrmToken);
+            List<ContainerId> ids = applicationContainerIdMap.get(amrmToken);
             ids.add(containerId);
             this.allocatedContainerMap.put(containerId, container);
           }
@@ -262,12 +335,13 @@ public class MockResourceManagerFacade
 
     if (request.getReleaseList() != null
         && request.getReleaseList().size() > 0) {
-      Log.info("Releasing containers: " + request.getReleaseList().size());
+      LOG.info("Releasing containers: " + request.getReleaseList().size());
       synchronized (applicationContainerIdMap) {
-        Assert.assertTrue(
-            "The application id is not registered before allocate(): "
-                + amrmToken,
-            applicationContainerIdMap.containsKey(amrmToken));
+        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()) {
@@ -279,10 +353,9 @@ public class MockResourceManagerFacade
             }
           }
 
-          Assert.assertTrue(
-              "ContainerId " + id
-                  + " being released is not valid for application: "
-                  + conf.get("AMRMTOKEN"), found);
+          Assert.assertTrue("ContainerId " + id
+              + " being released is not valid for application: "
+              + conf.get("AMRMTOKEN"), found);
 
           ids.remove(id);
 
@@ -292,9 +365,8 @@ public class MockResourceManagerFacade
           // 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());
+          ContainerId fakeContainerId = ContainerId.newInstance(
+              getApplicationAttemptId(1), containerIndex.incrementAndGet());
           Container fakeContainer = allocatedContainerMap.get(id);
           fakeContainer.setId(fakeContainerId);
           containerList.add(fakeContainer);
@@ -302,46 +374,44 @@ public class MockResourceManagerFacade
       }
     }
 
-    Log.info("Allocating containers: " + containerList.size()
+    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,
+    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 {
+      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));
+    report.setCurrentApplicationAttemptId(
+        ApplicationAttemptId.newInstance(request.getApplicationId(), 1));
     response.setApplicationReport(report);
     return response;
   }
 
   @Override
   public GetApplicationAttemptReportResponse getApplicationAttemptReport(
-      GetApplicationAttemptReportRequest request) throws YarnException,
-      IOException {
+      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);
+    report.setYarnApplicationAttemptState(YarnApplicationAttemptState.LAUNCHED);
     response.setApplicationAttemptReport(report);
     return response;
   }
@@ -349,172 +419,289 @@ public class MockResourceManagerFacade
   @Override
   public GetNewApplicationResponse getNewApplication(
       GetNewApplicationRequest request) throws YarnException, IOException {
-    return null;
+    return GetNewApplicationResponse.newInstance(null, null, null);
   }
 
   @Override
   public SubmitApplicationResponse submitApplication(
       SubmitApplicationRequest request) throws YarnException, IOException {
-    return null;
+    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 {
-    throw new NotImplementedException();
+    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);
   }
 
   @Override
   public GetClusterMetricsResponse getClusterMetrics(
       GetClusterMetricsRequest request) throws YarnException, IOException {
-    throw new NotImplementedException();
+    return GetClusterMetricsResponse.newInstance(null);
   }
 
   @Override
-  public GetApplicationsResponse getApplications(
-      GetApplicationsRequest request) throws YarnException, IOException {
-    throw new NotImplementedException();
+  public GetApplicationsResponse getApplications(GetApplicationsRequest request)
+      throws YarnException, IOException {
+    return GetApplicationsResponse.newInstance(null);
   }
 
   @Override
-  public GetClusterNodesResponse getClusterNodes(
-      GetClusterNodesRequest request) throws YarnException, IOException {
-    throw new NotImplementedException();
+  public GetClusterNodesResponse getClusterNodes(GetClusterNodesRequest request)
+      throws YarnException, IOException {
+    return GetClusterNodesResponse.newInstance(null);
   }
 
   @Override
   public GetQueueInfoResponse getQueueInfo(GetQueueInfoRequest request)
       throws YarnException, IOException {
-    throw new NotImplementedException();
+    return GetQueueInfoResponse.newInstance(null);
   }
 
   @Override
   public GetQueueUserAclsInfoResponse getQueueUserAcls(
-      GetQueueUserAclsInfoRequest request) throws YarnException,
-      IOException {
-    throw new NotImplementedException();
+      GetQueueUserAclsInfoRequest request) throws YarnException, IOException {
+    return GetQueueUserAclsInfoResponse.newInstance(null);
   }
 
   @Override
   public GetDelegationTokenResponse getDelegationToken(
       GetDelegationTokenRequest request) throws YarnException, IOException {
-    throw new NotImplementedException();
+    return GetDelegationTokenResponse.newInstance(null);
   }
 
   @Override
   public RenewDelegationTokenResponse renewDelegationToken(
-      RenewDelegationTokenRequest request) throws YarnException,
-      IOException {
-    throw new NotImplementedException();
+      RenewDelegationTokenRequest request) throws YarnException, IOException {
+    return RenewDelegationTokenResponse.newInstance(0);
   }
 
   @Override
   public CancelDelegationTokenResponse cancelDelegationToken(
-      CancelDelegationTokenRequest request) throws YarnException,
-      IOException {
-    throw new NotImplementedException();
+      CancelDelegationTokenRequest request) throws YarnException, IOException {
+    return CancelDelegationTokenResponse.newInstance();
   }
 
   @Override
   public MoveApplicationAcrossQueuesResponse moveApplicationAcrossQueues(
-      MoveApplicationAcrossQueuesRequest request) throws YarnException,
-      IOException {
-    throw new NotImplementedException();
+      MoveApplicationAcrossQueuesRequest request)
+      throws YarnException, IOException {
+    return MoveApplicationAcrossQueuesResponse.newInstance();
   }
 
   @Override
   public GetApplicationAttemptsResponse getApplicationAttempts(
-      GetApplicationAttemptsRequest request) throws YarnException,
-      IOException {
-    throw new NotImplementedException();
+      GetApplicationAttemptsRequest request) throws YarnException, IOException {
+    return GetApplicationAttemptsResponse.newInstance(null);
   }
 
   @Override
   public GetContainerReportResponse getContainerReport(
       GetContainerReportRequest request) throws YarnException, IOException {
-    throw new NotImplementedException();
+    return GetContainerReportResponse.newInstance(null);
   }
 
   @Override
   public GetContainersResponse getContainers(GetContainersRequest request)
       throws YarnException, IOException {
-    throw new NotImplementedException();
-  }
-
-  @Override
-  public GetNewReservationResponse getNewReservation(
-      GetNewReservationRequest request) throws YarnException, IOException {
-    throw new NotImplementedException();
+    return GetContainersResponse.newInstance(null);
   }
 
   @Override
   public ReservationSubmissionResponse submitReservation(
-      ReservationSubmissionRequest request) throws YarnException,
-      IOException {
-    throw new NotImplementedException();
+      ReservationSubmissionRequest request) throws YarnException, IOException {
+    return ReservationSubmissionResponse.newInstance();
   }
 
   @Override
   public ReservationListResponse listReservations(
-          ReservationListRequest request) throws YarnException,
-          IOException {
-      throw new NotImplementedException();
+      ReservationListRequest request) throws YarnException, IOException {
+    return ReservationListResponse
+        .newInstance(new ArrayList<ReservationAllocationState>());
   }
 
   @Override
   public ReservationUpdateResponse updateReservation(
       ReservationUpdateRequest request) throws YarnException, IOException {
-    throw new NotImplementedException();
+    return ReservationUpdateResponse.newInstance();
   }
 
   @Override
   public ReservationDeleteResponse deleteReservation(
       ReservationDeleteRequest request) throws YarnException, IOException {
-    throw new NotImplementedException();
+    return ReservationDeleteResponse.newInstance();
   }
 
   @Override
   public GetNodesToLabelsResponse getNodeToLabels(
       GetNodesToLabelsRequest request) throws YarnException, IOException {
-    throw new NotImplementedException();
+    return GetNodesToLabelsResponse
+        .newInstance(new HashMap<NodeId, Set<String>>());
   }
 
   @Override
   public GetClusterNodeLabelsResponse getClusterNodeLabels(
-      GetClusterNodeLabelsRequest request) throws YarnException,
-      IOException {
-    throw new NotImplementedException();
+      GetClusterNodeLabelsRequest request) throws YarnException, IOException {
+    return GetClusterNodeLabelsResponse.newInstance(new ArrayList<NodeLabel>());
   }
 
   @Override
   public GetLabelsToNodesResponse getLabelsToNodes(
       GetLabelsToNodesRequest request) throws YarnException, IOException {
-    return null;
+    return GetLabelsToNodesResponse.newInstance(null);
   }
 
   @Override
-  public UpdateApplicationPriorityResponse updateApplicationPriority(
-      UpdateApplicationPriorityRequest request) throws YarnException,
-      IOException {
-    return null;
+  public GetNewReservationResponse getNewReservation(
+      GetNewReservationRequest request) throws YarnException, IOException {
+    return GetNewReservationResponse
+        .newInstance(ReservationId.newInstance(0, 0));
   }
 
   @Override
-  public SignalContainerResponse signalToContainer(
-      SignalContainerRequest request) throws IOException {
-return null;
-}
-
-  @Override
   public FailApplicationAttemptResponse failApplicationAttempt(
       FailApplicationAttemptRequest request) throws YarnException, IOException {
-    throw new NotImplementedException();
+    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 {
-    throw new NotImplementedException();
+    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];
   }
 }


---------------------------------------------------------------------
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-3659. Federation: routing client invocations transparently to multiple RMs. (Giovanni Matteo Fumarola via Subru).

Posted by cu...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/169037cb/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) {

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


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

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

(cherry picked from commit f1eff212c5f13c62e8fc45b0af794b5bbeb577da)
(cherry picked from commit 712e97d4cfab15bec4cf4b11cc067b8f85c8bec8)


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

Branch: refs/heads/branch-2
Commit: 58e2458c2e8250cf028e0fc01e06dfd9670f86be
Parents: fdbd214
Author: Subru Krishnan <su...@apache.org>
Authored: Fri Jul 14 12:02:38 2017 -0700
Committer: Carlo Curino <cu...@apache.org>
Committed: Thu Sep 21 16:55:41 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/58e2458c/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 aa4d620..afa005e 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/58e2458c/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/58e2458c/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


[49/50] [abbrv] hadoop git commit: YARN-2280. Resource manager web service fields are not accessible (Krisztian Horvath via aw)

Posted by cu...@apache.org.
YARN-2280. Resource manager web service fields are not accessible (Krisztian Horvath via aw)

(cherry picked from commit a5cf985bf501fd032124d121dcae80538db9e380)


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

Branch: refs/heads/branch-2
Commit: baa2bec84399cf4c8dbd8306eeb8f3e0ea71e45a
Parents: d3afff7
Author: Allen Wittenauer <aw...@apache.org>
Authored: Tue Mar 10 16:32:20 2015 -0700
Committer: Carlo Curino <cu...@apache.org>
Committed: Thu Sep 21 17:52:25 2017 -0700

----------------------------------------------------------------------
 .../hadoop/yarn/server/resourcemanager/webapp/dao/NodesInfo.java  | 3 +++
 .../yarn/server/resourcemanager/webapp/dao/SchedulerTypeInfo.java | 3 +++
 2 files changed, 6 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/baa2bec8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/NodesInfo.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/NodesInfo.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/NodesInfo.java
index 7be9a6f..7dacd10 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/NodesInfo.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/NodesInfo.java
@@ -36,4 +36,7 @@ public class NodesInfo {
     node.add(nodeinfo);
   }
 
+  public ArrayList<NodeInfo> getNodes() {
+    return node;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/baa2bec8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/SchedulerTypeInfo.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/SchedulerTypeInfo.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/SchedulerTypeInfo.java
index 34078f1..22018d0 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/SchedulerTypeInfo.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/SchedulerTypeInfo.java
@@ -34,4 +34,7 @@ public class SchedulerTypeInfo {
     this.schedulerInfo = scheduler;
   }
 
+  public SchedulerInfo getSchedulerInfo() {
+    return schedulerInfo;
+  }
 }


---------------------------------------------------------------------
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-6203: Occasional test failure in TestWeightedRandomRouterPolicy (curino)

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

(cherry picked from commit 121d55bd29ac44289afad9a15e8737397ede298b)
(cherry picked from commit c4bbdfeb666f1f3f2fd4c9437f10b233ebedc2c5)


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

Branch: refs/heads/branch-2
Commit: c38ac05bb4a77b44768406fb7851e7c0b871d5a5
Parents: b9dcf92
Author: Carlo Curino <cu...@apache.org>
Authored: Thu Apr 13 12:09:39 2017 -0700
Committer: Carlo Curino <cu...@apache.org>
Committed: Thu Sep 21 16:25:11 2017 -0700

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


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


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

Posted by cu...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/dc0a2e6c/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


[45/50] [abbrv] hadoop git commit: YARN-5412. Create a proxy chain for ResourceManager REST API in the Router. (Contributed by Giovanni Matteo Fumarola via curino)

Posted by cu...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/bfd967d3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/RouterWebServices.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/webapp/RouterWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/RouterWebServices.java
new file mode 100644
index 0000000..bbb8326
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/RouterWebServices.java
@@ -0,0 +1,876 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, 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.webapp;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.DefaultValue;
+import javax.ws.rs.FormParam;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.http.JettyUtils;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authorize.AuthorizationException;
+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.server.resourcemanager.webapp.RMWSConsts;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWebServiceProtocol;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ActivitiesInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppActivitiesInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppAttemptsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppPriority;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppQueue;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppState;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppTimeoutInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppTimeoutsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ApplicationStatisticsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ApplicationSubmissionContextInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ClusterInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ClusterMetricsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.DelegationToken;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.LabelsToNodesInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeLabelsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeToLabelsEntryList;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeToLabelsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodesInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationDeleteRequestInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationSubmissionRequestInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationUpdateRequestInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.SchedulerTypeInfo;
+import org.apache.hadoop.yarn.server.router.Router;
+import org.apache.hadoop.yarn.server.webapp.dao.ContainerInfo;
+import org.apache.hadoop.yarn.server.webapp.dao.ContainersInfo;
+import org.apache.hadoop.yarn.util.LRUCacheHashMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.inject.Inject;
+import com.google.inject.Singleton;
+
+/**
+ * RouterWebServices is a service that runs on each router that can be used to
+ * intercept and inspect {@link RMWebServiceProtocol} messages from client to
+ * the cluster resource manager. It listens {@link RMWebServiceProtocol} REST
+ * messages from the client and creates a request intercepting pipeline instance
+ * for each client. The pipeline is a chain of {@link RESTRequestInterceptor}
+ * instances that can inspect and modify the request/response as needed. The
+ * main difference with AMRMProxyService is the protocol they implement.
+ **/
+@Singleton
+@Path("/ws/v1/cluster")
+public class RouterWebServices implements RMWebServiceProtocol {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RouterWebServices.class);
+  private final Router router;
+  private final Configuration conf;
+  private @Context HttpServletResponse response;
+
+  private Map<String, RequestInterceptorChainWrapper> userPipelineMap;
+
+  // -------Default values of QueryParams for RMWebServiceProtocol--------
+
+  public static final String DEFAULT_QUEUE = "default";
+  public static final String DEFAULT_RESERVATION_ID = "";
+  public static final String DEFAULT_START_TIME = "0";
+  public static final String DEFAULT_END_TIME = "-1";
+  public static final String DEFAULT_INCLUDE_RESOURCE = "false";
+
+  @Inject
+  public RouterWebServices(final Router router, Configuration conf) {
+    this.router = router;
+    this.conf = conf;
+    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));
+  }
+
+  /**
+   * 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 config) {
+    String configuredInterceptorClassNames =
+        config.get(YarnConfiguration.ROUTER_WEBAPP_INTERCEPTOR_CLASS_PIPELINE,
+            YarnConfiguration.DEFAULT_ROUTER_WEBAPP_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 void init() {
+    // clear content type
+    response.setContentType(null);
+  }
+
+  @VisibleForTesting
+  protected RequestInterceptorChainWrapper getInterceptorChain() {
+    String user = "";
+    try {
+      user = UserGroupInformation.getCurrentUser().getUserName();
+    } catch (IOException e) {
+      LOG.error("IOException " + e.getMessage());
+    }
+    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 RESTRequestInterceptor createRequestInterceptorChain() {
+
+    List<String> interceptorClassNames = getInterceptorClassNames(conf);
+
+    RESTRequestInterceptor pipeline = null;
+    RESTRequestInterceptor current = null;
+    for (String interceptorClassName : interceptorClassNames) {
+      try {
+        Class<?> interceptorClass = conf.getClassByName(interceptorClassName);
+        if (RESTRequestInterceptor.class.isAssignableFrom(interceptorClass)) {
+          RESTRequestInterceptor interceptorInstance =
+              (RESTRequestInterceptor) 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 "
+                  + RESTRequestInterceptor.class.getCanonicalName());
+        }
+      } catch (ClassNotFoundException e) {
+        throw new YarnRuntimeException(
+            "Could not instantiate RESTRequestInterceptor: "
+                + 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 {
+      RESTRequestInterceptor 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 RESTRequestInterceptor rootInterceptor;
+
+    /**
+     * Initializes the wrapper with the specified parameters.
+     *
+     * @param interceptor the first interceptor in the pipeline
+     */
+    public synchronized void init(RESTRequestInterceptor interceptor) {
+      this.rootInterceptor = interceptor;
+    }
+
+    /**
+     * Gets the root request intercepter.
+     *
+     * @return the root request intercepter
+     */
+    public synchronized RESTRequestInterceptor getRootInterceptor() {
+      return rootInterceptor;
+    }
+
+    /**
+     * Shutdown the chain of interceptors when the object is destroyed.
+     */
+    @Override
+    protected void finalize() {
+      rootInterceptor.shutdown();
+    }
+  }
+
+  @GET
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public ClusterInfo get() {
+    return getClusterInfo();
+  }
+
+  @GET
+  @Path(RMWSConsts.INFO)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public ClusterInfo getClusterInfo() {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getClusterInfo();
+  }
+
+  @GET
+  @Path(RMWSConsts.METRICS)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public ClusterMetricsInfo getClusterMetricsInfo() {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getClusterMetricsInfo();
+  }
+
+  @GET
+  @Path(RMWSConsts.SCHEDULER)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public SchedulerTypeInfo getSchedulerInfo() {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getSchedulerInfo();
+  }
+
+  @POST
+  @Path(RMWSConsts.SCHEDULER_LOGS)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public String dumpSchedulerLogs(@FormParam(RMWSConsts.TIME) String time,
+      @Context HttpServletRequest hsr) throws IOException {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().dumpSchedulerLogs(time, hsr);
+  }
+
+  @GET
+  @Path(RMWSConsts.NODES)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public NodesInfo getNodes(@QueryParam(RMWSConsts.STATES) String states) {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getNodes(states);
+  }
+
+  @GET
+  @Path(RMWSConsts.NODES_NODEID)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public NodeInfo getNode(@PathParam(RMWSConsts.NODEID) String nodeId) {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getNode(nodeId);
+  }
+
+  @GET
+  @Path(RMWSConsts.APPS)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public AppsInfo getApps(@Context HttpServletRequest hsr,
+      @QueryParam(RMWSConsts.STATE) String stateQuery,
+      @QueryParam(RMWSConsts.STATES) Set<String> statesQuery,
+      @QueryParam(RMWSConsts.FINAL_STATUS) String finalStatusQuery,
+      @QueryParam(RMWSConsts.USER) String userQuery,
+      @QueryParam(RMWSConsts.QUEUE) String queueQuery,
+      @QueryParam(RMWSConsts.LIMIT) String count,
+      @QueryParam(RMWSConsts.STARTED_TIME_BEGIN) String startedBegin,
+      @QueryParam(RMWSConsts.STARTED_TIME_END) String startedEnd,
+      @QueryParam(RMWSConsts.FINISHED_TIME_BEGIN) String finishBegin,
+      @QueryParam(RMWSConsts.FINISHED_TIME_END) String finishEnd,
+      @QueryParam(RMWSConsts.APPLICATION_TYPES) Set<String> applicationTypes,
+      @QueryParam(RMWSConsts.APPLICATION_TAGS) Set<String> applicationTags,
+      @QueryParam(RMWSConsts.DESELECTS) Set<String> unselectedFields) {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getApps(hsr, stateQuery, statesQuery,
+        finalStatusQuery, userQuery, queueQuery, count, startedBegin,
+        startedEnd, finishBegin, finishEnd, applicationTypes, applicationTags,
+        unselectedFields);
+  }
+
+  @GET
+  @Path(RMWSConsts.SCHEDULER_ACTIVITIES)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public ActivitiesInfo getActivities(@Context HttpServletRequest hsr,
+      @QueryParam(RMWSConsts.NODEID) String nodeId) {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getActivities(hsr, nodeId);
+  }
+
+  @GET
+  @Path(RMWSConsts.SCHEDULER_APP_ACTIVITIES)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public AppActivitiesInfo getAppActivities(@Context HttpServletRequest hsr,
+      @QueryParam(RMWSConsts.APP_ID) String appId,
+      @QueryParam(RMWSConsts.MAX_TIME) String time) {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getAppActivities(hsr, appId, time);
+  }
+
+  @GET
+  @Path(RMWSConsts.APP_STATISTICS)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public ApplicationStatisticsInfo getAppStatistics(
+      @Context HttpServletRequest hsr,
+      @QueryParam(RMWSConsts.STATES) Set<String> stateQueries,
+      @QueryParam(RMWSConsts.APPLICATION_TYPES) Set<String> typeQueries) {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getAppStatistics(hsr, stateQueries,
+        typeQueries);
+  }
+
+  @GET
+  @Path(RMWSConsts.APPS_APPID)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public AppInfo getApp(@Context HttpServletRequest hsr,
+      @PathParam(RMWSConsts.APPID) String appId,
+      @QueryParam(RMWSConsts.DESELECTS) Set<String> unselectedFields) {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getApp(hsr, appId, unselectedFields);
+  }
+
+  @GET
+  @Path(RMWSConsts.APPS_APPID_STATE)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public AppState getAppState(@Context HttpServletRequest hsr,
+      @PathParam(RMWSConsts.APPID) String appId) throws AuthorizationException {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getAppState(hsr, appId);
+  }
+
+  @PUT
+  @Path(RMWSConsts.APPS_APPID_STATE)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public Response updateAppState(AppState targetState,
+      @Context HttpServletRequest hsr,
+      @PathParam(RMWSConsts.APPID) String appId) throws AuthorizationException,
+      YarnException, InterruptedException, IOException {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().updateAppState(targetState, hsr,
+        appId);
+  }
+
+  @GET
+  @Path(RMWSConsts.GET_NODE_TO_LABELS)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public NodeToLabelsInfo getNodeToLabels(@Context HttpServletRequest hsr)
+      throws IOException {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getNodeToLabels(hsr);
+  }
+
+  @GET
+  @Path(RMWSConsts.LABEL_MAPPINGS)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public LabelsToNodesInfo getLabelsToNodes(
+      @QueryParam(RMWSConsts.LABELS) Set<String> labels) throws IOException {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getLabelsToNodes(labels);
+  }
+
+  @POST
+  @Path(RMWSConsts.REPLACE_NODE_TO_LABELS)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public Response replaceLabelsOnNodes(
+      final NodeToLabelsEntryList newNodeToLabels,
+      @Context HttpServletRequest hsr) throws Exception {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().replaceLabelsOnNodes(newNodeToLabels,
+        hsr);
+  }
+
+  @POST
+  @Path(RMWSConsts.NODES_NODEID_REPLACE_LABELS)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public Response replaceLabelsOnNode(
+      @QueryParam(RMWSConsts.LABELS) Set<String> newNodeLabelsName,
+      @Context HttpServletRequest hsr,
+      @PathParam(RMWSConsts.NODEID) String nodeId) throws Exception {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().replaceLabelsOnNode(newNodeLabelsName,
+        hsr, nodeId);
+  }
+
+  @GET
+  @Path(RMWSConsts.GET_NODE_LABELS)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public NodeLabelsInfo getClusterNodeLabels(@Context HttpServletRequest hsr)
+      throws IOException {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getClusterNodeLabels(hsr);
+  }
+
+  @POST
+  @Path(RMWSConsts.ADD_NODE_LABELS)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public Response addToClusterNodeLabels(NodeLabelsInfo newNodeLabels,
+      @Context HttpServletRequest hsr) throws Exception {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().addToClusterNodeLabels(newNodeLabels,
+        hsr);
+  }
+
+  @POST
+  @Path(RMWSConsts.REMOVE_NODE_LABELS)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public Response removeFromCluserNodeLabels(
+      @QueryParam(RMWSConsts.LABELS) Set<String> oldNodeLabels,
+      @Context HttpServletRequest hsr) throws Exception {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor()
+        .removeFromCluserNodeLabels(oldNodeLabels, hsr);
+  }
+
+  @GET
+  @Path(RMWSConsts.NODES_NODEID_GETLABELS)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public NodeLabelsInfo getLabelsOnNode(@Context HttpServletRequest hsr,
+      @PathParam(RMWSConsts.NODEID) String nodeId) throws IOException {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getLabelsOnNode(hsr, nodeId);
+  }
+
+  @GET
+  @Path(RMWSConsts.APPS_APPID_PRIORITY)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public AppPriority getAppPriority(@Context HttpServletRequest hsr,
+      @PathParam(RMWSConsts.APPID) String appId) throws AuthorizationException {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getAppPriority(hsr, appId);
+  }
+
+  @PUT
+  @Path(RMWSConsts.APPS_APPID_PRIORITY)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public Response updateApplicationPriority(AppPriority targetPriority,
+      @Context HttpServletRequest hsr,
+      @PathParam(RMWSConsts.APPID) String appId) throws AuthorizationException,
+      YarnException, InterruptedException, IOException {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor()
+        .updateApplicationPriority(targetPriority, hsr, appId);
+  }
+
+  @GET
+  @Path(RMWSConsts.APPS_APPID_QUEUE)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public AppQueue getAppQueue(@Context HttpServletRequest hsr,
+      @PathParam(RMWSConsts.APPID) String appId) throws AuthorizationException {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getAppQueue(hsr, appId);
+  }
+
+  @PUT
+  @Path(RMWSConsts.APPS_APPID_QUEUE)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public Response updateAppQueue(AppQueue targetQueue,
+      @Context HttpServletRequest hsr,
+      @PathParam(RMWSConsts.APPID) String appId) throws AuthorizationException,
+      YarnException, InterruptedException, IOException {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().updateAppQueue(targetQueue, hsr,
+        appId);
+  }
+
+  @POST
+  @Path(RMWSConsts.APPS_NEW_APPLICATION)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public Response createNewApplication(@Context HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().createNewApplication(hsr);
+  }
+
+  @POST
+  @Path(RMWSConsts.APPS)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public Response submitApplication(ApplicationSubmissionContextInfo newApp,
+      @Context HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().submitApplication(newApp, hsr);
+  }
+
+  @POST
+  @Path(RMWSConsts.DELEGATION_TOKEN)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public Response postDelegationToken(DelegationToken tokenData,
+      @Context HttpServletRequest hsr) throws AuthorizationException,
+      IOException, InterruptedException, Exception {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().postDelegationToken(tokenData, hsr);
+  }
+
+  @POST
+  @Path(RMWSConsts.DELEGATION_TOKEN_EXPIRATION)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public Response postDelegationTokenExpiration(@Context HttpServletRequest hsr)
+      throws AuthorizationException, IOException, Exception {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().postDelegationTokenExpiration(hsr);
+  }
+
+  @DELETE
+  @Path(RMWSConsts.DELEGATION_TOKEN)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public Response cancelDelegationToken(@Context HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException,
+      Exception {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().cancelDelegationToken(hsr);
+  }
+
+  @POST
+  @Path(RMWSConsts.RESERVATION_NEW)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public Response createNewReservation(@Context HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().createNewReservation(hsr);
+  }
+
+  @POST
+  @Path(RMWSConsts.RESERVATION_SUBMIT)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public Response submitReservation(ReservationSubmissionRequestInfo resContext,
+      @Context HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().submitReservation(resContext, hsr);
+  }
+
+  @POST
+  @Path(RMWSConsts.RESERVATION_UPDATE)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public Response updateReservation(ReservationUpdateRequestInfo resContext,
+      @Context HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().updateReservation(resContext, hsr);
+  }
+
+  @POST
+  @Path(RMWSConsts.RESERVATION_DELETE)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public Response deleteReservation(ReservationDeleteRequestInfo resContext,
+      @Context HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().deleteReservation(resContext, hsr);
+  }
+
+  @GET
+  @Path(RMWSConsts.RESERVATION_LIST)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public Response listReservation(
+      @QueryParam(RMWSConsts.QUEUE) @DefaultValue(DEFAULT_QUEUE) String queue,
+      @QueryParam(RMWSConsts.RESERVATION_ID) @DefaultValue(DEFAULT_RESERVATION_ID) String reservationId,
+      @QueryParam(RMWSConsts.START_TIME) @DefaultValue(DEFAULT_START_TIME) long startTime,
+      @QueryParam(RMWSConsts.END_TIME) @DefaultValue(DEFAULT_END_TIME) long endTime,
+      @QueryParam(RMWSConsts.INCLUDE_RESOURCE) @DefaultValue(DEFAULT_INCLUDE_RESOURCE) boolean includeResourceAllocations,
+      @Context HttpServletRequest hsr) throws Exception {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().listReservation(queue, reservationId,
+        startTime, endTime, includeResourceAllocations, hsr);
+  }
+
+  @GET
+  @Path(RMWSConsts.APPS_TIMEOUTS_TYPE)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public AppTimeoutInfo getAppTimeout(@Context HttpServletRequest hsr,
+      @PathParam(RMWSConsts.APPID) String appId,
+      @PathParam(RMWSConsts.TYPE) String type) throws AuthorizationException {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getAppTimeout(hsr, appId, type);
+  }
+
+  @GET
+  @Path(RMWSConsts.APPS_TIMEOUTS)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public AppTimeoutsInfo getAppTimeouts(@Context HttpServletRequest hsr,
+      @PathParam(RMWSConsts.APPID) String appId) throws AuthorizationException {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getAppTimeouts(hsr, appId);
+  }
+
+  @PUT
+  @Path(RMWSConsts.APPS_TIMEOUT)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public Response updateApplicationTimeout(AppTimeoutInfo appTimeout,
+      @Context HttpServletRequest hsr,
+      @PathParam(RMWSConsts.APPID) String appId) throws AuthorizationException,
+      YarnException, InterruptedException, IOException {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().updateApplicationTimeout(appTimeout,
+        hsr, appId);
+  }
+
+  @GET
+  @Path(RMWSConsts.APPS_APPID_APPATTEMPTS)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Override
+  public AppAttemptsInfo getAppAttempts(@Context HttpServletRequest hsr,
+      @PathParam(RMWSConsts.APPID) String appId) {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getAppAttempts(hsr, appId);
+  }
+
+  @GET
+  @Path(RMWSConsts.APPS_APPID_APPATTEMPTS_APPATTEMPTID)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  public org.apache.hadoop.yarn.server.webapp.dao.AppAttemptInfo getAppAttempt(
+      @Context HttpServletRequest req, @Context HttpServletResponse res,
+      @PathParam(RMWSConsts.APPID) String appId,
+      @PathParam(RMWSConsts.APPATTEMPTID) String appAttemptId) {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getAppAttempt(req, res, appId,
+        appAttemptId);
+  }
+
+  @GET
+  @Path(RMWSConsts.APPS_APPID_APPATTEMPTS_APPATTEMPTID_CONTAINERS)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  public ContainersInfo getContainers(@Context HttpServletRequest req,
+      @Context HttpServletResponse res,
+      @PathParam(RMWSConsts.APPID) String appId,
+      @PathParam(RMWSConsts.APPATTEMPTID) String appAttemptId) {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getContainers(req, res, appId,
+        appAttemptId);
+  }
+
+  @GET
+  @Path(RMWSConsts.GET_CONTAINER)
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  public ContainerInfo getContainer(@Context HttpServletRequest req,
+      @Context HttpServletResponse res,
+      @PathParam(RMWSConsts.APPID) String appId,
+      @PathParam(RMWSConsts.APPATTEMPTID) String appAttemptId,
+      @PathParam(RMWSConsts.CONTAINERID) String containerId) {
+    init();
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().getContainer(req, res, appId,
+        appAttemptId, containerId);
+  }
+
+  @VisibleForTesting
+  protected void setResponse(HttpServletResponse response) {
+    this.response = response;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bfd967d3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/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/webapp/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/webapp/package-info.java
new file mode 100644
index 0000000..bd94ead
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/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 WebApp package. **/
+package org.apache.hadoop.yarn.server.router.webapp;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bfd967d3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/BaseRouterWebServicesTest.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/webapp/BaseRouterWebServicesTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/BaseRouterWebServicesTest.java
new file mode 100644
index 0000000..223690f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/BaseRouterWebServicesTest.java
@@ -0,0 +1,601 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, 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.webapp;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+
+import javax.servlet.http.HttpServletResponse;
+import javax.ws.rs.core.Response;
+
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authorize.AuthorizationException;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ActivitiesInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppActivitiesInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppAttemptsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppPriority;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppQueue;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppState;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppTimeoutInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppTimeoutsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ApplicationStatisticsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ClusterInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ClusterMetricsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.LabelsToNodesInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeLabelsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeToLabelsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodesInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.SchedulerTypeInfo;
+import org.apache.hadoop.yarn.server.router.Router;
+import org.apache.hadoop.yarn.server.router.webapp.RouterWebServices.RequestInterceptorChainWrapper;
+import org.apache.hadoop.yarn.server.webapp.dao.AppAttemptInfo;
+import org.apache.hadoop.yarn.server.webapp.dao.ContainerInfo;
+import org.apache.hadoop.yarn.server.webapp.dao.ContainersInfo;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.mockito.Mockito;
+
+/**
+ * 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 BaseRouterWebServicesTest {
+
+  private YarnConfiguration conf;
+
+  private Router router;
+  public final static int TEST_MAX_CACHE_SIZE = 10;
+
+  private RouterWebServices routerWebService;
+
+  @Before
+  public void setup() {
+    conf = new YarnConfiguration();
+
+    String mockPassThroughInterceptorClass =
+        PassThroughRESTRequestInterceptor.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
+    conf.set(YarnConfiguration.ROUTER_WEBAPP_INTERCEPTOR_CLASS_PIPELINE,
+        mockPassThroughInterceptorClass + "," + mockPassThroughInterceptorClass
+            + "," + mockPassThroughInterceptorClass + ","
+            + MockRESTRequestInterceptor.class.getName());
+
+    conf.setInt(YarnConfiguration.ROUTER_PIPELINE_CACHE_MAX_SIZE,
+        TEST_MAX_CACHE_SIZE);
+
+    router = spy(new Router());
+    Mockito.doNothing().when(router).startWepApp();
+    routerWebService = new RouterWebServices(router, conf);
+    routerWebService.setResponse(mock(HttpServletResponse.class));
+
+    router.init(conf);
+    router.start();
+  }
+
+  @After
+  public void tearDown() {
+    if (router != null) {
+      router.stop();
+    }
+  }
+
+  protected RouterWebServices getRouterWebServices() {
+    Assert.assertNotNull(this.routerWebService);
+    return this.routerWebService;
+  }
+
+  protected ClusterInfo get(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<ClusterInfo>() {
+          @Override
+          public ClusterInfo run() throws Exception {
+            return routerWebService.get();
+          }
+        });
+  }
+
+  protected ClusterInfo getClusterInfo(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<ClusterInfo>() {
+          @Override
+          public ClusterInfo run() throws Exception {
+            return routerWebService.getClusterInfo();
+          }
+        });
+  }
+
+  protected ClusterMetricsInfo getClusterMetricsInfo(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<ClusterMetricsInfo>() {
+          @Override
+          public ClusterMetricsInfo run() throws Exception {
+            return routerWebService.getClusterMetricsInfo();
+          }
+        });
+  }
+
+  protected SchedulerTypeInfo getSchedulerInfo(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<SchedulerTypeInfo>() {
+          @Override
+          public SchedulerTypeInfo run() throws Exception {
+            return routerWebService.getSchedulerInfo();
+          }
+        });
+  }
+
+  protected String dumpSchedulerLogs(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<String>() {
+          @Override
+          public String run() throws Exception {
+            return routerWebService.dumpSchedulerLogs(null, null);
+          }
+        });
+  }
+
+  protected NodesInfo getNodes(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<NodesInfo>() {
+          @Override
+          public NodesInfo run() throws Exception {
+            return routerWebService.getNodes(null);
+          }
+        });
+  }
+
+  protected NodeInfo getNode(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<NodeInfo>() {
+          @Override
+          public NodeInfo run() throws Exception {
+            return routerWebService.getNode(null);
+          }
+        });
+  }
+
+  protected AppsInfo getApps(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<AppsInfo>() {
+          @Override
+          public AppsInfo run() throws Exception {
+            return routerWebService.getApps(null, null, null, null, null, null,
+                null, null, null, null, null, null, null, null);
+          }
+        });
+  }
+
+  protected ActivitiesInfo getActivities(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<ActivitiesInfo>() {
+          @Override
+          public ActivitiesInfo run() throws Exception {
+            return routerWebService.getActivities(null, null);
+          }
+        });
+  }
+
+  protected AppActivitiesInfo getAppActivities(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<AppActivitiesInfo>() {
+          @Override
+          public AppActivitiesInfo run() throws Exception {
+            return routerWebService.getAppActivities(null, null, null);
+          }
+        });
+  }
+
+  protected ApplicationStatisticsInfo getAppStatistics(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<ApplicationStatisticsInfo>() {
+          @Override
+          public ApplicationStatisticsInfo run() throws Exception {
+            return routerWebService.getAppStatistics(null, null, null);
+          }
+        });
+  }
+
+  protected AppInfo getApp(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<AppInfo>() {
+          @Override
+          public AppInfo run() throws Exception {
+            return routerWebService.getApp(null, null, null);
+          }
+        });
+  }
+
+  protected AppState getAppState(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<AppState>() {
+          @Override
+          public AppState run() throws Exception {
+            return routerWebService.getAppState(null, null);
+          }
+        });
+  }
+
+  protected Response updateAppState(String user) throws AuthorizationException,
+      YarnException, InterruptedException, IOException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<Response>() {
+          @Override
+          public Response run() throws Exception {
+            return routerWebService.updateAppState(null, null, null);
+          }
+        });
+  }
+
+  protected NodeToLabelsInfo getNodeToLabels(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<NodeToLabelsInfo>() {
+          @Override
+          public NodeToLabelsInfo run() throws Exception {
+            return routerWebService.getNodeToLabels(null);
+          }
+        });
+  }
+
+  protected LabelsToNodesInfo getLabelsToNodes(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<LabelsToNodesInfo>() {
+          @Override
+          public LabelsToNodesInfo run() throws Exception {
+            return routerWebService.getLabelsToNodes(null);
+          }
+        });
+  }
+
+  protected Response replaceLabelsOnNodes(String user) throws Exception {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<Response>() {
+          @Override
+          public Response run() throws Exception {
+            return routerWebService.replaceLabelsOnNodes(null, null);
+          }
+        });
+  }
+
+  protected Response replaceLabelsOnNode(String user) throws Exception {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<Response>() {
+          @Override
+          public Response run() throws Exception {
+            return routerWebService.replaceLabelsOnNode(null, null, null);
+          }
+        });
+  }
+
+  protected NodeLabelsInfo getClusterNodeLabels(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<NodeLabelsInfo>() {
+          @Override
+          public NodeLabelsInfo run() throws Exception {
+            return routerWebService.getClusterNodeLabels(null);
+          }
+        });
+  }
+
+  protected Response addToClusterNodeLabels(String user) throws Exception {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<Response>() {
+          @Override
+          public Response run() throws Exception {
+            return routerWebService.addToClusterNodeLabels(null, null);
+          }
+        });
+  }
+
+  protected Response removeFromCluserNodeLabels(String user) throws Exception {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<Response>() {
+          @Override
+          public Response run() throws Exception {
+            return routerWebService.removeFromCluserNodeLabels(null, null);
+          }
+        });
+  }
+
+  protected NodeLabelsInfo getLabelsOnNode(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<NodeLabelsInfo>() {
+          @Override
+          public NodeLabelsInfo run() throws Exception {
+            return routerWebService.getLabelsOnNode(null, null);
+          }
+        });
+  }
+
+  protected AppPriority getAppPriority(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<AppPriority>() {
+          @Override
+          public AppPriority run() throws Exception {
+            return routerWebService.getAppPriority(null, null);
+          }
+        });
+  }
+
+  protected Response updateApplicationPriority(String user)
+      throws AuthorizationException, YarnException, InterruptedException,
+      IOException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<Response>() {
+          @Override
+          public Response run() throws Exception {
+            return routerWebService.updateApplicationPriority(null, null, null);
+          }
+        });
+  }
+
+  protected AppQueue getAppQueue(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<AppQueue>() {
+          @Override
+          public AppQueue run() throws Exception {
+            return routerWebService.getAppQueue(null, null);
+          }
+        });
+  }
+
+  protected Response updateAppQueue(String user) throws AuthorizationException,
+      YarnException, InterruptedException, IOException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<Response>() {
+          @Override
+          public Response run() throws Exception {
+            return routerWebService.updateAppQueue(null, null, null);
+          }
+        });
+  }
+
+  protected Response createNewApplication(String user)
+      throws AuthorizationException, IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<Response>() {
+          @Override
+          public Response run() throws Exception {
+            return routerWebService.createNewApplication(null);
+          }
+        });
+  }
+
+  protected Response submitApplication(String user)
+      throws AuthorizationException, IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<Response>() {
+          @Override
+          public Response run() throws Exception {
+            return routerWebService.submitApplication(null, null);
+          }
+        });
+  }
+
+  protected Response postDelegationToken(String user)
+      throws AuthorizationException, IOException, InterruptedException,
+      Exception {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<Response>() {
+          @Override
+          public Response run() throws Exception {
+            return routerWebService.postDelegationToken(null, null);
+          }
+        });
+  }
+
+  protected Response postDelegationTokenExpiration(String user)
+      throws AuthorizationException, IOException, Exception {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<Response>() {
+          @Override
+          public Response run() throws Exception {
+            return routerWebService.postDelegationTokenExpiration(null);
+          }
+        });
+  }
+
+  protected Response cancelDelegationToken(String user)
+      throws AuthorizationException, IOException, InterruptedException,
+      Exception {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<Response>() {
+          @Override
+          public Response run() throws Exception {
+            return routerWebService.cancelDelegationToken(null);
+          }
+        });
+  }
+
+  protected Response createNewReservation(String user)
+      throws AuthorizationException, IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<Response>() {
+          @Override
+          public Response run() throws Exception {
+            return routerWebService.createNewReservation(null);
+          }
+        });
+  }
+
+  protected Response submitReservation(String user)
+      throws AuthorizationException, IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<Response>() {
+          @Override
+          public Response run() throws Exception {
+            return routerWebService.submitReservation(null, null);
+          }
+        });
+  }
+
+  protected Response updateReservation(String user)
+      throws AuthorizationException, IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<Response>() {
+          @Override
+          public Response run() throws Exception {
+            return routerWebService.updateReservation(null, null);
+          }
+        });
+  }
+
+  protected Response deleteReservation(String user)
+      throws AuthorizationException, IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<Response>() {
+          @Override
+          public Response run() throws Exception {
+            return routerWebService.deleteReservation(null, null);
+          }
+        });
+  }
+
+  protected Response listReservation(String user) throws Exception {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<Response>() {
+          @Override
+          public Response run() throws Exception {
+            return routerWebService.listReservation(null, null, 0, 0, false,
+                null);
+          }
+        });
+  }
+
+  protected AppTimeoutInfo getAppTimeout(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<AppTimeoutInfo>() {
+          @Override
+          public AppTimeoutInfo run() throws Exception {
+            return routerWebService.getAppTimeout(null, null, null);
+          }
+        });
+  }
+
+  protected AppTimeoutsInfo getAppTimeouts(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<AppTimeoutsInfo>() {
+          @Override
+          public AppTimeoutsInfo run() throws Exception {
+            return routerWebService.getAppTimeouts(null, null);
+          }
+        });
+  }
+
+  protected Response updateApplicationTimeout(String user)
+      throws AuthorizationException, YarnException, InterruptedException,
+      IOException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<Response>() {
+          @Override
+          public Response run() throws Exception {
+            return routerWebService.updateApplicationTimeout(null, null, null);
+          }
+        });
+  }
+
+  protected AppAttemptsInfo getAppAttempts(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<AppAttemptsInfo>() {
+          @Override
+          public AppAttemptsInfo run() throws Exception {
+            return routerWebService.getAppAttempts(null, null);
+          }
+        });
+  }
+
+  protected AppAttemptInfo getAppAttempt(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<AppAttemptInfo>() {
+          @Override
+          public AppAttemptInfo run() throws Exception {
+            return routerWebService.getAppAttempt(null, null, null, null);
+          }
+        });
+  }
+
+  protected ContainersInfo getContainers(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<ContainersInfo>() {
+          @Override
+          public ContainersInfo run() throws Exception {
+            return routerWebService.getContainers(null, null, null, null);
+          }
+        });
+  }
+
+  protected ContainerInfo getContainer(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<ContainerInfo>() {
+          @Override
+          public ContainerInfo run() throws Exception {
+            return routerWebService.getContainer(null, null, null, null, null);
+          }
+        });
+  }
+
+  protected RequestInterceptorChainWrapper getInterceptorChain(String user)
+      throws IOException, InterruptedException {
+    return UserGroupInformation.createRemoteUser(user)
+        .doAs(new PrivilegedExceptionAction<RequestInterceptorChainWrapper>() {
+          @Override
+          public RequestInterceptorChainWrapper run() throws Exception {
+            return routerWebService.getInterceptorChain();
+          }
+        });
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bfd967d3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/JavaProcess.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/webapp/JavaProcess.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/JavaProcess.java
new file mode 100644
index 0000000..d32013f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/JavaProcess.java
@@ -0,0 +1,52 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, 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.webapp;
+
+import java.io.File;
+import java.io.IOException;
+
+/**
+ * Helper class to start a new process.
+ */
+public class JavaProcess {
+
+  private Process process = null;
+
+  public JavaProcess(Class<?> klass) throws IOException, InterruptedException {
+    String javaHome = System.getProperty("java.home");
+    String javaBin =
+        javaHome + File.separator + "bin" + File.separator + "java";
+    String classpath = System.getProperty("java.class.path");
+    classpath = classpath.concat("./src/test/resources");
+    String className = klass.getCanonicalName();
+    ProcessBuilder builder =
+        new ProcessBuilder(javaBin, "-cp", classpath, className);
+    builder.inheritIO();
+    process = builder.start();
+  }
+
+  public void stop() throws InterruptedException {
+    if (process != null) {
+      process.destroy();
+      process.waitFor();
+      process.exitValue();
+    }
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bfd967d3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/MockRESTRequestInterceptor.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/webapp/MockRESTRequestInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/MockRESTRequestInterceptor.java
new file mode 100644
index 0000000..69afdea
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/MockRESTRequestInterceptor.java
@@ -0,0 +1,340 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, 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.webapp;
+
+import java.io.IOException;
+import java.util.Set;
+
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.Response.Status;
+
+import org.apache.hadoop.security.authorize.AuthorizationException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ActivitiesInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppActivitiesInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppAttemptsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppPriority;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppQueue;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppState;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppTimeoutInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppTimeoutsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ApplicationStatisticsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ApplicationSubmissionContextInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ClusterInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ClusterMetricsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.DelegationToken;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.LabelsToNodesInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeLabelsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeToLabelsEntryList;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeToLabelsInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodesInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationDeleteRequestInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationSubmissionRequestInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationUpdateRequestInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.SchedulerTypeInfo;
+import org.apache.hadoop.yarn.server.webapp.dao.AppAttemptInfo;
+import org.apache.hadoop.yarn.server.webapp.dao.ContainerInfo;
+import org.apache.hadoop.yarn.server.webapp.dao.ContainersInfo;
+
+/**
+ * This class mocks the RESTRequestInterceptor.
+ */
+public class MockRESTRequestInterceptor extends AbstractRESTRequestInterceptor {
+
+  @Override
+  public void setNextInterceptor(RESTRequestInterceptor next) {
+    throw new YarnRuntimeException(
+        "setNextInterceptor is being called on MockRESTRequestInterceptor,"
+            + "which should be the last one in the chain. "
+            + "Check if the interceptor pipeline configuration is correct");
+  }
+
+  @Override
+  public ClusterInfo get() {
+    return new ClusterInfo();
+  }
+
+  @Override
+  public ClusterInfo getClusterInfo() {
+    return new ClusterInfo();
+  }
+
+  @Override
+  public ClusterMetricsInfo getClusterMetricsInfo() {
+    return new ClusterMetricsInfo();
+  }
+
+  @Override
+  public SchedulerTypeInfo getSchedulerInfo() {
+    return new SchedulerTypeInfo();
+  }
+
+  @Override
+  public String dumpSchedulerLogs(String time, HttpServletRequest hsr)
+      throws IOException {
+    return "Done";
+  }
+
+  @Override
+  public NodesInfo getNodes(String states) {
+    return new NodesInfo();
+  }
+
+  @Override
+  public NodeInfo getNode(String nodeId) {
+    return new NodeInfo();
+  }
+
+  @SuppressWarnings("checkstyle:parameternumber")
+  @Override
+  public AppsInfo getApps(HttpServletRequest hsr, String stateQuery,
+      Set<String> statesQuery, String finalStatusQuery, String userQuery,
+      String queueQuery, String count, String startedBegin, String startedEnd,
+      String finishBegin, String finishEnd, Set<String> applicationTypes,
+      Set<String> applicationTags, Set<String> unselectedFields) {
+    return new AppsInfo();
+  }
+
+  @Override
+  public ActivitiesInfo getActivities(HttpServletRequest hsr, String nodeId) {
+    return new ActivitiesInfo();
+  }
+
+  @Override
+  public AppActivitiesInfo getAppActivities(HttpServletRequest hsr,
+      String appId, String time) {
+    return new AppActivitiesInfo();
+  }
+
+  @Override
+  public ApplicationStatisticsInfo getAppStatistics(HttpServletRequest hsr,
+      Set<String> stateQueries, Set<String> typeQueries) {
+    return new ApplicationStatisticsInfo();
+  }
+
+  @Override
+  public AppInfo getApp(HttpServletRequest hsr, String appId,
+      Set<String> unselectedFields) {
+    return new AppInfo();
+  }
+
+  @Override
+  public AppState getAppState(HttpServletRequest hsr, String appId)
+      throws AuthorizationException {
+    return new AppState();
+  }
+
+  @Override
+  public Response updateAppState(AppState targetState, HttpServletRequest hsr,
+      String appId) throws AuthorizationException, YarnException,
+      InterruptedException, IOException {
+    return Response.status(Status.OK).build();
+  }
+
+  @Override
+  public NodeToLabelsInfo getNodeToLabels(HttpServletRequest hsr)
+      throws IOException {
+    return new NodeToLabelsInfo();
+  }
+
+  @Override
+  public LabelsToNodesInfo getLabelsToNodes(Set<String> labels)
+      throws IOException {
+    return new LabelsToNodesInfo();
+  }
+
+  @Override
+  public Response replaceLabelsOnNodes(NodeToLabelsEntryList newNodeToLabels,
+      HttpServletRequest hsr) throws Exception {
+    return Response.status(Status.OK).build();
+  }
+
+  @Override
+  public Response replaceLabelsOnNode(Set<String> newNodeLabelsName,
+      HttpServletRequest hsr, String nodeId) throws Exception {
+    return Response.status(Status.OK).build();
+  }
+
+  @Override
+  public NodeLabelsInfo getClusterNodeLabels(HttpServletRequest hsr)
+      throws IOException {
+    return new NodeLabelsInfo();
+  }
+
+  @Override
+  public Response addToClusterNodeLabels(NodeLabelsInfo newNodeLabels,
+      HttpServletRequest hsr) throws Exception {
+    return Response.status(Status.OK).build();
+  }
+
+  @Override
+  public Response removeFromCluserNodeLabels(Set<String> oldNodeLabels,
+      HttpServletRequest hsr) throws Exception {
+    return Response.status(Status.OK).build();
+  }
+
+  @Override
+  public NodeLabelsInfo getLabelsOnNode(HttpServletRequest hsr, String nodeId)
+      throws IOException {
+    return new NodeLabelsInfo();
+  }
+
+  @Override
+  public AppPriority getAppPriority(HttpServletRequest hsr, String appId)
+      throws AuthorizationException {
+    return new AppPriority();
+  }
+
+  @Override
+  public Response updateApplicationPriority(AppPriority targetPriority,
+      HttpServletRequest hsr, String appId) throws AuthorizationException,
+      YarnException, InterruptedException, IOException {
+    return Response.status(Status.OK).build();
+  }
+
+  @Override
+  public AppQueue getAppQueue(HttpServletRequest hsr, String appId)
+      throws AuthorizationException {
+    return new AppQueue();
+  }
+
+  @Override
+  public Response updateAppQueue(AppQueue targetQueue, HttpServletRequest hsr,
+      String appId) throws AuthorizationException, YarnException,
+      InterruptedException, IOException {
+    return Response.status(Status.OK).build();
+  }
+
+  @Override
+  public Response createNewApplication(HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException {
+    return Response.status(Status.OK).build();
+  }
+
+  @Override
+  public Response submitApplication(ApplicationSubmissionContextInfo newApp,
+      HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException {
+    return Response.status(Status.OK).build();
+  }
+
+  @Override
+  public Response postDelegationToken(DelegationToken tokenData,
+      HttpServletRequest hsr) throws AuthorizationException, IOException,
+      InterruptedException, Exception {
+    return Response.status(Status.OK).build();
+  }
+
+  @Override
+  public Response postDelegationTokenExpiration(HttpServletRequest hsr)
+      throws AuthorizationException, IOException, Exception {
+    return Response.status(Status.OK).build();
+  }
+
+  @Override
+  public Response cancelDelegationToken(HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException,
+      Exception {
+    return Response.status(Status.OK).build();
+  }
+
+  @Override
+  public Response createNewReservation(HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException {
+    return Response.status(Status.OK).build();
+  }
+
+  @Override
+  public Response submitReservation(ReservationSubmissionRequestInfo resContext,
+      HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException {
+    return Response.status(Status.OK).build();
+  }
+
+  @Override
+  public Response updateReservation(ReservationUpdateRequestInfo resContext,
+      HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException {
+    return Response.status(Status.OK).build();
+  }
+
+  @Override
+  public Response deleteReservation(ReservationDeleteRequestInfo resContext,
+      HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException {
+    return Response.status(Status.OK).build();
+  }
+
+  @Override
+  public Response listReservation(String queue, String reservationId,
+      long startTime, long endTime, boolean includeResourceAllocations,
+      HttpServletRequest hsr) throws Exception {
+    return Response.status(Status.OK).build();
+  }
+
+  @Override
+  public AppTimeoutInfo getAppTimeout(HttpServletRequest hsr, String appId,
+      String type) throws AuthorizationException {
+    return new AppTimeoutInfo();
+  }
+
+  @Override
+  public AppTimeoutsInfo getAppTimeouts(HttpServletRequest hsr, String appId)
+      throws AuthorizationException {
+    return new AppTimeoutsInfo();
+  }
+
+  @Override
+  public Response updateApplicationTimeout(AppTimeoutInfo appTimeout,
+      HttpServletRequest hsr, String appId) throws AuthorizationException,
+      YarnException, InterruptedException, IOException {
+    return Response.status(Status.OK).build();
+  }
+
+  @Override
+  public AppAttemptsInfo getAppAttempts(HttpServletRequest hsr, String appId) {
+    return new AppAttemptsInfo();
+  }
+
+  @Override
+  public AppAttemptInfo getAppAttempt(HttpServletRequest req,
+      HttpServletResponse res, String appId, String appAttemptId) {
+    return new AppAttemptInfo();
+  }
+
+  @Override
+  public ContainersInfo getContainers(HttpServletRequest req,
+      HttpServletResponse res, String appId, String appAttemptId) {
+    return new ContainersInfo();
+  }
+
+  @Override
+  public ContainerInfo getContainer(HttpServletRequest req,
+      HttpServletResponse res, String appId, String appAttemptId,
+      String containerId) {
+    return new ContainerInfo();
+  }
+
+}
\ 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


[27/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 cu...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/7f00f938/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


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

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

(cherry picked from commit 8820693cd769065eed83193b673ec5d919d69500)
(cherry picked from commit 825108d08a9b49d7c19915624f9167101528cb95)


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

Branch: refs/heads/branch-2
Commit: 055138c5c4fc1c9663f0599873e774112b6e696c
Parents: d3fb171
Author: Subru Krishnan <su...@apache.org>
Authored: Thu Jul 13 18:51:06 2017 -0700
Committer: Carlo Curino <cu...@apache.org>
Committed: Thu Sep 21 16:55:40 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/055138c5/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/055138c5/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/055138c5/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/055138c5/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/055138c5/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/055138c5/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/055138c5/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/055138c5/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/055138c5/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


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

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

(cherry picked from commit 20893682eced98dfba55d88edd63296993087c85)
(cherry picked from commit 93d9fdeca65a97434936ec9c2e25c362ee016783)


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

Branch: refs/heads/branch-2
Commit: b3a1ab711c3f8bf3e298abd94cccf11679ad1acc
Parents: 5c5ab53
Author: Subru Krishnan <su...@apache.org>
Authored: Tue Nov 1 19:54:18 2016 -0700
Committer: Carlo Curino <cu...@apache.org>
Committed: Thu Sep 21 16:23:43 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/b3a1ab71/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/b3a1ab71/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/b3a1ab71/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/b3a1ab71/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/b3a1ab71/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/b3a1ab71/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/b3a1ab71/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/b3a1ab71/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/b3a1ab71/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/b3a1ab71/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


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

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

(cherry picked from commit 69e3ed26809e12dd62cb66d258ef51c66db3be0a)
(cherry picked from commit 8737c2a8e96799cf3fda730d25a01f6ea2f12e9b)


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

Branch: refs/heads/branch-2
Commit: 66980c00f6ec94c04a08c5326ca7a2ad3ba678bb
Parents: 58e2458
Author: Subru Krishnan <su...@apache.org>
Authored: Tue Jul 25 16:58:43 2017 -0700
Committer: Carlo Curino <cu...@apache.org>
Committed: Thu Sep 21 16:55:42 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/66980c00/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


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

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

(cherry picked from commit be99c1fe2eb150fabd69902118d65941f82971f6)
(cherry picked from commit 5272af8c7eab76d779c621eb0208bf29ffa25613)


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

Branch: refs/heads/branch-2
Commit: 3c5dfa0fe551f22e4550b61d3bf689857c9e418d
Parents: c38ac05
Author: Carlo Curino <cu...@apache.org>
Authored: Tue Apr 25 15:14:02 2017 -0700
Committer: Carlo Curino <cu...@apache.org>
Committed: Thu Sep 21 16:33:31 2017 -0700

----------------------------------------------------------------------
 LICENSE.txt                                     |   2 +
 hadoop-project/pom.xml                          |  26 +-
 .../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, 2242 insertions(+), 309 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3c5dfa0f/LICENSE.txt
----------------------------------------------------------------------
diff --git a/LICENSE.txt b/LICENSE.txt
index 65b9d42..b25ec5a 100644
--- a/LICENSE.txt
+++ b/LICENSE.txt
@@ -582,6 +582,7 @@ For:
 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
+Microsoft SQLServer - JDBC version 6.1.0.jre7
 --------------------------------------------------------------------------------
 
 Copyright jQuery Foundation and other contributors, https://jquery.org/
@@ -659,6 +660,7 @@ The binary distribution of this product bundles these dependencies under the
 following license:
 HSQLDB Database 2.3.4
 --------------------------------------------------------------------------------
+(HSQL License)
 "COPYRIGHTS AND LICENSES (based on BSD License)
 
 For work developed by the HSQL Development Group:

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3c5dfa0f/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index eb8fde9..152dcc7 100755
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -88,6 +88,11 @@
     <!-- Required for testing LDAP integration -->
     <apacheds.version>2.0.0-M15</apacheds.version>
 
+    <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.7</javac.version>
 
@@ -1073,7 +1078,26 @@
             <artifactId>jsonassert</artifactId>
             <version>1.3.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>
+        <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/3c5dfa0f/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 fffef1c..6fbf1a6 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
@@ -2561,6 +2561,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/3c5dfa0f/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 f1cc7e6..32ae209 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
@@ -92,6 +92,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/3c5dfa0f/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 629eab8..2d78885 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
@@ -117,6 +117,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/3c5dfa0f/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/3c5dfa0f/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/3c5dfa0f/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/3c5dfa0f/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/3c5dfa0f/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/3c5dfa0f/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/3c5dfa0f/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/3c5dfa0f/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/3c5dfa0f/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


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

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

(cherry picked from commit db26bade059ff9b38da0aa160f56653769bd0143)
(cherry picked from commit 91803305e526a4c05e95834b8a104c9b335cecb6)


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

Branch: refs/heads/branch-2
Commit: 002a77dcce0d6c3ba4866aba4ca2b0f14e45fdb5
Parents: e933a17
Author: Subru Krishnan <su...@apache.org>
Authored: Tue Nov 22 18:37:30 2016 -0800
Committer: Carlo Curino <cu...@apache.org>
Committed: Thu Sep 21 16:23:46 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/002a77dc/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/002a77dc/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/002a77dc/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/002a77dc/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/002a77dc/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/002a77dc/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


[47/50] [abbrv] hadoop git commit: YARN-6902. Update Microsoft JDBC Driver for SQL Server version in License.txt. (Botong Huang via Subru).

Posted by cu...@apache.org.
YARN-6902. Update Microsoft JDBC Driver for SQL Server version in License.txt. (Botong Huang via Subru).

(cherry picked from commit c581e9438444966345613cb6e0585482936a783a)
(cherry picked from commit 894ff83dd6e773645ab7f0ce7dfbbaa3453df26d)


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

Branch: refs/heads/branch-2
Commit: 190b79af8c07968b30bf32bdbc8ea31cee66d0dc
Parents: bfd967d
Author: Subru Krishnan <su...@apache.org>
Authored: Fri Jul 28 18:46:06 2017 -0700
Committer: Carlo Curino <cu...@apache.org>
Committed: Thu Sep 21 17:15:31 2017 -0700

----------------------------------------------------------------------
 LICENSE.txt | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/190b79af/LICENSE.txt
----------------------------------------------------------------------
diff --git a/LICENSE.txt b/LICENSE.txt
index b25ec5a..52642b0 100644
--- a/LICENSE.txt
+++ b/LICENSE.txt
@@ -582,7 +582,8 @@ For:
 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
-Microsoft SQLServer - JDBC version 6.1.0.jre7
+Apache HBase - Server which contains JQuery minified javascript library version 1.8.3
+Microsoft JDBC Driver for SQLServer - version 6.2.1.jre7
 --------------------------------------------------------------------------------
 
 Copyright jQuery Foundation and other contributors, https://jquery.org/


---------------------------------------------------------------------
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-5324. Stateless Federation router policies implementation. (Carlo Curino via Subru).

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

(cherry picked from commit 1298127bdad60a961441dc7e25cdc2553c119f33)
(cherry picked from commit 0662996b6af19deece21b95b961d9362accc5159)


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

Branch: refs/heads/branch-2
Commit: f792f7b0684634e577769b22323f692095c8829d
Parents: 23c4240
Author: Subru Krishnan <su...@apache.org>
Authored: Thu Sep 22 17:06:57 2016 -0700
Committer: Carlo Curino <cu...@apache.org>
Committed: Thu Sep 21 16:23:42 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/f792f7b0/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/f792f7b0/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/f792f7b0/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/f792f7b0/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/f792f7b0/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/f792f7b0/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/f792f7b0/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/f792f7b0/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/f792f7b0/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/f792f7b0/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/f792f7b0/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/f792f7b0/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/f792f7b0/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/f792f7b0/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/f792f7b0/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/f792f7b0/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/f792f7b0/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/f792f7b0/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


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

Posted by cu...@apache.org.
YARN-6511. Federation: transparently spanning application across multiple sub-clusters. (Botong Huang via Subru).

(cherry picked from commit 8c988d235eaf0972783985b1ab24680d029aea79)
(cherry picked from commit 70b1a757f13b01a9192ea5fb0820ba7babfd974e)


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

Branch: refs/heads/branch-2
Commit: 35a38330ef0ff598f82ee0621a8b2aecc5c1136d
Parents: 7ede8c1
Author: Subru Krishnan <su...@apache.org>
Authored: Wed Jun 7 14:45:51 2017 -0700
Committer: Carlo Curino <cu...@apache.org>
Committed: Thu Sep 21 16:49:55 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/35a38330/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/35a38330/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/35a38330/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/35a38330/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/35a38330/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


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

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

(cherry picked from commit cd9db822f0c1efc52005b1c069d52910d88038d9)
(cherry picked from commit 3e1dc7ece0303f4d06816b8c3c1f2fae14db8eff)


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

Branch: refs/heads/branch-2
Commit: fdbd214d4ac606b047e03654f7fe7fe6afa1aef3
Parents: 055138c
Author: Subru Krishnan <su...@apache.org>
Authored: Thu Jul 13 18:53:21 2017 -0700
Committer: Carlo Curino <cu...@apache.org>
Committed: Thu Sep 21 16:55:40 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/fdbd214d/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 0080112..a3a05e4 100644
--- a/hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-dist.xml
+++ b/hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-dist.xml
@@ -56,6 +56,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/fdbd214d/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/fdbd214d/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/fdbd214d/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/fdbd214d/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


[50/50] [abbrv] hadoop git commit: [YARN FEDERATION BACKPORT] Fix compilation issues due to: hadoop-router/pom.xml versions and Java 1.7

Posted by cu...@apache.org.
[YARN FEDERATION BACKPORT] Fix compilation issues due to: hadoop-router/pom.xml versions and Java 1.7


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

Branch: refs/heads/branch-2
Commit: d11be2dcac8116d1b7244112e85b91955a3627a4
Parents: baa2bec
Author: Carlo Curino <cu...@apache.org>
Authored: Thu Sep 21 17:56:46 2017 -0700
Committer: Carlo Curino <cu...@apache.org>
Committed: Thu Sep 21 17:56:46 2017 -0700

----------------------------------------------------------------------
 .../LocalityMulticastAMRMProxyPolicy.java       |   3 +-
 .../yarn/server/utils/AMRMClientUtils.java      |   2 +-
 .../policies/BaseFederationPoliciesTest.java    |   5 +-
 .../utils/FederationPoliciesTestUtil.java       |  21 ++-
 .../server/resourcemanager/ResourceManager.java |   4 +-
 .../hadoop-yarn-server-router/pom.xml           |   4 +-
 .../router/webapp/RouterWebServiceUtil.java     |   7 +-
 .../server/router/webapp/RouterWebServices.java | 132 +++++++------------
 .../router/rmadmin/BaseRouterRMAdminTest.java   |   2 +-
 .../webapp/TestRouterWebServicesREST.java       |  14 +-
 10 files changed, 79 insertions(+), 115 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d11be2dc/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 454962f..f50d3b0 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
@@ -494,7 +494,8 @@ public class LocalityMulticastAMRMProxyPolicy extends AbstractAMRMProxyPolicy {
           .checkArgument(!ResourceRequest.isAnyLocation(rr.getResourceName()));
 
       if (!countContainersPerRM.containsKey(rr.getAllocationRequestId())) {
-        countContainersPerRM.put(rr.getAllocationRequestId(), new HashMap<>());
+        countContainersPerRM.put(rr.getAllocationRequestId(),
+            new HashMap<SubClusterId, AtomicLong>());
       }
       if (!countContainersPerRM.get(rr.getAllocationRequestId())
           .containsKey(targetId)) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d11be2dc/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
index 7993bd8..9f15d90 100644
--- 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
@@ -161,7 +161,7 @@ public final class AMRMClientUtils {
       final Token<? extends TokenIdentifier> token) throws IOException {
     try {
       String rmClusterId = configuration.get(YarnConfiguration.RM_CLUSTER_ID,
-          YarnConfiguration.DEFAULT_RM_CLUSTER_ID);
+          "yarn_cluster");
       LOG.info("Creating RMProxy to RM {} for protocol {} for user {}",
           rmClusterId, protocol.getSimpleName(), user);
       if (token != null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d11be2dc/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 23978ed..208a46c 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
@@ -50,7 +50,8 @@ public abstract class BaseFederationPoliciesTest {
 
   private ConfigurableFederationPolicy policy;
   private WeightedPolicyInfo policyInfo = mock(WeightedPolicyInfo.class);
-  private Map<SubClusterId, SubClusterInfo> activeSubclusters = new HashMap<>();
+  private Map<SubClusterId, SubClusterInfo> activeSubclusters =
+      new HashMap<SubClusterId, SubClusterInfo>();
   private FederationPolicyInitializationContext federationPolicyContext;
   private ApplicationSubmissionContext applicationSubmissionContext =
       mock(ApplicationSubmissionContext.class);
@@ -99,7 +100,7 @@ public abstract class BaseFederationPoliciesTest {
   public void testNoSubclusters() throws YarnException {
     // empty the activeSubclusters map
     FederationPoliciesTestUtil.initializePolicyContext(getPolicy(),
-        getPolicyInfo(), new HashMap<>());
+        getPolicyInfo(), new HashMap<SubClusterId, SubClusterInfo>());
 
     ConfigurableFederationPolicy localPolicy = getPolicy();
     if (localPolicy instanceof FederationRouterPolicy) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d11be2dc/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 acc14dd..4954197 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
@@ -132,7 +132,8 @@ public final class FederationPoliciesTestUtil {
     GetSubClustersInfoResponse response = GetSubClustersInfoResponse
         .newInstance(new ArrayList<SubClusterInfo>(activeSubclusters.values()));
 
-    when(fss.getSubClusters(any())).thenReturn(response);
+    when(fss.getSubClusters(any(GetSubClustersInfoRequest.class)))
+        .thenReturn(response);
     facade.reinitialize(fss, new Configuration());
     fpc.setFederationStateStoreFacade(facade);
     policy.reinitialize(fpc);
@@ -192,20 +193,26 @@ public final class FederationPoliciesTestUtil {
     FederationStateStore fss = mock(FederationStateStore.class);
     GetSubClustersInfoResponse response = GetSubClustersInfoResponse
         .newInstance(subClusterInfos);
-    when(fss.getSubClusters(any())).thenReturn(response);
+    when(fss.getSubClusters(any(GetSubClustersInfoRequest.class)))
+        .thenReturn(response);
 
-    List<SubClusterPolicyConfiguration> configurations = new ArrayList<>();
+    List<SubClusterPolicyConfiguration> configurations =
+        new ArrayList<SubClusterPolicyConfiguration>();
     configurations.add(policyConfiguration);
 
     GetSubClusterPoliciesConfigurationsResponse policiesResponse =
         GetSubClusterPoliciesConfigurationsResponse
             .newInstance(configurations);
-    when(fss.getPoliciesConfigurations(any())).thenReturn(policiesResponse);
+    when(fss.getPoliciesConfigurations(
+        any(GetSubClusterPoliciesConfigurationsRequest.class)))
+            .thenReturn(policiesResponse);
 
     GetSubClusterPolicyConfigurationResponse policyResponse =
         GetSubClusterPolicyConfigurationResponse
             .newInstance(policyConfiguration);
-    when(fss.getPolicyConfiguration(any())).thenReturn(policyResponse);
+    when(fss.getPolicyConfiguration(
+        any(GetSubClusterPolicyConfigurationRequest.class)))
+            .thenReturn(policyResponse);
 
     goodFacade.reinitialize(fss, new Configuration());
     return goodFacade;
@@ -220,8 +227,8 @@ public final class FederationPoliciesTestUtil {
    * @throws YarnException in case the initialization is not successful.
    */
   public static FederationStateStoreFacade initFacade() throws YarnException {
-    return initFacade(new ArrayList<>(), mock(SubClusterPolicyConfiguration
-        .class));
+    return initFacade(new ArrayList<SubClusterInfo>(),
+        mock(SubClusterPolicyConfiguration.class));
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d11be2dc/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 2247819..aed295e 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
@@ -1000,13 +1000,11 @@ public class ResourceManager extends CompositeService implements Recoverable {
         fetcher = new AppReportFetcher(conf, getClientRMService());
       }
       builder.withServlet(ProxyUriUtils.PROXY_SERVLET_NAME,
-          ProxyUriUtils.PROXY_PATH_SPEC, WebAppProxyServlet.class);
+              ProxyUriUtils.PROXY_PATH_SPEC, WebAppProxyServlet.class);
       builder.withAttribute(WebAppProxy.FETCHER_ATTRIBUTE, fetcher);
       String[] proxyParts = proxyHostAndPort.split(":");
       builder.withAttribute(WebAppProxy.PROXY_HOST_ATTRIBUTE, proxyParts[0]);
     }
-
-    }
     webApp = builder.start(new RMWebApp(this));
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d11be2dc/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 e8b4d56..053918b 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-beta1-SNAPSHOT</version>
+    <version>2.9.0-SNAPSHOT</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
   <groupId>org.apache.hadoop</groupId>
   <artifactId>hadoop-yarn-server-router</artifactId>
-  <version>3.0.0-beta1-SNAPSHOT</version>
+  <version>2.9.0-SNAPSHOT</version>
   <name>Apache Hadoop YARN Router</name>
 
   <properties>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d11be2dc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/RouterWebServiceUtil.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/webapp/RouterWebServiceUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/RouterWebServiceUtil.java
index 18618ee..1c4332e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/RouterWebServiceUtil.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/RouterWebServiceUtil.java
@@ -73,9 +73,10 @@ public final class RouterWebServiceUtil {
    *          call in case the call has no servlet request
    * @return the retrieved entity from the REST call
    */
-  protected static <T> T genericForward(String webApp, HttpServletRequest hsr,
-      final Class<T> returnType, HTTPMethods method, String targetPath,
-      Object formParam, Map<String, String[]> additionalParam) {
+  protected static <T> T genericForward(final String webApp,
+      final HttpServletRequest hsr, final Class<T> returnType,
+      final HTTPMethods method, final String targetPath, final Object formParam,
+      final Map<String, String[]> additionalParam) {
 
     UserGroupInformation callerUGI = null;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d11be2dc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/RouterWebServices.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/webapp/RouterWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/RouterWebServices.java
index bbb8326..4bb6271 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/RouterWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/RouterWebServices.java
@@ -44,7 +44,6 @@ import javax.ws.rs.core.Response;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.http.JettyUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AuthorizationException;
 import org.apache.hadoop.util.ReflectionUtils;
@@ -134,7 +133,7 @@ public class RouterWebServices implements RMWebServiceProtocol {
   /**
    * Returns the comma separated intercepter class names from the configuration.
    *
-   * @param conf
+   * @param config
    * @return the intercepter class names as an instance of ArrayList
    */
   private List<String> getInterceptorClassNames(Configuration config) {
@@ -302,8 +301,7 @@ public class RouterWebServices implements RMWebServiceProtocol {
   }
 
   @GET
-  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
-      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
   @Override
   public ClusterInfo get() {
     return getClusterInfo();
@@ -311,8 +309,7 @@ public class RouterWebServices implements RMWebServiceProtocol {
 
   @GET
   @Path(RMWSConsts.INFO)
-  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
-      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
   @Override
   public ClusterInfo getClusterInfo() {
     init();
@@ -322,8 +319,7 @@ public class RouterWebServices implements RMWebServiceProtocol {
 
   @GET
   @Path(RMWSConsts.METRICS)
-  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
-      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
   @Override
   public ClusterMetricsInfo getClusterMetricsInfo() {
     init();
@@ -333,8 +329,7 @@ public class RouterWebServices implements RMWebServiceProtocol {
 
   @GET
   @Path(RMWSConsts.SCHEDULER)
-  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
-      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
   @Override
   public SchedulerTypeInfo getSchedulerInfo() {
     init();
@@ -344,8 +339,7 @@ public class RouterWebServices implements RMWebServiceProtocol {
 
   @POST
   @Path(RMWSConsts.SCHEDULER_LOGS)
-  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
-      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
   @Override
   public String dumpSchedulerLogs(@FormParam(RMWSConsts.TIME) String time,
       @Context HttpServletRequest hsr) throws IOException {
@@ -356,8 +350,7 @@ public class RouterWebServices implements RMWebServiceProtocol {
 
   @GET
   @Path(RMWSConsts.NODES)
-  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
-      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
   @Override
   public NodesInfo getNodes(@QueryParam(RMWSConsts.STATES) String states) {
     init();
@@ -367,8 +360,7 @@ public class RouterWebServices implements RMWebServiceProtocol {
 
   @GET
   @Path(RMWSConsts.NODES_NODEID)
-  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
-      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
   @Override
   public NodeInfo getNode(@PathParam(RMWSConsts.NODEID) String nodeId) {
     init();
@@ -378,8 +370,7 @@ public class RouterWebServices implements RMWebServiceProtocol {
 
   @GET
   @Path(RMWSConsts.APPS)
-  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
-      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
   @Override
   public AppsInfo getApps(@Context HttpServletRequest hsr,
       @QueryParam(RMWSConsts.STATE) String stateQuery,
@@ -405,8 +396,7 @@ public class RouterWebServices implements RMWebServiceProtocol {
 
   @GET
   @Path(RMWSConsts.SCHEDULER_ACTIVITIES)
-  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
-      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
   @Override
   public ActivitiesInfo getActivities(@Context HttpServletRequest hsr,
       @QueryParam(RMWSConsts.NODEID) String nodeId) {
@@ -417,8 +407,7 @@ public class RouterWebServices implements RMWebServiceProtocol {
 
   @GET
   @Path(RMWSConsts.SCHEDULER_APP_ACTIVITIES)
-  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
-      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
   @Override
   public AppActivitiesInfo getAppActivities(@Context HttpServletRequest hsr,
       @QueryParam(RMWSConsts.APP_ID) String appId,
@@ -430,8 +419,7 @@ public class RouterWebServices implements RMWebServiceProtocol {
 
   @GET
   @Path(RMWSConsts.APP_STATISTICS)
-  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
-      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
   @Override
   public ApplicationStatisticsInfo getAppStatistics(
       @Context HttpServletRequest hsr,
@@ -445,8 +433,7 @@ public class RouterWebServices implements RMWebServiceProtocol {
 
   @GET
   @Path(RMWSConsts.APPS_APPID)
-  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
-      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
   @Override
   public AppInfo getApp(@Context HttpServletRequest hsr,
       @PathParam(RMWSConsts.APPID) String appId,
@@ -458,8 +445,7 @@ public class RouterWebServices implements RMWebServiceProtocol {
 
   @GET
   @Path(RMWSConsts.APPS_APPID_STATE)
-  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
-      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
   @Override
   public AppState getAppState(@Context HttpServletRequest hsr,
       @PathParam(RMWSConsts.APPID) String appId) throws AuthorizationException {
@@ -470,8 +456,7 @@ public class RouterWebServices implements RMWebServiceProtocol {
 
   @PUT
   @Path(RMWSConsts.APPS_APPID_STATE)
-  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
-      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
   @Override
   public Response updateAppState(AppState targetState,
       @Context HttpServletRequest hsr,
@@ -485,8 +470,7 @@ public class RouterWebServices implements RMWebServiceProtocol {
 
   @GET
   @Path(RMWSConsts.GET_NODE_TO_LABELS)
-  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
-      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
   @Override
   public NodeToLabelsInfo getNodeToLabels(@Context HttpServletRequest hsr)
       throws IOException {
@@ -497,8 +481,7 @@ public class RouterWebServices implements RMWebServiceProtocol {
 
   @GET
   @Path(RMWSConsts.LABEL_MAPPINGS)
-  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
-      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
   @Override
   public LabelsToNodesInfo getLabelsToNodes(
       @QueryParam(RMWSConsts.LABELS) Set<String> labels) throws IOException {
@@ -509,8 +492,7 @@ public class RouterWebServices implements RMWebServiceProtocol {
 
   @POST
   @Path(RMWSConsts.REPLACE_NODE_TO_LABELS)
-  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
-      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
   @Override
   public Response replaceLabelsOnNodes(
       final NodeToLabelsEntryList newNodeToLabels,
@@ -523,8 +505,7 @@ public class RouterWebServices implements RMWebServiceProtocol {
 
   @POST
   @Path(RMWSConsts.NODES_NODEID_REPLACE_LABELS)
-  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
-      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
   @Override
   public Response replaceLabelsOnNode(
       @QueryParam(RMWSConsts.LABELS) Set<String> newNodeLabelsName,
@@ -538,8 +519,7 @@ public class RouterWebServices implements RMWebServiceProtocol {
 
   @GET
   @Path(RMWSConsts.GET_NODE_LABELS)
-  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
-      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
   @Override
   public NodeLabelsInfo getClusterNodeLabels(@Context HttpServletRequest hsr)
       throws IOException {
@@ -550,8 +530,7 @@ public class RouterWebServices implements RMWebServiceProtocol {
 
   @POST
   @Path(RMWSConsts.ADD_NODE_LABELS)
-  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
-      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
   @Override
   public Response addToClusterNodeLabels(NodeLabelsInfo newNodeLabels,
       @Context HttpServletRequest hsr) throws Exception {
@@ -563,8 +542,7 @@ public class RouterWebServices implements RMWebServiceProtocol {
 
   @POST
   @Path(RMWSConsts.REMOVE_NODE_LABELS)
-  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
-      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
   @Override
   public Response removeFromCluserNodeLabels(
       @QueryParam(RMWSConsts.LABELS) Set<String> oldNodeLabels,
@@ -577,8 +555,7 @@ public class RouterWebServices implements RMWebServiceProtocol {
 
   @GET
   @Path(RMWSConsts.NODES_NODEID_GETLABELS)
-  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
-      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
   @Override
   public NodeLabelsInfo getLabelsOnNode(@Context HttpServletRequest hsr,
       @PathParam(RMWSConsts.NODEID) String nodeId) throws IOException {
@@ -589,8 +566,7 @@ public class RouterWebServices implements RMWebServiceProtocol {
 
   @GET
   @Path(RMWSConsts.APPS_APPID_PRIORITY)
-  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
-      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
   @Override
   public AppPriority getAppPriority(@Context HttpServletRequest hsr,
       @PathParam(RMWSConsts.APPID) String appId) throws AuthorizationException {
@@ -601,8 +577,7 @@ public class RouterWebServices implements RMWebServiceProtocol {
 
   @PUT
   @Path(RMWSConsts.APPS_APPID_PRIORITY)
-  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
-      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
   @Override
   public Response updateApplicationPriority(AppPriority targetPriority,
       @Context HttpServletRequest hsr,
@@ -616,8 +591,7 @@ public class RouterWebServices implements RMWebServiceProtocol {
 
   @GET
   @Path(RMWSConsts.APPS_APPID_QUEUE)
-  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
-      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
   @Override
   public AppQueue getAppQueue(@Context HttpServletRequest hsr,
       @PathParam(RMWSConsts.APPID) String appId) throws AuthorizationException {
@@ -628,8 +602,7 @@ public class RouterWebServices implements RMWebServiceProtocol {
 
   @PUT
   @Path(RMWSConsts.APPS_APPID_QUEUE)
-  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
-      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
   @Override
   public Response updateAppQueue(AppQueue targetQueue,
       @Context HttpServletRequest hsr,
@@ -643,8 +616,7 @@ public class RouterWebServices implements RMWebServiceProtocol {
 
   @POST
   @Path(RMWSConsts.APPS_NEW_APPLICATION)
-  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
-      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
   @Override
   public Response createNewApplication(@Context HttpServletRequest hsr)
       throws AuthorizationException, IOException, InterruptedException {
@@ -655,8 +627,7 @@ public class RouterWebServices implements RMWebServiceProtocol {
 
   @POST
   @Path(RMWSConsts.APPS)
-  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
-      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
   @Override
   public Response submitApplication(ApplicationSubmissionContextInfo newApp,
       @Context HttpServletRequest hsr)
@@ -668,8 +639,7 @@ public class RouterWebServices implements RMWebServiceProtocol {
 
   @POST
   @Path(RMWSConsts.DELEGATION_TOKEN)
-  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
-      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
   @Override
   public Response postDelegationToken(DelegationToken tokenData,
       @Context HttpServletRequest hsr) throws AuthorizationException,
@@ -681,8 +651,7 @@ public class RouterWebServices implements RMWebServiceProtocol {
 
   @POST
   @Path(RMWSConsts.DELEGATION_TOKEN_EXPIRATION)
-  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
-      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
   @Override
   public Response postDelegationTokenExpiration(@Context HttpServletRequest hsr)
       throws AuthorizationException, IOException, Exception {
@@ -693,8 +662,7 @@ public class RouterWebServices implements RMWebServiceProtocol {
 
   @DELETE
   @Path(RMWSConsts.DELEGATION_TOKEN)
-  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
-      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
   @Override
   public Response cancelDelegationToken(@Context HttpServletRequest hsr)
       throws AuthorizationException, IOException, InterruptedException,
@@ -706,8 +674,7 @@ public class RouterWebServices implements RMWebServiceProtocol {
 
   @POST
   @Path(RMWSConsts.RESERVATION_NEW)
-  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
-      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
   @Override
   public Response createNewReservation(@Context HttpServletRequest hsr)
       throws AuthorizationException, IOException, InterruptedException {
@@ -718,8 +685,7 @@ public class RouterWebServices implements RMWebServiceProtocol {
 
   @POST
   @Path(RMWSConsts.RESERVATION_SUBMIT)
-  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
-      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
   @Override
   public Response submitReservation(ReservationSubmissionRequestInfo resContext,
       @Context HttpServletRequest hsr)
@@ -731,8 +697,7 @@ public class RouterWebServices implements RMWebServiceProtocol {
 
   @POST
   @Path(RMWSConsts.RESERVATION_UPDATE)
-  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
-      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
   @Override
   public Response updateReservation(ReservationUpdateRequestInfo resContext,
       @Context HttpServletRequest hsr)
@@ -744,8 +709,7 @@ public class RouterWebServices implements RMWebServiceProtocol {
 
   @POST
   @Path(RMWSConsts.RESERVATION_DELETE)
-  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
-      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
   @Override
   public Response deleteReservation(ReservationDeleteRequestInfo resContext,
       @Context HttpServletRequest hsr)
@@ -757,8 +721,7 @@ public class RouterWebServices implements RMWebServiceProtocol {
 
   @GET
   @Path(RMWSConsts.RESERVATION_LIST)
-  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
-      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
   @Override
   public Response listReservation(
       @QueryParam(RMWSConsts.QUEUE) @DefaultValue(DEFAULT_QUEUE) String queue,
@@ -775,8 +738,7 @@ public class RouterWebServices implements RMWebServiceProtocol {
 
   @GET
   @Path(RMWSConsts.APPS_TIMEOUTS_TYPE)
-  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
-      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
   @Override
   public AppTimeoutInfo getAppTimeout(@Context HttpServletRequest hsr,
       @PathParam(RMWSConsts.APPID) String appId,
@@ -788,8 +750,7 @@ public class RouterWebServices implements RMWebServiceProtocol {
 
   @GET
   @Path(RMWSConsts.APPS_TIMEOUTS)
-  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
-      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
   @Override
   public AppTimeoutsInfo getAppTimeouts(@Context HttpServletRequest hsr,
       @PathParam(RMWSConsts.APPID) String appId) throws AuthorizationException {
@@ -800,8 +761,7 @@ public class RouterWebServices implements RMWebServiceProtocol {
 
   @PUT
   @Path(RMWSConsts.APPS_TIMEOUT)
-  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
-      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
   @Override
   public Response updateApplicationTimeout(AppTimeoutInfo appTimeout,
       @Context HttpServletRequest hsr,
@@ -815,8 +775,7 @@ public class RouterWebServices implements RMWebServiceProtocol {
 
   @GET
   @Path(RMWSConsts.APPS_APPID_APPATTEMPTS)
-  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
-      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
   @Override
   public AppAttemptsInfo getAppAttempts(@Context HttpServletRequest hsr,
       @PathParam(RMWSConsts.APPID) String appId) {
@@ -827,8 +786,7 @@ public class RouterWebServices implements RMWebServiceProtocol {
 
   @GET
   @Path(RMWSConsts.APPS_APPID_APPATTEMPTS_APPATTEMPTID)
-  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
-      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
   public org.apache.hadoop.yarn.server.webapp.dao.AppAttemptInfo getAppAttempt(
       @Context HttpServletRequest req, @Context HttpServletResponse res,
       @PathParam(RMWSConsts.APPID) String appId,
@@ -841,8 +799,7 @@ public class RouterWebServices implements RMWebServiceProtocol {
 
   @GET
   @Path(RMWSConsts.APPS_APPID_APPATTEMPTS_APPATTEMPTID_CONTAINERS)
-  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
-      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
   public ContainersInfo getContainers(@Context HttpServletRequest req,
       @Context HttpServletResponse res,
       @PathParam(RMWSConsts.APPID) String appId,
@@ -855,8 +812,7 @@ public class RouterWebServices implements RMWebServiceProtocol {
 
   @GET
   @Path(RMWSConsts.GET_CONTAINER)
-  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
-      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
   public ContainerInfo getContainer(@Context HttpServletRequest req,
       @Context HttpServletResponse res,
       @PathParam(RMWSConsts.APPID) String appId,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d11be2dc/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
index d3eba61..86fb884 100644
--- 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
@@ -330,7 +330,7 @@ public abstract class BaseRouterRMAdminTest {
         });
   }
 
-  protected String[] getGroupsForUser(String user)
+  protected String[] getGroupsForUser(final String user)
       throws IOException, InterruptedException {
     return UserGroupInformation.createRemoteUser(user)
         .doAs(new PrivilegedExceptionAction<String[]>() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d11be2dc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestRouterWebServicesREST.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/webapp/TestRouterWebServicesREST.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestRouterWebServicesREST.java
index d7b1a0f..31a2ab2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestRouterWebServicesREST.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestRouterWebServicesREST.java
@@ -150,8 +150,8 @@ public class TestRouterWebServicesREST {
    * Performs 2 GET calls one to RM and the one to Router. In positive case, it
    * returns the 2 answers in a list.
    */
-  private static <T> List<T> performGetCalls(String path, Class<T> returnType,
-      String queryName, String queryValue)
+  private static <T> List<T> performGetCalls(String path,
+      final Class<T> returnType, String queryName, String queryValue)
       throws IOException, InterruptedException {
     Client clientToRouter = Client.create();
     WebResource toRouter = clientToRouter
@@ -161,8 +161,8 @@ public class TestRouterWebServicesREST {
     WebResource toRM = clientToRM
         .resource(WebAppUtils.getRMWebAppURLWithScheme(conf)).path(path);
 
-    Builder toRouterBuilder;
-    Builder toRMBuilder;
+    final Builder toRouterBuilder;
+    final Builder toRMBuilder;
 
     if (queryValue != null && queryName != null) {
       toRouterBuilder = toRouter.queryParam(queryName, queryValue)
@@ -197,9 +197,9 @@ public class TestRouterWebServicesREST {
   /**
    * Performs a POST/PUT/DELETE call to Router and returns the ClientResponse.
    */
-  private static ClientResponse performCall(String webAddress, String queryKey,
-      String queryValue, Object context, HTTPMethods method)
-      throws IOException, InterruptedException {
+  private static ClientResponse performCall(final String webAddress,
+      final String queryKey, final String queryValue, final Object context,
+      final HTTPMethods method) throws IOException, InterruptedException {
 
     return UserGroupInformation.createRemoteUser(userName)
         .doAs(new PrivilegedExceptionAction<ClientResponse>() {


---------------------------------------------------------------------
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: YARN-3659. Federation: routing client invocations transparently to multiple RMs. (Giovanni Matteo Fumarola via Subru).

Posted by cu...@apache.org.
YARN-3659. Federation: routing client invocations transparently to multiple RMs. (Giovanni Matteo Fumarola via Subru).

(cherry picked from commit 52daa6d971ae408d121b3737ea8c0575e7e8516d)
(cherry picked from commit 43a97174fe49aa0c25d03b8a970a46d4bebf1aa8)


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

Branch: refs/heads/branch-2
Commit: 169037cbb44e809b386f5fe9f7bc4f4ffcad5a3a
Parents: 8ed4dad
Author: Subru Krishnan <su...@apache.org>
Authored: Mon Jun 26 13:27:26 2017 -0700
Committer: Carlo Curino <cu...@apache.org>
Committed: Thu Sep 21 16:55:33 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  | 194 +++++-
 .../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, 1872 insertions(+), 62 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/169037cb/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 9e8a5ad..3600709 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
@@ -2621,6 +2621,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/169037cb/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 32ae209..910df9e 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
@@ -144,6 +144,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/169037cb/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/169037cb/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/169037cb/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/169037cb/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/169037cb/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/169037cb/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/169037cb/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/169037cb/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/169037cb/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/169037cb/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 4bdff64..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;
@@ -118,6 +119,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;
@@ -153,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;
 
 /**
@@ -174,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;
 
@@ -186,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();
@@ -207,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);
 
@@ -247,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);
 
@@ -283,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) {
@@ -390,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);
@@ -406,6 +444,8 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
       GetApplicationAttemptReportRequest request)
       throws YarnException, IOException {
 
+    validateRunning();
+
     GetApplicationAttemptReportResponse response =
         Records.newRecord(GetApplicationAttemptReportResponse.class);
     ApplicationAttemptReport report =
@@ -419,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();
@@ -437,32 +484,17 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
   @Override
   public KillApplicationResponse forceKillApplication(
       KillApplicationRequest request) throws YarnException, IOException {
-    String appId = "";
-    boolean foundApp = false;
+
+    validateRunning();
+
+    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);
   }
@@ -470,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();
   }
 
@@ -519,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>());
   }
@@ -556,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>>());
   }
@@ -575,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));
   }
@@ -594,6 +686,9 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
   @Override
   public FailApplicationAttemptResponse failApplicationAttempt(
       FailApplicationAttemptRequest request) throws YarnException, IOException {
+
+    validateRunning();
+
     return FailApplicationAttemptResponse.newInstance();
   }
 
@@ -601,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();
   }
 
@@ -614,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();
   }
 
@@ -633,6 +743,9 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
   public RefreshSuperUserGroupsConfigurationResponse refreshSuperUserGroupsConfiguration(
       RefreshSuperUserGroupsConfigurationRequest request)
       throws StandbyException, YarnException, IOException {
+
+    validateRunning();
+
     return RefreshSuperUserGroupsConfigurationResponse.newInstance();
   }
 
@@ -640,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();
   }
 
@@ -677,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();
   }
 
@@ -690,6 +827,9 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
   public CheckForDecommissioningNodesResponse checkForDecommissioningNodes(
       CheckForDecommissioningNodesRequest checkForDecommissioningNodesRequest)
       throws YarnException, IOException {
+
+    validateRunning();
+
     return CheckForDecommissioningNodesResponse.newInstance(null);
   }
 
@@ -697,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/169037cb/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/169037cb/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/169037cb/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/169037cb/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/169037cb/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/169037cb/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/169037cb/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();
+  }
+
+}


---------------------------------------------------------------------
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-5410. Bootstrap Router server module. (Giovanni Matteo Fumarola via Subru).

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

(cherry picked from commit 04f111394b08de7d16a86f8025221f6f5e0dbcc5)
(cherry picked from commit 193ec456d624a99c0d1aafb53bdd64fc29edef71)


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

Branch: refs/heads/branch-2
Commit: 2dca88b924ded1e56beb61011aea2635decea3bc
Parents: ce9110a
Author: Subru Krishnan <su...@apache.org>
Authored: Fri Feb 24 12:08:53 2017 -0800
Committer: Carlo Curino <cu...@apache.org>
Committed: Thu Sep 21 16:25:02 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/2dca88b9/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
old mode 100644
new mode 100755
index d7fd5a4..eb8fde9
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -328,6 +328,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/2dca88b9/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/2dca88b9/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/2dca88b9/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/2dca88b9/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/2dca88b9/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 80544bd..71feace 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/pom.xml
@@ -43,5 +43,6 @@
     <module>hadoop-yarn-server-tests</module>
     <module>hadoop-yarn-server-applicationhistoryservice</module>
     <module>hadoop-yarn-server-timeline-pluginstorage</module>
+    <module>hadoop-yarn-server-router</module>
   </modules>
 </project>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2dca88b9/hadoop-yarn-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/pom.xml b/hadoop-yarn-project/pom.xml
index 7ddb31a..9c3dd1a 100644
--- a/hadoop-yarn-project/pom.xml
+++ b/hadoop-yarn-project/pom.xml
@@ -75,6 +75,10 @@
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-yarn-server-web-proxy</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


[15/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 cu...@apache.org.
YARN-6093. Minor bugs with AMRMtoken renewal and state store availability when using FederationRMFailoverProxyProvider during RM failover. (Botong Huang via Subru).

(cherry picked from commit 66500f4fa6155d29435d7c92fd6d68079c4cab86)
(cherry picked from commit 98b45b0ed34a060e0a529069cd15676d91600dff)


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

Branch: refs/heads/branch-2
Commit: ce9110ab5c4b94857080383aa6a01332f9bfd103
Parents: 002a77d
Author: Subru Krishnan <su...@apache.org>
Authored: Wed Feb 22 13:16:22 2017 -0800
Committer: Carlo Curino <cu...@apache.org>
Committed: Thu Sep 21 16:23:47 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/ce9110ab/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/ce9110ab/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


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

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

(cherry picked from commit d7672ce2bddb40fbaa77d3f6fec8c99f5589177f)
(cherry picked from commit 083a05bc1420d6d6aa30d98ab48c2ed8ead5b810)


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

Branch: refs/heads/branch-2
Commit: d87d2b546e7337a56eb06ae67a46e28f8a146f02
Parents: b3a1ab7
Author: Subru Krishnan <su...@apache.org>
Authored: Wed Nov 16 19:39:25 2016 -0800
Committer: Carlo Curino <cu...@apache.org>
Committed: Thu Sep 21 16:23:44 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/d87d2b54/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 39305ce..4c4298d 100644
--- a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
+++ b/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
@@ -293,6 +293,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/d87d2b54/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 66bc377..055428d 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
@@ -2541,6 +2541,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/d87d2b54/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 c5a279d..f1cc7e6 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
@@ -80,6 +80,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/d87d2b54/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/d87d2b54/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/d87d2b54/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/d87d2b54/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/d87d2b54/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/d87d2b54/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/d87d2b54/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/d87d2b54/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


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

Posted by cu...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/5c5ab53f/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/5c5ab53f/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/5c5ab53f/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/5c5ab53f/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/5c5ab53f/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/5c5ab53f/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/5c5ab53f/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/5c5ab53f/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/5c5ab53f/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/5c5ab53f/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/5c5ab53f/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/5c5ab53f/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/5c5ab53f/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/5c5ab53f/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/5c5ab53f/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


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

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

(cherry picked from commit 9ca2aba9cc65090162b3517b194b5e655ee4a157)
(cherry picked from commit 2797507d51566ab3b8328e5fb1d0beb9fbce5bae)


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

Branch: refs/heads/branch-2
Commit: aac875512503130ad8e8b4e555caef25565c110a
Parents: 8409fef
Author: Jian He <ji...@apache.org>
Authored: Fri Sep 2 12:23:57 2016 +0800
Committer: Carlo Curino <cu...@apache.org>
Committed: Thu Sep 21 16:23:40 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/aac87551/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 6c1d014..39305ce 100644
--- a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
+++ b/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
@@ -276,7 +276,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/aac87551/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 c9db167..66bc377 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
@@ -149,6 +149,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/aac87551/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 38e2668..c5a279d 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
@@ -77,6 +77,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/aac87551/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 2eb69e0..2247819 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
@@ -718,6 +718,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/aac87551/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 71e1623..fbfcdb8 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
@@ -202,7 +202,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/aac87551/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/aac87551/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/aac87551/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/aac87551/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 dd14c41..dcf47ba 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
@@ -437,7 +437,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/aac87551/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 963aac8..fdac89c 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/aac87551/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 61088e1..68ed5c1 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
@@ -118,6 +118,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/aac87551/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 51adbe1..02ce9a6 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/aac87551/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 acfc60d..4db8b9f 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
@@ -188,6 +188,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


[48/50] [abbrv] hadoop git commit: YARN-6853. Add MySql Scripts for FederationStateStore. (Contributed by Giovanni Matteo Fumarola via curino)

Posted by cu...@apache.org.
YARN-6853. Add MySql Scripts for FederationStateStore. (Contributed by Giovanni Matteo Fumarola via curino)

(cherry picked from commit 874ddbf0b5b1d34aca70ee7fc303cbffdde67236)
(cherry picked from commit 9625a030dee1f567f3b91d74acccb8b15fe25428)


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

Branch: refs/heads/branch-2
Commit: d3afff76034b2318550f8c4c14cbb8408b6fc187
Parents: 190b79a
Author: Carlo Curino <cu...@apache.org>
Authored: Tue Aug 1 17:18:20 2017 -0700
Committer: Carlo Curino <cu...@apache.org>
Committed: Thu Sep 21 17:15:36 2017 -0700

----------------------------------------------------------------------
 .../MySQL/FederationStateStoreDatabase.sql      |  21 +++
 .../MySQL/FederationStateStoreStoredProcs.sql   | 162 +++++++++++++++++++
 .../MySQL/FederationStateStoreTables.sql        |  47 ++++++
 .../MySQL/FederationStateStoreUser.sql          |  25 +++
 .../FederationStateStore/MySQL/dropDatabase.sql |  21 +++
 .../MySQL/dropStoreProcedures.sql               |  47 ++++++
 .../FederationStateStore/MySQL/dropTables.sql   |  27 ++++
 .../bin/FederationStateStore/MySQL/dropUser.sql |  21 +++
 .../src/site/markdown/Federation.md             |  18 ++-
 9 files changed, 386 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3afff76/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/FederationStateStoreStoredProcs.sql
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/FederationStateStoreStoredProcs.sql b/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/FederationStateStoreStoredProcs.sql
new file mode 100644
index 0000000..eae882e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/FederationStateStoreStoredProcs.sql
@@ -0,0 +1,162 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+-- Script to generate all the stored procedures for the Federation StateStore in MySQL
+
+USE FederationStateStore
+
+DELIMITER //
+
+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)
+BEGIN
+   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(), state_IN, lastStartTime_IN, capability_IN);
+   SELECT ROW_COUNT() INTO rowCount_OUT;
+END //
+
+CREATE PROCEDURE sp_deregisterSubCluster(
+    IN subClusterId_IN varchar(256),
+    IN state_IN varchar(64),
+    OUT rowCount_OUT int)
+BEGIN
+    UPDATE membership SET state = state_IN
+    WHERE (subClusterId = subClusterId_IN AND state != state_IN);
+    SELECT ROW_COUNT() INTO rowCount_OUT;
+END //
+
+CREATE PROCEDURE sp_subClusterHeartbeat(
+    IN subClusterId_IN varchar(256), IN state_IN varchar(64),
+    IN capability_IN varchar(6000), OUT rowCount_OUT int)
+BEGIN
+   UPDATE membership
+   SET capability = capability_IN,
+       state = state_IN,
+       lastHeartBeat = NOW()
+   WHERE subClusterId = subClusterId_IN;
+   SELECT ROW_COUNT() INTO rowCount_OUT;
+END //
+
+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))
+BEGIN
+   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 //
+
+CREATE PROCEDURE sp_getSubClusters()
+BEGIN
+   SELECT subClusterId, amRMServiceAddress, clientRMServiceAddress,
+          rmAdminServiceAddress, rmWebServiceAddress, lastHeartBeat,
+          state, lastStartTime, capability
+   FROM membership;
+END //
+
+CREATE PROCEDURE sp_addApplicationHomeSubCluster(
+   IN applicationId_IN varchar(64), IN homeSubCluster_IN varchar(256),
+   OUT storedHomeSubCluster_OUT varchar(256), OUT rowCount_OUT int)
+BEGIN
+   INSERT INTO applicationsHomeSubCluster
+      (applicationId,homeSubCluster)
+      (SELECT applicationId_IN, homeSubCluster_IN
+       FROM applicationsHomeSubCluster
+       WHERE applicationId = applicationId_IN
+       HAVING COUNT(*) = 0 );
+   SELECT ROW_COUNT() INTO rowCount_OUT;
+   SELECT homeSubCluster INTO storedHomeSubCluster_OUT
+   FROM applicationsHomeSubCluster
+   WHERE applicationId = applicationID_IN;
+END //
+
+CREATE PROCEDURE sp_updateApplicationHomeSubCluster(
+   IN applicationId_IN varchar(64),
+   IN homeSubCluster_IN varchar(256), OUT rowCount_OUT int)
+BEGIN
+   UPDATE applicationsHomeSubCluster
+     SET homeSubCluster = homeSubCluster_IN
+   WHERE applicationId = applicationId_IN;
+   SELECT ROW_COUNT() INTO rowCount_OUT;
+END //
+
+CREATE PROCEDURE sp_getApplicationHomeSubCluster(
+   IN applicationId_IN varchar(64),
+   OUT homeSubCluster_OUT varchar(256))
+BEGIN
+   SELECT homeSubCluster INTO homeSubCluster_OUT
+   FROM applicationsHomeSubCluster
+   WHERE applicationId = applicationID_IN;
+END //
+
+CREATE PROCEDURE sp_getApplicationsHomeSubCluster()
+BEGIN
+   SELECT applicationId, homeSubCluster
+   FROM applicationsHomeSubCluster;
+END //
+
+CREATE PROCEDURE sp_deleteApplicationHomeSubCluster(
+   IN applicationId_IN varchar(64), OUT rowCount_OUT int)
+BEGIN
+   DELETE FROM applicationsHomeSubCluster
+   WHERE applicationId = applicationId_IN;
+   SELECT ROW_COUNT() INTO rowCount_OUT;
+END //
+
+CREATE PROCEDURE sp_setPolicyConfiguration(
+   IN queue_IN varchar(256), IN policyType_IN varchar(256),
+   IN params_IN varbinary(32768), OUT rowCount_OUT int)
+BEGIN
+   DELETE FROM policies WHERE queue = queue_IN;
+   INSERT INTO policies (queue, policyType, params)
+   VALUES (queue_IN, policyType_IN, params_IN);
+   SELECT ROW_COUNT() INTO rowCount_OUT;
+END //
+
+CREATE PROCEDURE sp_getPoliciesConfigurations()
+BEGIN
+   SELECT queue, policyType, params FROM policies;
+END //
+
+CREATE PROCEDURE sp_getPolicyConfiguration(
+   IN queue_IN varchar(256), OUT policyType_OUT varchar(256),
+   OUT params_OUT varbinary(32768))
+BEGIN
+   SELECT policyType, params INTO policyType_OUT, params_OUT
+   FROM policies WHERE queue = queue_IN;
+END //
+
+DELIMITER ;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3afff76/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/FederationStateStoreTables.sql
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/FederationStateStoreTables.sql b/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/FederationStateStoreTables.sql
new file mode 100644
index 0000000..67a1817
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/FederationStateStoreTables.sql
@@ -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.
+ */
+
+-- Script to generate all the tables for the Federation StateStore in MySQL
+
+USE FederationStateStore
+
+CREATE TABLE applicationsHomeSubCluster(
+   applicationId varchar(64) NOT NULL,
+   subClusterId varchar(256) NULL,
+   CONSTRAINT pk_applicationId PRIMARY KEY (applicationId)
+);
+
+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),
+   CONSTRAINT pk_subClusterId PRIMARY KEY (subClusterId)
+);
+
+CREATE TABLE policies(
+   queue varchar(256) NOT NULL,
+   policyType varchar(256) NOT NULL,
+   params varbinary(32768),
+   CONSTRAINT pk_queue PRIMARY KEY (queue)
+);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3afff76/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/FederationStateStoreUser.sql
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/FederationStateStoreUser.sql b/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/FederationStateStoreUser.sql
new file mode 100644
index 0000000..32f4933
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/FederationStateStoreUser.sql
@@ -0,0 +1,25 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+-- Script to create a new User in MySQL for the Federation StateStore
+
+CREATE USER 'FederationUser'@'%' IDENTIFIED BY 'FederationPassword';
+
+GRANT ALL PRIVILEGES ON FederationStateStore.* TO 'FederationUser'@'%';
+
+FLUSH PRIVILEGES;

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3afff76/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/dropStoreProcedures.sql
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/dropStoreProcedures.sql b/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/dropStoreProcedures.sql
new file mode 100644
index 0000000..f24f3fb
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/dropStoreProcedures.sql
@@ -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.
+ */
+
+-- Script to drop all the stored procedures for the Federation StateStore in MySQL
+
+USE FederationStateStore
+
+DROP PROCEDURE sp_registerSubCluster;
+
+DROP PROCEDURE sp_deregisterSubCluster;
+
+DROP PROCEDURE sp_subClusterHeartbeat;
+
+DROP PROCEDURE sp_getSubCluster;
+
+DROP PROCEDURE sp_getSubClusters;
+
+DROP PROCEDURE sp_addApplicationHomeSubCluster;
+
+DROP PROCEDURE sp_updateApplicationHomeSubCluster;
+
+DROP PROCEDURE sp_getApplicationHomeSubCluster;
+
+DROP PROCEDURE sp_getApplicationsHomeSubCluster;
+
+DROP PROCEDURE sp_deleteApplicationHomeSubCluster;
+
+DROP PROCEDURE sp_setPolicyConfiguration;
+
+DROP PROCEDURE sp_getPolicyConfiguration;
+
+DROP PROCEDURE sp_getPoliciesConfigurations;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3afff76/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/dropTables.sql
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/dropTables.sql b/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/dropTables.sql
new file mode 100644
index 0000000..ea6567b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/dropTables.sql
@@ -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.
+ */
+
+-- Script to drop all the tables from the Federation StateStore in MySQL
+
+USE FederationStateStore
+
+DROP TABLE applicationsHomeSubCluster;
+
+DROP TABLE membership;
+
+DROP TABLE policies;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3afff76/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/dropUser.sql
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/dropUser.sql b/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/dropUser.sql
new file mode 100644
index 0000000..7b4bb02
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/bin/FederationStateStore/MySQL/dropUser.sql
@@ -0,0 +1,21 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+-- Script to drop the user from Federation StateStore in MySQL
+
+DROP USER 'FederationUser'@'%';

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3afff76/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 61cb77f..ecf61c5 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
@@ -161,8 +161,8 @@ 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>;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.url` | `jdbc:mysql://<host>:<port>/FederationStateStore` | For SQLFederationStateStore the name of the DB where the state is stored. |
+|`yarn.federation.state-store.sql.jdbc-class` | `com.mysql.jdbc.jdbc2.optional.MysqlDataSource` | 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). |
@@ -238,7 +238,19 @@ Optional:
 
 ###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**
+Currently, we support only SQL based implementation of state-store (ZooKeeper is in the works), i.e. either MySQL or Microsoft SQL Server.
+
+For MySQL, one must download the latest jar version 5.x from [MVN Repository](https://mvnrepository.com/artifact/mysql/mysql-connector-java) and add it to the CLASSPATH.
+Then the DB schema is created by executing the following SQL scripts in the database:
+1. **sbin/FederationStateStore/MySQL/FederationStateStoreDatabase.sql**.
+2. **sbin/FederationStateStore/MySQL/FederationStateStoreUser.sql**.
+3. **sbin/FederationStateStore/MySQL/FederationStateStoreTables.sql**.
+4. **sbin/FederationStateStore/MySQL/FederationStateStoreStoredProcs.sql**.
+In the same directory we provide scripts to drop the Stored Procedures, the Tables, the User and the Database.
+**Note:** the FederationStateStoreUser.sql defines a default user/password for the DB that you are **highly encouraged** to set this to a proper strong password.
+
+For SQL-Server, the process is similar, but the jdbc driver is already included in the pom (license allows it).
+SQL-Server scripts are located in **sbin/FederationStateStore/SQLServer/**.
 
 Running a Sample Job
 --------------------


---------------------------------------------------------------------
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-6724. Add ability to blacklist sub-clusters when invoking Routing policies. (Giovanni Matteo Fumarola via Subru).

Posted by cu...@apache.org.
YARN-6724. Add ability to blacklist sub-clusters when invoking Routing policies. (Giovanni Matteo Fumarola via Subru).

(cherry picked from commit f8e5de59697cb78686f0e605dc7e93628b5f3297)
(cherry picked from commit 4cfec943b177e2123a935e70d39776521883c2bc)


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

Branch: refs/heads/branch-2
Commit: 8ed4daddc4f264876591722140fa7eb7ab8d482b
Parents: 35a3833
Author: Subru Krishnan <su...@apache.org>
Authored: Wed Jun 21 19:08:47 2017 -0700
Committer: Carlo Curino <cu...@apache.org>
Committed: Thu Sep 21 16:55:01 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/8ed4dadd/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/8ed4dadd/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/8ed4dadd/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/8ed4dadd/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/8ed4dadd/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/8ed4dadd/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/8ed4dadd/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/8ed4dadd/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/8ed4dadd/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/8ed4dadd/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/8ed4dadd/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/8ed4dadd/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/8ed4dadd/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/8ed4dadd/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/8ed4dadd/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/8ed4dadd/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/8ed4dadd/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


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

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

(cherry picked from commit 575137f41c27eb72d05d923337f3030a35403e8f)
(cherry picked from commit 4128c9522dcdc16bb3527f74a48ed1242458a165)


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

Branch: refs/heads/branch-2
Commit: d32ffa9e5ebb55b1fed4948f3750ce2159d235b4
Parents: d87d2b5
Author: Subru Krishnan <su...@apache.org>
Authored: Tue Nov 22 15:02:22 2016 -0800
Committer: Carlo Curino <cu...@apache.org>
Committed: Thu Sep 21 16:23:45 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/d32ffa9e/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 055428d..906d632 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
@@ -2547,7 +2547,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/d32ffa9e/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/d32ffa9e/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/d32ffa9e/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/d32ffa9e/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/d32ffa9e/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/d32ffa9e/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/d32ffa9e/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/d32ffa9e/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/d32ffa9e/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/d32ffa9e/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/d32ffa9e/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/d32ffa9e/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/d32ffa9e/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/d32ffa9e/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/d32ffa9e/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/d32ffa9e/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/d32ffa9e/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/d32ffa9e/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/d32ffa9e/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/d32ffa9e/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/d32ffa9e/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/d32ffa9e/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/d32ffa9e/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-5325. Stateless ARMRMProxy policies implementation. (Carlo Curino via Subru).

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

(cherry picked from commit 11c5336522d3504598fb94eee288d54df73418c6)
(cherry picked from commit 1dadd0b45a6a605da72eb304808edb49fc66da45)


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

Branch: refs/heads/branch-2
Commit: 5c5ab53f3daddc79e47bd14829cbf0b3a4afc7ca
Parents: f792f7b
Author: Subru Krishnan <su...@apache.org>
Authored: Thu Oct 13 17:59:13 2016 -0700
Committer: Carlo Curino <cu...@apache.org>
Committed: Thu Sep 21 16:23:42 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/5c5ab53f/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/5c5ab53f/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/5c5ab53f/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/5c5ab53f/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/5c5ab53f/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/5c5ab53f/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/5c5ab53f/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/5c5ab53f/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/5c5ab53f/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/5c5ab53f/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/5c5ab53f/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/5c5ab53f/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/5c5ab53f/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/5c5ab53f/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/5c5ab53f/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/5c5ab53f/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/5c5ab53f/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


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

Posted by cu...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/f792f7b0/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


[28/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 cu...@apache.org.
YARN-5413. Create a proxy chain for ResourceManager Admin API in the Router. (Giovanni Matteo Fumarola via Subru).

(cherry picked from commit 67846a5519b5905c2d925cf4c602f715b653e72c)
(cherry picked from commit 7444406d6d48846d06b04e82587885bb8632a16c)


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

Branch: refs/heads/branch-2
Commit: 7f00f938f253d87153c2b8a3938daa3b99cb6f42
Parents: dc0a2e6
Author: Subru Krishnan <su...@apache.org>
Authored: Tue May 9 19:19:27 2017 -0700
Committer: Carlo Curino <cu...@apache.org>
Committed: Thu Sep 21 16:47:14 2017 -0700

----------------------------------------------------------------------
 .../hadoop/yarn/conf/YarnConfiguration.java     |  22 +-
 .../src/main/resources/yarn-default.xml         |  21 +
 .../hadoop/yarn/util/TestLRUCacheHashMap.java   |   2 +-
 .../yarn/server/MockResourceManagerFacade.java  |   7 +-
 .../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, 1635 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7f00f938/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 a3b53d6..9e8a5ad 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
@@ -2586,6 +2586,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.";
 
@@ -2601,9 +2603,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/7f00f938/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 5d53e14..998e4cb 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
@@ -3132,4 +3132,25 @@
     <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>
+
 </configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7f00f938/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/7f00f938/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 499a5cb..bda41d4 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
@@ -124,8 +124,11 @@ import org.mortbay.log.Log;
  * implementation is expected by the unit test cases. So please change the
  * implementation with care.
  */
-public class MockResourceManagerFacade implements
-    ApplicationMasterProtocol, ApplicationClientProtocol {
+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>>();

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