You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by su...@apache.org on 2016/10/03 22:14:00 UTC

[01/50] [abbrv] hadoop git commit: HADOOP-13599. s3a close() to be non-synchronized, so avoid risk of deadlock on shutdown. Contributed by Steve Loughran. [Forced Update!]

Repository: hadoop
Updated Branches:
  refs/heads/YARN-2915 6f9133893 -> f43ad972e (forced update)


HADOOP-13599. s3a close() to be non-synchronized, so avoid risk of deadlock on shutdown. Contributed by Steve Loughran.


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

Branch: refs/heads/YARN-2915
Commit: 47f80922dc7cb2fa6d084e6fb1f354c4ec1d4c69
Parents: 84c6264
Author: Chris Nauroth <cn...@apache.org>
Authored: Wed Sep 28 15:53:17 2016 -0700
Committer: Chris Nauroth <cn...@apache.org>
Committed: Wed Sep 28 16:19:32 2016 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java    | 8 +++++++-
 .../java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java | 8 ++++++++
 2 files changed, 15 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/47f80922/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
index 6a6c26b..dffef15 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
@@ -31,6 +31,7 @@ import java.util.Map;
 import java.util.concurrent.ExecutorService;
 import java.util.Objects;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 import com.amazonaws.AmazonClientException;
 import com.amazonaws.AmazonServiceException;
@@ -121,6 +122,7 @@ public class S3AFileSystem extends FileSystem {
   private S3AStorageStatistics storageStatistics;
   private long readAhead;
   private S3AInputPolicy inputPolicy;
+  private final AtomicBoolean closed = new AtomicBoolean(false);
 
   // The maximum number of entries that can be deleted in any call to s3
   private static final int MAX_ENTRIES_TO_DELETE = 1000;
@@ -1414,7 +1416,11 @@ public class S3AFileSystem extends FileSystem {
    * @throws IOException IO problem
    */
   @Override
-  public synchronized void close() throws IOException {
+  public void close() throws IOException {
+    if (closed.getAndSet(true)) {
+      // already closed
+      return;
+    }
     try {
       super.close();
     } finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/47f80922/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java
index fca8e49..b08bfe9 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java
@@ -409,6 +409,14 @@ public class ITestS3AConfiguration {
         awsConf.getUserAgent());
   }
 
+  @Test
+  public void testCloseIdempotent() throws Throwable {
+    conf = new Configuration();
+    fs = S3ATestUtils.createTestFileSystem(conf);
+    fs.close();
+    fs.close();
+  }
+
   /**
    * Reads and returns a field from an object using reflection.  If the field
    * cannot be found, is null, or is not the expected type, then this method


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

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


[16/50] [abbrv] hadoop git commit: HADOOP-12974. Create a CachingGetSpaceUsed implementation that uses df. Contributed by Elliott Clark.

Posted by su...@apache.org.
HADOOP-12974. Create a CachingGetSpaceUsed implementation that uses df. Contributed by Elliott Clark.


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

Branch: refs/heads/YARN-2915
Commit: 57aec2b46b0e46b73a1f49927e30e2c41138d535
Parents: 7fad122
Author: Wei-Chiu Chuang <we...@apache.org>
Authored: Fri Sep 30 12:58:37 2016 -0700
Committer: Wei-Chiu Chuang <we...@apache.org>
Committed: Fri Sep 30 12:58:37 2016 -0700

----------------------------------------------------------------------
 .../apache/hadoop/fs/DFCachingGetSpaceUsed.java | 48 +++++++++++++
 .../src/main/java/org/apache/hadoop/fs/DU.java  |  8 +--
 .../hadoop/fs/TestDFCachingGetSpaceUsed.java    | 75 ++++++++++++++++++++
 3 files changed, 126 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/57aec2b4/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DFCachingGetSpaceUsed.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DFCachingGetSpaceUsed.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DFCachingGetSpaceUsed.java
new file mode 100644
index 0000000..6e8cd46
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DFCachingGetSpaceUsed.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.fs;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import java.io.IOException;
+
+/**
+ * Fast but inaccurate class to tell how much space HDFS is using.
+ * This class makes the assumption that the entire mount is used for
+ * HDFS and that no two hdfs data dirs are on the same disk.
+ *
+ * To use set fs.getspaceused.classname
+ * to org.apache.hadoop.fs.DFCachingGetSpaceUsed in your core-site.xml
+ *
+ */
+@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
+@InterfaceStability.Evolving
+public class DFCachingGetSpaceUsed extends CachingGetSpaceUsed {
+  private final DF df;
+
+  public DFCachingGetSpaceUsed(Builder builder) throws IOException {
+    super(builder);
+    this.df = new DF(builder.getPath(), builder.getInterval());
+  }
+
+  @Override
+  protected void refresh() {
+    this.used.set(df.getUsed());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/57aec2b4/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DU.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DU.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DU.java
index 20e8202..b64a19d 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DU.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DU.java
@@ -31,12 +31,13 @@ import java.io.IOException;
 @InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
 @InterfaceStability.Evolving
 public class DU extends CachingGetSpaceUsed {
-  private DUShell duShell;
+  private final DUShell duShell;
 
   @VisibleForTesting
-  public DU(File path, long interval, long jitter, long initialUsed)
+   public DU(File path, long interval, long jitter, long initialUsed)
       throws IOException {
     super(path, interval, jitter, initialUsed);
+    this.duShell = new DUShell();
   }
 
   public DU(CachingGetSpaceUsed.Builder builder) throws IOException {
@@ -48,9 +49,6 @@ public class DU extends CachingGetSpaceUsed {
 
   @Override
   protected synchronized void refresh() {
-    if (duShell == null) {
-      duShell = new DUShell();
-    }
     try {
       duShell.startRefresh();
     } catch (IOException ioe) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/57aec2b4/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestDFCachingGetSpaceUsed.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestDFCachingGetSpaceUsed.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestDFCachingGetSpaceUsed.java
new file mode 100644
index 0000000..3def5d5
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestDFCachingGetSpaceUsed.java
@@ -0,0 +1,75 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs;
+
+import org.apache.commons.lang.RandomStringUtils;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+
+
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test to make sure df can run and work.
+ */
+public class TestDFCachingGetSpaceUsed {
+  final static private File DF_DIR = GenericTestUtils.getTestDir("testdfspace");
+  public static final int FILE_SIZE = 1024;
+
+  @Before
+  public void setUp() {
+    FileUtil.fullyDelete(DF_DIR);
+    assertTrue(DF_DIR.mkdirs());
+  }
+
+  @After
+  public void tearDown() throws IOException {
+    FileUtil.fullyDelete(DF_DIR);
+  }
+
+  @Test
+  public void testCanBuildRun() throws Exception {
+    File file = writeFile("testCanBuild");
+
+    GetSpaceUsed instance = new CachingGetSpaceUsed.Builder()
+        .setPath(file)
+        .setInterval(50060)
+        .setKlass(DFCachingGetSpaceUsed.class)
+        .build();
+    assertTrue(instance instanceof DFCachingGetSpaceUsed);
+    assertTrue(instance.getUsed() >= FILE_SIZE - 20);
+    ((DFCachingGetSpaceUsed) instance).close();
+  }
+
+  private File writeFile(String fileName) throws IOException {
+    File f = new File(DF_DIR, fileName);
+    assertTrue(f.createNewFile());
+    RandomAccessFile randomAccessFile = new RandomAccessFile(f, "rws");
+    randomAccessFile.writeUTF(RandomStringUtils.randomAlphabetic(FILE_SIZE));
+    randomAccessFile.getFD().sync();
+    randomAccessFile.close();
+    return f;
+  }
+
+}
\ 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


[39/50] [abbrv] hadoop git commit: YARN-3664. Federation PolicyStore internal APIs

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


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

Branch: refs/heads/YARN-2915
Commit: ddb50245c1e25da76e5bb92bcf81525f189ef2cb
Parents: 89f43e6
Author: Subru Krishnan <su...@apache.org>
Authored: Fri Aug 5 12:34:58 2016 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon Oct 3 14:15:36 2016 -0700

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


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

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

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

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

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

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

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

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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ddb50245/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/impl/pb/GetSubClustersInfoResponsePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/impl/pb/GetSubClustersInfoResponsePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/impl/pb/GetSubClustersInfoResponsePBImpl.java
index 92bdf06..2efa3b7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/impl/pb/GetSubClustersInfoResponsePBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/impl/pb/GetSubClustersInfoResponsePBImpl.java
@@ -66,7 +66,7 @@ public class GetSubClustersInfoResponsePBImpl
 
   private void mergeLocalToBuilder() {
     if (this.subClusterInfos != null) {
-      addReservationResourcesToProto();
+      addSubClusterInfosToProto();
     }
   }
 
@@ -114,7 +114,7 @@ public class GetSubClustersInfoResponsePBImpl
     }
   }
 
-  private void addReservationResourcesToProto() {
+  private void addSubClusterInfosToProto() {
     maybeInitBuilder();
     builder.clearSubClusterInfos();
     if (subClusterInfos == null) {

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

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

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

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

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


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


[35/50] [abbrv] hadoop git commit: YARN-5390. Federation Subcluster Resolver. Contributed by Ellen Hui.

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


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

Branch: refs/heads/YARN-2915
Commit: 5ebd7f46621bcbc51dc3923aba5c782358f6d875
Parents: c65ec75
Author: Subru Krishnan <su...@apache.org>
Authored: Thu Aug 4 15:58:31 2016 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon Oct 3 14:15:36 2016 -0700

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5ebd7f46/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 4d43357..a80bb2b 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
@@ -2480,6 +2480,14 @@ public class YarnConfiguration extends Configuration {
   public static final int DEFAULT_SHARED_CACHE_NM_UPLOADER_THREAD_COUNT = 20;
 
   ////////////////////////////////
+  // Federation Configs
+  ////////////////////////////////
+
+  public static final String FEDERATION_PREFIX = YARN_PREFIX + "federation.";
+  public static final String FEDERATION_MACHINE_LIST =
+      FEDERATION_PREFIX + "machine-list";
+
+  ////////////////////////////////
   // Other Configs
   ////////////////////////////////
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5ebd7f46/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 524afec..9351519 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
@@ -2624,6 +2624,13 @@
   <!-- Other Configuration -->
 
   <property>
+    <description>
+      Machine list file to be loaded by the FederationSubCluster Resolver
+    </description>
+    <name>yarn.federation.machine-list</name>
+  </property>
+
+  <property>
     <description>The interval that the yarn client library uses to poll the
     completion status of the asynchronous API of application client protocol.
     </description>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5ebd7f46/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 4216f76..c16747a 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
@@ -180,6 +180,16 @@
           </filesets>
         </configuration>
       </plugin>
+      <plugin>
+        <groupId>org.apache.rat</groupId>
+        <artifactId>apache-rat-plugin</artifactId>
+        <configuration>
+          <excludes>
+            <exclude>src/test/resources/nodes</exclude>
+            <exclude>src/test/resources/nodes-malformed</exclude>
+          </excludes>
+        </configuration>
+      </plugin>
     </plugins>
   </build>
 </project>

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

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

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

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

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

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

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


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


[21/50] [abbrv] hadoop git commit: HDFS-10923. Make InstrumentedLock require ReentrantLock.

Posted by su...@apache.org.
HDFS-10923. Make InstrumentedLock require ReentrantLock.


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

Branch: refs/heads/YARN-2915
Commit: c7ce6fdc20fe053f0bb3bcf900ffc0e1db6feee5
Parents: 3a3697d
Author: Arpit Agarwal <ar...@apache.org>
Authored: Fri Sep 30 23:00:06 2016 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Fri Sep 30 23:00:06 2016 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hdfs/InstrumentedLock.java    | 185 ------------------
 .../hadoop/hdfs/InstrumentedReentrantLock.java  | 195 +++++++++++++++++++
 .../datanode/fsdataset/impl/FsDatasetImpl.java  |   4 +-
 .../hadoop/hdfs/TestInstrumentedLock.java       | 166 ----------------
 .../hdfs/TestInstrumentedReentrantLock.java     | 177 +++++++++++++++++
 5 files changed, 374 insertions(+), 353 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c7ce6fdc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/InstrumentedLock.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/InstrumentedLock.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/InstrumentedLock.java
deleted file mode 100644
index 6279e95..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/InstrumentedLock.java
+++ /dev/null
@@ -1,185 +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
- * <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.hdfs;
-
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.locks.Condition;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantLock;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.commons.logging.Log;
-import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.util.Timer;
-
-import com.google.common.annotations.VisibleForTesting;
-
-/**
- * This is a debugging class that can be used by callers to track
- * whether a specifc lock is being held for too long and periodically
- * log a warning and stack trace, if so.
- *
- * The logged warnings are throttled so that logs are not spammed.
- *
- * A new instance of InstrumentedLock can be created for each object
- * that needs to be instrumented.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class InstrumentedLock implements Lock {
-
-  private final Lock lock;
-  private final Log logger;
-  private final String name;
-  private final Timer clock;
-
-  /** Minimum gap between two lock warnings. */
-  private final long minLoggingGap;
-  /** Threshold for detecting long lock held time. */
-  private final long lockWarningThreshold;
-
-  // Tracking counters for lock statistics.
-  private volatile long lockAcquireTimestamp;
-  private final AtomicLong lastLogTimestamp;
-  private final AtomicLong warningsSuppressed = new AtomicLong(0);
-
-  /**
-   * Create a instrumented lock instance which logs a warning message
-   * when lock held time is above given threshold.
-   *
-   * @param name the identifier of the lock object
-   * @param logger this class does not have its own logger, will log to the
-   *               given logger instead
-   * @param minLoggingGapMs  the minimum time gap between two log messages,
-   *                         this is to avoid spamming to many logs
-   * @param lockWarningThresholdMs the time threshold to view lock held
-   *                               time as being "too long"
-   */
-  public InstrumentedLock(String name, Log logger, long minLoggingGapMs,
-      long lockWarningThresholdMs) {
-    this(name, logger, new ReentrantLock(),
-        minLoggingGapMs, lockWarningThresholdMs);
-  }
-
-  public InstrumentedLock(String name, Log logger, Lock lock,
-      long minLoggingGapMs, long lockWarningThresholdMs) {
-    this(name, logger, lock,
-        minLoggingGapMs, lockWarningThresholdMs, new Timer());
-  }
-
-  @VisibleForTesting
-  InstrumentedLock(String name, Log logger, Lock lock,
-      long minLoggingGapMs, long lockWarningThresholdMs, Timer clock) {
-    this.name = name;
-    this.lock = lock;
-    this.clock = clock;
-    this.logger = logger;
-    minLoggingGap = minLoggingGapMs;
-    lockWarningThreshold = lockWarningThresholdMs;
-    lastLogTimestamp = new AtomicLong(
-      clock.monotonicNow() - Math.max(minLoggingGap, lockWarningThreshold));
-  }
-
-  @Override
-  public void lock() {
-    lock.lock();
-    lockAcquireTimestamp = clock.monotonicNow();
-  }
-
-  @Override
-  public void lockInterruptibly() throws InterruptedException {
-    lock.lockInterruptibly();
-    lockAcquireTimestamp = clock.monotonicNow();
-  }
-
-  @Override
-  public boolean tryLock() {
-    if (lock.tryLock()) {
-      lockAcquireTimestamp = clock.monotonicNow();
-      return true;
-    }
-    return false;
-  }
-
-  @Override
-  public boolean tryLock(long time, TimeUnit unit) throws InterruptedException {
-    if (lock.tryLock(time, unit)) {
-      lockAcquireTimestamp = clock.monotonicNow();
-      return true;
-    }
-    return false;
-  }
-
-  @Override
-  public void unlock() {
-    long localLockReleaseTime = clock.monotonicNow();
-    long localLockAcquireTime = lockAcquireTimestamp;
-    lock.unlock();
-    check(localLockAcquireTime, localLockReleaseTime);
-  }
-
-  @Override
-  public Condition newCondition() {
-    return lock.newCondition();
-  }
-
-  @VisibleForTesting
-  void logWarning(long lockHeldTime, long suppressed) {
-    logger.warn(String.format("Lock held time above threshold: " +
-        "lock identifier: %s " +
-        "lockHeldTimeMs=%d ms. Suppressed %d lock warnings. " +
-        "The stack trace is: %s" ,
-        name, lockHeldTime, suppressed,
-        StringUtils.getStackTrace(Thread.currentThread())));
-  }
-
-  /**
-   * Log a warning if the lock was held for too long.
-   *
-   * Should be invoked by the caller immediately AFTER releasing the lock.
-   *
-   * @param acquireTime  - timestamp just after acquiring the lock.
-   * @param releaseTime - timestamp just before releasing the lock.
-   */
-  private void check(long acquireTime, long releaseTime) {
-    if (!logger.isWarnEnabled()) {
-      return;
-    }
-
-    final long lockHeldTime = releaseTime - acquireTime;
-    if (lockWarningThreshold - lockHeldTime < 0) {
-      long now;
-      long localLastLogTs;
-      do {
-        now = clock.monotonicNow();
-        localLastLogTs = lastLogTimestamp.get();
-        long deltaSinceLastLog = now - localLastLogTs;
-        // check should print log or not
-        if (deltaSinceLastLog - minLoggingGap < 0) {
-          warningsSuppressed.incrementAndGet();
-          return;
-        }
-      } while (!lastLogTimestamp.compareAndSet(localLastLogTs, now));
-      long suppressed = warningsSuppressed.getAndSet(0);
-      logWarning(lockHeldTime, suppressed);
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c7ce6fdc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/InstrumentedReentrantLock.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/InstrumentedReentrantLock.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/InstrumentedReentrantLock.java
new file mode 100644
index 0000000..010571e
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/InstrumentedReentrantLock.java
@@ -0,0 +1,195 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.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.hdfs;
+
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.commons.logging.Log;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.Timer;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * This is a debugging class that can be used by callers to track
+ * whether a specific lock is being held for too long and periodically
+ * log a warning and stack trace, if so.
+ *
+ * The logged warnings are throttled so that logs are not spammed.
+ *
+ * A new instance of InstrumentedLock can be created for each object
+ * that needs to be instrumented.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class InstrumentedReentrantLock implements Lock {
+
+  @VisibleForTesting
+  final ReentrantLock lock;
+  private final Log logger;
+  private final String name;
+  private final Timer clock;
+
+  /** Minimum gap between two lock warnings. */
+  private final long minLoggingGap;
+  /** Threshold for detecting long lock held time. */
+  private final long lockWarningThreshold;
+
+  // Tracking counters for lock statistics.
+  private volatile long lockAcquireTimestamp;
+  private final AtomicLong lastLogTimestamp;
+  private final AtomicLong warningsSuppressed = new AtomicLong(0);
+
+  /**
+   * Create a instrumented lock instance which logs a warning message
+   * when lock held time is above given threshold.
+   *
+   * @param name the identifier of the lock object
+   * @param logger this class does not have its own logger, will log to the
+   *               given logger instead
+   * @param minLoggingGapMs  the minimum time gap between two log messages,
+   *                         this is to avoid spamming to many logs
+   * @param lockWarningThresholdMs the time threshold to view lock held
+   *                               time as being "too long"
+   */
+  public InstrumentedReentrantLock(
+      String name, Log logger, long minLoggingGapMs,
+      long lockWarningThresholdMs) {
+    this(name, logger, new ReentrantLock(),
+        minLoggingGapMs, lockWarningThresholdMs);
+  }
+
+  public InstrumentedReentrantLock(
+      String name, Log logger, ReentrantLock lock,
+      long minLoggingGapMs, long lockWarningThresholdMs) {
+    this(name, logger, lock,
+        minLoggingGapMs, lockWarningThresholdMs, new Timer());
+  }
+
+  @VisibleForTesting
+  InstrumentedReentrantLock(
+      String name, Log logger, ReentrantLock lock,
+      long minLoggingGapMs, long lockWarningThresholdMs, Timer clock) {
+    this.name = name;
+    this.lock = lock;
+    this.clock = clock;
+    this.logger = logger;
+    minLoggingGap = minLoggingGapMs;
+    lockWarningThreshold = lockWarningThresholdMs;
+    lastLogTimestamp = new AtomicLong(
+      clock.monotonicNow() - Math.max(minLoggingGap, lockWarningThreshold));
+  }
+
+  @Override
+  public void lock() {
+    lock.lock();
+    if (lock.getHoldCount() == 1) {
+      lockAcquireTimestamp = clock.monotonicNow();
+    }
+  }
+
+  @Override
+  public void lockInterruptibly() throws InterruptedException {
+    lock.lockInterruptibly();
+    if (lock.getHoldCount() == 1) {
+      lockAcquireTimestamp = clock.monotonicNow();
+    }
+  }
+
+  @Override
+  public boolean tryLock() {
+    if (lock.tryLock() && lock.getHoldCount() == 1) {
+      lockAcquireTimestamp = clock.monotonicNow();
+      return true;
+    }
+    return false;
+  }
+
+  @Override
+  public boolean tryLock(long time, TimeUnit unit) throws InterruptedException {
+    if (lock.tryLock(time, unit) && lock.getHoldCount() == 1) {
+      lockAcquireTimestamp = clock.monotonicNow();
+      return true;
+    }
+    return false;
+  }
+
+  @Override
+  public void unlock() {
+    final boolean needReport = (lock.getHoldCount() == 1);
+    long localLockReleaseTime = clock.monotonicNow();
+    long localLockAcquireTime = lockAcquireTimestamp;
+    lock.unlock();
+    if (needReport) {
+      check(localLockAcquireTime, localLockReleaseTime);
+    }
+  }
+
+  @Override
+  public Condition newCondition() {
+    return lock.newCondition();
+  }
+
+  @VisibleForTesting
+  void logWarning(long lockHeldTime, long suppressed) {
+    logger.warn(String.format("Lock held time above threshold: " +
+        "lock identifier: %s " +
+        "lockHeldTimeMs=%d ms. Suppressed %d lock warnings. " +
+        "The stack trace is: %s" ,
+        name, lockHeldTime, suppressed,
+        StringUtils.getStackTrace(Thread.currentThread())));
+  }
+
+  /**
+   * Log a warning if the lock was held for too long.
+   *
+   * Should be invoked by the caller immediately AFTER releasing the lock.
+   *
+   * @param acquireTime  - timestamp just after acquiring the lock.
+   * @param releaseTime - timestamp just before releasing the lock.
+   */
+  private void check(long acquireTime, long releaseTime) {
+    if (!logger.isWarnEnabled()) {
+      return;
+    }
+
+    final long lockHeldTime = releaseTime - acquireTime;
+    if (lockWarningThreshold - lockHeldTime < 0) {
+      long now;
+      long localLastLogTs;
+      do {
+        now = clock.monotonicNow();
+        localLastLogTs = lastLogTimestamp.get();
+        long deltaSinceLastLog = now - localLastLogTs;
+        // check should print log or not
+        if (deltaSinceLastLog - minLoggingGap < 0) {
+          warningsSuppressed.incrementAndGet();
+          return;
+        }
+      } while (!lastLogTimestamp.compareAndSet(localLastLogTs, now));
+      long suppressed = warningsSuppressed.getAndSet(0);
+      logWarning(lockHeldTime, suppressed);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c7ce6fdc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
index 26a2e9f..ab31f25 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
@@ -58,7 +58,7 @@ import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.ExtendedBlockId;
-import org.apache.hadoop.hdfs.InstrumentedLock;
+import org.apache.hadoop.hdfs.InstrumentedReentrantLock;
 import org.apache.hadoop.util.AutoCloseableLock;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
@@ -266,7 +266,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     this.conf = conf;
     this.smallBufferSize = DFSUtilClient.getSmallBufferSize(conf);
     this.datasetLock = new AutoCloseableLock(
-        new InstrumentedLock(getClass().getName(), LOG,
+        new InstrumentedReentrantLock(getClass().getName(), LOG,
           conf.getTimeDuration(
             DFSConfigKeys.DFS_LOCK_SUPPRESS_WARNING_INTERVAL_KEY,
             DFSConfigKeys.DFS_LOCK_SUPPRESS_WARNING_INTERVAL_DEFAULT,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c7ce6fdc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestInstrumentedLock.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestInstrumentedLock.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestInstrumentedLock.java
deleted file mode 100644
index f470688..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestInstrumentedLock.java
+++ /dev/null
@@ -1,166 +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
- * <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.hdfs;
-
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.concurrent.locks.Lock;
-
-import org.apache.hadoop.util.AutoCloseableLock;
-import org.apache.hadoop.util.Timer;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-import static org.mockito.Mockito.*;
-import static org.junit.Assert.*;
-
-/**
- * A test class for InstrumentedLock.
- */
-public class TestInstrumentedLock {
-
-  static final Log LOG = LogFactory.getLog(TestInstrumentedLock.class);
-
-  @Rule public TestName name = new TestName();
-
-  /**
-   * Test exclusive access of the lock.
-   * @throws Exception
-   */
-  @Test(timeout=10000)
-  public void testMultipleThread() throws Exception {
-    String testname = name.getMethodName();
-    InstrumentedLock lock = new InstrumentedLock(testname, LOG, 0, 300);
-    lock.lock();
-    try {
-      Thread competingThread = new Thread() {
-        @Override
-        public void run() {
-          assertFalse(lock.tryLock());
-        }
-      };
-      competingThread.start();
-      competingThread.join();
-    } finally {
-      lock.unlock();
-    }
-  }
-
-  /**
-   * Test the correctness with try-with-resource syntax.
-   * @throws Exception
-   */
-  @Test(timeout=10000)
-  public void testTryWithResourceSyntax() throws Exception {
-    String testname = name.getMethodName();
-    final AtomicReference<Thread> lockThread = new AtomicReference<>(null);
-    Lock lock = new InstrumentedLock(testname, LOG, 0, 300) {
-      @Override
-      public void lock() {
-        super.lock();
-        lockThread.set(Thread.currentThread());
-      }
-      @Override
-      public void unlock() {
-        super.unlock();
-        lockThread.set(null);
-      }
-    };
-    AutoCloseableLock acl = new AutoCloseableLock(lock);
-    try (AutoCloseable localLock = acl.acquire()) {
-      assertEquals(acl, localLock);
-      Thread competingThread = new Thread() {
-        @Override
-        public void run() {
-          assertNotEquals(Thread.currentThread(), lockThread.get());
-          assertFalse(lock.tryLock());
-        }
-      };
-      competingThread.start();
-      competingThread.join();
-      assertEquals(Thread.currentThread(), lockThread.get());
-    }
-    assertNull(lockThread.get());
-  }
-
-  /**
-   * Test the lock logs warning when lock held time is greater than threshold
-   * and not log warning otherwise.
-   * @throws Exception
-   */
-  @Test(timeout=10000)
-  public void testLockLongHoldingReport() throws Exception {
-    String testname = name.getMethodName();
-    final AtomicLong time = new AtomicLong(0);
-    Timer mclock = new Timer() {
-      @Override
-      public long monotonicNow() {
-        return time.get();
-      }
-    };
-    Lock mlock = mock(Lock.class);
-
-    final AtomicLong wlogged = new AtomicLong(0);
-    final AtomicLong wsuppresed = new AtomicLong(0);
-    InstrumentedLock lock = new InstrumentedLock(
-        testname, LOG, mlock, 2000, 300, mclock) {
-      @Override
-      void logWarning(long lockHeldTime, long suppressed) {
-        wlogged.incrementAndGet();
-        wsuppresed.set(suppressed);
-      }
-    };
-
-    // do not log warning when the lock held time is short
-    lock.lock();   // t = 0
-    time.set(200);
-    lock.unlock(); // t = 200
-    assertEquals(0, wlogged.get());
-    assertEquals(0, wsuppresed.get());
-
-    lock.lock();   // t = 200
-    time.set(700);
-    lock.unlock(); // t = 700
-    assertEquals(1, wlogged.get());
-    assertEquals(0, wsuppresed.get());
-
-    // despite the lock held time is greater than threshold
-    // suppress the log warning due to the logging gap
-    // (not recorded in wsuppressed until next log message)
-    lock.lock();   // t = 700
-    time.set(1100);
-    lock.unlock(); // t = 1100
-    assertEquals(1, wlogged.get());
-    assertEquals(0, wsuppresed.get());
-
-    // log a warning message when the lock held time is greater the threshold
-    // and the logging time gap is satisfied. Also should display suppressed
-    // previous warnings.
-    time.set(2400);
-    lock.lock();   // t = 2400
-    time.set(2800);
-    lock.unlock(); // t = 2800
-    assertEquals(2, wlogged.get());
-    assertEquals(1, wsuppresed.get());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c7ce6fdc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestInstrumentedReentrantLock.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestInstrumentedReentrantLock.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestInstrumentedReentrantLock.java
new file mode 100644
index 0000000..3374b8a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestInstrumentedReentrantLock.java
@@ -0,0 +1,177 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.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.hdfs;
+
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+import org.apache.hadoop.util.AutoCloseableLock;
+import org.apache.hadoop.util.FakeTimer;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+import static org.mockito.Mockito.*;
+import static org.junit.Assert.*;
+
+/**
+ * A test class for {@link InstrumentedReentrantLock}.
+ */
+public class TestInstrumentedReentrantLock {
+
+  static final Log LOG = LogFactory.getLog(TestInstrumentedReentrantLock.class);
+
+  @Rule public TestName name = new TestName();
+
+  /**
+   * Test exclusive access of the lock.
+   * @throws Exception
+   */
+  @Test(timeout=10000)
+  public void testMultipleThread() throws Exception {
+    String testname = name.getMethodName();
+    InstrumentedReentrantLock lock =
+        new InstrumentedReentrantLock(testname, LOG, 0, 300);
+    lock.lock();
+    try {
+      Thread competingThread = new Thread() {
+        @Override
+        public void run() {
+          assertFalse(lock.tryLock());
+        }
+      };
+      competingThread.start();
+      competingThread.join();
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  /**
+   * Test the correctness with try-with-resource syntax.
+   * @throws Exception
+   */
+  @Test(timeout=10000)
+  public void testTryWithResourceSyntax() throws Exception {
+    String testname = name.getMethodName();
+    final AtomicReference<Thread> lockThread = new AtomicReference<>(null);
+    Lock lock = new InstrumentedReentrantLock(testname, LOG, 0, 300) {
+      @Override
+      public void lock() {
+        super.lock();
+        lockThread.set(Thread.currentThread());
+      }
+      @Override
+      public void unlock() {
+        super.unlock();
+        lockThread.set(null);
+      }
+    };
+    AutoCloseableLock acl = new AutoCloseableLock(lock);
+    try (AutoCloseable localLock = acl.acquire()) {
+      assertEquals(acl, localLock);
+      Thread competingThread = new Thread() {
+        @Override
+        public void run() {
+          assertNotEquals(Thread.currentThread(), lockThread.get());
+          assertFalse(lock.tryLock());
+        }
+      };
+      competingThread.start();
+      competingThread.join();
+      assertEquals(Thread.currentThread(), lockThread.get());
+    }
+    assertNull(lockThread.get());
+  }
+
+  /**
+   * Test the lock logs warning when lock held time is greater than threshold
+   * and not log warning otherwise.
+   * @throws Exception
+   */
+  @Test(timeout=10000)
+  public void testLockLongHoldingReport() throws Exception {
+    String testname = name.getMethodName();
+    FakeTimer mclock = new FakeTimer();
+    final int warningThreshold = 500;
+    final int minLoggingGap = warningThreshold * 10;
+
+    final AtomicLong wlogged = new AtomicLong(0);
+    final AtomicLong wsuppresed = new AtomicLong(0);
+    InstrumentedReentrantLock lock = new InstrumentedReentrantLock(
+        testname, LOG, new ReentrantLock(), minLoggingGap,
+        warningThreshold, mclock) {
+      @Override
+      void logWarning(long lockHeldTime, long suppressed) {
+        wlogged.incrementAndGet();
+        wsuppresed.set(suppressed);
+      }
+    };
+
+    // do not log warning when the lock held time is <= warningThreshold.
+    lock.lock();
+    mclock.advance(warningThreshold);
+    lock.unlock();
+    assertEquals(0, wlogged.get());
+    assertEquals(0, wsuppresed.get());
+
+    // log a warning when the lock held time exceeds the threshold.
+    lock.lock();
+    mclock.advance(warningThreshold + 1);
+    assertEquals(1, lock.lock.getHoldCount());
+    lock.unlock();
+    assertEquals(1, wlogged.get());
+    assertEquals(0, wsuppresed.get());
+
+    // despite the lock held time is greater than threshold
+    // suppress the log warning due to the logging gap
+    // (not recorded in wsuppressed until next log message)
+    lock.lock();
+    mclock.advance(warningThreshold + 1);
+    lock.unlock();
+    assertEquals(1, wlogged.get());
+    assertEquals(0, wsuppresed.get());
+
+    // log a warning message when the lock held time is greater the threshold
+    // and the logging time gap is satisfied. Also should display suppressed
+    // previous warnings.
+    lock.lock();
+    mclock.advance(minLoggingGap + 1);
+    lock.unlock(); // t = 2800
+    assertEquals(2, wlogged.get());
+    assertEquals(1, wsuppresed.get());
+
+    // Ensure that nested acquisitions do not log.
+    wlogged.set(0);
+    wsuppresed.set(0);
+    lock.lock();
+    lock.lock();
+    mclock.advance(minLoggingGap + 1);
+    lock.unlock();
+    assertEquals(0, wlogged.get());    // No warnings on nested release.
+    assertEquals(0, wsuppresed.get());
+    lock.unlock();
+    assertEquals(1, wlogged.get());    // Last release immediately logs.
+    assertEquals(0, wsuppresed.get());
+  }
+}


---------------------------------------------------------------------
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-3662. Federation Membership State Store internal APIs.

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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


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


[10/50] [abbrv] hadoop git commit: HDFS-10850. getEZForPath should NOT throw FNF. Contributed by Andrew Wang.

Posted by su...@apache.org.
HDFS-10850. getEZForPath should NOT throw FNF. Contributed by Andrew Wang.


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

Branch: refs/heads/YARN-2915
Commit: 0670149c88852cd7c4d6774bff06c7c588558739
Parents: 82c55dc
Author: Kihwal Lee <ki...@apache.org>
Authored: Fri Sep 30 08:44:18 2016 -0500
Committer: Kihwal Lee <ki...@apache.org>
Committed: Fri Sep 30 08:44:18 2016 -0500

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |  4 +-
 .../apache/hadoop/hdfs/client/HdfsAdmin.java    |  7 +--
 .../server/namenode/FSDirEncryptionZoneOp.java  |  4 --
 .../apache/hadoop/hdfs/TestEncryptionZones.java | 59 +++++++-------------
 4 files changed, 24 insertions(+), 50 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0670149c/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index 4c2a967..93c0ff0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -2599,8 +2599,8 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     try (TraceScope ignored = newPathTraceScope("getEZForPath", src)) {
       return namenode.getEZForPath(src);
     } catch (RemoteException re) {
-      throw re.unwrapRemoteException(FileNotFoundException.class,
-          AccessControlException.class, UnresolvedPathException.class);
+      throw re.unwrapRemoteException(AccessControlException.class,
+          UnresolvedPathException.class);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0670149c/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java
index 946b79d..bac2809 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java
@@ -328,14 +328,13 @@ public class HdfsAdmin {
    * Get the path of the encryption zone for a given file or directory.
    *
    * @param path The path to get the ez for.
-   *
-   * @return The EncryptionZone of the ez, or null if path is not in an ez.
+   * @return An EncryptionZone, or null if path does not exist or is not in an
+   * ez.
    * @throws IOException            if there was a general IO exception
    * @throws AccessControlException if the caller does not have access to path
-   * @throws FileNotFoundException  if the path does not exist
    */
   public EncryptionZone getEncryptionZoneForPath(Path path)
-    throws IOException, AccessControlException, FileNotFoundException {
+      throws IOException, AccessControlException {
     return dfs.getEZForPath(path);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0670149c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirEncryptionZoneOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirEncryptionZoneOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirEncryptionZoneOp.java
index 7501fc3..5457f08 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirEncryptionZoneOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirEncryptionZoneOp.java
@@ -19,7 +19,6 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.CRYPTO_XATTR_FILE_ENCRYPTION_INFO;
 
-import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.security.GeneralSecurityException;
 import java.security.PrivilegedExceptionAction;
@@ -184,9 +183,6 @@ final class FSDirEncryptionZoneOp {
     fsd.readLock();
     try {
       iip = fsd.resolvePath(pc, srcArg);
-      if (iip.getLastINode() == null) {
-        throw new FileNotFoundException("Path not found: " + iip.getPath());
-      }
       if (fsd.isPermissionEnabled()) {
         fsd.checkPathAccess(pc, iip, FsAction.READ);
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0670149c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java
index 9168ca6..18a0800 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java
@@ -97,6 +97,7 @@ import org.junit.Before;
 import org.junit.Test;
 import org.mockito.Mockito;
 
+import static org.junit.Assert.assertNotNull;
 import static org.mockito.Matchers.anyBoolean;
 import static org.mockito.Matchers.anyLong;
 import static org.mockito.Matchers.anyObject;
@@ -604,13 +605,8 @@ public class TestEncryptionZones {
           assertExceptionContains("Permission denied:", e);
         }
 
-        try {
-          userAdmin.getEncryptionZoneForPath(nonexistent);
-          fail("FileNotFoundException should be thrown for a non-existent"
-              + " file path");
-        } catch (FileNotFoundException e) {
-          assertExceptionContains("Path not found: " + nonexistent, e);
-        }
+        assertNull("expected null for nonexistent path",
+            userAdmin.getEncryptionZoneForPath(nonexistent));
 
         // Check operation with non-ez paths
         assertNull("expected null for non-ez path",
@@ -638,20 +634,10 @@ public class TestEncryptionZones {
         assertEquals("expected ez path", allPath.toString(),
             userAdmin.getEncryptionZoneForPath(
                 new Path(snapshottedAllPath)).getPath().toString());
-        try {
-          userAdmin.getEncryptionZoneForPath(allPathFile);
-          fail("FileNotFoundException should be thrown for a non-existent"
-              + " file path");
-        } catch (FileNotFoundException e) {
-          assertExceptionContains("Path not found: " + allPathFile, e);
-        }
-        try {
-          userAdmin.getEncryptionZoneForPath(allPath);
-          fail("FileNotFoundException should be thrown for a non-existent"
-              + " file path");
-        } catch (FileNotFoundException e) {
-          assertExceptionContains("Path not found: " + allPath, e);
-        }
+        assertNull("expected null for deleted file path",
+            userAdmin.getEncryptionZoneForPath(allPathFile));
+        assertNull("expected null for deleted directory path",
+            userAdmin.getEncryptionZoneForPath(allPath));
         return null;
       }
     });
@@ -1498,25 +1484,18 @@ public class TestEncryptionZones {
   }
 
   @Test(timeout = 60000)
-  public void testGetEncryptionZoneOnANonExistentZoneFile() throws Exception {
-    final Path ez = new Path("/ez");
-    fs.mkdirs(ez);
-    dfsAdmin.createEncryptionZone(ez, TEST_KEY, NO_TRASH);
-    Path zoneFile = new Path(ez, "file");
-    try {
-      fs.getEZForPath(zoneFile);
-      fail("FileNotFoundException should be thrown for a non-existent"
-          + " file path");
-    } catch (FileNotFoundException e) {
-      assertExceptionContains("Path not found: " + zoneFile, e);
-    }
-    try {
-      dfsAdmin.getEncryptionZoneForPath(zoneFile);
-      fail("FileNotFoundException should be thrown for a non-existent"
-          + " file path");
-    } catch (FileNotFoundException e) {
-      assertExceptionContains("Path not found: " + zoneFile, e);
-    }
+  public void testGetEncryptionZoneOnANonExistentPaths() throws Exception {
+    final Path ezPath = new Path("/ez");
+    fs.mkdirs(ezPath);
+    dfsAdmin.createEncryptionZone(ezPath, TEST_KEY, NO_TRASH);
+    Path zoneFile = new Path(ezPath, "file");
+    EncryptionZone ez = fs.getEZForPath(zoneFile);
+    assertNotNull("Expected EZ for non-existent path in EZ", ez);
+    ez = dfsAdmin.getEncryptionZoneForPath(zoneFile);
+    assertNotNull("Expected EZ for non-existent path in EZ", ez);
+    ez = dfsAdmin.getEncryptionZoneForPath(
+        new Path("/does/not/exist"));
+    assertNull("Expected null for non-existent path not in EZ", ez);
   }
 
   @Test(timeout = 120000)


---------------------------------------------------------------------
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-3662. Federation Membership State Store internal APIs.

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


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

Branch: refs/heads/YARN-2915
Commit: a48c8e65ed23fd546e33950a70a1e6a36f25cb6a
Parents: ad03510
Author: Subru Krishnan <su...@apache.org>
Authored: Fri Jul 29 16:53:40 2016 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon Oct 3 14:15:36 2016 -0700

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a48c8e65/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/BasePBImplRecordsTest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/BasePBImplRecordsTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/BasePBImplRecordsTest.java
new file mode 100644
index 0000000..98d8222
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/BasePBImplRecordsTest.java
@@ -0,0 +1,263 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.api;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.junit.Assert;
+
+import java.lang.reflect.*;
+import java.nio.ByteBuffer;
+import java.util.*;
+
+/**
+ * Generic helper class to validate protocol records.
+ */
+public class BasePBImplRecordsTest {
+  static final Log LOG = LogFactory.getLog(BasePBImplRecordsTest.class);
+
+  protected static HashMap<Type, Object> typeValueCache =
+      new HashMap<Type, Object>();
+  private static Random rand = new Random();
+  private static byte [] bytes = new byte[] {'1', '2', '3', '4'};
+
+  @SuppressWarnings({"rawtypes", "unchecked"})
+  private static Object genTypeValue(Type type) {
+    Object ret = typeValueCache.get(type);
+    if (ret != null) {
+      return ret;
+    }
+    // only use positive primitive values
+    if (type.equals(boolean.class)) {
+      return rand.nextBoolean();
+    } else if (type.equals(byte.class)) {
+      return bytes[rand.nextInt(4)];
+    } else if (type.equals(int.class) || type.equals(Integer.class)) {
+      return rand.nextInt(1000000);
+    } else if (type.equals(long.class)) {
+      return Long.valueOf(rand.nextInt(1000000));
+    } else if (type.equals(float.class)) {
+      return rand.nextFloat();
+    } else if (type.equals(double.class)) {
+      return rand.nextDouble();
+    } else if (type.equals(String.class)) {
+      return String.format("%c%c%c",
+          'a' + rand.nextInt(26),
+          'a' + rand.nextInt(26),
+          'a' + rand.nextInt(26));
+    } else if (type instanceof Class) {
+      Class clazz = (Class)type;
+      if (clazz.isArray()) {
+        Class compClass = clazz.getComponentType();
+        if (compClass != null) {
+          ret = Array.newInstance(compClass, 2);
+          Array.set(ret, 0, genTypeValue(compClass));
+          Array.set(ret, 1, genTypeValue(compClass));
+        }
+      } else if (clazz.isEnum()) {
+        Object [] values = clazz.getEnumConstants();
+        ret = values[rand.nextInt(values.length)];
+      } else if (clazz.equals(ByteBuffer.class)) {
+        // return new ByteBuffer every time
+        // to prevent potential side effects
+        ByteBuffer buff = ByteBuffer.allocate(4);
+        rand.nextBytes(buff.array());
+        return buff;
+      }
+    } else if (type instanceof ParameterizedType) {
+      ParameterizedType pt = (ParameterizedType)type;
+      Type rawType = pt.getRawType();
+      Type [] params = pt.getActualTypeArguments();
+      // only support EnumSet<T>, List<T>, Set<T>, Map<K,V>
+      if (rawType.equals(EnumSet.class)) {
+        if (params[0] instanceof Class) {
+          Class c = (Class)(params[0]);
+          return EnumSet.allOf(c);
+        }
+      } if (rawType.equals(List.class)) {
+        ret = Lists.newArrayList(genTypeValue(params[0]));
+      } else if (rawType.equals(Set.class)) {
+        ret = Sets.newHashSet(genTypeValue(params[0]));
+      } else if (rawType.equals(Map.class)) {
+        Map<Object, Object> map = Maps.newHashMap();
+        map.put(genTypeValue(params[0]), genTypeValue(params[1]));
+        ret = map;
+      }
+    }
+    if (ret == null) {
+      throw new IllegalArgumentException("type " + type + " is not supported");
+    }
+    typeValueCache.put(type, ret);
+    return ret;
+  }
+
+  /**
+   * this method generate record instance by calling newIntance
+   * using reflection, add register the generated value to typeValueCache
+   */
+  @SuppressWarnings("rawtypes")
+  protected static Object generateByNewInstance(Class clazz) throws Exception {
+    Object ret = typeValueCache.get(clazz);
+    if (ret != null) {
+      return ret;
+    }
+    Method newInstance = null;
+    Type [] paramTypes = new Type[0];
+    // get newInstance method with most parameters
+    for (Method m : clazz.getMethods()) {
+      int mod = m.getModifiers();
+      if (m.getDeclaringClass().equals(clazz) &&
+          Modifier.isPublic(mod) &&
+          Modifier.isStatic(mod) &&
+          m.getName().equals("newInstance")) {
+        Type [] pts = m.getGenericParameterTypes();
+        if (newInstance == null
+            || (pts.length > paramTypes.length)) {
+          newInstance = m;
+          paramTypes = pts;
+        }
+      }
+    }
+    if (newInstance == null) {
+      throw new IllegalArgumentException("type " + clazz.getName() +
+          " does not have newInstance method");
+    }
+    Object [] args = new Object[paramTypes.length];
+    for (int i=0;i<args.length;i++) {
+      args[i] = genTypeValue(paramTypes[i]);
+    }
+    ret = newInstance.invoke(null, args);
+    typeValueCache.put(clazz, ret);
+    return ret;
+  }
+
+  private class GetSetPair {
+    public String propertyName;
+    public Method getMethod;
+    public Method setMethod;
+    public Type type;
+    public Object testValue;
+
+    @Override
+    public String toString() {
+      return String.format("{ name=%s, class=%s, value=%s }", propertyName,
+          type, testValue);
+    }
+  }
+
+  private <R> Map<String, GetSetPair> getGetSetPairs(Class<R> recordClass)
+      throws Exception {
+    Map<String, GetSetPair> ret = new HashMap<String, GetSetPair>();
+    Method [] methods = recordClass.getDeclaredMethods();
+    // get all get methods
+    for (int i = 0; i < methods.length; i++) {
+      Method m = methods[i];
+      int mod = m.getModifiers();
+      if (m.getDeclaringClass().equals(recordClass) &&
+          Modifier.isPublic(mod) &&
+          (!Modifier.isStatic(mod))) {
+        String name = m.getName();
+        if (name.equals("getProto")) {
+          continue;
+        }
+        if ((name.length() > 3) && name.startsWith("get") &&
+            (m.getParameterTypes().length == 0)) {
+          String propertyName = name.substring(3);
+          Type valueType = m.getGenericReturnType();
+          GetSetPair p = ret.get(propertyName);
+          if (p == null) {
+            p = new GetSetPair();
+            p.propertyName = propertyName;
+            p.type = valueType;
+            p.getMethod = m;
+            ret.put(propertyName, p);
+          } else {
+            Assert.fail("Multiple get method with same name: " + recordClass
+                + p.propertyName);
+          }
+        }
+      }
+    }
+    // match get methods with set methods
+    for (int i = 0; i < methods.length; i++) {
+      Method m = methods[i];
+      int mod = m.getModifiers();
+      if (m.getDeclaringClass().equals(recordClass) &&
+          Modifier.isPublic(mod) &&
+          (!Modifier.isStatic(mod))) {
+        String name = m.getName();
+        if (name.startsWith("set") && (m.getParameterTypes().length == 1)) {
+          String propertyName = name.substring(3);
+          Type valueType = m.getGenericParameterTypes()[0];
+          GetSetPair p = ret.get(propertyName);
+          if (p != null && p.type.equals(valueType)) {
+            p.setMethod = m;
+          }
+        }
+      }
+    }
+    // exclude incomplete get/set pair, and generate test value
+    Iterator<Map.Entry<String, GetSetPair>> itr = ret.entrySet().iterator();
+    while (itr.hasNext()) {
+      Map.Entry<String, GetSetPair> cur = itr.next();
+      GetSetPair gsp = cur.getValue();
+      if ((gsp.getMethod == null) ||
+          (gsp.setMethod == null)) {
+        LOG.info(String.format("Exclude protential property: %s\n", gsp.propertyName));
+        itr.remove();
+      } else {
+        LOG.info(String.format("New property: %s type: %s", gsp.toString(), gsp.type));
+        gsp.testValue = genTypeValue(gsp.type);
+        LOG.info(String.format(" testValue: %s\n", gsp.testValue));
+      }
+    }
+    return ret;
+  }
+
+  protected  <R, P> void validatePBImplRecord(Class<R> recordClass,
+      Class<P> protoClass)
+      throws Exception {
+    LOG.info(String.format("Validate %s %s\n", recordClass.getName(),
+        protoClass.getName()));
+    Constructor<R> emptyConstructor = recordClass.getConstructor();
+    Constructor<R> pbConstructor = recordClass.getConstructor(protoClass);
+    Method getProto = recordClass.getDeclaredMethod("getProto");
+    Map<String, GetSetPair> getSetPairs = getGetSetPairs(recordClass);
+    R origRecord = emptyConstructor.newInstance();
+    for (GetSetPair gsp : getSetPairs.values()) {
+      gsp.setMethod.invoke(origRecord, gsp.testValue);
+    }
+    Object ret = getProto.invoke(origRecord);
+    Assert.assertNotNull(recordClass.getName() + "#getProto returns null", ret);
+    if (!(protoClass.isAssignableFrom(ret.getClass()))) {
+      Assert.fail("Illegal getProto method return type: " + ret.getClass());
+    }
+    R deserRecord = pbConstructor.newInstance(ret);
+    Assert.assertEquals("whole " + recordClass + " records should be equal",
+        origRecord, deserRecord);
+    for (GetSetPair gsp : getSetPairs.values()) {
+      Object origValue = gsp.getMethod.invoke(origRecord);
+      Object deserValue = gsp.getMethod.invoke(deserRecord);
+      Assert.assertEquals("property " + recordClass.getName() + "#"
+          + gsp.propertyName + " should be equal", origValue, deserValue);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a48c8e65/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
index 5270486..4b71282 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
@@ -16,26 +16,8 @@
  * limitations under the License.
  */
 package org.apache.hadoop.yarn.api;
-import java.io.IOException;
-import java.lang.reflect.Array;
-import java.lang.reflect.Constructor;
-import java.lang.reflect.Method;
-import java.lang.reflect.Modifier;
-import java.lang.reflect.ParameterizedType;
-import java.lang.reflect.Type;
-import java.nio.ByteBuffer;
-import java.util.EnumSet;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Random;
-import java.util.Set;
-
+import com.google.common.collect.ImmutableSet;
 import org.apache.commons.lang.math.LongRange;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.security.proto.SecurityProtos.CancelDelegationTokenRequestProto;
 import org.apache.hadoop.security.proto.SecurityProtos.CancelDelegationTokenResponseProto;
 import org.apache.hadoop.security.proto.SecurityProtos.GetDelegationTokenRequestProto;
@@ -343,127 +325,12 @@ import org.junit.BeforeClass;
 import org.junit.Ignore;
 import org.junit.Test;
 
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
-
-public class TestPBImplRecords {
-  static final Log LOG = LogFactory.getLog(TestPBImplRecords.class);
-
-  private static HashMap<Type, Object> typeValueCache = new HashMap<Type, Object>();
-  private static Random rand = new Random();
-  private static byte [] bytes = new byte[] {'1', '2', '3', '4'};
-
-  @SuppressWarnings({"rawtypes", "unchecked"})
-  private static Object genTypeValue(Type type) {
-    Object ret = typeValueCache.get(type);
-    if (ret != null) {
-      return ret;
-    }
-    // only use positive primitive values
-    if (type.equals(boolean.class)) {
-      return rand.nextBoolean();
-    } else if (type.equals(byte.class)) {
-      return bytes[rand.nextInt(4)];
-    } else if (type.equals(int.class) || type.equals(Integer.class)) {
-      return rand.nextInt(1000000);
-    } else if (type.equals(long.class) || type.equals(Long.class)) {
-      return Long.valueOf(rand.nextInt(1000000));
-    } else if (type.equals(float.class)) {
-      return rand.nextFloat();
-    } else if (type.equals(double.class)) {
-      return rand.nextDouble();
-    } else if (type.equals(String.class)) {
-      return String.format("%c%c%c",
-          'a' + rand.nextInt(26),
-          'a' + rand.nextInt(26),
-          'a' + rand.nextInt(26));
-    } else if (type instanceof Class) {
-      Class clazz = (Class)type;
-      if (clazz.isArray()) {
-        Class compClass = clazz.getComponentType();
-        if (compClass != null) {
-          ret = Array.newInstance(compClass, 2);
-          Array.set(ret, 0, genTypeValue(compClass));
-          Array.set(ret, 1, genTypeValue(compClass));
-        }
-      } else if (clazz.isEnum()) {
-        Object [] values = clazz.getEnumConstants();
-        ret = values[rand.nextInt(values.length)];
-      } else if (clazz.equals(ByteBuffer.class)) {
-        // return new ByteBuffer every time
-        // to prevent potential side effects
-        ByteBuffer buff = ByteBuffer.allocate(4);
-        rand.nextBytes(buff.array());
-        return buff;
-      }
-    } else if (type instanceof ParameterizedType) {
-      ParameterizedType pt = (ParameterizedType)type;
-      Type rawType = pt.getRawType();
-      Type [] params = pt.getActualTypeArguments();
-      // only support EnumSet<T>, List<T>, Set<T>, Map<K,V>
-      if (rawType.equals(EnumSet.class)) {
-        if (params[0] instanceof Class) {
-          Class c = (Class)(params[0]);
-          return EnumSet.allOf(c);
-        }
-      } if (rawType.equals(List.class)) {
-        ret = Lists.newArrayList(genTypeValue(params[0]));
-      } else if (rawType.equals(Set.class)) {
-        ret = Sets.newHashSet(genTypeValue(params[0]));
-      } else if (rawType.equals(Map.class)) {
-        Map<Object, Object> map = Maps.newHashMap();
-        map.put(genTypeValue(params[0]), genTypeValue(params[1]));
-        ret = map;
-      }
-    }
-    if (ret == null) {
-      throw new IllegalArgumentException("type " + type + " is not supported");
-    }
-    typeValueCache.put(type, ret);
-    return ret;
-  }
-
-  /**
-   * this method generate record instance by calling newIntance
-   * using reflection, add register the generated value to typeValueCache
-   */
-  @SuppressWarnings("rawtypes")
-  private static Object generateByNewInstance(Class clazz) throws Exception {
-    Object ret = typeValueCache.get(clazz);
-    if (ret != null) {
-      return ret;
-    }
-    Method newInstance = null;
-    Type [] paramTypes = new Type[0];
-    // get newInstance method with most parameters
-    for (Method m : clazz.getMethods()) {
-      int mod = m.getModifiers();
-      if (m.getDeclaringClass().equals(clazz) &&
-          Modifier.isPublic(mod) &&
-          Modifier.isStatic(mod) &&
-          m.getName().equals("newInstance")) {
-        Type [] pts = m.getGenericParameterTypes();
-        if (newInstance == null
-            || (pts.length > paramTypes.length)) {
-          newInstance = m;
-          paramTypes = pts;
-        }
-      }
-    }
-    if (newInstance == null) {
-      throw new IllegalArgumentException("type " + clazz.getName() +
-          " does not have newInstance method");
-    }
-    Object [] args = new Object[paramTypes.length];
-    for (int i=0;i<args.length;i++) {
-      args[i] = genTypeValue(paramTypes[i]);
-    }
-    ret = newInstance.invoke(null, args);
-    typeValueCache.put(clazz, ret);
-    return ret;
-  }
+import java.io.IOException;
+
+/**
+ * Test class for YARN API protocol records.
+ */
+public class TestPBImplRecords extends BasePBImplRecordsTest {
 
   @BeforeClass
   public static void setup() throws Exception {
@@ -532,118 +399,6 @@ public class TestPBImplRecords {
     generateByNewInstance(CommitResponse.class);
   }
 
-  private class GetSetPair {
-    public String propertyName;
-    public Method getMethod;
-    public Method setMethod;
-    public Type type;
-    public Object testValue;
-
-    @Override
-    public String toString() {
-      return String.format("{ name=%s, class=%s, value=%s }", propertyName,
-          type, testValue);
-    }
-  }
-
-  private <R> Map<String, GetSetPair> getGetSetPairs(Class<R> recordClass)
-      throws Exception {
-    Map<String, GetSetPair> ret = new HashMap<String, GetSetPair>();
-    Method [] methods = recordClass.getDeclaredMethods();
-    // get all get methods
-    for (int i = 0; i < methods.length; i++) {
-      Method m = methods[i];
-      int mod = m.getModifiers();
-      if (m.getDeclaringClass().equals(recordClass) &&
-          Modifier.isPublic(mod) &&
-          (!Modifier.isStatic(mod))) {
-        String name = m.getName();
-        if (name.equals("getProto")) {
-          continue;
-        }
-        if ((name.length() > 3) && name.startsWith("get") &&
-            (m.getParameterTypes().length == 0)) {
-          String propertyName = name.substring(3);
-          Type valueType = m.getGenericReturnType();
-          GetSetPair p = ret.get(propertyName);
-          if (p == null) {
-            p = new GetSetPair();
-            p.propertyName = propertyName;
-            p.type = valueType;
-            p.getMethod = m;
-            ret.put(propertyName, p);
-          } else {
-            Assert.fail("Multiple get method with same name: " + recordClass
-                + p.propertyName);
-          }
-        }
-      }
-    }
-    // match get methods with set methods
-    for (int i = 0; i < methods.length; i++) {
-      Method m = methods[i];
-      int mod = m.getModifiers();
-      if (m.getDeclaringClass().equals(recordClass) &&
-          Modifier.isPublic(mod) &&
-          (!Modifier.isStatic(mod))) {
-        String name = m.getName();
-        if (name.startsWith("set") && (m.getParameterTypes().length == 1)) {
-          String propertyName = name.substring(3);
-          Type valueType = m.getGenericParameterTypes()[0];
-          GetSetPair p = ret.get(propertyName);
-          if (p != null && p.type.equals(valueType)) {
-            p.setMethod = m;
-          }
-        }
-      }
-    }
-    // exclude incomplete get/set pair, and generate test value
-    Iterator<Entry<String, GetSetPair>> itr = ret.entrySet().iterator();
-    while (itr.hasNext()) {
-      Entry<String, GetSetPair> cur = itr.next();
-      GetSetPair gsp = cur.getValue();
-      if ((gsp.getMethod == null) ||
-          (gsp.setMethod == null)) {
-        LOG.info(String.format("Exclude protential property: %s\n", gsp.propertyName));
-        itr.remove();
-      } else {
-        LOG.info(String.format("New property: %s type: %s", gsp.toString(), gsp.type));
-        gsp.testValue = genTypeValue(gsp.type);
-        LOG.info(String.format(" testValue: %s\n", gsp.testValue));
-      }
-    }
-    return ret;
-  }
-
-  private <R, P> void validatePBImplRecord(Class<R> recordClass,
-      Class<P> protoClass)
-      throws Exception {
-    LOG.info(String.format("Validate %s %s\n", recordClass.getName(),
-        protoClass.getName()));
-    Constructor<R> emptyConstructor = recordClass.getConstructor();
-    Constructor<R> pbConstructor = recordClass.getConstructor(protoClass);
-    Method getProto = recordClass.getDeclaredMethod("getProto");
-    Map<String, GetSetPair> getSetPairs = getGetSetPairs(recordClass);
-    R origRecord = emptyConstructor.newInstance();
-    for (GetSetPair gsp : getSetPairs.values()) {
-      gsp.setMethod.invoke(origRecord, gsp.testValue);
-    }
-    Object ret = getProto.invoke(origRecord);
-    Assert.assertNotNull(recordClass.getName() + "#getProto returns null", ret);
-    if (!(protoClass.isAssignableFrom(ret.getClass()))) {
-      Assert.fail("Illegal getProto method return type: " + ret.getClass());
-    }
-    R deserRecord = pbConstructor.newInstance(ret);
-    Assert.assertEquals("whole " + recordClass + " records should be equal",
-        origRecord, deserRecord);
-    for (GetSetPair gsp : getSetPairs.values()) {
-      Object origValue = gsp.getMethod.invoke(origRecord);
-      Object deserValue = gsp.getMethod.invoke(deserRecord);
-      Assert.assertEquals("property " + recordClass.getName() + "#"
-          + gsp.propertyName + " should be equal", origValue, deserValue);
-    }
-  }
-
   @Test
   public void testAllocateRequestPBImpl() throws Exception {
     validatePBImplRecord(AllocateRequestPBImpl.class, AllocateRequestProto.class);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a48c8e65/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 b9e10ee..4216f76 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
@@ -59,6 +59,13 @@
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-yarn-common</artifactId>
     </dependency>
+    <!-- 'mvn dependency:analyze' fails to detect use of this dependency -->
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-common</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
 
     <dependency>
       <groupId>com.google.guava</groupId>
@@ -148,6 +155,7 @@
                   <include>yarn_server_common_protos.proto</include>
                   <include>yarn_server_common_service_protos.proto</include>
                   <include>yarn_server_common_service_protos.proto</include>
+                  <include>yarn_server_federation_protos.proto</include>
                   <include>ResourceTracker.proto</include>
                   <include>SCMUploader.proto</include>
                   <include>collectornodemanager_protocol.proto</include>

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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


---------------------------------------------------------------------
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: Revert "HADOOP-13081. add the ability to create multiple UGIs/subjects from one kerberos login. Contributed by Sergey Shelukhin."

Posted by su...@apache.org.
Revert "HADOOP-13081. add the ability to create multiple UGIs/subjects from one kerberos login. Contributed by Sergey Shelukhin."

This reverts commit 0458a2af6e925d023882714e8b7b0568eca7a775.


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

Branch: refs/heads/YARN-2915
Commit: 1e0ea27e9602efba102b2145d0240ecc9d5845a1
Parents: 236ac77
Author: Chris Nauroth <cn...@apache.org>
Authored: Thu Sep 29 13:59:09 2016 -0700
Committer: Chris Nauroth <cn...@apache.org>
Committed: Thu Sep 29 13:59:09 2016 -0700

----------------------------------------------------------------------
 .../hadoop/security/UserGroupInformation.java   | 29 +-------------------
 .../security/TestUserGroupInformation.java      | 27 ------------------
 2 files changed, 1 insertion(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1e0ea27e/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
index ed3a9d0..bcdfd53 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
@@ -38,7 +38,6 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
@@ -652,33 +651,7 @@ public class UserGroupInformation {
     }
     this.isKrbTkt = KerberosUtil.hasKerberosTicket(subject);
   }
-
-  /**
-   * Copies the Subject of this UGI and creates a new UGI with the new subject.
-   * This can be used to add credentials (e.g. tokens) to different copies of
-   * the same UGI, allowing multiple users with different tokens to reuse the
-   * UGI without re-authenticating with Kerberos.
-   * @return clone of the UGI with a new subject.
-   */
-  @InterfaceAudience.Public
-  @InterfaceStability.Evolving
-  public UserGroupInformation copySubjectAndUgi() {
-    Subject subj = getSubject();
-    // The ctor will set other fields automatically from the principals.
-    return new UserGroupInformation(new Subject(false, subj.getPrincipals(),
-        cloneCredentials(subj.getPublicCredentials()),
-        cloneCredentials(subj.getPrivateCredentials())));
-  }
-
-  private static Set<Object> cloneCredentials(Set<Object> old) {
-    Set<Object> set = new HashSet<>();
-    // Make sure Hadoop credentials objects do not reuse the maps.
-    for (Object o : old) {
-      set.add(o instanceof Credentials ? new Credentials((Credentials)o) : o);
-    }
-    return set;
-  }
-
+  
   /**
    * checks if logged in using kerberos
    * @return true if the subject logged via keytab or has a Kerberos TGT

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1e0ea27e/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java
index e45d70d..09a5807 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java
@@ -50,7 +50,6 @@ import java.security.PrivilegedExceptionAction;
 import java.util.Collection;
 import java.util.ConcurrentModificationException;
 import java.util.LinkedHashSet;
-import java.util.List;
 import java.util.Set;
 
 import static org.apache.hadoop.fs.CommonConfigurationKeys.HADOOP_USER_GROUP_METRICS_PERCENTILES_INTERVALS;
@@ -899,32 +898,6 @@ public class TestUserGroupInformation {
     assertEquals(1, tokens.size());
   }
 
-  @Test(timeout = 30000)
-  public void testCopySubjectAndUgi() throws IOException {
-    SecurityUtil.setAuthenticationMethod(AuthenticationMethod.SIMPLE, conf);
-    UserGroupInformation.setConfiguration(conf);
-    UserGroupInformation u1 = UserGroupInformation.getLoginUser();
-    assertNotNull(u1);
-    @SuppressWarnings("unchecked")
-    Token<? extends TokenIdentifier> tmpToken = mock(Token.class);
-    u1.addToken(tmpToken);
-
-    UserGroupInformation u2 = u1.copySubjectAndUgi();
-    assertEquals(u1.getAuthenticationMethod(), u2.getAuthenticationMethod());
-    assertNotSame(u1.getSubject(), u2.getSubject());
-    Credentials c1 = u1.getCredentials(), c2 = u2.getCredentials();
-    List<Text> sc1 = c1.getAllSecretKeys(), sc2 = c2.getAllSecretKeys();
-    assertArrayEquals(sc1.toArray(new Text[0]), sc2.toArray(new Text[0]));
-    Collection<Token<? extends TokenIdentifier>> ts1 = c1.getAllTokens(),
-        ts2 = c2.getAllTokens();
-    assertArrayEquals(ts1.toArray(new Token[0]), ts2.toArray(new Token[0]));
-    @SuppressWarnings("unchecked")
-    Token<? extends TokenIdentifier> token = mock(Token.class);
-    u2.addToken(token);
-    assertTrue(u2.getCredentials().getAllTokens().contains(token));
-    assertFalse(u1.getCredentials().getAllTokens().contains(token));
-  }
-
   /**
    * This test checks a race condition between getting and adding tokens for
    * the current user.  Calling UserGroupInformation.getCurrentUser() returns


---------------------------------------------------------------------
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-5323. Policies APIs for Federation Router and AMRMProxy policies. (Carlo Curino via Subru).

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


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

Branch: refs/heads/YARN-2915
Commit: ff47882d5bbddb5c78baa5506b80ce0f1b92c08c
Parents: 3c10a03
Author: Subru Krishnan <su...@apache.org>
Authored: Wed Sep 7 17:33:34 2016 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon Oct 3 14:34:03 2016 -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/ff47882d/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/ff47882d/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/ff47882d/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/ff47882d/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/ff47882d/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/ff47882d/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/ff47882d/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/ff47882d/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/ff47882d/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/ff47882d/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/ff47882d/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/ff47882d/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/ff47882d/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/ff47882d/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/ff47882d/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/ff47882d/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/ff47882d/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


[04/50] [abbrv] hadoop git commit: MAPREDUCE-6771. RMContainerAllocator sends container diagnostics event after corresponding completion event. Contributed by Haibo Chen

Posted by su...@apache.org.
MAPREDUCE-6771. RMContainerAllocator sends container diagnostics event after corresponding completion event. Contributed by Haibo Chen


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

Branch: refs/heads/YARN-2915
Commit: a1b8251bf7a7e9b776c4483fa01f7d453420eba4
Parents: 2ae5a3a
Author: Jason Lowe <jl...@apache.org>
Authored: Thu Sep 29 15:27:17 2016 +0000
Committer: Jason Lowe <jl...@apache.org>
Committed: Thu Sep 29 15:27:17 2016 +0000

----------------------------------------------------------------------
 .../v2/app/rm/RMContainerAllocator.java         | 51 ++++++++++++--------
 .../v2/app/rm/TestRMContainerAllocator.java     | 46 ++++++++++++++++++
 2 files changed, 77 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1b8251b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java
index ecd75db..4cb3cbe 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java
@@ -150,7 +150,7 @@ public class RMContainerAllocator extends RMContainerRequestor
     new LinkedList<ContainerRequest>();
 
   //holds information about the assigned containers to task attempts
-  private final AssignedRequests assignedRequests = new AssignedRequests();
+  private final AssignedRequests assignedRequests;
   
   //holds scheduled requests to be fulfilled by RM
   private final ScheduledRequests scheduledRequests = new ScheduledRequests();
@@ -200,6 +200,11 @@ public class RMContainerAllocator extends RMContainerRequestor
     this.preemptionPolicy = preemptionPolicy;
     this.stopped = new AtomicBoolean(false);
     this.clock = context.getClock();
+    this.assignedRequests = createAssignedRequests();
+  }
+
+  protected AssignedRequests createAssignedRequests() {
+    return new AssignedRequests();
   }
 
   @Override
@@ -833,29 +838,35 @@ public class RMContainerAllocator extends RMContainerRequestor
     }
 
     for (ContainerStatus cont : finishedContainers) {
-      LOG.info("Received completed container " + cont.getContainerId());
-      TaskAttemptId attemptID = assignedRequests.get(cont.getContainerId());
-      if (attemptID == null) {
-        LOG.error("Container complete event for unknown container id "
-            + cont.getContainerId());
-      } else {
-        pendingRelease.remove(cont.getContainerId());
-        assignedRequests.remove(attemptID);
-        
-        // send the container completed event to Task attempt
-        eventHandler.handle(createContainerFinishedEvent(cont, attemptID));
-        
-        // Send the diagnostics
-        String diagnostics = StringInterner.weakIntern(cont.getDiagnostics());
-        eventHandler.handle(new TaskAttemptDiagnosticsUpdateEvent(attemptID,
-            diagnostics));
-
-        preemptionPolicy.handleCompletedContainer(attemptID);
-      }
+      processFinishedContainer(cont);
     }
     return newContainers;
   }
 
+  @SuppressWarnings("unchecked")
+  @VisibleForTesting
+  void processFinishedContainer(ContainerStatus container) {
+    LOG.info("Received completed container " + container.getContainerId());
+    TaskAttemptId attemptID = assignedRequests.get(container.getContainerId());
+    if (attemptID == null) {
+      LOG.error("Container complete event for unknown container "
+          + container.getContainerId());
+    } else {
+      pendingRelease.remove(container.getContainerId());
+      assignedRequests.remove(attemptID);
+
+      // Send the diagnostics
+      String diagnostic = StringInterner.weakIntern(container.getDiagnostics());
+      eventHandler.handle(new TaskAttemptDiagnosticsUpdateEvent(attemptID,
+          diagnostic));
+
+      // send the container completed event to Task attempt
+      eventHandler.handle(createContainerFinishedEvent(container, attemptID));
+
+      preemptionPolicy.handleCompletedContainer(attemptID);
+    }
+  }
+
   private void applyConcurrentTaskLimits() {
     int numScheduledMaps = scheduledRequests.maps.size();
     if (maxRunningMaps > 0 && numScheduledMaps > 0) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a1b8251b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java
index a115b13..38a9731 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java
@@ -24,6 +24,7 @@ import static org.mockito.Matchers.anyInt;
 import static org.mockito.Matchers.isA;
 import static org.mockito.Mockito.doCallRealMethod;
 import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.inOrder;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.never;
 import static org.mockito.Mockito.times;
@@ -70,11 +71,13 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.JobEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.JobEventType;
 import org.apache.hadoop.mapreduce.v2.app.job.event.JobUpdatedNodesEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptContainerAssignedEvent;
+import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptDiagnosticsUpdateEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptKillEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.impl.JobImpl;
 import org.apache.hadoop.mapreduce.v2.app.job.impl.TaskAttemptImpl;
+import org.apache.hadoop.mapreduce.v2.app.rm.preemption.AMPreemptionPolicy;
 import org.apache.hadoop.mapreduce.v2.app.rm.preemption.NoopAMPreemptionPolicy;
 import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
@@ -144,6 +147,7 @@ import org.junit.Before;
 import org.junit.Test;
 
 import com.google.common.base.Supplier;
+import org.mockito.InOrder;
 
 @SuppressWarnings("unchecked")
 public class TestRMContainerAllocator {
@@ -3017,6 +3021,48 @@ public class TestRMContainerAllocator {
     }
   }
 
+  /**
+   * MAPREDUCE-6771. Test if RMContainerAllocator generates the events in the
+   * right order while processing finished containers.
+   */
+  @Test
+  public void testHandlingFinishedContainers() {
+    EventHandler eventHandler = mock(EventHandler.class);
+
+    AppContext context = mock(RunningAppContext.class);
+    when(context.getClock()).thenReturn(new ControlledClock());
+    when(context.getClusterInfo()).thenReturn(
+        new ClusterInfo(Resource.newInstance(10240, 1)));
+    when(context.getEventHandler()).thenReturn(eventHandler);
+    RMContainerAllocator containerAllocator =
+        new RMContainerAllocatorForFinishedContainer(null, context,
+            mock(AMPreemptionPolicy.class));
+
+    ContainerStatus finishedContainer = ContainerStatus.newInstance(
+        mock(ContainerId.class), ContainerState.COMPLETE, "", 0);
+    containerAllocator.processFinishedContainer(finishedContainer);
+
+    InOrder inOrder = inOrder(eventHandler);
+    inOrder.verify(eventHandler).handle(
+        isA(TaskAttemptDiagnosticsUpdateEvent.class));
+    inOrder.verify(eventHandler).handle(isA(TaskAttemptEvent.class));
+    inOrder.verifyNoMoreInteractions();
+  }
+
+  private static class RMContainerAllocatorForFinishedContainer
+      extends RMContainerAllocator {
+    public RMContainerAllocatorForFinishedContainer(ClientService clientService,
+        AppContext context, AMPreemptionPolicy preemptionPolicy) {
+      super(clientService, context, preemptionPolicy);
+    }
+    @Override
+    protected AssignedRequests createAssignedRequests() {
+      AssignedRequests assignedReqs = mock(AssignedRequests.class);
+      TaskAttemptId taskAttempt = mock(TaskAttemptId.class);
+      when(assignedReqs.get(any(ContainerId.class))).thenReturn(taskAttempt);
+      return assignedReqs;
+    }
+  }
 
   @Test
   public void testAvoidAskMoreReducersWhenReducerPreemptionIsRequired()


---------------------------------------------------------------------
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-5601. Make the RM epoch base value configurable. Contributed by Subru Krishnan

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


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

Branch: refs/heads/YARN-2915
Commit: 3c10a03232ef962d9b7d862da145dedee4428e4e
Parents: 911869a
Author: Jian He <ji...@apache.org>
Authored: Fri Sep 2 12:23:57 2016 +0800
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon Oct 3 14:34:03 2016 -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   | 8 ++++++--
 .../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, 34 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3c10a032/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 7f0a4e0..1d9e1ca 100644
--- a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
+++ b/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
@@ -297,7 +297,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/3c10a032/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 90520c1..9979d04 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
@@ -141,6 +141,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/3c10a032/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 56b13c4..56e7233 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
@@ -99,6 +99,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/3c10a032/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 9baccf2..a27d686 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
@@ -742,6 +742,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/3c10a032/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 a228037..30128d6 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/3c10a032/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 02f90dd..dfe78e5 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
@@ -250,7 +250,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/3c10a032/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 8b4ace3..5d7e1eb 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/3c10a032/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 de273c4..a3f7074 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
@@ -30,14 +30,12 @@ import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
 
 import javax.crypto.SecretKey;
 
-import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.ipc.CallerContext;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.token.delegation.DelegationKey;
 import org.apache.hadoop.service.AbstractService;
@@ -72,6 +70,8 @@ import org.apache.hadoop.yarn.state.MultipleArcTransition;
 import org.apache.hadoop.yarn.state.StateMachine;
 import org.apache.hadoop.yarn.state.StateMachineFactory;
 
+import com.google.common.annotations.VisibleForTesting;
+
 @Private
 @Unstable
 /**
@@ -95,6 +95,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;
@@ -665,6 +666,9 @@ public abstract class RMStateStore extends AbstractService {
     dispatcher.register(RMStateStoreEventType.class, 
                         new ForwardingEventHandler());
     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/3c10a032/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 51bb74d..b697171 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
@@ -352,7 +352,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/3c10a032/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 758bbae..40b8e9f 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;
@@ -558,13 +560,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/3c10a032/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/3c10a032/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 4297e73..d6f77d6 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
@@ -81,6 +81,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/3c10a032/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 19d3064..a2d3181 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
@@ -145,6 +145,7 @@ public class TestZKRMStateStore extends RMStateStoreTestBase {
       conf.set(YarnConfiguration.RM_ZK_ADDRESS,
           curatorTestingServer.getConnectString());
       conf.set(YarnConfiguration.ZK_RM_STATE_STORE_PARENT_PATH, workingZnode);
+      conf.setLong(YarnConfiguration.RM_EPOCH, epoch);
       this.store = new TestZKRMStateStoreInternal(conf, workingZnode);
       return this.store;
     }


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


[14/50] [abbrv] hadoop git commit: HDFS-10910. HDFS Erasure Coding doc should state its currently supported erasure coding policies. Contributed by Yiqun Lin.

Posted by su...@apache.org.
HDFS-10910. HDFS Erasure Coding doc should state its currently supported erasure coding policies. Contributed by Yiqun Lin.


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

Branch: refs/heads/YARN-2915
Commit: ee33a02234511ac69c1e491fd38490a141ec907e
Parents: 2ab1ef1
Author: Wei-Chiu Chuang <we...@apache.org>
Authored: Fri Sep 30 12:48:11 2016 -0700
Committer: Wei-Chiu Chuang <we...@apache.org>
Committed: Fri Sep 30 12:48:11 2016 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md      | 9 ++++-----
 1 file changed, 4 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee33a022/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
index 9066a15..76c1b3a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
@@ -59,9 +59,9 @@ Architecture
       1. _Read the data from source nodes:_ Input data is read in parallel from source nodes using a dedicated thread pool.
         Based on the EC policy, it schedules the read requests to all source targets and reads only the minimum number of input blocks for reconstruction.
 
-      1. _Decode the data and generate the output data:_ New data and parity blocks are decoded from the input data. All missing data and parity blocks are decoded together.
+      2. _Decode the data and generate the output data:_ New data and parity blocks are decoded from the input data. All missing data and parity blocks are decoded together.
 
-      1. _Transfer the generated data blocks to target nodes:_ Once decoding is finished, the recovered blocks are transferred to target DataNodes.
+      3. _Transfer the generated data blocks to target nodes:_ Once decoding is finished, the recovered blocks are transferred to target DataNodes.
 
  *  **ErasureCoding policy**
     To accommodate heterogeneous workloads, we allow files and directories in an HDFS cluster to have different replication and EC policies.
@@ -69,10 +69,9 @@ Architecture
 
       1. _The ECSchema:_ This includes the numbers of data and parity blocks in an EC group (e.g., 6+3), as well as the codec algorithm (e.g., Reed-Solomon).
 
-      1. _The size of a striping cell._ This determines the granularity of striped reads and writes, including buffer sizes and encoding work.
+      2. _The size of a striping cell._ This determines the granularity of striped reads and writes, including buffer sizes and encoding work.
 
-    Currently, HDFS supports the Reed-Solomon and XOR erasure coding algorithms. Additional algorithms are planned as future work.
-    The system default scheme is Reed-Solomon (6, 3) with a cell size of 64KB.
+    There are three policies currently being supported: RS-DEFAULT-3-2-64k, RS-DEFAULT-6-3-64k and RS-LEGACY-6-3-64k. All with default cell size of 64KB. The system default policy is RS-DEFAULT-6-3-64k which use the default schema RS_6_3_SCHEMA with a cell size of 64KB.
 
 
 Deployment


---------------------------------------------------------------------
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: HDFS-10896. Move lock logging logic from FSNamesystem into FSNamesystemLock. Contributed by Erik Krogen.

Posted by su...@apache.org.
HDFS-10896. Move lock logging logic from FSNamesystem into FSNamesystemLock. Contributed by Erik Krogen.


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

Branch: refs/heads/YARN-2915
Commit: 434c5ea75dc3d87513e49290ac9999148ff5163c
Parents: 57aec2b
Author: Zhe Zhang <zh...@apache.org>
Authored: Fri Sep 30 13:15:59 2016 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Fri Sep 30 13:16:08 2016 -0700

----------------------------------------------------------------------
 .../hdfs/server/namenode/FSNamesystem.java      | 147 +--------
 .../hdfs/server/namenode/FSNamesystemLock.java  | 187 ++++++++++-
 .../hdfs/server/namenode/TestFSNamesystem.java  | 292 -----------------
 .../server/namenode/TestFSNamesystemLock.java   | 317 +++++++++++++++++++
 4 files changed, 496 insertions(+), 447 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/434c5ea7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 4700263..1721b2c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -71,12 +71,6 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_MAX_OBJECTS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RESOURCE_CHECK_INTERVAL_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RESOURCE_CHECK_INTERVAL_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_WRITE_LOCK_REPORTING_THRESHOLD_MS_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_WRITE_LOCK_REPORTING_THRESHOLD_MS_DEFAULT;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_READ_LOCK_REPORTING_THRESHOLD_MS_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_READ_LOCK_REPORTING_THRESHOLD_MS_DEFAULT;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_LOCK_SUPPRESS_WARNING_INTERVAL_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_LOCK_SUPPRESS_WARNING_INTERVAL_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RETRY_CACHE_EXPIRYTIME_MILLIS_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RETRY_CACHE_EXPIRYTIME_MILLIS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RETRY_CACHE_HEAP_PERCENT_DEFAULT;
@@ -129,8 +123,6 @@ import java.util.TreeMap;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.locks.Condition;
 import java.util.concurrent.locks.ReentrantLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
@@ -284,7 +276,6 @@ import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.util.Timer;
 import org.apache.hadoop.util.VersionInfo;
 import org.apache.log4j.Appender;
 import org.apache.log4j.AsyncAppender;
@@ -713,12 +704,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       LOG.info("Enabling async auditlog");
       enableAsyncAuditLog();
     }
-    boolean fair = conf.getBoolean("dfs.namenode.fslock.fair", true);
-    LOG.info("fsLock is fair:" + fair);
-    fsLock = new FSNamesystemLock(fair);
-    cond = fsLock.writeLock().newCondition();
+    fsLock = new FSNamesystemLock(conf);
+    cond = fsLock.newWriteLockCondition();
     cpLock = new ReentrantLock();
-    setTimer(new Timer());
 
     this.fsImage = fsImage;
     try {
@@ -827,17 +815,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
           DFS_NAMENODE_MAX_LOCK_HOLD_TO_RELEASE_LEASE_MS_KEY,
           DFS_NAMENODE_MAX_LOCK_HOLD_TO_RELEASE_LEASE_MS_DEFAULT);
 
-      this.writeLockReportingThreshold = conf.getLong(
-          DFS_NAMENODE_WRITE_LOCK_REPORTING_THRESHOLD_MS_KEY,
-          DFS_NAMENODE_WRITE_LOCK_REPORTING_THRESHOLD_MS_DEFAULT);
-      this.readLockReportingThreshold = conf.getLong(
-          DFS_NAMENODE_READ_LOCK_REPORTING_THRESHOLD_MS_KEY,
-          DFS_NAMENODE_READ_LOCK_REPORTING_THRESHOLD_MS_DEFAULT);
-
-      this.lockSuppressWarningInterval = conf.getTimeDuration(
-          DFS_LOCK_SUPPRESS_WARNING_INTERVAL_KEY,
-          DFS_LOCK_SUPPRESS_WARNING_INTERVAL_DEFAULT, TimeUnit.MILLISECONDS);
-
       // For testing purposes, allow the DT secret manager to be started regardless
       // of whether security is enabled.
       alwaysUseDelegationTokensForTests = conf.getBoolean(
@@ -1516,131 +1493,25 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     return Util.stringCollectionAsURIs(dirNames);
   }
 
-  private final long lockSuppressWarningInterval;
-  /** Threshold (ms) for long holding write lock report. */
-  private final long writeLockReportingThreshold;
-  private int numWriteLockWarningsSuppressed = 0;
-  private long timeStampOfLastWriteLockReport = 0;
-  private long longestWriteLockHeldInterval = 0;
-  /** Last time stamp for write lock. Keep the longest one for multi-entrance.*/
-  private long writeLockHeldTimeStamp;
-  /** Threshold (ms) for long holding read lock report. */
-  private long readLockReportingThreshold;
-  private AtomicInteger numReadLockWarningsSuppressed = new AtomicInteger(0);
-  private AtomicLong timeStampOfLastReadLockReport = new AtomicLong(0);
-  private AtomicLong longestReadLockHeldInterval = new AtomicLong(0);
-  private Timer timer;
-  /**
-   * Last time stamp for read lock. Keep the longest one for
-   * multi-entrance. This is ThreadLocal since there could be
-   * many read locks held simultaneously.
-   */
-  private static ThreadLocal<Long> readLockHeldTimeStamp =
-      new ThreadLocal<Long>() {
-        @Override
-        public Long initialValue() {
-          return Long.MAX_VALUE;
-        }
-      };
-
   @Override
   public void readLock() {
-    this.fsLock.readLock().lock();
-    if (this.fsLock.getReadHoldCount() == 1) {
-      readLockHeldTimeStamp.set(timer.monotonicNow());
-    }
+    this.fsLock.readLock();
   }
   @Override
   public void readUnlock() {
-    final boolean needReport = this.fsLock.getReadHoldCount() == 1;
-    final long readLockInterval = timer.monotonicNow() -
-        readLockHeldTimeStamp.get();
-    if (needReport) {
-      readLockHeldTimeStamp.remove();
-    }
-
-    this.fsLock.readLock().unlock();
-
-    if (needReport && readLockInterval >= this.readLockReportingThreshold) {
-      long localLongestReadLock;
-      do {
-        localLongestReadLock = longestReadLockHeldInterval.get();
-      } while (localLongestReadLock - readLockInterval < 0
-          && !longestReadLockHeldInterval.compareAndSet(localLongestReadLock,
-                                                        readLockInterval));
-
-      long localTimeStampOfLastReadLockReport;
-      long now;
-      do {
-        now = timer.monotonicNow();
-        localTimeStampOfLastReadLockReport = timeStampOfLastReadLockReport
-            .get();
-        if (now - localTimeStampOfLastReadLockReport <
-            lockSuppressWarningInterval) {
-          numReadLockWarningsSuppressed.incrementAndGet();
-          return;
-        }
-      } while (!timeStampOfLastReadLockReport.compareAndSet(
-          localTimeStampOfLastReadLockReport, now));
-      int numSuppressedWarnings = numReadLockWarningsSuppressed.getAndSet(0);
-      long longestLockHeldInterval = longestReadLockHeldInterval.getAndSet(0);
-      LOG.info("FSNamesystem read lock held for " + readLockInterval +
-          " ms via\n" + StringUtils.getStackTrace(Thread.currentThread()) +
-          "\tNumber of suppressed read-lock reports: " +
-          numSuppressedWarnings + "\n\tLongest read-lock held interval: " +
-          longestLockHeldInterval);
-    }
+    this.fsLock.readUnlock();
   }
   @Override
   public void writeLock() {
-    this.fsLock.writeLock().lock();
-    if (fsLock.getWriteHoldCount() == 1) {
-      writeLockHeldTimeStamp = timer.monotonicNow();
-    }
+    this.fsLock.writeLock();
   }
   @Override
   public void writeLockInterruptibly() throws InterruptedException {
-    this.fsLock.writeLock().lockInterruptibly();
-    if (fsLock.getWriteHoldCount() == 1) {
-      writeLockHeldTimeStamp = timer.monotonicNow();
-    }
+    this.fsLock.writeLockInterruptibly();
   }
   @Override
   public void writeUnlock() {
-    final boolean needReport = fsLock.getWriteHoldCount() == 1 &&
-        fsLock.isWriteLockedByCurrentThread();
-    final long currentTime = timer.monotonicNow();
-    final long writeLockInterval = currentTime - writeLockHeldTimeStamp;
-
-    boolean logReport = false;
-    int numSuppressedWarnings = 0;
-    long longestLockHeldInterval = 0;
-    if (needReport && writeLockInterval >= this.writeLockReportingThreshold) {
-      if (writeLockInterval > longestWriteLockHeldInterval) {
-        longestWriteLockHeldInterval = writeLockInterval;
-      }
-      if (currentTime - timeStampOfLastWriteLockReport > this
-          .lockSuppressWarningInterval) {
-        logReport = true;
-        numSuppressedWarnings = numWriteLockWarningsSuppressed;
-        numWriteLockWarningsSuppressed = 0;
-        longestLockHeldInterval = longestWriteLockHeldInterval;
-        longestWriteLockHeldInterval = 0;
-        timeStampOfLastWriteLockReport = currentTime;
-      } else {
-        numWriteLockWarningsSuppressed++;
-      }
-    }
-
-    this.fsLock.writeLock().unlock();
-
-    if (logReport) {
-      LOG.info("FSNamesystem write lock held for " + writeLockInterval +
-          " ms via\n" + StringUtils.getStackTrace(Thread.currentThread()) +
-          "\tNumber of suppressed write-lock reports: " +
-          numSuppressedWarnings + "\n\tLongest write-lock held interval: " +
-              longestLockHeldInterval);
-    }
+    this.fsLock.writeUnlock();
   }
   @Override
   public boolean hasWriteLock() {
@@ -7173,9 +7044,5 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         .size();
   }
 
-  @VisibleForTesting
-  void setTimer(Timer newTimer) {
-    this.timer = newTimer;
-  }
 }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/434c5ea7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystemLock.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystemLock.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystemLock.java
index d239796..043f569 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystemLock.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystemLock.java
@@ -19,33 +19,186 @@
 
 package org.apache.hadoop.hdfs.server.namenode;
 
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantLock;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.Condition;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.Timer;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_LOCK_SUPPRESS_WARNING_INTERVAL_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_LOCK_SUPPRESS_WARNING_INTERVAL_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_READ_LOCK_REPORTING_THRESHOLD_MS_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_READ_LOCK_REPORTING_THRESHOLD_MS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_WRITE_LOCK_REPORTING_THRESHOLD_MS_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_WRITE_LOCK_REPORTING_THRESHOLD_MS_KEY;
 
 /**
- * Mimics a ReentrantReadWriteLock so more sophisticated locking capabilities
- * are possible.
+ * Mimics a ReentrantReadWriteLock but does not directly implement the interface
+ * so more sophisticated locking capabilities and logging/metrics are possible.
  */
-class FSNamesystemLock implements ReadWriteLock {
+class FSNamesystemLock {
   @VisibleForTesting
   protected ReentrantReadWriteLock coarseLock;
-  
-  FSNamesystemLock(boolean fair) {
+
+  private final Timer timer;
+
+  /**
+   * Log statements about long lock hold times will not be produced more
+   * frequently than this interval.
+   */
+  private final long lockSuppressWarningInterval;
+
+  /** Threshold (ms) for long holding write lock report. */
+  private final long writeLockReportingThreshold;
+  /** Last time stamp for write lock. Keep the longest one for multi-entrance.*/
+  private long writeLockHeldTimeStamp;
+  private int numWriteLockWarningsSuppressed = 0;
+  private long timeStampOfLastWriteLockReport = 0;
+  private long longestWriteLockHeldInterval = 0;
+
+  /** Threshold (ms) for long holding read lock report. */
+  private final long readLockReportingThreshold;
+  /**
+   * Last time stamp for read lock. Keep the longest one for
+   * multi-entrance. This is ThreadLocal since there could be
+   * many read locks held simultaneously.
+   */
+  private final ThreadLocal<Long> readLockHeldTimeStamp =
+      new ThreadLocal<Long>() {
+        @Override
+        public Long initialValue() {
+          return Long.MAX_VALUE;
+        }
+      };
+  private final AtomicInteger numReadLockWarningsSuppressed =
+      new AtomicInteger(0);
+  private final AtomicLong timeStampOfLastReadLockReport = new AtomicLong(0);
+  private final AtomicLong longestReadLockHeldInterval = new AtomicLong(0);
+
+  FSNamesystemLock(Configuration conf) {
+    this(conf, new Timer());
+  }
+
+  @VisibleForTesting
+  FSNamesystemLock(Configuration conf, Timer timer) {
+    boolean fair = conf.getBoolean("dfs.namenode.fslock.fair", true);
+    FSNamesystem.LOG.info("fsLock is fair: " + fair);
     this.coarseLock = new ReentrantReadWriteLock(fair);
+    this.timer = timer;
+
+    this.writeLockReportingThreshold = conf.getLong(
+        DFS_NAMENODE_WRITE_LOCK_REPORTING_THRESHOLD_MS_KEY,
+        DFS_NAMENODE_WRITE_LOCK_REPORTING_THRESHOLD_MS_DEFAULT);
+    this.readLockReportingThreshold = conf.getLong(
+        DFS_NAMENODE_READ_LOCK_REPORTING_THRESHOLD_MS_KEY,
+        DFS_NAMENODE_READ_LOCK_REPORTING_THRESHOLD_MS_DEFAULT);
+    this.lockSuppressWarningInterval = conf.getTimeDuration(
+        DFS_LOCK_SUPPRESS_WARNING_INTERVAL_KEY,
+        DFS_LOCK_SUPPRESS_WARNING_INTERVAL_DEFAULT, TimeUnit.MILLISECONDS);
   }
-  
-  @Override
-  public Lock readLock() {
-    return coarseLock.readLock();
+
+  public void readLock() {
+    coarseLock.readLock().lock();
+    if (coarseLock.getReadHoldCount() == 1) {
+      readLockHeldTimeStamp.set(timer.monotonicNow());
+    }
+  }
+
+  public void readUnlock() {
+    final boolean needReport = coarseLock.getReadHoldCount() == 1;
+    final long readLockInterval =
+        timer.monotonicNow() - readLockHeldTimeStamp.get();
+    coarseLock.readLock().unlock();
+
+    if (needReport) {
+      readLockHeldTimeStamp.remove();
+    }
+    if (needReport && readLockInterval >= this.readLockReportingThreshold) {
+      long localLongestReadLock;
+      do {
+        localLongestReadLock = longestReadLockHeldInterval.get();
+      } while (localLongestReadLock - readLockInterval < 0 &&
+          !longestReadLockHeldInterval.compareAndSet(localLongestReadLock,
+              readLockInterval));
+
+      long localTimeStampOfLastReadLockReport;
+      long now;
+      do {
+        now = timer.monotonicNow();
+        localTimeStampOfLastReadLockReport =
+            timeStampOfLastReadLockReport.get();
+        if (now - localTimeStampOfLastReadLockReport <
+            lockSuppressWarningInterval) {
+          numReadLockWarningsSuppressed.incrementAndGet();
+          return;
+        }
+      } while (!timeStampOfLastReadLockReport.compareAndSet(
+          localTimeStampOfLastReadLockReport, now));
+      int numSuppressedWarnings = numReadLockWarningsSuppressed.getAndSet(0);
+      long longestLockHeldInterval = longestReadLockHeldInterval.getAndSet(0);
+      FSNamesystem.LOG.info("FSNamesystem read lock held for " +
+          readLockInterval + " ms via\n" +
+          StringUtils.getStackTrace(Thread.currentThread()) +
+          "\tNumber of suppressed read-lock reports: " + numSuppressedWarnings +
+          "\n\tLongest read-lock held interval: " + longestLockHeldInterval);
+    }
   }
   
-  @Override
-  public Lock writeLock() {
-    return coarseLock.writeLock();
+  public void writeLock() {
+    coarseLock.writeLock().lock();
+    if (coarseLock.getWriteHoldCount() == 1) {
+      writeLockHeldTimeStamp = timer.monotonicNow();
+    }
+  }
+
+  public void writeLockInterruptibly() throws InterruptedException {
+    coarseLock.writeLock().lockInterruptibly();
+    if (coarseLock.getWriteHoldCount() == 1) {
+      writeLockHeldTimeStamp = timer.monotonicNow();
+    }
+  }
+
+  public void writeUnlock() {
+    final boolean needReport = coarseLock.getWriteHoldCount() == 1 &&
+        coarseLock.isWriteLockedByCurrentThread();
+    final long currentTime = timer.monotonicNow();
+    final long writeLockInterval = currentTime - writeLockHeldTimeStamp;
+
+    boolean logReport = false;
+    int numSuppressedWarnings = 0;
+    long longestLockHeldInterval = 0;
+    if (needReport && writeLockInterval >= this.writeLockReportingThreshold) {
+      if (writeLockInterval > longestWriteLockHeldInterval) {
+        longestWriteLockHeldInterval = writeLockInterval;
+      }
+      if (currentTime - timeStampOfLastWriteLockReport >
+          this.lockSuppressWarningInterval) {
+        logReport = true;
+        numSuppressedWarnings = numWriteLockWarningsSuppressed;
+        numWriteLockWarningsSuppressed = 0;
+        longestLockHeldInterval = longestWriteLockHeldInterval;
+        longestWriteLockHeldInterval = 0;
+        timeStampOfLastWriteLockReport = currentTime;
+      } else {
+        numWriteLockWarningsSuppressed++;
+      }
+    }
+
+    coarseLock.writeLock().unlock();
+
+    if (logReport) {
+      FSNamesystem.LOG.info("FSNamesystem write lock held for " +
+          writeLockInterval + " ms via\n" +
+          StringUtils.getStackTrace(Thread.currentThread()) +
+          "\tNumber of suppressed write-lock reports: " +
+          numSuppressedWarnings + "\n\tLongest write-lock held interval: " +
+          longestLockHeldInterval);
+    }
   }
 
   public int getReadHoldCount() {
@@ -60,6 +213,10 @@ class FSNamesystemLock implements ReadWriteLock {
     return coarseLock.isWriteLockedByCurrentThread();
   }
 
+  public Condition newWriteLockCondition() {
+    return coarseLock.writeLock().newCondition();
+  }
+
   /**
    * Returns the QueueLength of waiting threads.
    *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/434c5ea7/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystem.java
index 47d549b..f02c679 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystem.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY;
-import org.apache.hadoop.util.FakeTimer;
 import static org.hamcrest.CoreMatchers.either;
 import static org.hamcrest.CoreMatchers.instanceOf;
 import static org.junit.Assert.*;
@@ -31,7 +30,6 @@ import java.net.InetAddress;
 import java.net.URI;
 import java.util.Collection;
 
-import com.google.common.base.Supplier;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileUtil;
@@ -45,22 +43,12 @@ import org.apache.hadoop.hdfs.server.namenode.ha.HAContext;
 import org.apache.hadoop.hdfs.server.namenode.ha.HAState;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.hdfs.server.namenode.top.TopAuditLogger;
-import org.apache.hadoop.test.GenericTestUtils;
-import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
-import org.apache.log4j.Level;
 import org.junit.After;
-import org.junit.Assert;
 import org.junit.Test;
 import org.mockito.Mockito;
 import org.mockito.internal.util.reflection.Whitebox;
 
 import java.util.List;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-import java.util.regex.Pattern;
 
 public class TestFSNamesystem {
 
@@ -165,59 +153,6 @@ public class TestFSNamesystem {
     assertTrue("Replication queues weren't being populated after entering "
       + "safemode 2nd time", bm.isPopulatingReplQueues());
   }
-  
-  @Test
-  public void testFsLockFairness() throws IOException, InterruptedException{
-    Configuration conf = new Configuration();
-
-    FSEditLog fsEditLog = Mockito.mock(FSEditLog.class);
-    FSImage fsImage = Mockito.mock(FSImage.class);
-    Mockito.when(fsImage.getEditLog()).thenReturn(fsEditLog);
-
-    conf.setBoolean("dfs.namenode.fslock.fair", true);
-    FSNamesystem fsNamesystem = new FSNamesystem(conf, fsImage);
-    assertTrue(fsNamesystem.getFsLockForTests().isFair());
-    
-    conf.setBoolean("dfs.namenode.fslock.fair", false);
-    fsNamesystem = new FSNamesystem(conf, fsImage);
-    assertFalse(fsNamesystem.getFsLockForTests().isFair());
-  }  
-  
-  @Test
-  public void testFSNamesystemLockCompatibility() {
-    FSNamesystemLock rwLock = new FSNamesystemLock(true);
-
-    assertEquals(0, rwLock.getReadHoldCount());
-    rwLock.readLock().lock();
-    assertEquals(1, rwLock.getReadHoldCount());
-
-    rwLock.readLock().lock();
-    assertEquals(2, rwLock.getReadHoldCount());
-
-    rwLock.readLock().unlock();
-    assertEquals(1, rwLock.getReadHoldCount());
-
-    rwLock.readLock().unlock();
-    assertEquals(0, rwLock.getReadHoldCount());
-
-    assertFalse(rwLock.isWriteLockedByCurrentThread());
-    assertEquals(0, rwLock.getWriteHoldCount());
-    rwLock.writeLock().lock();
-    assertTrue(rwLock.isWriteLockedByCurrentThread());
-    assertEquals(1, rwLock.getWriteHoldCount());
-    
-    rwLock.writeLock().lock();
-    assertTrue(rwLock.isWriteLockedByCurrentThread());
-    assertEquals(2, rwLock.getWriteHoldCount());
-
-    rwLock.writeLock().unlock();
-    assertTrue(rwLock.isWriteLockedByCurrentThread());
-    assertEquals(1, rwLock.getWriteHoldCount());
-
-    rwLock.writeLock().unlock();
-    assertFalse(rwLock.isWriteLockedByCurrentThread());
-    assertEquals(0, rwLock.getWriteHoldCount());
-  }
 
   @Test
   public void testReset() throws Exception {
@@ -258,233 +193,6 @@ public class TestFSNamesystem {
   }
 
   @Test
-  public void testFSLockGetWaiterCount() throws InterruptedException {
-    final int threadCount = 3;
-    final CountDownLatch latch = new CountDownLatch(threadCount);
-    final FSNamesystemLock rwLock = new FSNamesystemLock(true);
-    rwLock.writeLock().lock();
-    ExecutorService helper = Executors.newFixedThreadPool(threadCount);
-
-    for (int x = 0; x < threadCount; x++) {
-      helper.execute(new Runnable() {
-        @Override
-        public void run() {
-          latch.countDown();
-          rwLock.readLock().lock();
-        }
-      });
-    }
-
-    latch.await();
-    try {
-      GenericTestUtils.waitFor(new Supplier<Boolean>() {
-        @Override
-        public Boolean get() {
-          return (threadCount == rwLock.getQueueLength());
-        }
-      }, 10, 1000);
-    } catch (TimeoutException e) {
-      fail("Expected number of blocked thread not found");
-    }
-  }
-
-  /**
-   * Test when FSNamesystem write lock is held for a long time,
-   * logger will report it.
-   */
-  @Test(timeout=45000)
-  public void testFSWriteLockLongHoldingReport() throws Exception {
-    final long writeLockReportingThreshold = 100L;
-    final long writeLockSuppressWarningInterval = 10000L;
-    Configuration conf = new Configuration();
-    conf.setLong(DFSConfigKeys.DFS_NAMENODE_WRITE_LOCK_REPORTING_THRESHOLD_MS_KEY,
-        writeLockReportingThreshold);
-    conf.setTimeDuration(DFSConfigKeys.DFS_LOCK_SUPPRESS_WARNING_INTERVAL_KEY,
-        writeLockSuppressWarningInterval, TimeUnit.MILLISECONDS);
-    FSImage fsImage = Mockito.mock(FSImage.class);
-    FSEditLog fsEditLog = Mockito.mock(FSEditLog.class);
-    Mockito.when(fsImage.getEditLog()).thenReturn(fsEditLog);
-    FSNamesystem fsn = new FSNamesystem(conf, fsImage);
-
-    FakeTimer timer = new FakeTimer();
-    fsn.setTimer(timer);
-    timer.advance(writeLockSuppressWarningInterval);
-
-    LogCapturer logs = LogCapturer.captureLogs(FSNamesystem.LOG);
-    GenericTestUtils.setLogLevel(FSNamesystem.LOG, Level.INFO);
-
-    // Don't report if the write lock is held for a short time
-    fsn.writeLock();
-    fsn.writeUnlock();
-    assertFalse(logs.getOutput().contains(GenericTestUtils.getMethodName()));
-
-    // Report the first write lock warning if it is held for a long time
-    fsn.writeLock();
-    timer.advance(writeLockReportingThreshold + 10);
-    logs.clearOutput();
-    fsn.writeUnlock();
-    assertTrue(logs.getOutput().contains(GenericTestUtils.getMethodName()));
-
-    // Track but do not Report if the write lock is held (interruptibly) for
-    // a long time but time since last report does not exceed the suppress
-    // warning interval
-    fsn.writeLockInterruptibly();
-    timer.advance(writeLockReportingThreshold + 10);
-    logs.clearOutput();
-    fsn.writeUnlock();
-    assertFalse(logs.getOutput().contains(GenericTestUtils.getMethodName()));
-
-    // Track but do not Report if it's held for a long time when re-entering
-    // write lock but time since last report does not exceed the suppress
-    // warning interval
-    fsn.writeLock();
-    timer.advance(writeLockReportingThreshold/ 2 + 1);
-    fsn.writeLockInterruptibly();
-    timer.advance(writeLockReportingThreshold/ 2 + 1);
-    fsn.writeLock();
-    timer.advance(writeLockReportingThreshold/ 2);
-    logs.clearOutput();
-    fsn.writeUnlock();
-    assertFalse(logs.getOutput().contains(GenericTestUtils.getMethodName()));
-    logs.clearOutput();
-    fsn.writeUnlock();
-    assertFalse(logs.getOutput().contains(GenericTestUtils.getMethodName()));
-    logs.clearOutput();
-    fsn.writeUnlock();
-    assertFalse(logs.getOutput().contains(GenericTestUtils.getMethodName()));
-
-    // Report if it's held for a long time and time since last report exceeds
-    // the supress warning interval
-    timer.advance(writeLockSuppressWarningInterval);
-    fsn.writeLock();
-    timer.advance(writeLockReportingThreshold + 100);
-    logs.clearOutput();
-    fsn.writeUnlock();
-    assertTrue(logs.getOutput().contains(GenericTestUtils.getMethodName()));
-    assertTrue(logs.getOutput().contains("Number of suppressed write-lock " +
-        "reports: 2"));
-  }
-
-  /**
-   * Test when FSNamesystem read lock is held for a long time,
-   * logger will report it.
-   */
-  @Test(timeout=45000)
-  public void testFSReadLockLongHoldingReport() throws Exception {
-    final long readLockReportingThreshold = 100L;
-    final long readLockSuppressWarningInterval = 10000L;
-    final String readLockLogStmt = "FSNamesystem read lock held for ";
-    Configuration conf = new Configuration();
-    conf.setLong(
-        DFSConfigKeys.DFS_NAMENODE_READ_LOCK_REPORTING_THRESHOLD_MS_KEY,
-        readLockReportingThreshold);
-    conf.setTimeDuration(DFSConfigKeys.DFS_LOCK_SUPPRESS_WARNING_INTERVAL_KEY,
-        readLockSuppressWarningInterval, TimeUnit.MILLISECONDS);
-    FSImage fsImage = Mockito.mock(FSImage.class);
-    FSEditLog fsEditLog = Mockito.mock(FSEditLog.class);
-    Mockito.when(fsImage.getEditLog()).thenReturn(fsEditLog);
-    FSNamesystem fsn = new FSNamesystem(conf, fsImage);
-
-    FakeTimer timer = new FakeTimer();
-    fsn.setTimer(timer);
-    timer.advance(readLockSuppressWarningInterval);
-
-    LogCapturer logs = LogCapturer.captureLogs(FSNamesystem.LOG);
-    GenericTestUtils.setLogLevel(FSNamesystem.LOG, Level.INFO);
-
-    // Don't report if the read lock is held for a short time
-    fsn.readLock();
-    fsn.readUnlock();
-    assertFalse(logs.getOutput().contains(GenericTestUtils.getMethodName()) &&
-        logs.getOutput().contains(readLockLogStmt));
-
-    // Report the first read lock warning if it is held for a long time
-    fsn.readLock();
-    timer.advance(readLockReportingThreshold + 10);
-    logs.clearOutput();
-    fsn.readUnlock();
-    assertTrue(logs.getOutput().contains(GenericTestUtils.getMethodName())
-        && logs.getOutput().contains(readLockLogStmt));
-
-    // Track but do not Report if the write lock is held for a long time but
-    // time since last report does not exceed the suppress warning interval
-    fsn.readLock();
-    timer.advance(readLockReportingThreshold + 10);
-    logs.clearOutput();
-    fsn.readUnlock();
-    assertFalse(logs.getOutput().contains(GenericTestUtils.getMethodName())
-        && logs.getOutput().contains(readLockLogStmt));
-
-    // Track but do not Report if it's held for a long time when re-entering
-    // read lock but time since last report does not exceed the suppress
-    // warning interval
-    fsn.readLock();
-    timer.advance(readLockReportingThreshold / 2 + 1);
-    fsn.readLock();
-    timer.advance(readLockReportingThreshold / 2 + 1);
-    logs.clearOutput();
-    fsn.readUnlock();
-    assertFalse(logs.getOutput().contains(GenericTestUtils.getMethodName()) ||
-        logs.getOutput().contains(readLockLogStmt));
-    logs.clearOutput();
-    fsn.readUnlock();
-    assertFalse(logs.getOutput().contains(GenericTestUtils.getMethodName()) &&
-        logs.getOutput().contains(readLockLogStmt));
-
-    // Report if it's held for a long time (and time since last report
-    // exceeds the suppress warning interval) while another thread also has the
-    // read lock. Let one thread hold the lock long enough to activate an
-    // alert, then have another thread grab the read lock to ensure that this
-    // doesn't reset the timing.
-    timer.advance(readLockSuppressWarningInterval);
-    logs.clearOutput();
-    CountDownLatch barrier = new CountDownLatch(1);
-    CountDownLatch barrier2 = new CountDownLatch(1);
-    Thread t1 = new Thread() {
-      @Override
-      public void run() {
-        try {
-          fsn.readLock();
-          timer.advance(readLockReportingThreshold + 1);
-          barrier.countDown(); // Allow for t2 to acquire the read lock
-          barrier2.await(); // Wait until t2 has the read lock
-          fsn.readUnlock();
-        } catch (InterruptedException e) {
-          fail("Interrupted during testing");
-        }
-      }
-    };
-    Thread t2 = new Thread() {
-      @Override
-      public void run() {
-        try {
-          barrier.await(); // Wait until t1 finishes sleeping
-          fsn.readLock();
-          barrier2.countDown(); // Allow for t1 to unlock
-          fsn.readUnlock();
-        } catch (InterruptedException e) {
-          fail("Interrupted during testing");
-        }
-      }
-    };
-    t1.start();
-    t2.start();
-    t1.join();
-    t2.join();
-    // Look for the differentiating class names in the stack trace
-    String stackTracePatternString =
-        String.format("INFO.+%s(.+\n){4}\\Q%%s\\E\\.run", readLockLogStmt);
-    Pattern t1Pattern = Pattern.compile(
-        String.format(stackTracePatternString, t1.getClass().getName()));
-    assertTrue(t1Pattern.matcher(logs.getOutput()).find());
-    Pattern t2Pattern = Pattern.compile(
-        String.format(stackTracePatternString, t2.getClass().getName()));
-    assertFalse(t2Pattern.matcher(logs.getOutput()).find());
-    assertTrue(logs.getOutput().contains("Number of suppressed read-lock " +
-        "reports: 2"));
-  }
-
-  @Test
   public void testSafemodeReplicationConf() throws IOException {
     Configuration conf = new Configuration();
     FSImage fsImage = Mockito.mock(FSImage.class);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/434c5ea7/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystemLock.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystemLock.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystemLock.java
new file mode 100644
index 0000000..08900ec
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystemLock.java
@@ -0,0 +1,317 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.server.namenode;
+
+import com.google.common.base.Supplier;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
+import org.apache.hadoop.util.FakeTimer;
+import org.apache.log4j.Level;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.regex.Pattern;
+
+import static org.junit.Assert.*;
+
+/**
+ * Tests the FSNamesystemLock, looking at lock compatibilities and
+ * proper logging of lock hold times.
+ */
+public class TestFSNamesystemLock {
+
+  @Test
+  public void testFsLockFairness() throws IOException, InterruptedException{
+    Configuration conf = new Configuration();
+
+    conf.setBoolean("dfs.namenode.fslock.fair", true);
+    FSNamesystemLock fsnLock = new FSNamesystemLock(conf);
+    assertTrue(fsnLock.coarseLock.isFair());
+
+    conf.setBoolean("dfs.namenode.fslock.fair", false);
+    fsnLock = new FSNamesystemLock(conf);
+    assertFalse(fsnLock.coarseLock.isFair());
+  }
+
+  @Test
+  public void testFSNamesystemLockCompatibility() {
+    FSNamesystemLock rwLock = new FSNamesystemLock(new Configuration());
+
+    assertEquals(0, rwLock.getReadHoldCount());
+    rwLock.readLock();
+    assertEquals(1, rwLock.getReadHoldCount());
+
+    rwLock.readLock();
+    assertEquals(2, rwLock.getReadHoldCount());
+
+    rwLock.readUnlock();
+    assertEquals(1, rwLock.getReadHoldCount());
+
+    rwLock.readUnlock();
+    assertEquals(0, rwLock.getReadHoldCount());
+
+    assertFalse(rwLock.isWriteLockedByCurrentThread());
+    assertEquals(0, rwLock.getWriteHoldCount());
+    rwLock.writeLock();
+    assertTrue(rwLock.isWriteLockedByCurrentThread());
+    assertEquals(1, rwLock.getWriteHoldCount());
+
+    rwLock.writeLock();
+    assertTrue(rwLock.isWriteLockedByCurrentThread());
+    assertEquals(2, rwLock.getWriteHoldCount());
+
+    rwLock.writeUnlock();
+    assertTrue(rwLock.isWriteLockedByCurrentThread());
+    assertEquals(1, rwLock.getWriteHoldCount());
+
+    rwLock.writeUnlock();
+    assertFalse(rwLock.isWriteLockedByCurrentThread());
+    assertEquals(0, rwLock.getWriteHoldCount());
+  }
+
+  @Test
+  public void testFSLockGetWaiterCount() throws InterruptedException {
+    final int threadCount = 3;
+    final CountDownLatch latch = new CountDownLatch(threadCount);
+    final Configuration conf = new Configuration();
+    conf.setBoolean("dfs.namenode.fslock.fair", true);
+    final FSNamesystemLock rwLock = new FSNamesystemLock(conf);
+    rwLock.writeLock();
+    ExecutorService helper = Executors.newFixedThreadPool(threadCount);
+
+    for (int x = 0; x < threadCount; x++) {
+      helper.execute(new Runnable() {
+        @Override
+        public void run() {
+          latch.countDown();
+          rwLock.readLock();
+        }
+      });
+    }
+
+    latch.await();
+    try {
+      GenericTestUtils.waitFor(new Supplier<Boolean>() {
+        @Override
+        public Boolean get() {
+          return (threadCount == rwLock.getQueueLength());
+        }
+      }, 10, 1000);
+    } catch (TimeoutException e) {
+      fail("Expected number of blocked thread not found");
+    }
+  }
+
+  /**
+   * Test when FSNamesystem write lock is held for a long time,
+   * logger will report it.
+   */
+  @Test(timeout=45000)
+  public void testFSWriteLockLongHoldingReport() throws Exception {
+    final long writeLockReportingThreshold = 100L;
+    final long writeLockSuppressWarningInterval = 10000L;
+    Configuration conf = new Configuration();
+    conf.setLong(
+        DFSConfigKeys.DFS_NAMENODE_WRITE_LOCK_REPORTING_THRESHOLD_MS_KEY,
+        writeLockReportingThreshold);
+    conf.setTimeDuration(DFSConfigKeys.DFS_LOCK_SUPPRESS_WARNING_INTERVAL_KEY,
+        writeLockSuppressWarningInterval, TimeUnit.MILLISECONDS);
+
+    final FakeTimer timer = new FakeTimer();
+    final FSNamesystemLock fsnLock = new FSNamesystemLock(conf, timer);
+    timer.advance(writeLockSuppressWarningInterval);
+
+    LogCapturer logs = LogCapturer.captureLogs(FSNamesystem.LOG);
+    GenericTestUtils.setLogLevel(FSNamesystem.LOG, Level.INFO);
+
+    // Don't report if the write lock is held for a short time
+    fsnLock.writeLock();
+    fsnLock.writeUnlock();
+    assertFalse(logs.getOutput().contains(GenericTestUtils.getMethodName()));
+
+    // Report if the write lock is held for a long time
+    fsnLock.writeLock();
+    timer.advance(writeLockReportingThreshold + 10);
+    logs.clearOutput();
+    fsnLock.writeUnlock();
+    assertTrue(logs.getOutput().contains(GenericTestUtils.getMethodName()));
+
+    // Track but do not report if the write lock is held (interruptibly) for
+    // a long time but time since last report does not exceed the suppress
+    // warning interval
+    fsnLock.writeLockInterruptibly();
+    timer.advance(writeLockReportingThreshold + 10);
+    logs.clearOutput();
+    fsnLock.writeUnlock();
+    assertFalse(logs.getOutput().contains(GenericTestUtils.getMethodName()));
+
+    // Track but do not report if it's held for a long time when re-entering
+    // write lock but time since last report does not exceed the suppress
+    // warning interval
+    fsnLock.writeLock();
+    timer.advance(writeLockReportingThreshold / 2 + 1);
+    fsnLock.writeLockInterruptibly();
+    timer.advance(writeLockReportingThreshold / 2 + 1);
+    fsnLock.writeLock();
+    timer.advance(writeLockReportingThreshold / 2);
+    logs.clearOutput();
+    fsnLock.writeUnlock();
+    assertFalse(logs.getOutput().contains(GenericTestUtils.getMethodName()));
+    logs.clearOutput();
+    fsnLock.writeUnlock();
+    assertFalse(logs.getOutput().contains(GenericTestUtils.getMethodName()));
+    logs.clearOutput();
+    fsnLock.writeUnlock();
+    assertFalse(logs.getOutput().contains(GenericTestUtils.getMethodName()));
+
+    // Report if it's held for a long time and time since last report exceeds
+    // the supress warning interval
+    timer.advance(writeLockSuppressWarningInterval);
+    fsnLock.writeLock();
+    timer.advance(writeLockReportingThreshold + 100);
+    logs.clearOutput();
+    fsnLock.writeUnlock();
+    assertTrue(logs.getOutput().contains(GenericTestUtils.getMethodName()));
+    assertTrue(logs.getOutput().contains(
+        "Number of suppressed write-lock reports: 2"));
+  }
+
+  /**
+   * Test when FSNamesystem read lock is held for a long time,
+   * logger will report it.
+   */
+  @Test(timeout=45000)
+  public void testFSReadLockLongHoldingReport() throws Exception {
+    final long readLockReportingThreshold = 100L;
+    final long readLockSuppressWarningInterval = 10000L;
+    final String readLockLogStmt = "FSNamesystem read lock held for ";
+    Configuration conf = new Configuration();
+    conf.setLong(
+        DFSConfigKeys.DFS_NAMENODE_READ_LOCK_REPORTING_THRESHOLD_MS_KEY,
+        readLockReportingThreshold);
+    conf.setTimeDuration(DFSConfigKeys.DFS_LOCK_SUPPRESS_WARNING_INTERVAL_KEY,
+        readLockSuppressWarningInterval, TimeUnit.MILLISECONDS);
+
+    final FakeTimer timer = new FakeTimer();
+    final FSNamesystemLock fsnLock = new FSNamesystemLock(conf, timer);
+    timer.advance(readLockSuppressWarningInterval);
+
+    LogCapturer logs = LogCapturer.captureLogs(FSNamesystem.LOG);
+    GenericTestUtils.setLogLevel(FSNamesystem.LOG, Level.INFO);
+
+    // Don't report if the read lock is held for a short time
+    fsnLock.readLock();
+    fsnLock.readUnlock();
+    assertFalse(logs.getOutput().contains(GenericTestUtils.getMethodName()) &&
+        logs.getOutput().contains(readLockLogStmt));
+
+    // Report the first read lock warning if it is held for a long time
+    fsnLock.readLock();
+    timer.advance(readLockReportingThreshold + 10);
+    logs.clearOutput();
+    fsnLock.readUnlock();
+    assertTrue(logs.getOutput().contains(GenericTestUtils.getMethodName()) &&
+        logs.getOutput().contains(readLockLogStmt));
+
+    // Track but do not Report if the write lock is held for a long time but
+    // time since last report does not exceed the suppress warning interval
+    fsnLock.readLock();
+    timer.advance(readLockReportingThreshold + 10);
+    logs.clearOutput();
+    fsnLock.readUnlock();
+    assertFalse(logs.getOutput().contains(GenericTestUtils.getMethodName()) &&
+        logs.getOutput().contains(readLockLogStmt));
+
+    // Track but do not Report if it's held for a long time when re-entering
+    // read lock but time since last report does not exceed the suppress
+    // warning interval
+    fsnLock.readLock();
+    timer.advance(readLockReportingThreshold / 2 + 1);
+    fsnLock.readLock();
+    timer.advance(readLockReportingThreshold / 2 + 1);
+    logs.clearOutput();
+    fsnLock.readUnlock();
+    assertFalse(logs.getOutput().contains(GenericTestUtils.getMethodName()) ||
+        logs.getOutput().contains(readLockLogStmt));
+    logs.clearOutput();
+    fsnLock.readUnlock();
+    assertFalse(logs.getOutput().contains(GenericTestUtils.getMethodName()) &&
+        logs.getOutput().contains(readLockLogStmt));
+
+    // Report if it's held for a long time (and time since last report
+    // exceeds the suppress warning interval) while another thread also has the
+    // read lock. Let one thread hold the lock long enough to activate an
+    // alert, then have another thread grab the read lock to ensure that this
+    // doesn't reset the timing.
+    timer.advance(readLockSuppressWarningInterval);
+    logs.clearOutput();
+    final CountDownLatch barrier = new CountDownLatch(1);
+    final CountDownLatch barrier2 = new CountDownLatch(1);
+    Thread t1 = new Thread() {
+      @Override
+      public void run() {
+        try {
+          fsnLock.readLock();
+          timer.advance(readLockReportingThreshold + 1);
+          barrier.countDown(); // Allow for t2 to acquire the read lock
+          barrier2.await(); // Wait until t2 has the read lock
+          fsnLock.readUnlock();
+        } catch (InterruptedException e) {
+          fail("Interrupted during testing");
+        }
+      }
+    };
+    Thread t2 = new Thread() {
+      @Override
+      public void run() {
+        try {
+          barrier.await(); // Wait until t1 finishes sleeping
+          fsnLock.readLock();
+          barrier2.countDown(); // Allow for t1 to unlock
+          fsnLock.readUnlock();
+        } catch (InterruptedException e) {
+          fail("Interrupted during testing");
+        }
+      }
+    };
+    t1.start();
+    t2.start();
+    t1.join();
+    t2.join();
+    // Look for the differentiating class names in the stack trace
+    String stackTracePatternString =
+        String.format("INFO.+%s(.+\n){4}\\Q%%s\\E\\.run", readLockLogStmt);
+    Pattern t1Pattern = Pattern.compile(
+        String.format(stackTracePatternString, t1.getClass().getName()));
+    assertTrue(t1Pattern.matcher(logs.getOutput()).find());
+    Pattern t2Pattern = Pattern.compile(
+        String.format(stackTracePatternString, t2.getClass().getName()));
+    assertFalse(t2Pattern.matcher(logs.getOutput()).find());
+    assertTrue(logs.getOutput().contains(
+        "Number of suppressed read-lock reports: 2"));
+  }
+
+}
\ No newline at end of file


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


[13/50] [abbrv] hadoop git commit: HDFS-10908. Improve StripedBlockReader#createBlockReader error logging. Contributed by Manoj Govindassamy.

Posted by su...@apache.org.
HDFS-10908. Improve StripedBlockReader#createBlockReader error logging. Contributed by Manoj Govindassamy.


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

Branch: refs/heads/YARN-2915
Commit: 2ab1ef15c5e0b05fed5106d6bbecb3ead2b25f9a
Parents: d6afcf3
Author: Wei-Chiu Chuang <we...@apache.org>
Authored: Fri Sep 30 12:35:47 2016 -0700
Committer: Wei-Chiu Chuang <we...@apache.org>
Committed: Fri Sep 30 12:35:47 2016 -0700

----------------------------------------------------------------------
 .../hdfs/server/datanode/erasurecode/StripedBlockReader.java       | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2ab1ef15/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedBlockReader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedBlockReader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedBlockReader.java
index 8f976c2..a27de9b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedBlockReader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedBlockReader.java
@@ -122,7 +122,7 @@ class StripedBlockReader {
           "", newConnectedPeer(block, dnAddr, blockToken, source), source,
           null, stripedReader.getCachingStrategy(), datanode.getTracer(), -1);
     } catch (IOException e) {
-      LOG.debug("Exception while creating remote block reader, datanode {}",
+      LOG.info("Exception while creating remote block reader, datanode {}",
           source, e);
       return null;
     }


---------------------------------------------------------------------
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: HDFS-10851. FSDirStatAndListingOp: stop passing path as string. Contributed by Daryn Sharp.

Posted by su...@apache.org.
HDFS-10851. FSDirStatAndListingOp: stop passing path as string. Contributed by Daryn Sharp.


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

Branch: refs/heads/YARN-2915
Commit: a0730aa5ced7666a8c92f9fb830b615f5f9f477a
Parents: 0670149
Author: Kihwal Lee <ki...@apache.org>
Authored: Fri Sep 30 13:03:24 2016 -0500
Committer: Kihwal Lee <ki...@apache.org>
Committed: Fri Sep 30 13:03:24 2016 -0500

----------------------------------------------------------------------
 .../hadoop/hdfs/server/namenode/FSDirAclOp.java |   4 +-
 .../server/namenode/FSDirStatAndListingOp.java  | 256 +++++++------------
 .../hdfs/server/namenode/FSDirXAttrOp.java      |   6 +-
 .../hdfs/server/namenode/FSDirectory.java       |  17 +-
 .../hdfs/server/namenode/FSEditLogLoader.java   |  18 +-
 .../server/namenode/INodeAttributeProvider.java |  10 +-
 .../hdfs/server/namenode/INodesInPath.java      |   2 +-
 .../server/namenode/TestSnapshotPathINodes.java |   6 +
 8 files changed, 126 insertions(+), 193 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a0730aa5/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAclOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAclOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAclOp.java
index 2153f02..afafd78 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAclOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAclOp.java
@@ -152,7 +152,6 @@ class FSDirAclOp {
     fsd.readLock();
     try {
       INodesInPath iip = fsd.resolvePath(pc, src);
-      src = iip.getPath();
       // There is no real inode for the path ending in ".snapshot", so return a
       // non-null, unpopulated AclStatus.  This is similar to getFileInfo.
       if (iip.isDotSnapshotDir() && fsd.getINode4DotSnapshot(iip) != null) {
@@ -163,8 +162,7 @@ class FSDirAclOp {
       }
       INode inode = FSDirectory.resolveLastINode(iip);
       int snapshotId = iip.getPathSnapshotId();
-      List<AclEntry> acl = AclStorage.readINodeAcl(fsd.getAttributes(src,
-              inode.getLocalNameBytes(), inode, snapshotId));
+      List<AclEntry> acl = AclStorage.readINodeAcl(fsd.getAttributes(iip));
       FsPermission fsPermission = inode.getFsPermission(snapshotId);
       return new AclStatus.Builder()
           .owner(inode.getUserName()).group(inode.getGroupName())

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a0730aa5/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
index 5072d68..4876fb1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
@@ -53,15 +53,12 @@ import static org.apache.hadoop.util.Time.now;
 class FSDirStatAndListingOp {
   static DirectoryListing getListingInt(FSDirectory fsd, final String srcArg,
       byte[] startAfter, boolean needLocation) throws IOException {
-    String src = null;
-
     final INodesInPath iip;
     if (fsd.isPermissionEnabled()) {
       FSPermissionChecker pc = fsd.getPermissionChecker();
       iip = fsd.resolvePath(pc, srcArg);
-      src = iip.getPath();
     } else {
-      src = FSDirectory.resolvePath(srcArg, fsd);
+      String src = FSDirectory.resolvePath(srcArg, fsd);
       iip = fsd.getINodesInPath(src, true);
     }
 
@@ -92,7 +89,7 @@ class FSDirStatAndListingOp {
       }
       isSuperUser = pc.isSuperUser();
     }
-    return getListing(fsd, iip, src, startAfter, needLocation, isSuperUser);
+    return getListing(fsd, iip, startAfter, needLocation, isSuperUser);
   }
 
   /**
@@ -161,7 +158,6 @@ class FSDirStatAndListingOp {
         "Negative length is not supported. File: " + src);
     CacheManager cm = fsd.getFSNamesystem().getCacheManager();
     BlockManager bm = fsd.getBlockManager();
-    boolean isReservedName = FSDirectory.isReservedRawName(src);
     fsd.readLock();
     try {
       final INodesInPath iip = fsd.resolvePath(pc, src);
@@ -184,7 +180,7 @@ class FSDirStatAndListingOp {
         isUc = false;
       }
 
-      final FileEncryptionInfo feInfo = isReservedName ? null
+      final FileEncryptionInfo feInfo = iip.isRaw() ? null
           : FSDirEncryptionZoneOp.getFileEncryptionInfo(fsd, inode,
           iip.getPathSnapshotId(), iip);
       final ErasureCodingPolicy ecPolicy = FSDirErasureCodingOp.
@@ -225,42 +221,39 @@ class FSDirStatAndListingOp {
    * @param fsd FSDirectory
    * @param iip the INodesInPath instance containing all the INodes along the
    *            path
-   * @param src the directory name
    * @param startAfter the name to start listing after
    * @param needLocation if block locations are returned
+   * @param includeStoragePolicy if storage policy is returned
    * @return a partial listing starting after startAfter
    */
   private static DirectoryListing getListing(FSDirectory fsd, INodesInPath iip,
-      String src, byte[] startAfter, boolean needLocation, boolean isSuperUser)
+      byte[] startAfter, boolean needLocation, boolean includeStoragePolicy)
       throws IOException {
-    String srcs = FSDirectory.normalizePath(src);
-    if (FSDirectory.isExactReservedName(srcs)) {
+    if (FSDirectory.isExactReservedName(iip.getPathComponents())) {
       return getReservedListing(fsd);
     }
 
     fsd.readLock();
     try {
-      if (srcs.endsWith(HdfsConstants.SEPARATOR_DOT_SNAPSHOT_DIR)) {
-        return getSnapshotsListing(fsd, srcs, startAfter);
+      if (iip.isDotSnapshotDir()) {
+        return getSnapshotsListing(fsd, iip, startAfter);
       }
       final int snapshot = iip.getPathSnapshotId();
       final INode targetNode = iip.getLastINode();
-      if (targetNode == null)
+      if (targetNode == null) {
         return null;
-      byte parentStoragePolicy = isSuperUser ?
-          targetNode.getStoragePolicyID() : HdfsConstants
-          .BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
+      }
+
+      byte parentStoragePolicy = includeStoragePolicy
+          ? targetNode.getStoragePolicyID()
+          : HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
 
       if (!targetNode.isDirectory()) {
         // return the file's status. note that the iip already includes the
         // target INode
-        INodeAttributes nodeAttrs = getINodeAttributes(
-            fsd, src, HdfsFileStatus.EMPTY_NAME, targetNode,
-            snapshot);
         return new DirectoryListing(
             new HdfsFileStatus[]{ createFileStatus(
-                fsd, HdfsFileStatus.EMPTY_NAME, nodeAttrs,
-                needLocation, parentStoragePolicy, iip)
+                fsd, iip, null, parentStoragePolicy, needLocation)
             }, 0);
       }
 
@@ -274,20 +267,15 @@ class FSDirStatAndListingOp {
       int listingCnt = 0;
       HdfsFileStatus listing[] = new HdfsFileStatus[numOfListing];
       for (int i = 0; i < numOfListing && locationBudget > 0; i++) {
-        INode cur = contents.get(startChild+i);
-        byte curPolicy = isSuperUser && !cur.isSymlink()?
-            cur.getLocalStoragePolicyID():
-            HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
-        INodeAttributes nodeAttrs = getINodeAttributes(
-            fsd, src, cur.getLocalNameBytes(), cur,
-            snapshot);
-        final INodesInPath iipWithChild = INodesInPath.append(iip, cur,
-            cur.getLocalNameBytes());
-        listing[i] = createFileStatus(fsd, cur.getLocalNameBytes(), nodeAttrs,
-            needLocation, getStoragePolicyID(curPolicy, parentStoragePolicy),
-            iipWithChild);
+        INode child = contents.get(startChild+i);
+        byte childStoragePolicy = (includeStoragePolicy && !child.isSymlink())
+            ? getStoragePolicyID(child.getLocalStoragePolicyID(),
+                                 parentStoragePolicy)
+            : parentStoragePolicy;
+        listing[i] =
+            createFileStatus(fsd, iip, child, childStoragePolicy, needLocation);
         listingCnt++;
-        if (needLocation) {
+        if (listing[i] instanceof HdfsLocatedFileStatus) {
             // Once we  hit lsLimit locations, stop.
             // This helps to prevent excessively large response payloads.
             // Approximate #locations with locatedBlockCount() * repl_factor
@@ -312,17 +300,16 @@ class FSDirStatAndListingOp {
    * Get a listing of all the snapshots of a snapshottable directory
    */
   private static DirectoryListing getSnapshotsListing(
-      FSDirectory fsd, String src, byte[] startAfter)
+      FSDirectory fsd, INodesInPath iip, byte[] startAfter)
       throws IOException {
     Preconditions.checkState(fsd.hasReadLock());
-    Preconditions.checkArgument(
-        src.endsWith(HdfsConstants.SEPARATOR_DOT_SNAPSHOT_DIR),
-        "%s does not end with %s", src, HdfsConstants.SEPARATOR_DOT_SNAPSHOT_DIR);
-
-    final String dirPath = FSDirectory.normalizePath(src.substring(0,
-        src.length() - HdfsConstants.DOT_SNAPSHOT_DIR.length()));
-
-    final INode node = fsd.getINode(dirPath);
+    Preconditions.checkArgument(iip.isDotSnapshotDir(),
+        "%s does not end with %s",
+        iip.getPath(), HdfsConstants.SEPARATOR_DOT_SNAPSHOT_DIR);
+    // drop off the null .snapshot component
+    iip = iip.getParentINodesInPath();
+    final String dirPath = iip.getPath();
+    final INode node = iip.getLastINode();
     final INodeDirectory dirNode = INodeDirectory.valueOf(node, dirPath);
     final DirectorySnapshottableFeature sf = dirNode.getDirectorySnapshottableFeature();
     if (sf == null) {
@@ -336,13 +323,8 @@ class FSDirStatAndListingOp {
     final HdfsFileStatus listing[] = new HdfsFileStatus[numOfListing];
     for (int i = 0; i < numOfListing; i++) {
       Snapshot.Root sRoot = snapshots.get(i + skipSize).getRoot();
-      INodeAttributes nodeAttrs = getINodeAttributes(
-          fsd, src, sRoot.getLocalNameBytes(),
-          node, Snapshot.CURRENT_STATE_ID);
-      listing[i] = createFileStatus(
-          fsd, sRoot.getLocalNameBytes(), nodeAttrs,
-          HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED,
-          INodesInPath.fromINode(sRoot));
+      listing[i] = createFileStatus(fsd, iip, sRoot,
+          HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED, false);
     }
     return new DirectoryListing(
         listing, snapshots.size() - skipSize - numOfListing);
@@ -360,7 +342,6 @@ class FSDirStatAndListingOp {
   /** Get the file info for a specific file.
    * @param fsd FSDirectory
    * @param iip The path to the file, the file is included
-   * @param isRawPath true if a /.reserved/raw pathname was passed by the user
    * @param includeStoragePolicy whether to include storage policy
    * @return object containing information regarding the file
    *         or null if file not found
@@ -373,15 +354,10 @@ class FSDirStatAndListingOp {
       if (node == null) {
         return null;
       }
-
-      byte policyId = includeStoragePolicy && !node.isSymlink() ?
-          node.getStoragePolicyID() :
-          HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
-      INodeAttributes nodeAttrs = getINodeAttributes(fsd, iip.getPath(),
-                                                     HdfsFileStatus.EMPTY_NAME,
-                                                     node, iip.getPathSnapshotId());
-      return createFileStatus(fsd, HdfsFileStatus.EMPTY_NAME, nodeAttrs,
-                              policyId, iip);
+      byte policy = (includeStoragePolicy && !node.isSymlink())
+          ? node.getStoragePolicyID()
+          : HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
+      return createFileStatus(fsd, iip, null, policy, false);
     } finally {
       fsd.readUnlock();
     }
@@ -408,48 +384,41 @@ class FSDirStatAndListingOp {
   }
 
   /**
-   * create an hdfs file status from an inode
+   * create a hdfs file status from an iip.
+   * @param fsd FSDirectory
+   * @param iip The INodesInPath containing the INodeFile and its ancestors
+   * @return HdfsFileStatus without locations or storage policy
+   */
+  static HdfsFileStatus createFileStatusForEditLog(
+      FSDirectory fsd, INodesInPath iip) throws IOException {
+    return createFileStatus(fsd, iip,
+        null, HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED, false);
+  }
+
+  /**
+   * create a hdfs file status from an iip.
    *
    * @param fsd FSDirectory
-   * @param path the local name
+   * @param iip The INodesInPath containing the INodeFile and its ancestors.
+   * @param child for a directory listing of the iip, else null
+   * @param storagePolicy for the path or closest ancestor
    * @param needLocation if block locations need to be included or not
-   * @param isRawPath true if this is being called on behalf of a path in
-   *                  /.reserved/raw
-   * @param iip the INodesInPath containing the target INode and its ancestors
+   * @param includeStoragePolicy if storage policy should be returned
    * @return a file status
    * @throws java.io.IOException if any error occurs
    */
   private static HdfsFileStatus createFileStatus(
-      FSDirectory fsd, byte[] path, INodeAttributes nodeAttrs,
-      boolean needLocation, byte storagePolicy, INodesInPath iip)
-      throws IOException {
-    if (needLocation) {
-      return createLocatedFileStatus(fsd, path, nodeAttrs, storagePolicy, iip);
-    } else {
-      return createFileStatus(fsd, path, nodeAttrs, storagePolicy, iip);
+      FSDirectory fsd, INodesInPath iip, INode child, byte storagePolicy,
+      boolean needLocation) throws IOException {
+    assert fsd.hasReadLock();
+    // only directory listing sets the status name.
+    byte[] name = HdfsFileStatus.EMPTY_NAME;
+    if (child != null) {
+      name = child.getLocalNameBytes();
+      // have to do this for EC and EZ lookups...
+      iip = INodesInPath.append(iip, child, name);
     }
-  }
 
-  /**
-   * Create FileStatus for an given INodeFile.
-   * @param iip The INodesInPath containing the INodeFile and its ancestors
-   */
-  static HdfsFileStatus createFileStatusForEditLog(
-      FSDirectory fsd, String fullPath, byte[] path,
-      byte storagePolicy, int snapshot, boolean isRawPath,
-      INodesInPath iip) throws IOException {
-    INodeAttributes nodeAttrs = getINodeAttributes(
-        fsd, fullPath, path, iip.getLastINode(), snapshot);
-    return createFileStatus(fsd, path, nodeAttrs, storagePolicy, iip);
-  }
-
-  /**
-   * create file status for a given INode
-   * @param iip the INodesInPath containing the target INode and its ancestors
-   */
-  static HdfsFileStatus createFileStatus(
-      FSDirectory fsd, byte[] path, INodeAttributes nodeAttrs,
-      byte storagePolicy, INodesInPath iip) throws IOException {
     long size = 0;     // length is zero for directories
     short replication = 0;
     long blocksize = 0;
@@ -457,6 +426,7 @@ class FSDirStatAndListingOp {
     final INode node = iip.getLastINode();
     final int snapshot = iip.getPathSnapshotId();
     final boolean isRawPath = iip.isRaw();
+    LocatedBlocks loc = null;
 
     final FileEncryptionInfo feInfo = isRawPath ? null : FSDirEncryptionZoneOp
         .getFileEncryptionInfo(fsd, node, snapshot, iip);
@@ -471,6 +441,18 @@ class FSDirStatAndListingOp {
       blocksize = fileNode.getPreferredBlockSize();
       isEncrypted = (feInfo != null)
           || (isRawPath && FSDirEncryptionZoneOp.isInAnEZ(fsd, iip));
+      if (needLocation) {
+        final boolean inSnapshot = snapshot != Snapshot.CURRENT_STATE_ID;
+        final boolean isUc = !inSnapshot && fileNode.isUnderConstruction();
+        final long fileSize = !inSnapshot && isUc
+            ? fileNode.computeFileSizeNotIncludingLastUcBlock() : size;
+        loc = fsd.getBlockManager().createLocatedBlocks(
+            fileNode.getBlocks(snapshot), fileSize, isUc, 0L, size, false,
+            inSnapshot, feInfo, ecPolicy);
+        if (loc == null) {
+          loc = new LocatedBlocks();
+        }
+      }
     } else {
       isEncrypted = FSDirEncryptionZoneOp.isInAnEZ(fsd, iip);
     }
@@ -478,7 +460,8 @@ class FSDirStatAndListingOp {
     int childrenNum = node.isDirectory() ?
         node.asDirectory().getChildrenNum(snapshot) : 0;
 
-    return new HdfsFileStatus(
+    INodeAttributes nodeAttrs = fsd.getAttributes(iip);
+    HdfsFileStatus status = createFileStatus(
         size,
         node.isDirectory(),
         replication,
@@ -489,73 +472,13 @@ class FSDirStatAndListingOp {
         nodeAttrs.getUserName(),
         nodeAttrs.getGroupName(),
         node.isSymlink() ? node.asSymlink().getSymlink() : null,
-        path,
+        name,
         node.getId(),
         childrenNum,
         feInfo,
         storagePolicy,
-        ecPolicy);
-  }
-
-  private static INodeAttributes getINodeAttributes(
-      FSDirectory fsd, String fullPath, byte[] path, INode node, int snapshot) {
-    return fsd.getAttributes(fullPath, path, node, snapshot);
-  }
-
-  /**
-   * Create FileStatus with location info by file INode
-   * @param iip the INodesInPath containing the target INode and its ancestors
-   */
-  private static HdfsFileStatus createLocatedFileStatus(
-      FSDirectory fsd, byte[] path, INodeAttributes nodeAttrs,
-      byte storagePolicy, INodesInPath iip) throws IOException {
-    assert fsd.hasReadLock();
-    long size = 0; // length is zero for directories
-    short replication = 0;
-    long blocksize = 0;
-    LocatedBlocks loc = null;
-    final boolean isEncrypted;
-    final INode node = iip.getLastINode();
-    final int snapshot = iip.getPathSnapshotId();
-    final boolean isRawPath = iip.isRaw();
-
-    final FileEncryptionInfo feInfo = isRawPath ? null : FSDirEncryptionZoneOp
-        .getFileEncryptionInfo(fsd, node, snapshot, iip);
-    final ErasureCodingPolicy ecPolicy = FSDirErasureCodingOp.getErasureCodingPolicy(
-        fsd.getFSNamesystem(), iip);
-    if (node.isFile()) {
-      final INodeFile fileNode = node.asFile();
-      size = fileNode.computeFileSize(snapshot);
-      replication = fileNode.getFileReplication(snapshot);
-      blocksize = fileNode.getPreferredBlockSize();
-
-      final boolean inSnapshot = snapshot != Snapshot.CURRENT_STATE_ID;
-      final boolean isUc = !inSnapshot && fileNode.isUnderConstruction();
-      final long fileSize = !inSnapshot && isUc ?
-          fileNode.computeFileSizeNotIncludingLastUcBlock() : size;
-
-      loc = fsd.getBlockManager().createLocatedBlocks(
-          fileNode.getBlocks(snapshot), fileSize, isUc, 0L, size, false,
-          inSnapshot, feInfo, ecPolicy);
-      if (loc == null) {
-        loc = new LocatedBlocks();
-      }
-      isEncrypted = (feInfo != null)
-          || (isRawPath && FSDirEncryptionZoneOp.isInAnEZ(fsd, iip));
-    } else {
-      isEncrypted = FSDirEncryptionZoneOp.isInAnEZ(fsd, iip);
-    }
-    int childrenNum = node.isDirectory() ?
-        node.asDirectory().getChildrenNum(snapshot) : 0;
-
-    HdfsLocatedFileStatus status =
-        new HdfsLocatedFileStatus(size, node.isDirectory(), replication,
-          blocksize, node.getModificationTime(snapshot),
-          node.getAccessTime(snapshot),
-          getPermissionForFileStatus(nodeAttrs, isEncrypted),
-          nodeAttrs.getUserName(), nodeAttrs.getGroupName(),
-          node.isSymlink() ? node.asSymlink().getSymlink() : null, path,
-          node.getId(), loc, childrenNum, feInfo, storagePolicy, ecPolicy);
+        ecPolicy,
+        loc);
     // Set caching information for the located blocks.
     if (loc != null) {
       CacheManager cacheManager = fsd.getFSNamesystem().getCacheManager();
@@ -566,6 +489,23 @@ class FSDirStatAndListingOp {
     return status;
   }
 
+  private static HdfsFileStatus createFileStatus(long length, boolean isdir,
+      int replication, long blocksize, long mtime,
+      long atime, FsPermission permission, String owner, String group,
+      byte[] symlink, byte[] path, long fileId, int childrenNum,
+      FileEncryptionInfo feInfo, byte storagePolicy,
+      ErasureCodingPolicy ecPolicy, LocatedBlocks locations) {
+    if (locations == null) {
+      return new HdfsFileStatus(length, isdir, replication, blocksize,
+          mtime, atime, permission, owner, group, symlink, path, fileId,
+          childrenNum, feInfo, storagePolicy, ecPolicy);
+    } else {
+      return new HdfsLocatedFileStatus(length, isdir, replication, blocksize,
+          mtime, atime, permission, owner, group, symlink, path, fileId,
+          locations, childrenNum, feInfo, storagePolicy, ecPolicy);
+    }
+  }
+
   /**
    * Returns an inode's FsPermission for use in an outbound FileStatus.  If the
    * inode has an ACL or is for an encrypted file/dir, then this method will

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a0730aa5/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirXAttrOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirXAttrOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirXAttrOp.java
index 746fdb7..08016c3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirXAttrOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirXAttrOp.java
@@ -429,11 +429,7 @@ class FSDirXAttrOp {
       throws IOException {
     fsd.readLock();
     try {
-      String src = iip.getPath();
-      INode inode = FSDirectory.resolveLastINode(iip);
-      int snapshotId = iip.getPathSnapshotId();
-      return XAttrStorage.readINodeXAttrs(fsd.getAttributes(src,
-              inode.getLocalNameBytes(), inode, snapshotId));
+      return XAttrStorage.readINodeXAttrs(fsd.getAttributes(iip));
     } finally {
       fsd.readUnlock();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a0730aa5/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
index 2c7a268..7db2106 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
@@ -1807,14 +1807,19 @@ public class FSDirectory implements Closeable {
     inodeId.setCurrentValue(newValue);
   }
 
-  INodeAttributes getAttributes(String fullPath, byte[] path,
-      INode node, int snapshot) {
+  INodeAttributes getAttributes(INodesInPath iip)
+      throws FileNotFoundException {
+    INode node = FSDirectory.resolveLastINode(iip);
+    int snapshot = iip.getPathSnapshotId();
     INodeAttributes nodeAttrs = node.getSnapshotINode(snapshot);
     if (attributeProvider != null) {
-      fullPath = fullPath
-          + (fullPath.endsWith(Path.SEPARATOR) ? "" : Path.SEPARATOR)
-          + DFSUtil.bytes2String(path);
-      nodeAttrs = attributeProvider.getAttributes(fullPath, nodeAttrs);
+      // permission checking sends the full components array including the
+      // first empty component for the root.  however file status
+      // related calls are expected to strip out the root component according
+      // to TestINodeAttributeProvider.
+      byte[][] components = iip.getPathComponents();
+      components = Arrays.copyOfRange(components, 1, components.length);
+      nodeAttrs = attributeProvider.getAttributes(components, nodeAttrs);
     }
     return nodeAttrs;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a0730aa5/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
index 25f5a4f..ee6206a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
@@ -383,10 +383,8 @@ public class FSEditLogLoader {
 
         // add the op into retry cache if necessary
         if (toAddRetryCache) {
-          HdfsFileStatus stat = FSDirStatAndListingOp.createFileStatusForEditLog(
-              fsNamesys.dir, path, HdfsFileStatus.EMPTY_NAME,
-              HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED, Snapshot.CURRENT_STATE_ID,
-              false, iip);
+          HdfsFileStatus stat =
+              FSDirStatAndListingOp.createFileStatusForEditLog(fsDir, iip);
           fsNamesys.addCacheEntryWithPayload(addCloseOp.rpcClientId,
               addCloseOp.rpcCallId, stat);
         }
@@ -402,10 +400,8 @@ public class FSEditLogLoader {
               false);
           // add the op into retry cache if necessary
           if (toAddRetryCache) {
-            HdfsFileStatus stat = FSDirStatAndListingOp.createFileStatusForEditLog(
-                fsNamesys.dir, path, HdfsFileStatus.EMPTY_NAME,
-                HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED,
-                Snapshot.CURRENT_STATE_ID, false, iip);
+            HdfsFileStatus stat =
+                FSDirStatAndListingOp.createFileStatusForEditLog(fsDir, iip);
             fsNamesys.addCacheEntryWithPayload(addCloseOp.rpcClientId,
                 addCloseOp.rpcCallId, new LastBlockWithStatus(lb, stat));
           }
@@ -480,10 +476,8 @@ public class FSEditLogLoader {
             false, false);
         // add the op into retry cache if necessary
         if (toAddRetryCache) {
-          HdfsFileStatus stat = FSDirStatAndListingOp.createFileStatusForEditLog(
-              fsNamesys.dir, path, HdfsFileStatus.EMPTY_NAME,
-              HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED,
-              Snapshot.CURRENT_STATE_ID, false, iip);
+          HdfsFileStatus stat =
+              FSDirStatAndListingOp.createFileStatusForEditLog(fsDir, iip);
           fsNamesys.addCacheEntryWithPayload(appendOp.rpcClientId,
               appendOp.rpcCallId, new LastBlockWithStatus(lb, stat));
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a0730aa5/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeAttributeProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeAttributeProvider.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeAttributeProvider.java
index 2e0775b..2f9bc37 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeAttributeProvider.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeAttributeProvider.java
@@ -17,13 +17,6 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Set;
-
-import com.google.common.annotations.VisibleForTesting;
-
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -87,7 +80,7 @@ public abstract class INodeAttributeProvider {
    */
   public abstract void stop();
 
-  @VisibleForTesting
+  @Deprecated
   String[] getPathElements(String path) {
     path = path.trim();
     if (path.charAt(0) != Path.SEPARATOR_CHAR) {
@@ -115,6 +108,7 @@ public abstract class INodeAttributeProvider {
     return pathElements;
   }
 
+  @Deprecated
   public INodeAttributes getAttributes(String fullPath, INodeAttributes inode) {
     return getAttributes(getPathElements(fullPath), inode);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a0730aa5/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java
index 8f65ff8..04d3bda 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java
@@ -399,7 +399,7 @@ public class INodesInPath {
    */
   private INodesInPath getAncestorINodesInPath(int length) {
     Preconditions.checkArgument(length >= 0 && length < inodes.length);
-    Preconditions.checkState(!isSnapshot());
+    Preconditions.checkState(isDotSnapshotDir() || !isSnapshot());
     final INode[] anodes = new INode[length];
     final byte[][] apath = new byte[length][];
     System.arraycopy(this.inodes, 0, anodes, 0, length);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a0730aa5/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSnapshotPathINodes.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSnapshotPathINodes.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSnapshotPathINodes.java
index 24ec1a2..3a318bc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSnapshotPathINodes.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSnapshotPathINodes.java
@@ -166,6 +166,9 @@ public class TestSnapshotPathINodes {
     assertEquals(sub1.toString(), nodesInPath.getPath(2));
     assertEquals(file1.toString(), nodesInPath.getPath(3));
 
+    assertEquals(file1.getParent().toString(),
+                 nodesInPath.getParentINodesInPath().getPath());
+
     nodesInPath = INodesInPath.resolve(fsdir.rootDir, components, false);
     assertEquals(nodesInPath.length(), components.length);
     assertSnapshot(nodesInPath, false, null, -1);
@@ -212,6 +215,9 @@ public class TestSnapshotPathINodes {
     // The number of INodes returned should still be components.length
     // since we put a null in the inode array for ".snapshot"
     assertEquals(nodesInPath.length(), components.length);
+    // ensure parent inodes can strip the .snapshot
+    assertEquals(sub1.toString(),
+        nodesInPath.getParentINodesInPath().getPath());
 
     // No SnapshotRoot dir is included in the resolved inodes  
     assertSnapshot(nodesInPath, true, snapshot, -1);


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


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

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


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

Branch: refs/heads/YARN-2915
Commit: ad03510b7b23c098730d8d657447ca3349d273ea
Parents: 607705c
Author: Subru Krishnan <su...@apache.org>
Authored: Tue Jul 19 15:08:25 2016 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon Oct 3 14:09:39 2016 -0700

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ad03510b/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 01b1da7..7f0a4e0 100644
--- a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
+++ b/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
@@ -21,6 +21,9 @@
     <Package name="org.apache.hadoop.yarn.proto" />
   </Match>
   <Match>
+    <Package name="org.apache.hadoop.yarn.federation.proto" />
+  </Match>
+  <Match>
     <Class name="~org\.apache\.hadoop\.yarn\.ipc\.RpcProtos.*" />
   </Match>
 

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


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


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

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


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

Branch: refs/heads/YARN-2915
Commit: 760d0be71ad976d74c87fa0a1d560e33462f9b00
Parents: 1191de4
Author: Subru Krishnan <su...@apache.org>
Authored: Mon Aug 15 14:47:02 2016 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon Oct 3 14:15:36 2016 -0700

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


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

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

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

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

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

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


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


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

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


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


[24/50] [abbrv] hadoop git commit: YARN-4855. Should check if node exists when replace nodelabels. Contributeed by Tao Jie

Posted by su...@apache.org.
YARN-4855. Should check if node exists when replace nodelabels. Contributeed by Tao Jie


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

Branch: refs/heads/YARN-2915
Commit: 6e130c308cf1b97e8386b6a43c26d72d2850119c
Parents: 8285703
Author: Naganarasimha <na...@apache.org>
Authored: Mon Oct 3 02:02:26 2016 -0400
Committer: Naganarasimha <na...@apache.org>
Committed: Mon Oct 3 02:02:26 2016 -0400

----------------------------------------------------------------------
 .../ReplaceLabelsOnNodeRequest.java             |   8 ++
 ..._server_resourcemanager_service_protos.proto |   2 +-
 .../hadoop/yarn/client/cli/RMAdminCLI.java      |  39 ++++---
 .../hadoop/yarn/client/cli/TestRMAdminCLI.java  |   3 +-
 .../pb/ReplaceLabelsOnNodeRequestPBImpl.java    |  14 ++-
 .../server/resourcemanager/AdminService.java    |  46 +++++++++
 .../resourcemanager/TestRMAdminService.java     | 103 ++++++++++++++++++-
 7 files changed, 197 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6e130c30/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/ReplaceLabelsOnNodeRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/ReplaceLabelsOnNodeRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/ReplaceLabelsOnNodeRequest.java
index 28e261a..1b8e687 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/ReplaceLabelsOnNodeRequest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/ReplaceLabelsOnNodeRequest.java
@@ -44,4 +44,12 @@ public abstract class ReplaceLabelsOnNodeRequest {
   @Public
   @Evolving
   public abstract Map<NodeId, Set<String>> getNodeToLabels();
+
+  @Public
+  @Evolving
+  public abstract void setFailOnUnknownNodes(boolean failOnUnknownNodes);
+
+  @Public
+  @Evolving
+  public abstract boolean getFailOnUnknownNodes();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6e130c30/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/yarn_server_resourcemanager_service_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/yarn_server_resourcemanager_service_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/yarn_server_resourcemanager_service_protos.proto
index b9f30db..16d8097 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/yarn_server_resourcemanager_service_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/yarn_server_resourcemanager_service_protos.proto
@@ -99,10 +99,10 @@ message RemoveFromClusterNodeLabelsResponseProto {
 
 message ReplaceLabelsOnNodeRequestProto {
   repeated NodeIdToLabelsNameProto nodeToLabels = 1;
+  optional bool failOnUnknownNodes = 2;
 }
 
 message ReplaceLabelsOnNodeResponseProto {
-  
 }
 
 message UpdateNodeLabelsResponseProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6e130c30/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/RMAdminCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/RMAdminCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/RMAdminCLI.java
index 7a898a1..640f8e3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/RMAdminCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/RMAdminCLI.java
@@ -130,11 +130,13 @@ public class RMAdminCLI extends HAAdmin {
               new UsageInfo("<label1,label2,label3> (label splitted by \",\")",
                   "remove from cluster node labels"))
           .put("-replaceLabelsOnNode",
-              new UsageInfo(
+              new UsageInfo("[-failOnUnknownNodes] " +
                   "<\"node1[:port]=label1,label2 node2[:port]=label1,label2\">",
-                  "replace labels on nodes"
-                      + " (please note that we do not support specifying multiple"
-                      + " labels on a single host for now.)"))
+              "replace labels on nodes"
+                  + " (please note that we do not support specifying multiple"
+                  + " labels on a single host for now.)\n\t\t"
+                  + "[-failOnUnknownNodes] is optional, when we set this"
+                  + " option, it will fail if specified nodes are unknown."))
           .put("-directlyAccessNodeLabelStore",
               new UsageInfo("", "This is DEPRECATED, will be removed in future releases. Directly access node label store, "
                   + "with this option, all node label related operations"
@@ -246,8 +248,8 @@ public class RMAdminCLI extends HAAdmin {
         " [-addToClusterNodeLabels <\"label1(exclusive=true),"
             + "label2(exclusive=false),label3\">]" +
         " [-removeFromClusterNodeLabels <label1,label2,label3>]" +
-        " [-replaceLabelsOnNode <\"node1[:port]=label1,label2" +
-        " node2[:port]=label1\">]" +
+        " [-replaceLabelsOnNode [-failOnUnknownNodes] "
+            + "<\"node1[:port]=label1,label2 node2[:port]=label1\">]" +
         " [-directlyAccessNodeLabelStore]" +
         " [-refreshClusterMaxPriority]" +
         " [-updateNodeResource [NodeID] [MemSize] [vCores]" +
@@ -302,7 +304,7 @@ public class RMAdminCLI extends HAAdmin {
     return ClientRMProxy.createRMProxy(conf,
         ResourceManagerAdministrationProtocol.class);
   }
-  
+
   private int refreshQueues() throws IOException, YarnException {
     // Refresh the queue properties
     ResourceManagerAdministrationProtocol adminProtocol = createAdminProtocol();
@@ -657,14 +659,14 @@ public class RMAdminCLI extends HAAdmin {
     return map;
   }
 
-  private int replaceLabelsOnNodes(String args) throws IOException,
-      YarnException {
+  private int replaceLabelsOnNodes(String args, boolean failOnUnknownNodes)
+      throws IOException, YarnException {
     Map<NodeId, Set<String>> map = buildNodeLabelsMapFromStr(args);
-    return replaceLabelsOnNodes(map);
+    return replaceLabelsOnNodes(map, failOnUnknownNodes);
   }
 
-  private int replaceLabelsOnNodes(Map<NodeId, Set<String>> map)
-      throws IOException, YarnException {
+  private int replaceLabelsOnNodes(Map<NodeId, Set<String>> map,
+      boolean failOnUnknownNodes) throws IOException, YarnException {
     if (directlyAccessNodeLabelStore) {
       getNodeLabelManagerInstance(getConf()).replaceLabelsOnNode(map);
     } else {
@@ -672,11 +674,12 @@ public class RMAdminCLI extends HAAdmin {
           createAdminProtocol();
       ReplaceLabelsOnNodeRequest request =
           ReplaceLabelsOnNodeRequest.newInstance(map);
+      request.setFailOnUnknownNodes(failOnUnknownNodes);
       adminProtocol.replaceLabelsOnNode(request);
     }
     return 0;
   }
-  
+
   @Override
   public int run(String[] args) throws Exception {
     // -directlyAccessNodeLabelStore is a additional option for node label
@@ -783,8 +786,16 @@ public class RMAdminCLI extends HAAdmin {
           System.err.println(NO_MAPPING_ERR_MSG);
           printUsage("", isHAEnabled);
           exitCode = -1;
+        } else if ("-failOnUnknownNodes".equals(args[i])) {
+          if (i + 1 >= args.length) {
+            System.err.println(NO_MAPPING_ERR_MSG);
+            printUsage("", isHAEnabled);
+            exitCode = -1;
+          } else {
+            exitCode = replaceLabelsOnNodes(args[i + 1], true);
+          }
         } else {
-          exitCode = replaceLabelsOnNodes(args[i]);
+          exitCode = replaceLabelsOnNodes(args[i], false);
         }
       } else {
         exitCode = -1;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6e130c30/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestRMAdminCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestRMAdminCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestRMAdminCLI.java
index bea6e39..9e20a43 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestRMAdminCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestRMAdminCLI.java
@@ -469,7 +469,7 @@ public class TestRMAdminCLI {
               "[username]] [-addToClusterNodeLabels " +
               "<\"label1(exclusive=true),label2(exclusive=false),label3\">] " +
               "[-removeFromClusterNodeLabels <label1,label2,label3>] " +
-              "[-replaceLabelsOnNode " +
+              "[-replaceLabelsOnNode [-failOnUnknownNodes] " +
               "<\"node1[:port]=label1,label2 node2[:port]=label1\">] " +
               "[-directlyAccessNodeLabelStore] [-refreshClusterMaxPriority] " +
               "[-updateNodeResource [NodeID] [MemSize] [vCores] " +
@@ -564,6 +564,7 @@ public class TestRMAdminCLI {
               + " [username]] [-addToClusterNodeLabels <\"label1(exclusive=true),"
                   + "label2(exclusive=false),label3\">]"
               + " [-removeFromClusterNodeLabels <label1,label2,label3>] [-replaceLabelsOnNode "
+              + "[-failOnUnknownNodes] "
               + "<\"node1[:port]=label1,label2 node2[:port]=label1\">] [-directlyAccessNodeLabelStore] "
               + "[-refreshClusterMaxPriority] "
               + "[-updateNodeResource [NodeID] [MemSize] [vCores] "

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6e130c30/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/ReplaceLabelsOnNodeRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/ReplaceLabelsOnNodeRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/ReplaceLabelsOnNodeRequestPBImpl.java
index 22e561c..3b15b27 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/ReplaceLabelsOnNodeRequestPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/ReplaceLabelsOnNodeRequestPBImpl.java
@@ -146,10 +146,22 @@ public class ReplaceLabelsOnNodeRequestPBImpl extends
     nodeIdToLabels.putAll(map);
   }
 
+  @Override
+  public boolean getFailOnUnknownNodes() {
+    ReplaceLabelsOnNodeRequestProtoOrBuilder p = viaProto ? proto : builder;
+    return p.getFailOnUnknownNodes();
+  }
+
+  @Override
+  public void setFailOnUnknownNodes(boolean failOnUnknownNodes) {
+    maybeInitBuilder();
+    builder.setFailOnUnknownNodes(failOnUnknownNodes);
+  }
+
   private NodeIdProto convertToProtoFormat(NodeId t) {
     return ((NodeIdPBImpl) t).getProto();
   }
-  
+
   @Override
   public int hashCode() {
     assert false : "hashCode not designed";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6e130c30/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.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/AdminService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
index db55264..33daf7f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
@@ -21,6 +21,9 @@ package org.apache.hadoop.yarn.server.resourcemanager;
 import java.io.IOException;
 import java.io.InputStream;
 import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
@@ -806,6 +809,49 @@ public class AdminService extends CompositeService implements
 
     ReplaceLabelsOnNodeResponse response =
         recordFactory.newRecordInstance(ReplaceLabelsOnNodeResponse.class);
+
+    if (request.getFailOnUnknownNodes()) {
+      // verify if nodes have registered to RM
+      List<NodeId> unknownNodes = new ArrayList<>();
+      for (NodeId requestedNode : request.getNodeToLabels().keySet()) {
+        boolean isKnown = false;
+        // both active and inactive nodes are recognized as known nodes
+        if (requestedNode.getPort() != 0) {
+          if (rmContext.getRMNodes().containsKey(requestedNode)
+              || rmContext.getInactiveRMNodes().containsKey(requestedNode)) {
+            isKnown = true;
+          }
+        } else {
+          for (NodeId knownNode : rmContext.getRMNodes().keySet()) {
+            if (knownNode.getHost().equals(requestedNode.getHost())) {
+              isKnown = true;
+              break;
+            }
+          }
+          if (!isKnown) {
+            for (NodeId knownNode : rmContext.getInactiveRMNodes().keySet()) {
+              if (knownNode.getHost().equals(requestedNode.getHost())) {
+                isKnown = true;
+                break;
+              }
+            }
+          }
+        }
+        if (!isKnown) {
+          unknownNodes.add(requestedNode);
+        }
+      }
+
+      if (!unknownNodes.isEmpty()) {
+        RMAuditLogger.logFailure(user.getShortUserName(), operation, "",
+            "AdminService",
+            "Failed to replace labels as there are unknown nodes:"
+                + Arrays.toString(unknownNodes.toArray()));
+        throw RPCUtil.getRemoteException(new IOException(
+            "Failed to replace labels as there are unknown nodes:"
+                + Arrays.toString(unknownNodes.toArray())));
+      }
+    }
     try {
       rmContext.getNodeLabelManager().replaceLabelsOnNode(
           request.getNodeToLabels());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6e130c30/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.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/TestRMAdminService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.java
index 0b65c0b..a3022f7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.java
@@ -28,6 +28,7 @@ import java.io.IOException;
 import java.io.PrintWriter;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 
 import org.apache.hadoop.conf.Configuration;
@@ -64,9 +65,9 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.ReplaceLabelsOnNodeRequ
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.resource.DynamicResourceConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
+import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
-import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -1086,6 +1087,106 @@ public class TestRMAdminService {
   }
 
   @Test
+  public void testModifyLabelsOnUnknownNodes() throws IOException,
+      YarnException {
+    // create RM and set it's ACTIVE, and set distributed node label
+    // configuration to true
+    rm = new MockRM();
+
+    ((RMContextImpl) rm.getRMContext())
+        .setHAServiceState(HAServiceState.ACTIVE);
+    Map<NodeId, RMNode> rmNodes = rm.getRMContext().getRMNodes();
+    rmNodes.put(NodeId.newInstance("host1", 1111),
+        new RMNodeImpl(null, rm.getRMContext(), "host1", 0, 0, null, null,
+                null));
+    rmNodes.put(NodeId.newInstance("host2", 2222),
+            new RMNodeImpl(null, rm.getRMContext(), "host2", 0, 0, null, null,
+                null));
+    rmNodes.put(NodeId.newInstance("host3", 3333),
+            new RMNodeImpl(null, rm.getRMContext(), "host3", 0, 0, null, null,
+                null));
+    Map<NodeId, RMNode> rmInactiveNodes = rm.getRMContext()
+        .getInactiveRMNodes();
+    rmInactiveNodes.put(NodeId.newInstance("host4", 4444),
+        new RMNodeImpl(null, rm.getRMContext(), "host4", 0, 0, null, null,
+                null));
+    RMNodeLabelsManager labelMgr = rm.rmContext.getNodeLabelManager();
+
+    // by default, distributed configuration for node label is disabled, this
+    // should pass
+    labelMgr.addToCluserNodeLabelsWithDefaultExclusivity(ImmutableSet.of("x",
+        "y"));
+    // replace known node
+    ReplaceLabelsOnNodeRequest request1 = ReplaceLabelsOnNodeRequest
+        .newInstance(ImmutableMap.of(NodeId.newInstance("host1", 1111),
+            (Set<String>) ImmutableSet.of("x")));
+    request1.setFailOnUnknownNodes(true);
+    try {
+      rm.adminService.replaceLabelsOnNode(request1);
+    } catch (Exception ex) {
+      fail("should not fail on known node");
+    }
+
+    // replace known node with wildcard port
+    ReplaceLabelsOnNodeRequest request2 = ReplaceLabelsOnNodeRequest
+        .newInstance(ImmutableMap.of(NodeId.newInstance("host1", 0),
+            (Set<String>) ImmutableSet.of("x")));
+    request2.setFailOnUnknownNodes(true);
+    try {
+      rm.adminService.replaceLabelsOnNode(request2);
+    } catch (Exception ex) {
+      fail("should not fail on known node");
+    }
+
+    // replace unknown node
+    ReplaceLabelsOnNodeRequest request3 = ReplaceLabelsOnNodeRequest
+        .newInstance(ImmutableMap.of(NodeId.newInstance("host5", 0),
+            (Set<String>) ImmutableSet.of("x")));
+    request3.setFailOnUnknownNodes(true);
+    try {
+      rm.adminService.replaceLabelsOnNode(request3);
+      fail("Should fail on unknown node");
+    } catch (Exception ex) {
+    }
+
+    // replace known node but wrong port
+    ReplaceLabelsOnNodeRequest request4 = ReplaceLabelsOnNodeRequest
+        .newInstance(ImmutableMap.of(NodeId.newInstance("host2", 1111),
+            (Set<String>) ImmutableSet.of("x")));
+    request4.setFailOnUnknownNodes(true);
+    try {
+      rm.adminService.replaceLabelsOnNode(request4);
+      fail("Should fail on node with wrong port");
+    } catch (Exception ex) {
+    }
+
+    // replace non-exist node but not check
+    ReplaceLabelsOnNodeRequest request5 = ReplaceLabelsOnNodeRequest
+        .newInstance(ImmutableMap.of(NodeId.newInstance("host5", 0),
+            (Set<String>) ImmutableSet.of("x")));
+    request5.setFailOnUnknownNodes(false);
+    try {
+      rm.adminService.replaceLabelsOnNode(request5);
+    } catch (Exception ex) {
+      fail("Should not fail on unknown node when "
+          + "fail-on-unkown-nodes is set false");
+    }
+
+    // replace on inactive node
+    ReplaceLabelsOnNodeRequest request6 = ReplaceLabelsOnNodeRequest
+        .newInstance(ImmutableMap.of(NodeId.newInstance("host4", 0),
+            (Set<String>) ImmutableSet.of("x")));
+    request6.setFailOnUnknownNodes(true);
+    try {
+      rm.adminService.replaceLabelsOnNode(request6);
+    } catch (Exception ex) {
+      fail("should not fail on inactive node");
+    }
+
+    rm.close();
+  }
+
+  @Test
   public void testRemoveClusterNodeLabelsWithCentralizedConfigurationDisabled()
       throws IOException, YarnException {
     // create RM and set it's ACTIVE


---------------------------------------------------------------------
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-4205. Add a service for monitoring application life time out. Contributed by Rohith Sharma K S

Posted by su...@apache.org.
YARN-4205. Add a service for monitoring application life time out. Contributed by Rohith Sharma K S


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

Branch: refs/heads/YARN-2915
Commit: 2ae5a3a5bf5ea355370469a53eeccff0b5220081
Parents: 1518cb9
Author: Jian He <ji...@apache.org>
Authored: Thu Sep 29 22:00:31 2016 +0800
Committer: Jian He <ji...@apache.org>
Committed: Thu Sep 29 22:00:31 2016 +0800

----------------------------------------------------------------------
 .../records/ApplicationSubmissionContext.java   |  21 +++
 .../api/records/ApplicationTimeoutType.java     |  41 +++++
 .../hadoop/yarn/conf/YarnConfiguration.java     |   6 +
 .../src/main/proto/yarn_protos.proto            |  10 ++
 .../pb/ApplicationSubmissionContextPBImpl.java  |  83 ++++++++++
 .../yarn/api/records/impl/pb/ProtoUtils.java    |  19 +++
 .../yarn/util/AbstractLivelinessMonitor.java    |  32 ++--
 .../src/main/resources/yarn-default.xml         |   9 +
 .../hadoop/yarn/api/TestPBImplRecords.java      |   2 +-
 .../resourcemanager/RMActiveServiceContext.java |  16 ++
 .../server/resourcemanager/RMAppManager.java    |   4 +
 .../yarn/server/resourcemanager/RMContext.java  |   5 +
 .../server/resourcemanager/RMContextImpl.java   |  12 ++
 .../server/resourcemanager/RMServerUtils.java   |  16 ++
 .../server/resourcemanager/ResourceManager.java |   9 +
 .../server/resourcemanager/rmapp/RMAppImpl.java |  47 +++++-
 .../rmapp/monitor/RMAppLifetimeMonitor.java     | 130 +++++++++++++++
 .../rmapp/monitor/RMAppToMonitor.java           |  77 +++++++++
 .../rmapp/monitor/package-info.java             |  28 ++++
 .../yarn/server/resourcemanager/MockRM.java     |  22 ++-
 .../rmapp/TestApplicationLifetimeMonitor.java   | 165 +++++++++++++++++++
 21 files changed, 738 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2ae5a3a5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java
index 21cd1bb..83f601a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn.api.records;
 
+import java.util.Map;
 import java.util.Set;
 
 import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
@@ -535,4 +536,24 @@ public abstract class ApplicationSubmissionContext {
   @Public
   @Unstable
   public abstract void setReservationID(ReservationId reservationID);
+
+  /**
+   * Get <code>ApplicationTimeouts</code> of the application. Timeout value is
+   * in seconds.
+   * @return all <code>ApplicationTimeouts</code> of the application.
+   */
+  @Public
+  @Unstable
+  public abstract Map<ApplicationTimeoutType, Long> getApplicationTimeouts();
+
+  /**
+   * Set the <code>ApplicationTimeouts</code> for the application in seconds.
+   * All pre-existing Map entries are cleared before adding the new Map.
+   * @param applicationTimeouts <code>ApplicationTimeouts</code>s for the
+   *          application
+   */
+  @Public
+  @Unstable
+  public abstract void setApplicationTimeouts(
+      Map<ApplicationTimeoutType, Long> applicationTimeouts);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2ae5a3a5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationTimeoutType.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationTimeoutType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationTimeoutType.java
new file mode 100644
index 0000000..edde1b0
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationTimeoutType.java
@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.api.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+
+/**
+ * Application timeout type.
+ */
+@Public
+@Unstable
+public enum ApplicationTimeoutType {
+
+  /**
+   * <p>
+   * Timeout imposed on overall application life time. It includes actual
+   * run-time plus non-runtime. Non-runtime delays are time elapsed by scheduler
+   * to allocate container, time taken to store in RMStateStore and etc.
+   * </p>
+   * If this is set, then timeout monitoring start from application submission
+   * time.
+   */
+  LIFETIME;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2ae5a3a5/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 1421873..4d43357 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
@@ -1533,6 +1533,12 @@ public class YarnConfiguration extends Configuration {
       false;
 
 
+  // Configurations for applicaiton life time monitor feature
+  public static final String RM_APPLICATION_LIFETIME_MONITOR_INTERVAL_MS =
+      RM_PREFIX + "application-timeouts.lifetime-monitor.interval-ms";
+
+  public static final long DEFAULT_RM_APPLICATION_LIFETIME_MONITOR_INTERVAL_MS =
+      60000;
 
   /**
    * Interval of time the linux container executor should try cleaning up

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2ae5a3a5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
index 2d6007e..f788295 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
@@ -365,6 +365,16 @@ message ApplicationSubmissionContextProto {
   optional ReservationIdProto reservation_id = 15;
   optional string node_label_expression = 16;
   optional ResourceRequestProto am_container_resource_request = 17;
+  repeated ApplicationTimeoutMapProto application_timeouts = 18;
+}
+
+enum ApplicationTimeoutTypeProto {
+  APP_TIMEOUT_LIFETIME = 1;
+}
+
+message ApplicationTimeoutMapProto {
+  optional ApplicationTimeoutTypeProto application_timeout_type = 1;
+  optional int64 timeout = 2;
 }
 
 message LogAggregationContextProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2ae5a3a5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationSubmissionContextPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationSubmissionContextPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationSubmissionContextPBImpl.java
index 67e3a84..62b54e7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationSubmissionContextPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationSubmissionContextPBImpl.java
@@ -18,7 +18,11 @@
 
 package org.apache.hadoop.yarn.api.records.impl.pb;
 
+import java.util.HashMap;
 import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
 import java.util.Set;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
@@ -26,6 +30,7 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.api.records.ApplicationTimeoutType;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.LogAggregationContext;
 import org.apache.hadoop.yarn.api.records.Priority;
@@ -36,6 +41,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationSubmissionContextProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationSubmissionContextProtoOrBuilder;
+import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationTimeoutMapProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerLaunchContextProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.LogAggregationContextProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.PriorityProto;
@@ -63,6 +69,7 @@ extends ApplicationSubmissionContext {
   private ResourceRequest amResourceRequest = null;
   private LogAggregationContext logAggregationContext = null;
   private ReservationId reservationId = null;
+  private Map<ApplicationTimeoutType, Long> applicationTimeouts = null;
 
   public ApplicationSubmissionContextPBImpl() {
     builder = ApplicationSubmissionContextProto.newBuilder();
@@ -131,6 +138,9 @@ extends ApplicationSubmissionContext {
     if (this.reservationId != null) {
       builder.setReservationId(convertToProtoFormat(this.reservationId));
     }
+    if (this.applicationTimeouts != null) {
+      addApplicationTimeouts();
+    }
   }
 
   private void mergeLocalToProto() {
@@ -548,4 +558,77 @@ extends ApplicationSubmissionContext {
   private ReservationIdProto convertToProtoFormat(ReservationId t) {
     return ((ReservationIdPBImpl) t).getProto();
   }
+
+  @Override
+  public Map<ApplicationTimeoutType, Long> getApplicationTimeouts() {
+    initApplicationTimeout();
+    return this.applicationTimeouts;
+  }
+
+  private void initApplicationTimeout() {
+    if (this.applicationTimeouts != null) {
+      return;
+    }
+    ApplicationSubmissionContextProtoOrBuilder p = viaProto ? proto : builder;
+    List<ApplicationTimeoutMapProto> lists = p.getApplicationTimeoutsList();
+    this.applicationTimeouts =
+        new HashMap<ApplicationTimeoutType, Long>(lists.size());
+    for (ApplicationTimeoutMapProto timeoutProto : lists) {
+      this.applicationTimeouts.put(
+          ProtoUtils
+              .convertFromProtoFormat(timeoutProto.getApplicationTimeoutType()),
+          timeoutProto.getTimeout());
+    }
+  }
+
+  @Override
+  public void setApplicationTimeouts(
+      Map<ApplicationTimeoutType, Long> appTimeouts) {
+    if (appTimeouts == null) {
+      return;
+    }
+    initApplicationTimeout();
+    this.applicationTimeouts.clear();
+    this.applicationTimeouts.putAll(appTimeouts);
+  }
+
+  private void addApplicationTimeouts() {
+    maybeInitBuilder();
+    builder.clearApplicationTimeouts();
+    if (applicationTimeouts == null) {
+      return;
+    }
+    Iterable<? extends ApplicationTimeoutMapProto> values =
+        new Iterable<ApplicationTimeoutMapProto>() {
+
+          @Override
+          public Iterator<ApplicationTimeoutMapProto> iterator() {
+            return new Iterator<ApplicationTimeoutMapProto>() {
+              private Iterator<ApplicationTimeoutType> iterator =
+                  applicationTimeouts.keySet().iterator();
+
+              @Override
+              public boolean hasNext() {
+                return iterator.hasNext();
+              }
+
+              @Override
+              public ApplicationTimeoutMapProto next() {
+                ApplicationTimeoutType key = iterator.next();
+                return ApplicationTimeoutMapProto.newBuilder()
+                    .setTimeout(applicationTimeouts.get(key))
+                    .setApplicationTimeoutType(
+                        ProtoUtils.convertToProtoFormat(key))
+                    .build();
+              }
+
+              @Override
+              public void remove() {
+                throw new UnsupportedOperationException();
+              }
+            };
+          }
+        };
+    this.builder.addAllApplicationTimeouts(values);
+  }
 }  

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2ae5a3a5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java
index 128120e..ab283e7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ProtoUtils.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.ApplicationsRequestScope;
 import org.apache.hadoop.yarn.api.records.AMCommand;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
+import org.apache.hadoop.yarn.api.records.ApplicationTimeoutType;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerRetryPolicy;
@@ -51,6 +52,7 @@ import org.apache.hadoop.yarn.proto.YarnProtos;
 import org.apache.hadoop.yarn.proto.YarnProtos.AMCommandProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationAccessTypeProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationResourceUsageReportProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationTimeoutTypeProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStateProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.FinalApplicationStatusProto;
@@ -259,6 +261,23 @@ public class ProtoUtils {
     return ApplicationAccessType.valueOf(e.name().replace(
         APP_ACCESS_TYPE_PREFIX, ""));
   }
+
+  /*
+   * ApplicationTimeoutType
+   */
+  private static String APP_TIMEOUT_TYPE_PREFIX = "APP_TIMEOUT_";
+
+  public static ApplicationTimeoutTypeProto convertToProtoFormat(
+      ApplicationTimeoutType e) {
+    return ApplicationTimeoutTypeProto
+        .valueOf(APP_TIMEOUT_TYPE_PREFIX + e.name());
+  }
+
+  public static ApplicationTimeoutType convertFromProtoFormat(
+      ApplicationTimeoutTypeProto e) {
+    return ApplicationTimeoutType
+        .valueOf(e.name().replace(APP_TIMEOUT_TYPE_PREFIX, ""));
+  }
   
   /*
    * Reservation Request interpreter type

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2ae5a3a5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/AbstractLivelinessMonitor.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/AbstractLivelinessMonitor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/AbstractLivelinessMonitor.java
index e80d032..b605026 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/AbstractLivelinessMonitor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/AbstractLivelinessMonitor.java
@@ -44,8 +44,8 @@ public abstract class AbstractLivelinessMonitor<O> extends AbstractService {
   private Thread checkerThread;
   private volatile boolean stopped;
   public static final int DEFAULT_EXPIRE = 5*60*1000;//5 mins
-  private int expireInterval = DEFAULT_EXPIRE;
-  private int monitorInterval = expireInterval/3;
+  private long expireInterval = DEFAULT_EXPIRE;
+  private long monitorInterval = expireInterval / 3;
 
   private final Clock clock;
 
@@ -85,7 +85,12 @@ public abstract class AbstractLivelinessMonitor<O> extends AbstractService {
     this.expireInterval = expireInterval;
   }
 
-  protected void setMonitorInterval(int monitorInterval) {
+  protected long getExpireInterval(O o) {
+    // by-default return for all the registered object interval.
+    return this.expireInterval;
+  }
+
+  protected void setMonitorInterval(long monitorInterval) {
     this.monitorInterval = monitorInterval;
   }
 
@@ -97,7 +102,11 @@ public abstract class AbstractLivelinessMonitor<O> extends AbstractService {
   }
 
   public synchronized void register(O ob) {
-    running.put(ob, clock.getTime());
+    register(ob, clock.getTime());
+  }
+
+  public synchronized void register(O ob, long monitorStartTime) {
+    running.put(ob, monitorStartTime);
   }
 
   public synchronized void unregister(O ob) {
@@ -117,19 +126,20 @@ public abstract class AbstractLivelinessMonitor<O> extends AbstractService {
     public void run() {
       while (!stopped && !Thread.currentThread().isInterrupted()) {
         synchronized (AbstractLivelinessMonitor.this) {
-          Iterator<Map.Entry<O, Long>> iterator = 
-            running.entrySet().iterator();
+          Iterator<Map.Entry<O, Long>> iterator = running.entrySet().iterator();
 
-          //avoid calculating current time everytime in loop
+          // avoid calculating current time everytime in loop
           long currentTime = clock.getTime();
 
           while (iterator.hasNext()) {
             Map.Entry<O, Long> entry = iterator.next();
-            if (currentTime > entry.getValue() + expireInterval) {
+            O key = entry.getKey();
+            long interval = getExpireInterval(key);
+            if (currentTime > entry.getValue() + interval) {
               iterator.remove();
-              expire(entry.getKey());
-              LOG.info("Expired:" + entry.getKey().toString() + 
-                      " Timed out after " + expireInterval/1000 + " secs");
+              expire(key);
+              LOG.info("Expired:" + entry.getKey().toString()
+                  + " Timed out after " + interval / 1000 + " secs");
             }
           }
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2ae5a3a5/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 965b575..524afec 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
@@ -3075,4 +3075,13 @@
     <name>yarn.resourcemanager.node-removal-untracked.timeout-ms</name>
     <value>60000</value>
   </property>
+
+  <property>
+    <description>
+    The RMAppLifetimeMonitor Service uses this value as lifetime monitor interval
+    </description>
+    <name>yarn.resourcemanager.application-timeouts.lifetime-monitor.interval-ms</name>
+    <value>60000</value>
+  </property>
+
 </configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2ae5a3a5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
index e57a5a2..5270486 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
@@ -368,7 +368,7 @@ public class TestPBImplRecords {
       return bytes[rand.nextInt(4)];
     } else if (type.equals(int.class) || type.equals(Integer.class)) {
       return rand.nextInt(1000000);
-    } else if (type.equals(long.class)) {
+    } else if (type.equals(long.class) || type.equals(Long.class)) {
       return Long.valueOf(rand.nextInt(1000000));
     } else if (type.equals(float.class)) {
       return rand.nextFloat();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2ae5a3a5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMActiveServiceContext.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/RMActiveServiceContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMActiveServiceContext.java
index caa0ff13..0e305a9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMActiveServiceContext.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMActiveServiceContext.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSystem;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AMLivelinessMonitor;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.monitor.RMAppLifetimeMonitor;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
@@ -105,6 +106,8 @@ public class RMActiveServiceContext {
   private boolean isSchedulerReady = false;
   private PlacementManager queuePlacementManager = null;
 
+  private RMAppLifetimeMonitor rmAppLifetimeMonitor;
+
   public RMActiveServiceContext() {
     queuePlacementManager = new PlacementManager();
   }
@@ -467,4 +470,17 @@ public class RMActiveServiceContext {
   public void setQueuePlacementManager(PlacementManager placementMgr) {
     this.queuePlacementManager = placementMgr;
   }
+
+  @Private
+  @Unstable
+  public void setRMAppLifetimeMonitor(
+      RMAppLifetimeMonitor lifetimeMonitor) {
+    this.rmAppLifetimeMonitor = lifetimeMonitor;
+  }
+
+  @Private
+  @Unstable
+  public RMAppLifetimeMonitor getRMAppLifetimeMonitor() {
+    return this.rmAppLifetimeMonitor;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2ae5a3a5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.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/RMAppManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
index 136dee0..7352a28 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
@@ -385,6 +385,10 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
       }
     }
 
+    // fail the submission if configured application timeout value is invalid
+    RMServerUtils.validateApplicationTimeouts(
+        submissionContext.getApplicationTimeouts());
+
     // Create RMApp
     RMAppImpl application =
         new RMAppImpl(applicationId, rmContext, this.conf,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2ae5a3a5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.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/RMContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java
index 2ba445c..c9d185f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSystem;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AMLivelinessMonitor;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.monitor.RMAppLifetimeMonitor;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
@@ -149,4 +150,8 @@ public interface RMContext {
   LeaderElectorService getLeaderElectorService();
 
   QueueLimitCalculator getNodeManagerQueueLimitCalculator();
+
+  void setRMAppLifetimeMonitor(RMAppLifetimeMonitor rmAppLifetimeMonitor);
+
+  RMAppLifetimeMonitor getRMAppLifetimeMonitor();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2ae5a3a5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.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/RMContextImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
index 1e702de..dc8f7d1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSystem;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AMLivelinessMonitor;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.monitor.RMAppLifetimeMonitor;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
@@ -499,4 +500,15 @@ public class RMContextImpl implements RMContext {
       QueueLimitCalculator limitCalculator) {
     this.queueLimitCalculator = limitCalculator;
   }
+
+  @Override
+  public void setRMAppLifetimeMonitor(
+      RMAppLifetimeMonitor rmAppLifetimeMonitor) {
+    this.activeServiceContext.setRMAppLifetimeMonitor(rmAppLifetimeMonitor);
+  }
+
+  @Override
+  public RMAppLifetimeMonitor getRMAppLifetimeMonitor() {
+    return this.activeServiceContext.getRMAppLifetimeMonitor();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2ae5a3a5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServerUtils.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/RMServerUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServerUtils.java
index 7fcabab..b90e499 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServerUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServerUtils.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.security.authorize.ProxyUsers;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
+import org.apache.hadoop.yarn.api.records.ApplicationTimeoutType;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.api.records.QueueInfo;
@@ -51,6 +52,7 @@ import org.apache.hadoop.yarn.exceptions.InvalidContainerReleaseException;
 import org.apache.hadoop.yarn.exceptions
     .InvalidResourceBlacklistRequestException;
 import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException;
+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;
@@ -470,4 +472,18 @@ public class RMServerUtils {
       conf.set(entry.getKey(), entry.getValue());
     }
   }
+
+  public static void validateApplicationTimeouts(
+      Map<ApplicationTimeoutType, Long> timeouts) throws YarnException {
+    if (timeouts != null) {
+      for (Map.Entry<ApplicationTimeoutType, Long> timeout : timeouts
+          .entrySet()) {
+        if (timeout.getValue() < 0) {
+          String message = "Invalid application timeout, value="
+              + timeout.getValue() + " for type=" + timeout.getKey();
+          throw new YarnException(message);
+        }
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2ae5a3a5/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 8a6997d..5e9bece 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
@@ -96,6 +96,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AMLivelinessM
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.monitor.RMAppLifetimeMonitor;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEvent;
@@ -556,6 +557,10 @@ public class ResourceManager extends CompositeService implements Recoverable {
       addService(amFinishingMonitor);
       rmContext.setAMFinishingMonitor(amFinishingMonitor);
       
+      RMAppLifetimeMonitor rmAppLifetimeMonitor = createRMAppLifetimeMonitor();
+      addService(rmAppLifetimeMonitor);
+      rmContext.setRMAppLifetimeMonitor(rmAppLifetimeMonitor);
+
       RMNodeLabelsManager nlm = createNodeLabelManager();
       nlm.setRMContext(rmContext);
       addService(nlm);
@@ -1398,4 +1403,8 @@ public class ResourceManager extends CompositeService implements Recoverable {
     out.println("                            "
         + "[-remove-application-from-state-store <appId>]" + "\n");
   }
+
+  protected RMAppLifetimeMonitor createRMAppLifetimeMonitor() {
+    return new RMAppLifetimeMonitor(this.rmContext);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2ae5a3a5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.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/rmapp/RMAppImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
index e5bde32..727703b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
@@ -56,6 +56,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.api.records.ApplicationTimeoutType;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.LogAggregationStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
@@ -527,6 +528,8 @@ public class RMAppImpl implements RMApp, Recoverable {
             DEFAULT_AM_SCHEDULING_NODE_BLACKLISTING_DISABLE_THRESHOLD;
       }
     }
+
+
   }
 
   /**
@@ -1106,6 +1109,20 @@ public class RMAppImpl implements RMApp, Recoverable {
         }
       }
 
+      long applicationLifetime =
+          app.getApplicationLifetime(ApplicationTimeoutType.LIFETIME);
+      if (applicationLifetime > 0) {
+        app.rmContext.getRMAppLifetimeMonitor().registerApp(app.applicationId,
+            ApplicationTimeoutType.LIFETIME, app.submitTime,
+            applicationLifetime * 1000);
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Application " + app.applicationId
+              + " is registered for timeout monitor, type="
+              + ApplicationTimeoutType.LIFETIME + " value="
+              + applicationLifetime + " seconds");
+        }
+      }
+
       // No existent attempts means the attempt associated with this app was not
       // started or started but not yet saved.
       if (app.attempts.isEmpty()) {
@@ -1152,6 +1169,13 @@ public class RMAppImpl implements RMApp, Recoverable {
 
     @Override
     public RMAppState transition(RMAppImpl app, RMAppEvent event) {
+      Map<ApplicationTimeoutType, Long> timeouts =
+          app.submissionContext.getApplicationTimeouts();
+      if (timeouts != null && timeouts.size() > 0) {
+        app.rmContext.getRMAppLifetimeMonitor()
+            .unregisterApp(app.getApplicationId(), timeouts.keySet());
+      }
+
       if (app.transitionTodo instanceof SingleArcTransition) {
         ((SingleArcTransition) app.transitionTodo).transition(app,
           app.eventCausingFinalSaving);
@@ -1160,7 +1184,6 @@ public class RMAppImpl implements RMApp, Recoverable {
           app.eventCausingFinalSaving);
       }
       return app.targetedFinalState;
-
     }
   }
 
@@ -1209,6 +1232,18 @@ public class RMAppImpl implements RMApp, Recoverable {
     @Override
     public void transition(RMAppImpl app, RMAppEvent event) {
 
+      long applicationLifetime =
+          app.getApplicationLifetime(ApplicationTimeoutType.LIFETIME);
+      if (applicationLifetime > 0) {
+        app.rmContext.getRMAppLifetimeMonitor().registerApp(app.applicationId,
+            ApplicationTimeoutType.LIFETIME, app.submitTime,
+            applicationLifetime * 1000);
+        LOG.debug("Application " + app.applicationId
+            + " is registered for timeout monitor, type="
+            + ApplicationTimeoutType.LIFETIME + " value=" + applicationLifetime
+            + " seconds");
+      }
+
       // If recovery is enabled then store the application information in a
       // non-blocking call so make sure that RM has stored the information
       // needed to restart the AM after RM restart without further client
@@ -1922,4 +1957,14 @@ public class RMAppImpl implements RMApp, Recoverable {
   public int getNextAttemptId() {
     return nextAttemptId;
   }
+
+  private long getApplicationLifetime(ApplicationTimeoutType type) {
+    Map<ApplicationTimeoutType, Long> timeouts =
+        this.submissionContext.getApplicationTimeouts();
+    long applicationLifetime = -1;
+    if (timeouts != null && timeouts.containsKey(type)) {
+      applicationLifetime = timeouts.get(type);
+    }
+    return applicationLifetime;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2ae5a3a5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/monitor/RMAppLifetimeMonitor.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/rmapp/monitor/RMAppLifetimeMonitor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/monitor/RMAppLifetimeMonitor.java
new file mode 100644
index 0000000..e550c97
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/monitor/RMAppLifetimeMonitor.java
@@ -0,0 +1,130 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.rmapp.monitor;
+
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+
+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.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationTimeoutType;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
+import org.apache.hadoop.yarn.util.AbstractLivelinessMonitor;
+import org.apache.hadoop.yarn.util.SystemClock;
+
+/**
+ * This service will monitor the applications against the lifetime value given.
+ * The applications will be killed if it running beyond the given time.
+ */
+public class RMAppLifetimeMonitor
+    extends AbstractLivelinessMonitor<RMAppToMonitor> {
+
+  private static final Log LOG = LogFactory.getLog(RMAppLifetimeMonitor.class);
+
+  private RMContext rmContext;
+  private Map<RMAppToMonitor, Long> monitoredApps =
+      new HashMap<RMAppToMonitor, Long>();
+
+  private static final EnumSet<RMAppState> COMPLETED_APP_STATES =
+      EnumSet.of(RMAppState.FINISHED, RMAppState.FINISHING, RMAppState.FAILED,
+          RMAppState.KILLED, RMAppState.FINAL_SAVING, RMAppState.KILLING);
+
+  public RMAppLifetimeMonitor(RMContext rmContext) {
+    super(RMAppLifetimeMonitor.class.getName(), SystemClock.getInstance());
+    this.rmContext = rmContext;
+  }
+
+  @Override
+  protected void serviceInit(Configuration conf) throws Exception {
+    long monitorInterval = conf.getLong(
+        YarnConfiguration.RM_APPLICATION_LIFETIME_MONITOR_INTERVAL_MS,
+        YarnConfiguration.DEFAULT_RM_APPLICATION_LIFETIME_MONITOR_INTERVAL_MS);
+    if (monitorInterval <= 0) {
+      monitorInterval =
+          YarnConfiguration.DEFAULT_RM_APPLICATION_LIFETIME_MONITOR_INTERVAL_MS;
+    }
+    setMonitorInterval(monitorInterval);
+    LOG.info("Application lifelime monitor interval set to " + monitorInterval
+        + " ms.");
+    super.serviceInit(conf);
+  }
+
+  @SuppressWarnings("unchecked")
+  @Override
+  protected synchronized void expire(RMAppToMonitor monitoredAppKey) {
+    Long remove = monitoredApps.remove(monitoredAppKey);
+    ApplicationId appId = monitoredAppKey.getApplicationId();
+    RMApp app = rmContext.getRMApps().get(appId);
+    if (app == null) {
+      return;
+    }
+    // Don't trigger a KILL event if application is in completed states
+    if (!COMPLETED_APP_STATES.contains(app.getState())) {
+      String diagnostics =
+          "Application killed due to exceeding its lifetime period " + remove
+              + " milliseconds";
+      rmContext.getDispatcher().getEventHandler()
+          .handle(new RMAppEvent(appId, RMAppEventType.KILL, diagnostics));
+    } else {
+      LOG.info("Application " + appId
+          + " is about to complete. So not killing the application.");
+    }
+  }
+
+  public synchronized void registerApp(ApplicationId appId,
+      ApplicationTimeoutType timeoutType, long monitorStartTime, long timeout) {
+    RMAppToMonitor appToMonitor = new RMAppToMonitor(appId, timeoutType);
+    register(appToMonitor, monitorStartTime);
+    monitoredApps.putIfAbsent(appToMonitor, timeout);
+  }
+
+  @Override
+  protected synchronized long getExpireInterval(
+      RMAppToMonitor monitoredAppKey) {
+    return monitoredApps.get(monitoredAppKey);
+  }
+
+  public synchronized void unregisterApp(ApplicationId appId,
+      ApplicationTimeoutType timeoutType) {
+    RMAppToMonitor appToRemove = new RMAppToMonitor(appId, timeoutType);
+    unregister(appToRemove);
+    monitoredApps.remove(appToRemove);
+  }
+
+  public synchronized void unregisterApp(ApplicationId appId,
+      Set<ApplicationTimeoutType> types) {
+    for (ApplicationTimeoutType type : types) {
+      unregisterApp(appId, type);
+    }
+  }
+
+  public synchronized void updateApplicationTimeouts(ApplicationId appId,
+      Map<ApplicationTimeoutType, Long> timeouts) {
+    // TODO in YARN-5611
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2ae5a3a5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/monitor/RMAppToMonitor.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/rmapp/monitor/RMAppToMonitor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/monitor/RMAppToMonitor.java
new file mode 100644
index 0000000..1cf2132
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/monitor/RMAppToMonitor.java
@@ -0,0 +1,77 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, 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.rmapp.monitor;
+
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationTimeoutType;
+
+/**
+ * This class used for monitor application with applicationId+appTimeoutType.
+ */
+public class RMAppToMonitor {
+
+  private ApplicationId applicationId;
+  private ApplicationTimeoutType appTimeoutType;
+
+  RMAppToMonitor(ApplicationId appId, ApplicationTimeoutType timeoutType) {
+    this.applicationId = appId;
+    this.appTimeoutType = timeoutType;
+  }
+
+  public ApplicationId getApplicationId() {
+    return applicationId;
+  }
+
+  public ApplicationTimeoutType getAppTimeoutType() {
+    return appTimeoutType;
+  }
+
+  @Override
+  public int hashCode() {
+    return applicationId.hashCode() + appTimeoutType.hashCode();
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (getClass() != obj.getClass()) {
+      return false;
+    }
+    RMAppToMonitor other = (RMAppToMonitor) obj;
+    if (!this.applicationId.equals(other.getApplicationId())) {
+      return false;
+    }
+    if (this.appTimeoutType != other.getAppTimeoutType()) {
+      return false;
+    }
+    return true;
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append(applicationId.toString()).append("_").append(appTimeoutType);
+    return sb.toString();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2ae5a3a5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/monitor/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/rmapp/monitor/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/rmapp/monitor/package-info.java
new file mode 100644
index 0000000..a3cc7ef
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/monitor/package-info.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.resourcemanager.rmapp.monitor contains
+ * classes related to application monitor.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.yarn.server.resourcemanager.rmapp.monitor;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2ae5a3a5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.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/MockRM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
index f843261..25a8288 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
@@ -50,6 +50,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 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.ApplicationTimeoutType;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerState;
@@ -460,7 +461,7 @@ public class MockRM extends ResourceManager {
     return submitApp(resource, name, user, acls, false, queue,
       super.getConfig().getInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS,
       YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS), null, null, true, false,
-      false, null, 0, null, true, priority, amLabel);
+        false, null, 0, null, true, priority, amLabel, null);
   }
 
   public RMApp submitApp(Resource resource, String name, String user,
@@ -561,7 +562,7 @@ public class MockRM extends ResourceManager {
     return submitApp(capability, name, user, acls, unmanaged, queue,
       maxAppAttempts, ts, appType, waitForAccepted, keepContainers,
       isAppIdProvided, applicationId, attemptFailuresValidityInterval,
-      logAggregationContext, cancelTokensWhenComplete, priority, "");
+        logAggregationContext, cancelTokensWhenComplete, priority, "", null);
   }
 
   public RMApp submitApp(Resource capability, String name, String user,
@@ -570,7 +571,8 @@ public class MockRM extends ResourceManager {
       boolean waitForAccepted, boolean keepContainers, boolean isAppIdProvided,
       ApplicationId applicationId, long attemptFailuresValidityInterval,
       LogAggregationContext logAggregationContext,
-      boolean cancelTokensWhenComplete, Priority priority, String amLabel)
+      boolean cancelTokensWhenComplete, Priority priority, String amLabel,
+      Map<ApplicationTimeoutType, Long> applicationTimeouts)
       throws Exception {
     ApplicationId appId = isAppIdProvided ? applicationId : null;
     ApplicationClientProtocol client = getClientRMService();
@@ -587,6 +589,9 @@ public class MockRM extends ResourceManager {
     sub.setApplicationId(appId);
     sub.setApplicationName(name);
     sub.setMaxAppAttempts(maxAppAttempts);
+    if (applicationTimeouts != null && applicationTimeouts.size() > 0) {
+      sub.setApplicationTimeouts(applicationTimeouts);
+    }
     if (unmanaged) {
       sub.setUnmanagedAM(true);
     }
@@ -1073,4 +1078,15 @@ public class MockRM extends ResourceManager {
         !apps.containsKey(appId));
     LOG.info("app is removed from scheduler, " + appId);
   }
+
+  public RMApp submitApp(int masterMemory, Priority priority,
+      Map<ApplicationTimeoutType, Long> applicationTimeouts) throws Exception {
+    Resource resource = Resource.newInstance(masterMemory, 0);
+    return submitApp(
+        resource, "", UserGroupInformation.getCurrentUser().getShortUserName(),
+        null, false, null,
+        super.getConfig().getInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS,
+            YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS), null, null, true,
+        false, false, null, 0, null, true, priority, null, applicationTimeouts);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2ae5a3a5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestApplicationLifetimeMonitor.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/rmapp/TestApplicationLifetimeMonitor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestApplicationLifetimeMonitor.java
new file mode 100644
index 0000000..3f2db1d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestApplicationLifetimeMonitor.java
@@ -0,0 +1,165 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, 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.rmapp;
+
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.records.ApplicationTimeoutType;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerState;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus;
+import org.apache.hadoop.yarn.server.resourcemanager.MockAM;
+import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
+import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
+import org.apache.hadoop.yarn.server.resourcemanager.TestRMRestart;
+import org.apache.hadoop.yarn.server.resourcemanager.TestWorkPreservingRMRestart;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.MemoryRMStateStore;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
+import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeImpl;
+import org.apache.log4j.Level;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test class for application life time monitor feature test.
+ */
+public class TestApplicationLifetimeMonitor {
+  private YarnConfiguration conf;
+
+  @Before
+  public void setup() throws IOException {
+    conf = new YarnConfiguration();
+    Logger rootLogger = LogManager.getRootLogger();
+    rootLogger.setLevel(Level.DEBUG);
+    UserGroupInformation.setConfiguration(conf);
+    conf.setBoolean(YarnConfiguration.RECOVERY_ENABLED, true);
+    conf.setBoolean(YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_ENABLED,
+        true);
+    conf.set(YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName());
+    conf.setLong(YarnConfiguration.RM_APPLICATION_LIFETIME_MONITOR_INTERVAL_MS,
+        3000L);
+  }
+
+  @Test(timeout = 90000)
+  public void testApplicationLifetimeMonitor() throws Exception {
+    MockRM rm = null;
+    try {
+      rm = new MockRM(conf);
+      rm.start();
+      Priority appPriority = Priority.newInstance(0);
+      MockNM nm1 = rm.registerNode("127.0.0.1:1234", 16 * 1024);
+
+      Map<ApplicationTimeoutType, Long> timeouts =
+          new HashMap<ApplicationTimeoutType, Long>();
+      timeouts.put(ApplicationTimeoutType.LIFETIME, 10L);
+      RMApp app1 = rm.submitApp(1024, appPriority, timeouts);
+      nm1.nodeHeartbeat(true);
+      // Send launch Event
+      MockAM am1 =
+          rm.sendAMLaunched(app1.getCurrentAppAttempt().getAppAttemptId());
+      am1.registerAppAttempt();
+      rm.waitForState(app1.getApplicationId(), RMAppState.KILLED);
+      Assert.assertTrue("Applicaiton killed before lifetime value",
+          (System.currentTimeMillis() - app1.getSubmitTime()) > 10000);
+    } finally {
+      stopRM(rm);
+    }
+  }
+
+  @SuppressWarnings("rawtypes")
+  @Test(timeout = 180000)
+  public void testApplicationLifetimeOnRMRestart() throws Exception {
+    MemoryRMStateStore memStore = new MemoryRMStateStore();
+    memStore.init(conf);
+    MockRM rm1 = new MockRM(conf, memStore);
+    rm1.start();
+    MockNM nm1 =
+        new MockNM("127.0.0.1:1234", 8192, rm1.getResourceTrackerService());
+    nm1.registerNode();
+    nm1.nodeHeartbeat(true);
+
+    long appLifetime = 60L;
+    Map<ApplicationTimeoutType, Long> timeouts =
+        new HashMap<ApplicationTimeoutType, Long>();
+    timeouts.put(ApplicationTimeoutType.LIFETIME, appLifetime);
+    RMApp app1 = rm1.submitApp(200, Priority.newInstance(0), timeouts);
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
+
+    // Re-start RM
+    MockRM rm2 = new MockRM(conf, memStore);
+    rm2.start();
+    nm1.setResourceTrackerService(rm2.getResourceTrackerService());
+
+    // recover app
+    RMApp recoveredApp1 =
+        rm2.getRMContext().getRMApps().get(app1.getApplicationId());
+
+    NMContainerStatus amContainer = TestRMRestart.createNMContainerStatus(
+        am1.getApplicationAttemptId(), 1, ContainerState.RUNNING);
+    NMContainerStatus runningContainer = TestRMRestart.createNMContainerStatus(
+        am1.getApplicationAttemptId(), 2, ContainerState.RUNNING);
+
+    nm1.registerNode(Arrays.asList(amContainer, runningContainer), null);
+
+    // Wait for RM to settle down on recovering containers;
+    TestWorkPreservingRMRestart.waitForNumContainersToRecover(2, rm2,
+        am1.getApplicationAttemptId());
+    Set<ContainerId> launchedContainers =
+        ((RMNodeImpl) rm2.getRMContext().getRMNodes().get(nm1.getNodeId()))
+            .getLaunchedContainers();
+    assertTrue(launchedContainers.contains(amContainer.getContainerId()));
+    assertTrue(launchedContainers.contains(runningContainer.getContainerId()));
+
+    // check RMContainers are re-recreated and the container state is correct.
+    rm2.waitForState(nm1, amContainer.getContainerId(),
+        RMContainerState.RUNNING);
+    rm2.waitForState(nm1, runningContainer.getContainerId(),
+        RMContainerState.RUNNING);
+
+    // re register attempt to rm2
+    rm2.waitForState(recoveredApp1.getApplicationId(), RMAppState.ACCEPTED);
+    am1.setAMRMProtocol(rm2.getApplicationMasterService(), rm2.getRMContext());
+    am1.registerAppAttempt();
+    rm2.waitForState(recoveredApp1.getApplicationId(), RMAppState.RUNNING);
+
+    // wait for app life time and application to be in killed state.
+    rm2.waitForState(recoveredApp1.getApplicationId(), RMAppState.KILLED);
+    Assert.assertTrue("Applicaiton killed before lifetime value",
+        (System.currentTimeMillis()
+            - recoveredApp1.getSubmitTime()) > appLifetime);
+  }
+
+  private void stopRM(MockRM rm) {
+    if (rm != null) {
+      rm.stop();
+    }
+  }
+}


---------------------------------------------------------------------
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-5406. In-memory based implementation of the FederationMembershipStateStore. Contributed by Ellen Hui.

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


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

Branch: refs/heads/YARN-2915
Commit: c65ec7598df1e9c910ed204c958f5dcb78cdba2d
Parents: a48c8e6
Author: Subru Krishnan <su...@apache.org>
Authored: Thu Aug 4 15:54:38 2016 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon Oct 3 14:15:36 2016 -0700

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


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

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

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

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

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

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


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


[20/50] [abbrv] hadoop git commit: YARN-5384. Expose priority in ReservationSystem submission APIs. (Sean Po via Subru).

Posted by su...@apache.org.
YARN-5384. Expose priority in ReservationSystem submission APIs. (Sean Po via Subru).


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

Branch: refs/heads/YARN-2915
Commit: 3a3697deab3e3397082222deb66fb613d86ff9ae
Parents: 89bd6d2
Author: Subru Krishnan <su...@apache.org>
Authored: Fri Sep 30 19:41:43 2016 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Fri Sep 30 19:41:43 2016 -0700

----------------------------------------------------------------------
 .../yarn/api/records/ReservationDefinition.java | 44 ++++++++++++++++----
 .../src/main/proto/yarn_protos.proto            |  1 +
 .../impl/pb/ReservationDefinitionPBImpl.java    | 31 ++++++++++++++
 .../webapp/dao/ReservationDefinitionInfo.java   | 11 +++++
 .../reservation/ReservationSystemTestUtil.java  | 10 ++++-
 .../src/site/markdown/ResourceManagerRest.md    |  3 ++
 6 files changed, 91 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a3697de/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationDefinition.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationDefinition.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationDefinition.java
index 8ef881b..bb9bca2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationDefinition.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ReservationDefinition.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.yarn.api.records;
 
 import org.apache.hadoop.classification.InterfaceAudience.Public;
-import org.apache.hadoop.classification.InterfaceStability.Evolving;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.util.Records;
 
@@ -38,7 +37,7 @@ public abstract class ReservationDefinition {
   @Unstable
   public static ReservationDefinition newInstance(long arrival, long deadline,
       ReservationRequests reservationRequests, String name,
-      String recurrenceExpression) {
+      String recurrenceExpression, Priority priority) {
     ReservationDefinition rDefinition =
         Records.newRecord(ReservationDefinition.class);
     rDefinition.setArrival(arrival);
@@ -46,6 +45,7 @@ public abstract class ReservationDefinition {
     rDefinition.setReservationRequests(reservationRequests);
     rDefinition.setReservationName(name);
     rDefinition.setRecurrenceExpression(recurrenceExpression);
+    rDefinition.setPriority(priority);
     return rDefinition;
   }
 
@@ -53,8 +53,8 @@ public abstract class ReservationDefinition {
   @Unstable
   public static ReservationDefinition newInstance(long arrival, long deadline,
       ReservationRequests reservationRequests, String name) {
-    ReservationDefinition rDefinition =
-        newInstance(arrival, deadline, reservationRequests, name, "0");
+    ReservationDefinition rDefinition = newInstance(arrival, deadline,
+        reservationRequests, name, "0", Priority.UNDEFINED);
     return rDefinition;
   }
 
@@ -130,7 +130,7 @@ public abstract class ReservationDefinition {
    *         allocation in the scheduler
    */
   @Public
-  @Evolving
+  @Unstable
   public abstract String getReservationName();
 
   /**
@@ -142,7 +142,7 @@ public abstract class ReservationDefinition {
    *          allocation in the scheduler
    */
   @Public
-  @Evolving
+  @Unstable
   public abstract void setReservationName(String name);
 
   /**
@@ -160,7 +160,7 @@ public abstract class ReservationDefinition {
    * @return recurrence of this reservation
    */
   @Public
-  @Evolving
+  @Unstable
   public abstract String getRecurrenceExpression();
 
   /**
@@ -178,7 +178,35 @@ public abstract class ReservationDefinition {
    * @param recurrenceExpression recurrence interval of this reservation
    */
   @Public
-  @Evolving
+  @Unstable
   public abstract void setRecurrenceExpression(String recurrenceExpression);
 
+  /**
+   * Get the priority for this reservation. A lower number for priority
+   * indicates a higher priority reservation. Recurring reservations are
+   * always higher priority than non-recurring reservations. Priority for
+   * non-recurring reservations are only compared with non-recurring
+   * reservations. Likewise for recurring reservations.
+   *
+   * @return int representing the priority of the reserved resource
+   *         allocation in the scheduler
+   */
+  @Public
+  @Unstable
+  public abstract Priority getPriority();
+
+  /**
+   * Set the priority for this reservation. A lower number for priority
+   * indicates a higher priority reservation. Recurring reservations are
+   * always higher priority than non-recurring reservations. Priority for
+   * non-recurring reservations are only compared with non-recurring
+   * reservations. Likewise for recurring reservations.
+   *
+   * @param priority representing the priority of the reserved resource
+   *          allocation in the scheduler
+   */
+  @Public
+  @Unstable
+  public abstract void setPriority(Priority priority);
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a3697de/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
index f788295..9c746fd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
@@ -489,6 +489,7 @@ message ReservationDefinitionProto {
   optional int64 deadline = 3;
   optional string reservation_name = 4;
   optional string recurrence_expression = 5 [default = "0"];
+  optional PriorityProto priority = 6;
 }
 
 message ResourceAllocationRequestProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a3697de/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ReservationDefinitionPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ReservationDefinitionPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ReservationDefinitionPBImpl.java
index b30cd2a..49aef11 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ReservationDefinitionPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ReservationDefinitionPBImpl.java
@@ -18,8 +18,10 @@
 
 package org.apache.hadoop.yarn.api.records.impl.pb;
 
+import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.ReservationDefinition;
 import org.apache.hadoop.yarn.api.records.ReservationRequests;
+import org.apache.hadoop.yarn.proto.YarnProtos;
 import org.apache.hadoop.yarn.proto.YarnProtos.ReservationDefinitionProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ReservationDefinitionProtoOrBuilder;
 import org.apache.hadoop.yarn.proto.YarnProtos.ReservationRequestsProto;
@@ -32,6 +34,7 @@ public class ReservationDefinitionPBImpl extends ReservationDefinition {
   boolean viaProto = false;
 
   private ReservationRequests reservationReqs;
+  private Priority priority = null;
 
   public ReservationDefinitionPBImpl() {
     builder = ReservationDefinitionProto.newBuilder();
@@ -150,6 +153,33 @@ public class ReservationDefinitionPBImpl extends ReservationDefinition {
     builder.setReservationName(name);
   }
 
+  @Override
+  public Priority getPriority() {
+    ReservationDefinitionProtoOrBuilder p = viaProto ? proto : builder;
+    if (this.priority != null) {
+      return this.priority;
+    }
+    if (!p.hasPriority()) {
+      return Priority.UNDEFINED;
+    }
+    this.priority = convertFromProtoFormat(p.getPriority());
+    return this.priority;
+  }
+
+  @Override
+  public void setPriority(Priority priority) {
+    maybeInitBuilder();
+    if (priority == null) {
+      this.priority = Priority.UNDEFINED;
+    }
+    this.priority = priority;
+  }
+
+  private PriorityPBImpl convertFromProtoFormat(
+      YarnProtos.PriorityProto p) {
+    return new PriorityPBImpl(p);
+  }
+
   private ReservationRequestsPBImpl convertFromProtoFormat(
       ReservationRequestsProto p) {
     return new ReservationRequestsPBImpl(p);
@@ -164,6 +194,7 @@ public class ReservationDefinitionPBImpl extends ReservationDefinition {
     return "{Arrival: " + getArrival() + ", Deadline: " + getDeadline()
         + ", Reservation Name: " + getReservationName()
         + ", Recurrence expression: " + getRecurrenceExpression()
+        + ", Priority: " + getPriority().toString()
         + ", Resources: " + getReservationRequests() + "}";
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a3697de/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ReservationDefinitionInfo.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/ReservationDefinitionInfo.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/ReservationDefinitionInfo.java
index 71ee924..42a07af 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/ReservationDefinitionInfo.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/ReservationDefinitionInfo.java
@@ -44,6 +44,9 @@ public class ReservationDefinitionInfo {
   @XmlElement(name = "reservation-name")
   private String reservationName;
 
+  @XmlElement(name = "priority")
+  private int priority;
+
   public ReservationDefinitionInfo() {
 
   }
@@ -89,4 +92,12 @@ public class ReservationDefinitionInfo {
     this.reservationName = reservationName;
   }
 
+  public int getPriority() {
+    return priority;
+  }
+
+  public void setPriority(int priority) {
+    this.priority = priority;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a3697de/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.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/reservation/ReservationSystemTestUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java
index 24c386a..1ff6a1a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationSystemTestUtil.java
@@ -31,6 +31,7 @@ import java.util.TreeMap;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest;
+import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.ReservationDefinition;
 import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.ReservationRequest;
@@ -199,6 +200,13 @@ public class ReservationSystemTestUtil {
   public static ReservationSubmissionRequest createSimpleReservationRequest(
       ReservationId reservationId, int numContainers, long arrival,
       long deadline, long duration) {
+    return createSimpleReservationRequest(reservationId, numContainers,
+        arrival, deadline, duration, Priority.UNDEFINED);
+  }
+
+  public static ReservationSubmissionRequest createSimpleReservationRequest(
+      ReservationId reservationId, int numContainers, long arrival,
+      long deadline, long duration, Priority priority) {
     // create a request with a single atomic ask
     ReservationRequest r =
         ReservationRequest.newInstance(Resource.newInstance(1024, 1),
@@ -208,7 +216,7 @@ public class ReservationSystemTestUtil {
             ReservationRequestInterpreter.R_ALL);
     ReservationDefinition rDef =
         ReservationDefinition.newInstance(arrival, deadline, reqs,
-            "testClientRMService#reservation");
+            "testClientRMService#reservation", "0", priority);
     ReservationSubmissionRequest request =
         ReservationSubmissionRequest.newInstance(rDef,
             reservationQ, reservationId);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a3697de/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md
index 5862506..051509c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md
@@ -3237,6 +3237,7 @@ The Cluster Reservation API can be used to list reservations. When listing reser
 | deadline | long | The UTC time representation of the latest time within which this reservation can be allocated. |
 | reservation-name | string | A mnemonic name of the reservation (not a valid identifier). |
 | reservation-requests | object | A list of "stages" or phases of this reservation, each describing resource requirements and duration |
+| priority | int | An integer representing the priority of the reservation. A lower number for priority indicates a higher priority reservation. Recurring reservations are always higher priority than non-recurring reservations. Priority for non-recurring reservations are only compared with non-recurring reservations. Likewise with recurring reservations. |
 
 ### Elements of the *reservation-requests* object
 
@@ -3500,6 +3501,7 @@ Elements of the *reservation-definition* object
 | deadline | long | The UTC time representation of the latest time within which this reservation can be allocated. |
 | reservation-name | string | A mnemonic name of the reservation (not a valid identifier). |
 | reservation-requests | object | A list of "stages" or phases of this reservation, each describing resource requirements and duration |
+| priority | int | An integer representing the priority of the reservation. A lower number for priority indicates a higher priority reservation. Recurring reservations are always higher priority than non-recurring reservations. Priority for non-recurring reservations are only compared with non-recurring reservations. Likewise with recurring reservations. |
 
 Elements of the *reservation-requests* object
 
@@ -3675,6 +3677,7 @@ Elements of the *reservation-definition* object
 | deadline | long | The UTC time representation of the latest time within which this reservation can be allocated. |
 | reservation-name | string | A mnemonic name of the reservation (not a valid identifier). |
 | reservation-requests | object | A list of "stages" or phases of this reservation, each describing resource requirements and duration |
+| priority | int | An integer representing the priority of the reservation. A lower number for priority indicates a higher priority reservation. Recurring reservations are always higher priority than non-recurring reservations. Priority for non-recurring reservations are only compared with non-recurring reservations. Likewise with recurring reservations. |
 
 Elements of the *reservation-requests* object
 


---------------------------------------------------------------------
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: HDFS-10907. Fix Erasure Coding documentation. Contributed by Manoj Govindassamy.

Posted by su...@apache.org.
HDFS-10907. Fix Erasure Coding documentation. Contributed by Manoj Govindassamy.


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

Branch: refs/heads/YARN-2915
Commit: 7fad1221d6f35e84b320fab82174525c067ad521
Parents: ee33a02
Author: Wei-Chiu Chuang <we...@apache.org>
Authored: Fri Sep 30 12:51:27 2016 -0700
Committer: Wei-Chiu Chuang <we...@apache.org>
Committed: Fri Sep 30 12:51:27 2016 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md  | 2 +-
 .../hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md             | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7fad1221/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
index 9f9fba5..e923b86 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
@@ -463,7 +463,7 @@ Runs the diskbalancer CLI. See [HDFS Diskbalancer](./HDFSDiskbalancer.html) for
 Usage:
 
        hdfs erasurecode [generic options]
-         [-setPolicy [-s <policyName>] <path>]
+         [-setPolicy [-p <policyName>] <path>]
          [-getPolicy <path>]
          [-listPolicies]
          [-usage [cmd ...]]

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7fad1221/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
index 76c1b3a..18b3a25 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
@@ -125,7 +125,7 @@ Below are the details about each command.
 
       `path`: An directory in HDFS. This is a mandatory parameter. Setting a policy only affects newly created files, and does not affect existing files.
 
-      `policyName`: The ErasureCoding policy to be used for files under this directory. This is an optional parameter, specified using \u2018-s\u2019 flag. If no policy is specified, the system default ErasureCodingPolicy will be used.
+      `policyName`: The ErasureCoding policy to be used for files under this directory. This is an optional parameter, specified using \u2018-p\u2019 flag. If no policy is specified, the system default ErasureCodingPolicy will be used.
 
  *  `[-getPolicy <path>]`
 


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


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

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


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

Branch: refs/heads/YARN-2915
Commit: 1191de4104b54416da60e9a17cf48a03d6d8d07e
Parents: 75d47b0
Author: Subru Krishnan <su...@apache.org>
Authored: Tue Aug 9 16:07:55 2016 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon Oct 3 14:15:36 2016 -0700

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


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

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

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

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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1191de41/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/impl/pb/GetApplicationHomeSubClusterRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/impl/pb/GetApplicationHomeSubClusterRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/impl/pb/GetApplicationHomeSubClusterRequestPBImpl.java
index 865d0c4..585ba81 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/impl/pb/GetApplicationHomeSubClusterRequestPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/impl/pb/GetApplicationHomeSubClusterRequestPBImpl.java
@@ -108,6 +108,10 @@ public class GetApplicationHomeSubClusterRequestPBImpl
   public ApplicationId getApplicationId() {
     GetApplicationHomeSubClusterRequestProtoOrBuilder p =
         viaProto ? proto : builder;
+    if (applicationId != null) {
+      return applicationId;
+    }
+
     if (!p.hasApplicationId()) {
       return null;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1191de41/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/impl/pb/SetSubClusterPolicyConfigurationRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/impl/pb/SetSubClusterPolicyConfigurationRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/impl/pb/SetSubClusterPolicyConfigurationRequestPBImpl.java
index 5e29bd5..7b7f89d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/impl/pb/SetSubClusterPolicyConfigurationRequestPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/impl/pb/SetSubClusterPolicyConfigurationRequestPBImpl.java
@@ -107,23 +107,6 @@ public class SetSubClusterPolicyConfigurationRequestPBImpl
   }
 
   @Override
-  public String getQueue() {
-    SetSubClusterPolicyConfigurationRequestProtoOrBuilder p =
-        viaProto ? proto : builder;
-    return p.getQueue();
-  }
-
-  @Override
-  public void setQueue(String queueName) {
-    maybeInitBuilder();
-    if (queueName == null) {
-      builder.clearQueue();
-      return;
-    }
-    builder.setQueue(queueName);
-  }
-
-  @Override
   public SubClusterPolicyConfiguration getPolicyConfiguration() {
     SetSubClusterPolicyConfigurationRequestProtoOrBuilder p =
         viaProto ? proto : builder;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1191de41/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/impl/pb/SubClusterPolicyConfigurationPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/impl/pb/SubClusterPolicyConfigurationPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/impl/pb/SubClusterPolicyConfigurationPBImpl.java
index fe9d9db..305a8d3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/impl/pb/SubClusterPolicyConfigurationPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/records/impl/pb/SubClusterPolicyConfigurationPBImpl.java
@@ -87,6 +87,23 @@ public class SubClusterPolicyConfigurationPBImpl
   }
 
   @Override
+  public String getQueue() {
+    SubClusterPolicyConfigurationProtoOrBuilder p = viaProto ? proto : builder;
+    return p.getQueue();
+  }
+
+  @Override
+  public void setQueue(String queueName) {
+    maybeInitBuilder();
+    if (queueName == null) {
+      builder.clearType();
+      return;
+    }
+    builder.setQueue(queueName);
+
+  }
+
+  @Override
   public String getType() {
     SubClusterPolicyConfigurationProtoOrBuilder p = viaProto ? proto : builder;
     return p.getType();

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

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

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


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


[06/50] [abbrv] hadoop git commit: HADOOP-13537. Support external calls in the RPC call queue. Contributed by Daryn Sharp.

Posted by su...@apache.org.
HADOOP-13537. Support external calls in the RPC call queue. Contributed by Daryn Sharp.


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

Branch: refs/heads/YARN-2915
Commit: 236ac773c964fa21d6d5f1496023cd61818dd3b1
Parents: ee0c722
Author: Kihwal Lee <ki...@apache.org>
Authored: Thu Sep 29 13:27:30 2016 -0500
Committer: Kihwal Lee <ki...@apache.org>
Committed: Thu Sep 29 13:27:30 2016 -0500

----------------------------------------------------------------------
 .../dev-support/findbugsExcludeFile.xml         |  5 ++
 .../org/apache/hadoop/ipc/ExternalCall.java     | 91 ++++++++++++++++++++
 .../main/java/org/apache/hadoop/ipc/Server.java | 63 +++++++++-----
 .../java/org/apache/hadoop/ipc/TestRPC.java     | 85 ++++++++++++++++++
 4 files changed, 221 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/236ac773/hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml b/hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml
index ec7c396..bded4b99 100644
--- a/hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml
+++ b/hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml
@@ -405,4 +405,9 @@
     <Bug pattern="NP_NULL_PARAM_DEREF"/>
   </Match>
 
+  <Match>
+    <Class name="org.apache.hadoop.ipc.ExternalCall"/>
+    <Filed name="done"/>
+    <Bug pattern="JLM_JSR166_UTILCONCURRENT_MONITORENTER"/>
+  </Match>
 </FindBugsFilter>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/236ac773/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ExternalCall.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ExternalCall.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ExternalCall.java
new file mode 100644
index 0000000..9b4cbcf
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ExternalCall.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.ipc;
+
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.hadoop.ipc.Server.Call;
+import org.apache.hadoop.security.UserGroupInformation;
+
+public abstract class ExternalCall<T> extends Call {
+  private final PrivilegedExceptionAction<T> action;
+  private final AtomicBoolean done = new AtomicBoolean();
+  private T result;
+  private Throwable error;
+
+  public ExternalCall(PrivilegedExceptionAction<T> action) {
+    this.action = action;
+  }
+
+  public abstract UserGroupInformation getRemoteUser();
+
+  public final T get() throws IOException, InterruptedException {
+    waitForCompletion();
+    if (error != null) {
+      if (error instanceof IOException) {
+        throw (IOException)error;
+      } else {
+        throw new IOException(error);
+      }
+    }
+    return result;
+  }
+
+  // wait for response to be triggered to support postponed calls
+  private void waitForCompletion() throws InterruptedException {
+    synchronized(done) {
+      while (!done.get()) {
+        try {
+          done.wait();
+        } catch (InterruptedException ie) {
+          if (Thread.interrupted()) {
+            throw ie;
+          }
+        }
+      }
+    }
+  }
+
+  boolean isDone() {
+    return done.get();
+  }
+
+  // invoked by ipc handler
+  @Override
+  public final Void run() throws IOException {
+    try {
+      result = action.run();
+      sendResponse();
+    } catch (Throwable t) {
+      abortResponse(t);
+    }
+    return null;
+  }
+
+  @Override
+  final void doResponse(Throwable t) {
+    synchronized(done) {
+      error = t;
+      done.set(true);
+      done.notify();
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/236ac773/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
index f509d71..1c7e76a 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
@@ -384,6 +384,11 @@ public abstract class Server {
     return (call != null) ? call.getRemoteUser() : null;
   }
 
+  public static String getProtocol() {
+    Call call = CurCall.get();
+    return (call != null) ? call.getProtocol() : null;
+  }
+
   /** Return true if the invocation was through an RPC.
    */
   public static boolean isRpcInvocation() {
@@ -672,6 +677,11 @@ public abstract class Server {
     private int priorityLevel;
     // the priority level assigned by scheduler, 0 by default
 
+    Call() {
+      this(RpcConstants.INVALID_CALL_ID, RpcConstants.INVALID_RETRY_COUNT,
+        RPC.RpcKind.RPC_BUILTIN, RpcConstants.DUMMY_CLIENT_ID);
+    }
+
     Call(Call call) {
       this(call.callId, call.retryCount, call.rpcKind, call.clientId,
           call.traceScope, call.callerContext);
@@ -703,6 +713,7 @@ public abstract class Server {
       return "Call#" + callId + " Retry#" + retryCount;
     }
 
+    @Override
     public Void run() throws Exception {
       return null;
     }
@@ -718,6 +729,10 @@ public abstract class Server {
       return (addr != null) ? addr.getHostAddress() : null;
     }
 
+    public String getProtocol() {
+      return null;
+    }
+
     /**
      * Allow a IPC response to be postponed instead of sent immediately
      * after the handler returns from the proxy method.  The intended use
@@ -800,6 +815,11 @@ public abstract class Server {
     }
 
     @Override
+    public String getProtocol() {
+      return "rpc";
+    }
+
+    @Override
     public UserGroupInformation getRemoteUser() {
       return connection.user;
     }
@@ -2333,33 +2353,15 @@ public abstract class Server {
       // Save the priority level assignment by the scheduler
       call.setPriorityLevel(callQueue.getPriorityLevel(call));
 
-      if (callQueue.isClientBackoffEnabled()) {
-        // if RPC queue is full, we will ask the RPC client to back off by
-        // throwing RetriableException. Whether RPC client will honor
-        // RetriableException and retry depends on client ipc retry policy.
-        // For example, FailoverOnNetworkExceptionRetry handles
-        // RetriableException.
-        queueRequestOrAskClientToBackOff(call);
-      } else {
-        callQueue.put(call);              // queue the call; maybe blocked here
+      try {
+        queueCall(call);
+      } catch (IOException ioe) {
+        throw new WrappedRpcServerException(
+            RpcErrorCodeProto.ERROR_RPC_SERVER, ioe);
       }
       incRpcCount();  // Increment the rpc count
     }
 
-    private void queueRequestOrAskClientToBackOff(Call call)
-        throws WrappedRpcServerException, InterruptedException {
-      // If rpc scheduler indicates back off based on performance
-      // degradation such as response time or rpc queue is full,
-      // we will ask the client to back off.
-      if (callQueue.shouldBackOff(call) || !callQueue.offer(call)) {
-        rpcMetrics.incrClientBackoff();
-        RetriableException retriableException =
-            new RetriableException("Server is too busy.");
-        throw new WrappedRpcServerExceptionSuppressed(
-            RpcErrorCodeProto.ERROR_RPC_SERVER, retriableException);
-      }
-    }
-
     /**
      * Establish RPC connection setup by negotiating SASL if required, then
      * reading and authorizing the connection header
@@ -2487,6 +2489,21 @@ public abstract class Server {
     }
   }
 
+  public void queueCall(Call call) throws IOException, InterruptedException {
+    if (!callQueue.isClientBackoffEnabled()) {
+      callQueue.put(call); // queue the call; maybe blocked here
+    } else if (callQueue.shouldBackOff(call) || !callQueue.offer(call)) {
+      // If rpc scheduler indicates back off based on performance degradation
+      // such as response time or rpc queue is full, we will ask the client
+      // to back off by throwing RetriableException. Whether the client will
+      // honor RetriableException and retry depends the client and its policy.
+      // For example, IPC clients using FailoverOnNetworkExceptionRetry handle
+      // RetriableException.
+      rpcMetrics.incrClientBackoff();
+      throw new RetriableException("Server is too busy.");
+    }
+  }
+
   /** Handles queued calls . */
   private class Handler extends Thread {
     public Handler(int instanceNumber) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/236ac773/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java
index ff6b25e..92d9183 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java
@@ -64,6 +64,7 @@ import java.net.ConnectException;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.SocketTimeoutException;
+import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -926,6 +927,90 @@ public class TestRPC extends TestRpcBase {
     }
   }
 
+  @Test(timeout=30000)
+  public void testExternalCall() throws Exception {
+    final UserGroupInformation ugi = UserGroupInformation
+        .createUserForTesting("user123", new String[0]);
+    final IOException expectedIOE = new IOException("boom");
+
+    // use 1 handler so the callq can be plugged
+    final Server server = setupTestServer(conf, 1);
+    try {
+      final AtomicBoolean result = new AtomicBoolean();
+
+      ExternalCall<String> remoteUserCall = newExtCall(ugi,
+          new PrivilegedExceptionAction<String>() {
+            @Override
+            public String run() throws Exception {
+              return UserGroupInformation.getCurrentUser().getUserName();
+            }
+          });
+
+      ExternalCall<String> exceptionCall = newExtCall(ugi,
+          new PrivilegedExceptionAction<String>() {
+            @Override
+            public String run() throws Exception {
+              throw expectedIOE;
+            }
+          });
+
+      final CountDownLatch latch = new CountDownLatch(1);
+      final CyclicBarrier barrier = new CyclicBarrier(2);
+
+      ExternalCall<Void> barrierCall = newExtCall(ugi,
+          new PrivilegedExceptionAction<Void>() {
+            @Override
+            public Void run() throws Exception {
+              // notify we are in a handler and then wait to keep the callq
+              // plugged up
+              latch.countDown();
+              barrier.await();
+              return null;
+            }
+          });
+
+      server.queueCall(barrierCall);
+      server.queueCall(exceptionCall);
+      server.queueCall(remoteUserCall);
+
+      // wait for barrier call to enter the handler, check that the other 2
+      // calls are actually queued
+      latch.await();
+      assertEquals(2, server.getCallQueueLen());
+
+      // unplug the callq
+      barrier.await();
+      barrierCall.get();
+
+      // verify correct ugi is used
+      String answer = remoteUserCall.get();
+      assertEquals(ugi.getUserName(), answer);
+
+      try {
+        exceptionCall.get();
+        fail("didn't throw");
+      } catch (IOException ioe) {
+        assertEquals(expectedIOE.getMessage(), ioe.getMessage());
+      }
+    } finally {
+      server.stop();
+    }
+  }
+
+  private <T> ExternalCall<T> newExtCall(UserGroupInformation ugi,
+      PrivilegedExceptionAction<T> callable) {
+    return new ExternalCall<T>(callable) {
+      @Override
+      public String getProtocol() {
+        return "test";
+      }
+      @Override
+      public UserGroupInformation getRemoteUser() {
+        return ugi;
+      }
+    };
+  }
+
   @Test
   public void testRpcMetrics() throws Exception {
     Server server;


---------------------------------------------------------------------
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: HDFS-10619. Cache path in InodesInPath. Contributed by Daryn Sharp.

Posted by su...@apache.org.
HDFS-10619. Cache path in InodesInPath. Contributed by Daryn Sharp.


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

Branch: refs/heads/YARN-2915
Commit: 90020624b05230ad4a7fbd666d0177ecb107a4d6
Parents: 0da54e8
Author: Kihwal Lee <ki...@apache.org>
Authored: Mon Oct 3 09:13:04 2016 -0500
Committer: Kihwal Lee <ki...@apache.org>
Committed: Mon Oct 3 09:13:04 2016 -0500

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/server/namenode/INodesInPath.java    | 5 ++++-
 1 file changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/90020624/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java
index 04d3bda..f05fa37 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java
@@ -278,6 +278,8 @@ public class INodesInPath {
   }
 
   private final byte[][] path;
+  private final String pathname;
+
   /**
    * Array with the specified number of INodes resolved for a given path.
    */
@@ -306,6 +308,7 @@ public class INodesInPath {
     Preconditions.checkArgument(inodes != null && path != null);
     this.inodes = inodes;
     this.path = path;
+    this.pathname = DFSUtil.byteArray2PathString(path);
     this.isRaw = isRaw;
     this.isSnapshot = isSnapshot;
     this.snapshotId = snapshotId;
@@ -366,7 +369,7 @@ public class INodesInPath {
 
   /** @return the full path in string form */
   public String getPath() {
-    return DFSUtil.byteArray2PathString(path);
+    return pathname;
   }
 
   public String getParentPath() {


---------------------------------------------------------------------
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: MAPREDUCE-6638. Do not attempt to recover progress from previous job attempts if spill encryption is enabled. (Haibo Chen via kasha)

Posted by su...@apache.org.
MAPREDUCE-6638. Do not attempt to recover progress from previous job attempts if spill encryption is enabled. (Haibo Chen via kasha)


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

Branch: refs/heads/YARN-2915
Commit: de7a0a92ca1983b35ca4beb7ab712fd700a9e6e0
Parents: 7442084
Author: Karthik Kambatla <ka...@cloudera.com>
Authored: Mon Oct 3 10:30:22 2016 -0700
Committer: Karthik Kambatla <ka...@cloudera.com>
Committed: Mon Oct 3 10:30:22 2016 -0700

----------------------------------------------------------------------
 .../hadoop/mapreduce/v2/app/MRAppMaster.java    | 90 ++++++++++++++------
 .../hadoop/mapreduce/v2/app/TestRecovery.java   | 66 ++++++++++++++
 2 files changed, 129 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/de7a0a92/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
index d94f8a5..4a8a90e 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
@@ -149,7 +149,6 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
 import org.apache.hadoop.yarn.security.client.ClientToAMTokenSecretManager;
 import org.apache.hadoop.yarn.util.Clock;
-import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.util.SystemClock;
 import org.apache.log4j.LogManager;
 
@@ -1303,44 +1302,77 @@ public class MRAppMaster extends CompositeService {
   }
 
   private void processRecovery() throws IOException{
-    if (appAttemptID.getAttemptId() == 1) {
-      return;  // no need to recover on the first attempt
+    boolean attemptRecovery = shouldAttemptRecovery();
+    boolean recoverySucceeded = true;
+    if (attemptRecovery) {
+      LOG.info("Attempting to recover.");
+      try {
+        parsePreviousJobHistory();
+      } catch (IOException e) {
+        LOG.warn("Unable to parse prior job history, aborting recovery", e);
+        recoverySucceeded = false;
+      }
+    }
+
+    if (!isFirstAttempt() && (!attemptRecovery || !recoverySucceeded)) {
+      amInfos.addAll(readJustAMInfos());
+    }
+  }
+
+  private boolean isFirstAttempt() {
+    return appAttemptID.getAttemptId() == 1;
+  }
+
+  /**
+   * Check if the current job attempt should try to recover from previous
+   * job attempts if any.
+   */
+  private boolean shouldAttemptRecovery() throws IOException {
+    if (isFirstAttempt()) {
+      return false;  // no need to recover on the first attempt
     }
 
     boolean recoveryEnabled = getConfig().getBoolean(
         MRJobConfig.MR_AM_JOB_RECOVERY_ENABLE,
         MRJobConfig.MR_AM_JOB_RECOVERY_ENABLE_DEFAULT);
+    if (!recoveryEnabled) {
+      LOG.info("Not attempting to recover. Recovery disabled. To enable " +
+          "recovery, set " + MRJobConfig.MR_AM_JOB_RECOVERY_ENABLE);
+      return false;
+    }
 
     boolean recoverySupportedByCommitter = isRecoverySupported();
+    if (!recoverySupportedByCommitter) {
+      LOG.info("Not attempting to recover. Recovery is not supported by " +
+          committer.getClass() + ". Use an OutputCommitter that supports" +
+              " recovery.");
+      return false;
+    }
 
-    // If a shuffle secret was not provided by the job client then this app
-    // attempt will generate one.  However that disables recovery if there
-    // are reducers as the shuffle secret would be app attempt specific.
-    int numReduceTasks = getConfig().getInt(MRJobConfig.NUM_REDUCES, 0);
+    int reducerCount = getConfig().getInt(MRJobConfig.NUM_REDUCES, 0);
+
+    // If a shuffle secret was not provided by the job client, one will be
+    // generated in this job attempt. However, that disables recovery if
+    // there are reducers as the shuffle secret would be job attempt specific.
     boolean shuffleKeyValidForRecovery =
         TokenCache.getShuffleSecretKey(jobCredentials) != null;
+    if (reducerCount > 0 && !shuffleKeyValidForRecovery) {
+      LOG.info("Not attempting to recover. The shuffle key is invalid for " +
+          "recovery.");
+      return false;
+    }
 
-    if (recoveryEnabled && recoverySupportedByCommitter
-        && (numReduceTasks <= 0 || shuffleKeyValidForRecovery)) {
-      LOG.info("Recovery is enabled. "
-          + "Will try to recover from previous life on best effort basis.");
-      try {
-        parsePreviousJobHistory();
-      } catch (IOException e) {
-        LOG.warn("Unable to parse prior job history, aborting recovery", e);
-        // try to get just the AMInfos
-        amInfos.addAll(readJustAMInfos());
-      }
-    } else {
-      LOG.info("Will not try to recover. recoveryEnabled: "
-            + recoveryEnabled + " recoverySupportedByCommitter: "
-            + recoverySupportedByCommitter + " numReduceTasks: "
-            + numReduceTasks + " shuffleKeyValidForRecovery: "
-            + shuffleKeyValidForRecovery + " ApplicationAttemptID: "
-            + appAttemptID.getAttemptId());
-      // Get the amInfos anyways whether recovery is enabled or not
-      amInfos.addAll(readJustAMInfos());
+    // If the intermediate data is encrypted, recovering the job requires the
+    // access to the key. Until the encryption key is persisted, we should
+    // avoid attempts to recover.
+    boolean spillEncrypted = CryptoUtils.isEncryptedSpillEnabled(getConfig());
+    if (reducerCount > 0 && spillEncrypted) {
+      LOG.info("Not attempting to recover. Intermediate spill encryption" +
+          " is enabled.");
+      return false;
     }
+
+    return true;
   }
 
   private static FSDataInputStream getPreviousJobHistoryStream(
@@ -1440,6 +1472,10 @@ public class MRAppMaster extends CompositeService {
     return amInfos;
   }
 
+  public boolean recovered() {
+    return recoveredJobStartTime > 0;
+  }
+
   /**
    * This can be overridden to instantiate multiple jobs and create a 
    * workflow.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/de7a0a92/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRecovery.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRecovery.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRecovery.java
index 9d5f0ae..071575a 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRecovery.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRecovery.java
@@ -579,6 +579,72 @@ public class TestRecovery {
     app.verifyCompleted();
   }
 
+  @Test
+  public void testRecoveryWithSpillEncryption() throws Exception {
+    int runCount = 0;
+    MRApp app = new MRAppWithHistory(1, 1, false, this.getClass().getName(),
+        true, ++runCount) {
+    };
+    Configuration conf = new Configuration();
+    conf.setBoolean(MRJobConfig.MR_AM_JOB_RECOVERY_ENABLE, true);
+    conf.setBoolean("mapred.mapper.new-api", true);
+    conf.setBoolean("mapred.reducer.new-api", true);
+    conf.setBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, false);
+    conf.set(FileOutputFormat.OUTDIR, outputDir.toString());
+    conf.setBoolean(MRJobConfig.MR_ENCRYPTED_INTERMEDIATE_DATA, true);
+
+    // run the MR job at the first attempt
+    Job jobAttempt1 = app.submit(conf);
+    app.waitForState(jobAttempt1, JobState.RUNNING);
+
+    Iterator<Task> tasks = jobAttempt1.getTasks().values().iterator();
+
+    // finish the map task but the reduce task
+    Task mapper = tasks.next();
+    app.waitForState(mapper, TaskState.RUNNING);
+    TaskAttempt mapAttempt = mapper.getAttempts().values().iterator().next();
+    app.waitForState(mapAttempt, TaskAttemptState.RUNNING);
+    app.getContext().getEventHandler().handle(
+        new TaskAttemptEvent(mapAttempt.getID(), TaskAttemptEventType.TA_DONE));
+    app.waitForState(mapper, TaskState.SUCCEEDED);
+
+    // crash the first attempt of the MR job
+    app.stop();
+
+    // run the MR job again at the second attempt
+    app = new MRAppWithHistory(1, 1, false, this.getClass().getName(), false,
+        ++runCount);
+    Job jobAttempt2 = app.submit(conf);
+    Assert.assertTrue("Recovery from previous job attempt is processed even " +
+        "though intermediate data encryption is enabled.", !app.recovered());
+
+    // The map task succeeded from previous job attempt will not be recovered
+    // because the data spill encryption is enabled.
+    // Let's finish the job at the second attempt and verify its completion.
+    app.waitForState(jobAttempt2, JobState.RUNNING);
+    tasks = jobAttempt2.getTasks().values().iterator();
+    mapper = tasks.next();
+    Task reducer = tasks.next();
+
+    // finish the map task first
+    app.waitForState(mapper, TaskState.RUNNING);
+    mapAttempt = mapper.getAttempts().values().iterator().next();
+    app.waitForState(mapAttempt, TaskAttemptState.RUNNING);
+    app.getContext().getEventHandler().handle(
+        new TaskAttemptEvent(mapAttempt.getID(), TaskAttemptEventType.TA_DONE));
+    app.waitForState(mapper, TaskState.SUCCEEDED);
+
+    // then finish the reduce task
+    TaskAttempt redAttempt = reducer.getAttempts().values().iterator().next();
+    app.waitForState(redAttempt, TaskAttemptState.RUNNING);
+    app.getContext().getEventHandler().handle(
+        new TaskAttemptEvent(redAttempt.getID(), TaskAttemptEventType.TA_DONE));
+    app.waitForState(reducer, TaskState.SUCCEEDED);
+
+    // verify that the job succeeds at the 2rd attempt
+    app.waitForState(jobAttempt2, JobState.SUCCEEDED);
+  }
+
   /**
    * This test case primarily verifies if the recovery is controlled through config
    * property. In this case, recover is turned OFF. AM with 3 maps and 0 reduce.


---------------------------------------------------------------------
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-3671. Integrate Federation services with ResourceManager. Contributed by Subru Krishnan

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


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

Branch: refs/heads/YARN-2915
Commit: 24b9180882ca55951674f55a3d13c9af2b05ca55
Parents: 0cd1680
Author: Jian He <ji...@apache.org>
Authored: Tue Aug 30 12:20:52 2016 +0800
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon Oct 3 14:15:36 2016 -0700

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/24b91808/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 63c8192b..90520c1 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
@@ -2492,9 +2492,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";
 
@@ -2507,6 +2504,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/24b91808/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 63413eb..56b13c4 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
@@ -96,9 +96,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/24b91808/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/24b91808/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/failover/FederationRMFailoverProxyProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/failover/FederationRMFailoverProxyProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/failover/FederationRMFailoverProxyProvider.java
index 90a9239..0ffab0b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/failover/FederationRMFailoverProxyProvider.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/failover/FederationRMFailoverProxyProvider.java
@@ -74,8 +74,8 @@ public class FederationRMFailoverProxyProvider<T>
     this.protocol = proto;
     this.rmProxy.checkAllowedProtocols(this.protocol);
     String clusterId =
-        configuration.get(YarnConfiguration.FEDERATION_SUBCLUSTER_ID);
-    Preconditions.checkNotNull(clusterId, "Missing Federation SubClusterId");
+        configuration.get(YarnConfiguration.RM_CLUSTER_ID);
+    Preconditions.checkNotNull(clusterId, "Missing RM ClusterId");
     this.subClusterId = SubClusterId.newInstance(clusterId);
     this.facade = facade.getInstance();
     if (configuration instanceof YarnConfiguration) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/24b91808/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/24b91808/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/24b91808/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 d2d706d..9baccf2 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
@@ -74,6 +74,7 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWriter;
 import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.ApplicationMasterLauncher;
+import org.apache.hadoop.yarn.server.resourcemanager.federation.FederationStateStoreService;
 import org.apache.hadoop.yarn.server.resourcemanager.metrics.NoOpSystemMetricPublisher;
 import org.apache.hadoop.yarn.server.resourcemanager.metrics.SystemMetricsPublisher;
 import org.apache.hadoop.yarn.server.resourcemanager.metrics.TimelineServiceV1Publisher;
@@ -176,6 +177,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;
@@ -468,6 +470,10 @@ public class ResourceManager extends CompositeService implements Recoverable {
     return new RMTimelineCollectorManager(rmContext);
   }
 
+  private FederationStateStoreService createFederationStateStoreService() {
+    return new FederationStateStoreService(rmContext);
+  }
+
   protected SystemMetricsPublisher createSystemMetricsPublisher() {
     SystemMetricsPublisher publisher;
     if (YarnConfiguration.timelineServiceEnabled(conf) &&
@@ -694,6 +700,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);
@@ -1273,6 +1293,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/24b91808/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/24b91808/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/24b91808/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/24b91808/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 1789e09..a7f563d 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
@@ -64,7 +64,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/24b91808/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


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

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


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

Branch: refs/heads/YARN-2915
Commit: 0cd1680d36ef744f7b644baf1a12655a87f02938
Parents: a8c46ad
Author: Jian He <ji...@apache.org>
Authored: Mon Aug 22 14:43:07 2016 +0800
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon Oct 3 14:15:36 2016 -0700

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


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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0cd1680d/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 a3f9ad0..63c8192b 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
@@ -2485,6 +2485,16 @@ public class YarnConfiguration extends Configuration {
 
   public static final String FEDERATION_PREFIX = YARN_PREFIX + "federation.";
 
+  public static final String FEDERATION_ENABLED = FEDERATION_PREFIX + "enabled";
+  public static final boolean DEFAULT_FEDERATION_ENABLED = false;
+
+  public static final String FEDERATION_FAILOVER_ENABLED =
+      FEDERATION_PREFIX + "failover.enabled";
+  public static final boolean DEFAULT_FEDERATION_FAILOVER_ENABLED = true;
+
+  public static final String FEDERATION_SUBCLUSTER_ID =
+      FEDERATION_PREFIX + "sub-cluster.id";
+
   public static final String FEDERATION_STATESTORE_CLIENT_CLASS =
       FEDERATION_PREFIX + "state-store.class";
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0cd1680d/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 000f5de..63413eb 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
@@ -95,6 +95,10 @@ public class TestYarnConfigurationFields extends TestConfigurationFieldsBase {
     // Federation default configs to be ignored
     configurationPropsToSkipCompare
         .add(YarnConfiguration.DEFAULT_FEDERATION_STATESTORE_CLIENT_CLASS);
+    configurationPropsToSkipCompare
+        .add(YarnConfiguration.FEDERATION_SUBCLUSTER_ID);
+    configurationPropsToSkipCompare
+        .add(YarnConfiguration.FEDERATION_FAILOVER_ENABLED);
 
     // Ignore blacklisting nodes for AM failures feature since it is still a
     // "work in progress"

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

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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0cd1680d/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 910c296..ebbbbf9 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
@@ -2625,6 +2625,13 @@
   <!-- Federation Configuration -->
   <property>
     <description>
+      Flag to indicate whether the RM is participating in Federation or not.
+    </description>
+    <name>yarn.federation.enabled</name>
+    <value>false</value>
+  </property>
+  <property>
+    <description>
       Machine list file to be loaded by the FederationSubCluster Resolver
     </description>
     <name>yarn.federation.machine-list</name>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0cd1680d/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 b6fd0c5..1faf754 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
@@ -112,12 +112,10 @@
     <dependency>
       <groupId>javax.cache</groupId>
       <artifactId>cache-api</artifactId>
-      <version>${jcache.version}</version>
     </dependency>
     <dependency>
       <groupId>org.ehcache</groupId>
       <artifactId>ehcache</artifactId>
-      <version>${ehcache.version}</version>
     </dependency>
   </dependencies>
 

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

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

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

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


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


[38/50] [abbrv] hadoop git commit: YARN-5307. Federation Application State Store internal APIs

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


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

Branch: refs/heads/YARN-2915
Commit: 89f43e6320deea5f00feeebd7c7b64448c1ce38d
Parents: 5ebd7f4
Author: Subru Krishnan <su...@apache.org>
Authored: Fri Aug 5 11:52:44 2016 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon Oct 3 14:15:36 2016 -0700

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


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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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


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


[25/50] [abbrv] hadoop git commit: YARN-5672. FairScheduler: Wrong queue name in log when adding application. (Wilfred Spiegelenburg via kasha)

Posted by su...@apache.org.
YARN-5672. FairScheduler: Wrong queue name in log when adding application. (Wilfred Spiegelenburg via kasha)


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

Branch: refs/heads/YARN-2915
Commit: 0da54e8848764c71a31473516d23ada582013f8c
Parents: 6e130c3
Author: Karthik Kambatla <ka...@apache.org>
Authored: Mon Oct 3 06:03:46 2016 -0700
Committer: Karthik Kambatla <ka...@apache.org>
Committed: Mon Oct 3 06:03:46 2016 -0700

----------------------------------------------------------------------
 .../server/resourcemanager/scheduler/fair/FairScheduler.java  | 7 ++++---
 1 file changed, 4 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0da54e88/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.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/scheduler/fair/FairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
index 310f2f9..920052f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
@@ -668,11 +668,12 @@ public class FairScheduler extends
     queue.getMetrics().submitApp(user);
 
     LOG.info("Accepted application " + applicationId + " from user: " + user
-        + ", in queue: " + queueName + ", currently num of applications: "
-        + applications.size());
+        + ", in queue: " + queue.getName()
+        + ", currently num of applications: " + applications.size());
     if (isAppRecovering) {
       if (LOG.isDebugEnabled()) {
-        LOG.debug(applicationId + " is recovering. Skip notifying APP_ACCEPTED");
+        LOG.debug(applicationId
+            + " is recovering. Skip notifying APP_ACCEPTED");
       }
     } else {
       rmContext.getDispatcher().getEventHandler()


---------------------------------------------------------------------
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: HDFS-10940. Reduce performance penalty of block caching when not used. Contributed by Daryn Sharp.

Posted by su...@apache.org.
HDFS-10940. Reduce performance penalty of block caching when not used. Contributed by Daryn Sharp.


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

Branch: refs/heads/YARN-2915
Commit: 744208431f7365bf054e6b773b86af2583001e1d
Parents: 9002062
Author: Kihwal Lee <ki...@apache.org>
Authored: Mon Oct 3 11:27:23 2016 -0500
Committer: Kihwal Lee <ki...@apache.org>
Committed: Mon Oct 3 11:27:23 2016 -0500

----------------------------------------------------------------------
 .../hdfs/server/blockmanagement/BlockManager.java  | 10 +++++++++-
 .../hadoop/hdfs/server/namenode/CacheManager.java  | 12 +++++++++++-
 .../server/namenode/FSDirStatAndListingOp.java     | 17 +----------------
 .../hdfs/server/namenode/TestCacheDirectives.java  | 10 ++++++++++
 4 files changed, 31 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/74420843/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index 886984a..9b426bb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -103,6 +103,7 @@ import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
 import org.apache.hadoop.hdfs.util.FoldedTreeSet;
 import org.apache.hadoop.hdfs.util.LightWeightHashSet;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.apache.hadoop.hdfs.server.namenode.CacheManager;
 
 import static org.apache.hadoop.hdfs.util.StripedBlockUtil.getInternalBlockLength;
 
@@ -1145,9 +1146,16 @@ public class BlockManager implements BlockStatsMXBean {
             fileSizeExcludeBlocksUnderConstruction, mode);
         isComplete = true;
       }
-      return new LocatedBlocks(fileSizeExcludeBlocksUnderConstruction,
+      LocatedBlocks locations = new LocatedBlocks(
+          fileSizeExcludeBlocksUnderConstruction,
           isFileUnderConstruction, locatedblocks, lastlb, isComplete, feInfo,
           ecPolicy);
+      // Set caching information for the located blocks.
+      CacheManager cm = namesystem.getCacheManager();
+      if (cm != null) {
+        cm.setCachedLocations(locations);
+      }
+      return locations;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/74420843/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CacheManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CacheManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CacheManager.java
index 366dd9b..24bf751 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CacheManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CacheManager.java
@@ -63,6 +63,7 @@ import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolInfoProto;
 import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
@@ -902,7 +903,16 @@ public final class CacheManager {
     return new BatchedListEntries<CachePoolEntry>(results, false);
   }
 
-  public void setCachedLocations(LocatedBlock block) {
+  public void setCachedLocations(LocatedBlocks locations) {
+    // don't attempt lookups if there are no cached blocks
+    if (cachedBlocks.size() > 0) {
+      for (LocatedBlock lb : locations.getLocatedBlocks()) {
+        setCachedLocations(lb);
+      }
+    }
+  }
+
+  private void setCachedLocations(LocatedBlock block) {
     CachedBlock cachedBlock =
         new CachedBlock(block.getBlock().getBlockId(),
             (short)0, false);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/74420843/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
index 4876fb1..f56d83d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
@@ -36,7 +36,6 @@ import org.apache.hadoop.hdfs.protocol.FsPermissionExtension;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
-import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.SnapshotException;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
@@ -156,7 +155,6 @@ class FSDirStatAndListingOp {
         "Negative offset is not supported. File: " + src);
     Preconditions.checkArgument(length >= 0,
         "Negative length is not supported. File: " + src);
-    CacheManager cm = fsd.getFSNamesystem().getCacheManager();
     BlockManager bm = fsd.getBlockManager();
     fsd.readLock();
     try {
@@ -190,11 +188,6 @@ class FSDirStatAndListingOp {
           inode.getBlocks(iip.getPathSnapshotId()), fileSize, isUc, offset,
           length, needBlockToken, iip.isSnapshot(), feInfo, ecPolicy);
 
-      // Set caching information for the located blocks.
-      for (LocatedBlock lb : blocks.getLocatedBlocks()) {
-        cm.setCachedLocations(lb);
-      }
-
       final long now = now();
       boolean updateAccessTime = fsd.isAccessTimeSupported()
           && !iip.isSnapshot()
@@ -461,7 +454,7 @@ class FSDirStatAndListingOp {
         node.asDirectory().getChildrenNum(snapshot) : 0;
 
     INodeAttributes nodeAttrs = fsd.getAttributes(iip);
-    HdfsFileStatus status = createFileStatus(
+    return createFileStatus(
         size,
         node.isDirectory(),
         replication,
@@ -479,14 +472,6 @@ class FSDirStatAndListingOp {
         storagePolicy,
         ecPolicy,
         loc);
-    // Set caching information for the located blocks.
-    if (loc != null) {
-      CacheManager cacheManager = fsd.getFSNamesystem().getCacheManager();
-      for (LocatedBlock lb: loc.getLocatedBlocks()) {
-        cacheManager.setCachedLocations(lb);
-      }
-    }
-    return status;
   }
 
   private static HdfsFileStatus createFileStatus(long length, boolean isdir,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/74420843/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCacheDirectives.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCacheDirectives.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCacheDirectives.java
index efb5cf8..658e4ca 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCacheDirectives.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCacheDirectives.java
@@ -72,6 +72,7 @@ import org.apache.hadoop.hdfs.protocol.CachePoolStats;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.CachedBlocksList.Type;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
@@ -89,6 +90,7 @@ import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
+import org.mockito.Mockito;
 
 import com.google.common.base.Supplier;
 
@@ -1531,4 +1533,12 @@ public class TestCacheDirectives {
       DataNodeTestUtils.setCacheReportsDisabledForTests(cluster, false);
     }
   }
+
+  @Test
+  public void testNoLookupsWhenNotUsed() throws Exception {
+    CacheManager cm = cluster.getNamesystem().getCacheManager();
+    LocatedBlocks locations = Mockito.mock(LocatedBlocks.class);
+    cm.setCachedLocations(locations);
+    Mockito.verifyZeroInteractions(locations);
+  }
 }


---------------------------------------------------------------------
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: HADOOP-13640. Fix findbugs warning in VersionInfoMojo.java. Contributed by Yuanbo Liu.

Posted by su...@apache.org.
HADOOP-13640. Fix findbugs warning in VersionInfoMojo.java. Contributed by Yuanbo Liu.


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

Branch: refs/heads/YARN-2915
Commit: 82c55dcbc8e3d5314aae9f8f600c660759213e45
Parents: 10be459
Author: Akira Ajisaka <aa...@apache.org>
Authored: Fri Sep 30 18:17:30 2016 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Fri Sep 30 18:17:30 2016 +0900

----------------------------------------------------------------------
 .../apache/hadoop/maven/plugin/versioninfo/VersionInfoMojo.java    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/82c55dcb/hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/versioninfo/VersionInfoMojo.java
----------------------------------------------------------------------
diff --git a/hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/versioninfo/VersionInfoMojo.java b/hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/versioninfo/VersionInfoMojo.java
index cd2651b..f6faea0 100644
--- a/hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/versioninfo/VersionInfoMojo.java
+++ b/hadoop-maven-plugins/src/main/java/org/apache/hadoop/maven/plugin/versioninfo/VersionInfoMojo.java
@@ -160,7 +160,7 @@ public class VersionInfoMojo extends AbstractMojo {
       if (index > -1) {
         res[0] = path.substring(0, index - 1);
         int branchIndex = index + "branches".length() + 1;
-        index = path.indexOf("/", branchIndex);
+        index = path.indexOf('/', branchIndex);
         if (index > -1) {
           res[1] = path.substring(branchIndex, index);
         } else {


---------------------------------------------------------------------
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-5486. Update OpportunisticContainerAllocatorAMService::allocate method to handle OPPORTUNISTIC container requests. (Konstantinos Karanasos via asuresh)

Posted by su...@apache.org.
YARN-5486. Update OpportunisticContainerAllocatorAMService::allocate method to handle OPPORTUNISTIC container requests. (Konstantinos Karanasos via asuresh)


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

Branch: refs/heads/YARN-2915
Commit: 10be45986cdf86a89055065b752959bd6369d54f
Parents: 1e0ea27
Author: Arun Suresh <as...@apache.org>
Authored: Thu Sep 29 15:11:41 2016 -0700
Committer: Arun Suresh <as...@apache.org>
Committed: Thu Sep 29 15:11:41 2016 -0700

----------------------------------------------------------------------
 .../TestOpportunisticContainerAllocation.java   | 398 +++++++++++++++++++
 .../OpportunisticContainerAllocator.java        |  22 +-
 .../OpportunisticContainerContext.java          |  49 ++-
 .../yarn/server/nodemanager/NodeManager.java    |   3 +-
 .../amrmproxy/DefaultRequestInterceptor.java    |   4 +-
 .../scheduler/DistributedScheduler.java         |  59 +--
 ...pportunisticContainerAllocatorAMService.java | 215 ++++++----
 .../server/resourcemanager/ResourceManager.java |  12 +-
 .../scheduler/SchedulerApplicationAttempt.java  |  58 ++-
 .../distributed/NodeQueueLoadMonitor.java       |  45 ++-
 ...pportunisticContainerAllocatorAMService.java |  10 +-
 11 files changed, 707 insertions(+), 168 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/10be4598/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestOpportunisticContainerAllocation.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestOpportunisticContainerAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestOpportunisticContainerAllocation.java
new file mode 100644
index 0000000..b9b4b02
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestOpportunisticContainerAllocation.java
@@ -0,0 +1,398 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.client.api.impl;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.service.Service;
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
+import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+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.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.hadoop.yarn.api.records.ContainerState;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.ExecutionType;
+import org.apache.hadoop.yarn.api.records.ExecutionTypeRequest;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.api.records.NMToken;
+import org.apache.hadoop.yarn.api.records.NodeReport;
+import org.apache.hadoop.yarn.api.records.NodeState;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.api.records.Token;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
+import org.apache.hadoop.yarn.client.ClientRMProxy;
+import org.apache.hadoop.yarn.client.api.AMRMClient;
+import org.apache.hadoop.yarn.client.api.NMTokenCache;
+import org.apache.hadoop.yarn.client.api.YarnClient;
+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.resourcemanager.rmapp.attempt.RMAppAttempt;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
+import org.apache.hadoop.yarn.server.utils.BuilderUtils;
+import org.apache.hadoop.yarn.util.Records;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Set;
+import java.util.TreeSet;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Class that tests the allocation of OPPORTUNISTIC containers through the
+ * centralized ResourceManager.
+ */
+public class TestOpportunisticContainerAllocation {
+  private static Configuration conf = null;
+  private static MiniYARNCluster yarnCluster = null;
+  private static YarnClient yarnClient = null;
+  private static List<NodeReport> nodeReports = null;
+  private static ApplicationAttemptId attemptId = null;
+  private static int nodeCount = 3;
+
+  private static final int ROLLING_INTERVAL_SEC = 13;
+  private static final long AM_EXPIRE_MS = 4000;
+
+  private static Resource capability;
+  private static Priority priority;
+  private static Priority priority2;
+  private static String node;
+  private static String rack;
+  private static String[] nodes;
+  private static String[] racks;
+  private final static int DEFAULT_ITERATION = 3;
+
+  @BeforeClass
+  public static void setup() throws Exception {
+    // start minicluster
+    conf = new YarnConfiguration();
+    conf.setLong(
+        YarnConfiguration.RM_AMRM_TOKEN_MASTER_KEY_ROLLING_INTERVAL_SECS,
+        ROLLING_INTERVAL_SEC);
+    conf.setLong(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, AM_EXPIRE_MS);
+    conf.setInt(YarnConfiguration.RM_NM_HEARTBEAT_INTERVAL_MS, 100);
+    // set the minimum allocation so that resource decrease can go under 1024
+    conf.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 512);
+    conf.setBoolean(
+        YarnConfiguration.OPPORTUNISTIC_CONTAINER_ALLOCATION_ENABLED, true);
+    conf.setLong(YarnConfiguration.NM_LOG_RETAIN_SECONDS, 1);
+    yarnCluster =
+        new MiniYARNCluster(TestAMRMClient.class.getName(), nodeCount, 1, 1);
+    yarnCluster.init(conf);
+    yarnCluster.start();
+
+    // start rm client
+    yarnClient = YarnClient.createYarnClient();
+    yarnClient.init(conf);
+    yarnClient.start();
+
+    // get node info
+    nodeReports = yarnClient.getNodeReports(NodeState.RUNNING);
+
+    priority = Priority.newInstance(1);
+    priority2 = Priority.newInstance(2);
+    capability = Resource.newInstance(1024, 1);
+
+    node = nodeReports.get(0).getNodeId().getHost();
+    rack = nodeReports.get(0).getRackName();
+    nodes = new String[]{node};
+    racks = new String[]{rack};
+  }
+
+  @Before
+  public void startApp() throws Exception {
+    // submit new app
+    ApplicationSubmissionContext appContext =
+        yarnClient.createApplication().getApplicationSubmissionContext();
+    ApplicationId appId = appContext.getApplicationId();
+    // set the application name
+    appContext.setApplicationName("Test");
+    // Set the priority for the application master
+    Priority pri = Records.newRecord(Priority.class);
+    pri.setPriority(0);
+    appContext.setPriority(pri);
+    // Set the queue to which this application is to be submitted in the RM
+    appContext.setQueue("default");
+    // Set up the container launch context for the application master
+    ContainerLaunchContext amContainer = BuilderUtils.newContainerLaunchContext(
+        Collections.<String, LocalResource>emptyMap(),
+        new HashMap<String, String>(), Arrays.asList("sleep", "100"),
+        new HashMap<String, ByteBuffer>(), null,
+        new HashMap<ApplicationAccessType, String>());
+    appContext.setAMContainerSpec(amContainer);
+    appContext.setResource(Resource.newInstance(1024, 1));
+    // Create the request to send to the applications manager
+    SubmitApplicationRequest appRequest =
+        Records.newRecord(SubmitApplicationRequest.class);
+    appRequest.setApplicationSubmissionContext(appContext);
+    // Submit the application to the applications manager
+    yarnClient.submitApplication(appContext);
+
+    // wait for app to start
+    RMAppAttempt appAttempt = null;
+    while (true) {
+      ApplicationReport appReport = yarnClient.getApplicationReport(appId);
+      if (appReport.getYarnApplicationState() ==
+          YarnApplicationState.ACCEPTED) {
+        attemptId = appReport.getCurrentApplicationAttemptId();
+        appAttempt = yarnCluster.getResourceManager().getRMContext().getRMApps()
+            .get(attemptId.getApplicationId()).getCurrentAppAttempt();
+        while (true) {
+          if (appAttempt.getAppAttemptState() == RMAppAttemptState.LAUNCHED) {
+            break;
+          }
+        }
+        break;
+      }
+    }
+    // Just dig into the ResourceManager and get the AMRMToken just for the sake
+    // of testing.
+    UserGroupInformation.setLoginUser(UserGroupInformation
+        .createRemoteUser(UserGroupInformation.getCurrentUser().getUserName()));
+
+    // emulate RM setup of AMRM token in credentials by adding the token
+    // *before* setting the token service
+    UserGroupInformation.getCurrentUser().addToken(appAttempt.getAMRMToken());
+    appAttempt.getAMRMToken()
+        .setService(ClientRMProxy.getAMRMTokenService(conf));
+  }
+
+  @After
+  public void cancelApp() throws YarnException, IOException {
+    yarnClient.killApplication(attemptId.getApplicationId());
+    attemptId = null;
+  }
+
+  @AfterClass
+  public static void tearDown() {
+    if (yarnClient != null &&
+        yarnClient.getServiceState() == Service.STATE.STARTED) {
+      yarnClient.stop();
+    }
+    if (yarnCluster != null &&
+        yarnCluster.getServiceState() == Service.STATE.STARTED) {
+      yarnCluster.stop();
+    }
+  }
+
+  @Test(timeout = 60000)
+  public void testAMRMClient() throws YarnException, IOException {
+    AMRMClient<AMRMClient.ContainerRequest> amClient = null;
+    try {
+      // start am rm client
+      amClient = AMRMClient.<AMRMClient.ContainerRequest>createAMRMClient();
+
+      //setting an instance NMTokenCache
+      amClient.setNMTokenCache(new NMTokenCache());
+      //asserting we are not using the singleton instance cache
+      Assert.assertNotSame(NMTokenCache.getSingleton(),
+          amClient.getNMTokenCache());
+
+      amClient.init(conf);
+      amClient.start();
+
+      amClient.registerApplicationMaster("Host", 10000, "");
+
+      testAllocation((AMRMClientImpl<AMRMClient.ContainerRequest>)amClient);
+
+      amClient
+          .unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED, null,
+              null);
+
+    } finally {
+      if (amClient != null &&
+          amClient.getServiceState() == Service.STATE.STARTED) {
+        amClient.stop();
+      }
+    }
+  }
+
+  private void testAllocation(
+      final AMRMClientImpl<AMRMClient.ContainerRequest> amClient)
+      throws YarnException, IOException {
+    // setup container request
+
+    assertEquals(0, amClient.ask.size());
+    assertEquals(0, amClient.release.size());
+
+    amClient.addContainerRequest(
+        new AMRMClient.ContainerRequest(capability, nodes, racks, priority));
+    amClient.addContainerRequest(
+        new AMRMClient.ContainerRequest(capability, nodes, racks, priority));
+    amClient.addContainerRequest(
+        new AMRMClient.ContainerRequest(capability, nodes, racks, priority));
+    amClient.addContainerRequest(
+        new AMRMClient.ContainerRequest(capability, nodes, racks, priority));
+    amClient.addContainerRequest(
+        new AMRMClient.ContainerRequest(capability, null, null, priority2, 0,
+            true, null,
+            ExecutionTypeRequest.newInstance(
+                ExecutionType.OPPORTUNISTIC, true)));
+    amClient.addContainerRequest(
+        new AMRMClient.ContainerRequest(capability, null, null, priority2, 0,
+            true, null,
+            ExecutionTypeRequest.newInstance(
+                ExecutionType.OPPORTUNISTIC, true)));
+
+    amClient.removeContainerRequest(
+        new AMRMClient.ContainerRequest(capability, nodes, racks, priority));
+    amClient.removeContainerRequest(
+        new AMRMClient.ContainerRequest(capability, nodes, racks, priority));
+    amClient.removeContainerRequest(
+        new AMRMClient.ContainerRequest(capability, null, null, priority2, 0,
+            true, null,
+            ExecutionTypeRequest.newInstance(
+                ExecutionType.OPPORTUNISTIC, true)));
+
+    int containersRequestedNode = amClient.getTable(0).get(priority,
+        node, ExecutionType.GUARANTEED, capability).remoteRequest
+        .getNumContainers();
+    int containersRequestedRack = amClient.getTable(0).get(priority,
+        rack, ExecutionType.GUARANTEED, capability).remoteRequest
+        .getNumContainers();
+    int containersRequestedAny = amClient.getTable(0).get(priority,
+        ResourceRequest.ANY, ExecutionType.GUARANTEED, capability)
+        .remoteRequest.getNumContainers();
+    int oppContainersRequestedAny =
+        amClient.getTable(0).get(priority2, ResourceRequest.ANY,
+            ExecutionType.OPPORTUNISTIC, capability).remoteRequest
+            .getNumContainers();
+
+    assertEquals(2, containersRequestedNode);
+    assertEquals(2, containersRequestedRack);
+    assertEquals(2, containersRequestedAny);
+    assertEquals(1, oppContainersRequestedAny);
+
+    assertEquals(4, amClient.ask.size());
+    assertEquals(0, amClient.release.size());
+
+    // RM should allocate container within 2 calls to allocate()
+    int allocatedContainerCount = 0;
+    int allocatedOpportContainerCount = 0;
+    int iterationsLeft = 10;
+    Set<ContainerId> releases = new TreeSet<>();
+
+    amClient.getNMTokenCache().clearCache();
+    Assert.assertEquals(0,
+        amClient.getNMTokenCache().numberOfTokensInCache());
+    HashMap<String, Token> receivedNMTokens = new HashMap<>();
+
+    while (allocatedContainerCount <
+        containersRequestedAny + oppContainersRequestedAny
+        && iterationsLeft-- > 0) {
+      AllocateResponse allocResponse = amClient.allocate(0.1f);
+      assertEquals(0, amClient.ask.size());
+      assertEquals(0, amClient.release.size());
+
+      allocatedContainerCount += allocResponse.getAllocatedContainers()
+          .size();
+      for (Container container : allocResponse.getAllocatedContainers()) {
+        if (container.getExecutionType() == ExecutionType.OPPORTUNISTIC) {
+          allocatedOpportContainerCount++;
+        }
+        ContainerId rejectContainerId = container.getId();
+        releases.add(rejectContainerId);
+      }
+
+      for (NMToken token : allocResponse.getNMTokens()) {
+        String nodeID = token.getNodeId().toString();
+        receivedNMTokens.put(nodeID, token.getToken());
+      }
+
+      if (allocatedContainerCount < containersRequestedAny) {
+        // sleep to let NM's heartbeat to RM and trigger allocations
+        sleep(100);
+      }
+    }
+
+    assertEquals(allocatedContainerCount,
+        containersRequestedAny + oppContainersRequestedAny);
+    assertEquals(allocatedOpportContainerCount, oppContainersRequestedAny);
+    for (ContainerId rejectContainerId : releases) {
+      amClient.releaseAssignedContainer(rejectContainerId);
+    }
+    assertEquals(3, amClient.release.size());
+    assertEquals(0, amClient.ask.size());
+
+    // need to tell the AMRMClient that we don't need these resources anymore
+    amClient.removeContainerRequest(
+        new AMRMClient.ContainerRequest(capability, nodes, racks, priority));
+    amClient.removeContainerRequest(
+        new AMRMClient.ContainerRequest(capability, nodes, racks, priority));
+    amClient.removeContainerRequest(
+        new AMRMClient.ContainerRequest(capability, nodes, racks, priority2, 0,
+            true, null,
+            ExecutionTypeRequest.newInstance(
+                ExecutionType.OPPORTUNISTIC, true)));
+    assertEquals(4, amClient.ask.size());
+
+    iterationsLeft = 3;
+    // do a few iterations to ensure RM is not going to send new containers
+    while (iterationsLeft-- > 0) {
+      // inform RM of rejection
+      AllocateResponse allocResponse = amClient.allocate(0.1f);
+      // RM did not send new containers because AM does not need any
+      assertEquals(0, allocResponse.getAllocatedContainers().size());
+      if (allocResponse.getCompletedContainersStatuses().size() > 0) {
+        for (ContainerStatus cStatus : allocResponse
+            .getCompletedContainersStatuses()) {
+          if (releases.contains(cStatus.getContainerId())) {
+            assertEquals(cStatus.getState(), ContainerState.COMPLETE);
+            assertEquals(-100, cStatus.getExitStatus());
+            releases.remove(cStatus.getContainerId());
+          }
+        }
+      }
+      if (iterationsLeft > 0) {
+        // sleep to make sure NM's heartbeat
+        sleep(100);
+      }
+    }
+    assertEquals(0, amClient.ask.size());
+    assertEquals(0, amClient.release.size());
+  }
+
+  private void sleep(int sleepTime) {
+    try {
+      Thread.sleep(sleepTime);
+    } catch (InterruptedException e) {
+      e.printStackTrace();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10be4598/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/scheduler/OpportunisticContainerAllocator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/scheduler/OpportunisticContainerAllocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/scheduler/OpportunisticContainerAllocator.java
index 9b2fd38..9c158e9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/scheduler/OpportunisticContainerAllocator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/scheduler/OpportunisticContainerAllocator.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.yarn.util.resource.Resources;
 
 import java.net.InetSocketAddress;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -145,15 +146,6 @@ public class OpportunisticContainerAllocator {
     }
 
     /**
-     * Sets the underlying Atomic Long. To be used when implementation needs to
-     * share the underlying AtomicLong of an existing counter.
-     * @param counter AtomicLong
-     */
-    public void setContainerIdCounter(AtomicLong counter) {
-      this.containerIdCounter = counter;
-    }
-
-    /**
      * Generates a new long value. Default implementation increments the
      * underlying AtomicLong. Sub classes are encouraged to over-ride this
      * behaviour.
@@ -213,6 +205,10 @@ public class OpportunisticContainerAllocator {
     PartitionedResourceRequests partitionedAsks =
         partitionAskList(request.getAskList());
 
+    if (partitionedAsks.getOpportunistic().isEmpty()) {
+      return Collections.emptyList();
+    }
+
     List<ContainerId> releasedContainers = request.getReleaseList();
     int numReleasedContainers = releasedContainers.size();
     if (numReleasedContainers > 0) {
@@ -236,8 +232,8 @@ public class OpportunisticContainerAllocator {
         appContext.getOutstandingOpReqs().descendingKeySet()) {
       // Allocated containers :
       //  Key = Requested Capability,
-      //  Value = List of Containers of given Cap (The actual container size
-      //          might be different than what is requested.. which is why
+      //  Value = List of Containers of given cap (the actual container size
+      //          might be different than what is requested, which is why
       //          we need the requested capability (key) to match against
       //          the outstanding reqs)
       Map<Resource, List<Container>> allocated = allocate(rmIdentifier,
@@ -290,6 +286,10 @@ public class OpportunisticContainerAllocator {
       }
       nodesForScheduling.add(nodeEntry.getValue());
     }
+    if (nodesForScheduling.isEmpty()) {
+      LOG.warn("No nodes available for allocating opportunistic containers.");
+      return;
+    }
     int numAllocated = 0;
     int nextNodeToSchedule = 0;
     for (int numCont = 0; numCont < toAllocate; numCont++) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10be4598/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/scheduler/OpportunisticContainerContext.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/scheduler/OpportunisticContainerContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/scheduler/OpportunisticContainerContext.java
index 1b701ea..6fcddf8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/scheduler/OpportunisticContainerContext.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/scheduler/OpportunisticContainerContext.java
@@ -18,9 +18,11 @@
 
 package org.apache.hadoop.yarn.server.scheduler;
 
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.NMToken;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.ExecutionType;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
@@ -28,9 +30,11 @@ import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.LinkedHashMap;
+import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -56,15 +60,13 @@ public class OpportunisticContainerContext {
   private ContainerIdGenerator containerIdGenerator =
       new ContainerIdGenerator();
 
-  private Map<String, NodeId> nodeMap = new LinkedHashMap<>();
+  private volatile List<NodeId> nodeList = new LinkedList<>();
+  private final Map<String, NodeId> nodeMap = new LinkedHashMap<>();
 
-  // Mapping of NodeId to NodeTokens. Populated either from RM response or
-  // generated locally if required.
-  private Map<NodeId, NMToken> nodeTokens = new HashMap<>();
   private final Set<String> blacklist = new HashSet<>();
 
   // This maintains a map of outstanding OPPORTUNISTIC Reqs. Key-ed by Priority,
-  // Resource Name (Host/rack/any) and capability. This mapping is required
+  // Resource Name (host/rack/any) and capability. This mapping is required
   // to match a received Container to an outstanding OPPORTUNISTIC
   // ResourceRequest (ask).
   private final TreeMap<Priority, Map<Resource, ResourceRequest>>
@@ -74,7 +76,7 @@ public class OpportunisticContainerContext {
     return containersAllocated;
   }
 
-  public OpportunisticContainerAllocator.AllocationParams getAppParams() {
+  public AllocationParams getAppParams() {
     return appParams;
   }
 
@@ -88,11 +90,29 @@ public class OpportunisticContainerContext {
   }
 
   public Map<String, NodeId> getNodeMap() {
-    return nodeMap;
+    return Collections.unmodifiableMap(nodeMap);
   }
 
-  public Map<NodeId, NMToken> getNodeTokens() {
-    return nodeTokens;
+  public synchronized void updateNodeList(List<NodeId> newNodeList) {
+    // This is an optimization for centralized placement. The
+    // OppContainerAllocatorAMService has a cached list of nodes which it sets
+    // here. The nodeMap needs to be updated only if the backing node list is
+    // modified.
+    if (newNodeList != nodeList) {
+      nodeList = newNodeList;
+      nodeMap.clear();
+      for (NodeId n : nodeList) {
+        nodeMap.put(n.getHost(), n);
+      }
+    }
+  }
+
+  public void updateAllocationParams(Resource minResource, Resource maxResource,
+      Resource incrResource, int containerTokenExpiryInterval) {
+    appParams.setMinResource(minResource);
+    appParams.setMaxResource(maxResource);
+    appParams.setIncrementResource(incrResource);
+    appParams.setContainerTokenExpiryInterval(containerTokenExpiryInterval);
   }
 
   public Set<String> getBlacklist() {
@@ -104,6 +124,15 @@ public class OpportunisticContainerContext {
     return outstandingOpReqs;
   }
 
+  public void updateCompletedContainers(AllocateResponse allocateResponse) {
+    for (ContainerStatus cs :
+        allocateResponse.getCompletedContainersStatuses()) {
+      if (cs.getExecutionType() == ExecutionType.OPPORTUNISTIC) {
+        containersAllocated.remove(cs.getContainerId());
+      }
+    }
+  }
+
   /**
    * Takes a list of ResourceRequests (asks), extracts the key information viz.
    * (Priority, ResourceName, Capability) and adds to the outstanding

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10be4598/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.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/NodeManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
index 7f13334..37f67c4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
@@ -336,8 +336,7 @@ public class NodeManager extends CompositeService
     addService(nodeHealthChecker);
 
     boolean isDistSchedulingEnabled =
-        conf.getBoolean(YarnConfiguration.
-            OPPORTUNISTIC_CONTAINER_ALLOCATION_ENABLED,
+        conf.getBoolean(YarnConfiguration.DIST_SCHEDULING_ENABLED,
             YarnConfiguration.DIST_SCHEDULING_ENABLED_DEFAULT);
 
     this.context = createNMContext(containerTokenSecretManager,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10be4598/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 efbdfb4..22fc8f6 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
@@ -152,7 +152,7 @@ public final class DefaultRequestInterceptor extends
       return ((DistributedSchedulingAMProtocol)rmClient)
           .registerApplicationMasterForDistributedScheduling(request);
     } else {
-      throw new YarnException("Distributed Scheduling is not enabled !!");
+      throw new YarnException("Distributed Scheduling is not enabled.");
     }
   }
 
@@ -174,7 +174,7 @@ public final class DefaultRequestInterceptor extends
       }
       return allocateResponse;
     } else {
-      throw new YarnException("Distributed Scheduling is not enabled !!");
+      throw new YarnException("Distributed Scheduling is not enabled.");
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10be4598/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/scheduler/DistributedScheduler.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/scheduler/DistributedScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/scheduler/DistributedScheduler.java
index 368858c..8a40337 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/scheduler/DistributedScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/scheduler/DistributedScheduler.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.yarn.server.nodemanager.scheduler;
 import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
+import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.server.api.protocolrecords.DistributedSchedulingAllocateRequest;
@@ -32,8 +33,6 @@ import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterReque
 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.ContainerStatus;
-import org.apache.hadoop.yarn.api.records.ExecutionType;
 import org.apache.hadoop.yarn.api.records.NMToken;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.exceptions.YarnException;
@@ -48,7 +47,9 @@ import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 
 /**
  * <p>The DistributedScheduler runs on the NodeManager and is modeled as an
@@ -74,6 +75,9 @@ public final class DistributedScheduler extends AbstractRequestInterceptor {
   private OpportunisticContainerContext oppContainerContext =
       new OpportunisticContainerContext();
 
+  // Mapping of NodeId to NodeTokens. Populated either from RM response or
+  // generated locally if required.
+  private Map<NodeId, NMToken> nodeTokens = new HashMap<>();
   private ApplicationAttemptId applicationAttemptId;
   private OpportunisticContainerAllocator containerAllocator;
   private NMTokenSecretManagerInNM nmSecretManager;
@@ -157,17 +161,17 @@ public final class DistributedScheduler extends AbstractRequestInterceptor {
   }
 
   /**
-   * Check if we already have a NMToken. if Not, generate the Token and
-   * add it to the response
+   * Adds all the newly allocated Containers to the allocate Response.
+   * Additionally, in case the NMToken for one of the nodes does not exist, it
+   * generates one and adds it to the response.
    */
-  private void updateResponseWithNMTokens(AllocateResponse response,
+  private void updateAllocateResponse(AllocateResponse response,
       List<NMToken> nmTokens, List<Container> allocatedContainers) {
     List<NMToken> newTokens = new ArrayList<>();
     if (allocatedContainers.size() > 0) {
       response.getAllocatedContainers().addAll(allocatedContainers);
       for (Container alloc : allocatedContainers) {
-        if (!oppContainerContext.getNodeTokens().containsKey(
-            alloc.getNodeId())) {
+        if (!nodeTokens.containsKey(alloc.getNodeId())) {
           newTokens.add(nmSecretManager.generateNMToken(appSubmitter, alloc));
         }
       }
@@ -179,17 +183,14 @@ public final class DistributedScheduler extends AbstractRequestInterceptor {
 
   private void updateParameters(
       RegisterDistributedSchedulingAMResponse registerResponse) {
-    oppContainerContext.getAppParams().setMinResource(
-        registerResponse.getMinContainerResource());
-    oppContainerContext.getAppParams().setMaxResource(
-        registerResponse.getMaxContainerResource());
-    oppContainerContext.getAppParams().setIncrementResource(
-        registerResponse.getIncrContainerResource());
-    if (oppContainerContext.getAppParams().getIncrementResource() == null) {
-      oppContainerContext.getAppParams().setIncrementResource(
-          oppContainerContext.getAppParams().getMinResource());
+    Resource incrementResource = registerResponse.getIncrContainerResource();
+    if (incrementResource == null) {
+      incrementResource = registerResponse.getMinContainerResource();
     }
-    oppContainerContext.getAppParams().setContainerTokenExpiryInterval(
+    oppContainerContext.updateAllocationParams(
+        registerResponse.getMinContainerResource(),
+        registerResponse.getMaxContainerResource(),
+        incrementResource,
         registerResponse.getContainerTokenExpiryInterval());
 
     oppContainerContext.getContainerIdGenerator()
@@ -198,14 +199,7 @@ public final class DistributedScheduler extends AbstractRequestInterceptor {
   }
 
   private void setNodeList(List<NodeId> nodeList) {
-    oppContainerContext.getNodeMap().clear();
-    addToNodeList(nodeList);
-  }
-
-  private void addToNodeList(List<NodeId> nodes) {
-    for (NodeId n : nodes) {
-      oppContainerContext.getNodeMap().put(n.getHost(), n);
-    }
+    oppContainerContext.updateNodeList(nodeList);
   }
 
   @Override
@@ -243,23 +237,14 @@ public final class DistributedScheduler extends AbstractRequestInterceptor {
     setNodeList(dsResp.getNodesForScheduling());
     List<NMToken> nmTokens = dsResp.getAllocateResponse().getNMTokens();
     for (NMToken nmToken : nmTokens) {
-      oppContainerContext.getNodeTokens().put(nmToken.getNodeId(), nmToken);
+      nodeTokens.put(nmToken.getNodeId(), nmToken);
     }
 
-    List<ContainerStatus> completedContainers =
-        dsResp.getAllocateResponse().getCompletedContainersStatuses();
-
-    // Only account for opportunistic containers
-    for (ContainerStatus cs : completedContainers) {
-      if (cs.getExecutionType() == ExecutionType.OPPORTUNISTIC) {
-        oppContainerContext.getContainersAllocated()
-            .remove(cs.getContainerId());
-      }
-    }
+    oppContainerContext.updateCompletedContainers(dsResp.getAllocateResponse());
 
     // Check if we have NM tokens for all the allocated containers. If not
     // generate one and update the response.
-    updateResponseWithNMTokens(
+    updateAllocateResponse(
         dsResp.getAllocateResponse(), nmTokens, allocatedContainers);
 
     if (LOG.isDebugEnabled()) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10be4598/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/OpportunisticContainerAllocatorAMService.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/OpportunisticContainerAllocatorAMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/OpportunisticContainerAllocatorAMService.java
index a473b14..a7c0a50 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/OpportunisticContainerAllocatorAMService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/OpportunisticContainerAllocatorAMService.java
@@ -24,9 +24,11 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.yarn.api.ApplicationMasterProtocolPB;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
 import org.apache.hadoop.yarn.server.api.DistributedSchedulingAMProtocol;
 import org.apache.hadoop.yarn.api.impl.pb.service.ApplicationMasterProtocolPBServiceImpl;
 
@@ -65,12 +67,14 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateS
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.security.AMRMTokenSecretManager;
 
+
+import org.apache.hadoop.yarn.server.scheduler.OpportunisticContainerAllocator;
+import org.apache.hadoop.yarn.server.scheduler.OpportunisticContainerContext;
+import org.apache.hadoop.yarn.server.utils.YarnServerSecurityUtils;
+
 import java.io.IOException;
 import java.net.InetSocketAddress;
-import java.util.HashSet;
 import java.util.List;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
 
 /**
  * The OpportunisticContainerAllocatorAMService is started instead of the
@@ -88,17 +92,20 @@ public class OpportunisticContainerAllocatorAMService
       LogFactory.getLog(OpportunisticContainerAllocatorAMService.class);
 
   private final NodeQueueLoadMonitor nodeMonitor;
+  private final OpportunisticContainerAllocator oppContainerAllocator;
 
-  private final ConcurrentHashMap<String, Set<NodeId>> rackToNode =
-      new ConcurrentHashMap<>();
-  private final ConcurrentHashMap<String, Set<NodeId>> hostToNode =
-      new ConcurrentHashMap<>();
   private final int k;
 
+  private final long cacheRefreshInterval;
+  private List<NodeId> cachedNodeIds;
+  private long lastCacheUpdateTime;
+
   public OpportunisticContainerAllocatorAMService(RMContext rmContext,
       YarnScheduler scheduler) {
     super(OpportunisticContainerAllocatorAMService.class.getName(),
         rmContext, scheduler);
+    this.oppContainerAllocator = new OpportunisticContainerAllocator(
+        rmContext.getContainerTokenSecretManager(), 0);
     this.k = rmContext.getYarnConfiguration().getInt(
         YarnConfiguration.OPP_CONTAINER_ALLOCATION_NODES_NUMBER_USED,
         YarnConfiguration.OPP_CONTAINER_ALLOCATION_NODES_NUMBER_USED_DEFAULT);
@@ -106,6 +113,8 @@ public class OpportunisticContainerAllocatorAMService
         YarnConfiguration.NM_CONTAINER_QUEUING_SORTING_NODES_INTERVAL_MS,
         YarnConfiguration.
             NM_CONTAINER_QUEUING_SORTING_NODES_INTERVAL_MS_DEFAULT);
+    this.cacheRefreshInterval = nodeSortInterval;
+    this.lastCacheUpdateTime = System.currentTimeMillis();
     NodeQueueLoadMonitor.LoadComparator comparator =
         NodeQueueLoadMonitor.LoadComparator.valueOf(
             rmContext.getYarnConfiguration().get(
@@ -172,6 +181,27 @@ public class OpportunisticContainerAllocatorAMService
   public RegisterApplicationMasterResponse registerApplicationMaster
       (RegisterApplicationMasterRequest request) throws YarnException,
       IOException {
+    final ApplicationAttemptId appAttemptId = getAppAttemptId();
+    SchedulerApplicationAttempt appAttempt = ((AbstractYarnScheduler)
+        rmContext.getScheduler()).getApplicationAttempt(appAttemptId);
+    if (appAttempt.getOpportunisticContainerContext() == null) {
+      OpportunisticContainerContext opCtx = new OpportunisticContainerContext();
+      opCtx.setContainerIdGenerator(new OpportunisticContainerAllocator
+          .ContainerIdGenerator() {
+        @Override
+        public long generateContainerId() {
+          return appAttempt.getAppSchedulingInfo().getNewContainerId();
+        }
+      });
+      int tokenExpiryInterval = getConfig()
+          .getInt(YarnConfiguration.OPPORTUNISTIC_CONTAINERS_TOKEN_EXPIRY_MS,
+              YarnConfiguration.
+                  OPPORTUNISTIC_CONTAINERS_TOKEN_EXPIRY_MS_DEFAULT);
+      opCtx.updateAllocationParams(createMinContainerResource(),
+          createMaxContainerResource(), createIncrContainerResource(),
+          tokenExpiryInterval);
+      appAttempt.setOpportunisticContainerContext(opCtx);
+    }
     return super.registerApplicationMaster(request);
   }
 
@@ -185,7 +215,30 @@ public class OpportunisticContainerAllocatorAMService
   @Override
   public AllocateResponse allocate(AllocateRequest request) throws
       YarnException, IOException {
-    return super.allocate(request);
+
+    final ApplicationAttemptId appAttemptId = getAppAttemptId();
+    SchedulerApplicationAttempt appAttempt = ((AbstractYarnScheduler)
+        rmContext.getScheduler()).getApplicationAttempt(appAttemptId);
+    OpportunisticContainerContext oppCtx =
+        appAttempt.getOpportunisticContainerContext();
+    oppCtx.updateNodeList(getLeastLoadedNodes());
+    List<Container> oppContainers =
+        oppContainerAllocator.allocateContainers(request, appAttemptId, oppCtx,
+        ResourceManager.getClusterTimeStamp(), appAttempt.getUser());
+
+    if (!oppContainers.isEmpty()) {
+      handleNewContainers(oppContainers, false);
+      appAttempt.updateNMTokens(oppContainers);
+    }
+
+    // Allocate all guaranteed containers
+    AllocateResponse allocateResp = super.allocate(request);
+
+    oppCtx.updateCompletedContainers(allocateResp);
+
+    // Add all opportunistic containers
+    allocateResp.getAllocatedContainers().addAll(oppContainers);
+    return allocateResp;
   }
 
   @Override
@@ -198,39 +251,9 @@ public class OpportunisticContainerAllocatorAMService
     RegisterDistributedSchedulingAMResponse dsResp = recordFactory
         .newRecordInstance(RegisterDistributedSchedulingAMResponse.class);
     dsResp.setRegisterResponse(response);
-    dsResp.setMinContainerResource(
-        Resource.newInstance(
-            getConfig().getInt(
-                YarnConfiguration.OPPORTUNISTIC_CONTAINERS_MIN_MEMORY_MB,
-                YarnConfiguration.
-                    OPPORTUNISTIC_CONTAINERS_MIN_MEMORY_MB_DEFAULT),
-            getConfig().getInt(
-                YarnConfiguration.OPPORTUNISTIC_CONTAINERS_MIN_VCORES,
-                YarnConfiguration.OPPORTUNISTIC_CONTAINERS_MIN_VCORES_DEFAULT)
-        )
-    );
-    dsResp.setMaxContainerResource(
-        Resource.newInstance(
-            getConfig().getInt(
-                YarnConfiguration.OPPORTUNISTIC_CONTAINERS_MAX_MEMORY_MB,
-                YarnConfiguration
-                    .OPPORTUNISTIC_CONTAINERS_MAX_MEMORY_MB_DEFAULT),
-            getConfig().getInt(
-                YarnConfiguration.OPPORTUNISTIC_CONTAINERS_MAX_VCORES,
-                YarnConfiguration.OPPORTUNISTIC_CONTAINERS_MAX_VCORES_DEFAULT)
-        )
-    );
-    dsResp.setIncrContainerResource(
-        Resource.newInstance(
-            getConfig().getInt(
-                YarnConfiguration.OPPORTUNISTIC_CONTAINERS_INCR_MEMORY_MB,
-                YarnConfiguration.
-                    OPPORTUNISTIC_CONTAINERS_INCR_MEMORY_MB_DEFAULT),
-            getConfig().getInt(
-                YarnConfiguration.OPPORTUNISTIC_CONTAINERS_INCR_VCORES,
-                YarnConfiguration.OPPORTUNISTIC_CONTAINERS_INCR_VCORES_DEFAULT)
-        )
-    );
+    dsResp.setMinContainerResource(createMinContainerResource());
+    dsResp.setMaxContainerResource(createMaxContainerResource());
+    dsResp.setIncrContainerResource(createIncrContainerResource());
     dsResp.setContainerTokenExpiryInterval(
         getConfig().getInt(
             YarnConfiguration.OPPORTUNISTIC_CONTAINERS_TOKEN_EXPIRY_MS,
@@ -240,8 +263,7 @@ public class OpportunisticContainerAllocatorAMService
         this.rmContext.getEpoch() << ResourceManager.EPOCH_BIT_SHIFT);
 
     // Set nodes to be used for scheduling
-    dsResp.setNodesForScheduling(
-        this.nodeMonitor.selectLeastLoadedNodes(this.k));
+    dsResp.setNodesForScheduling(getLeastLoadedNodes());
     return dsResp;
   }
 
@@ -250,47 +272,30 @@ public class OpportunisticContainerAllocatorAMService
       DistributedSchedulingAllocateRequest request)
       throws YarnException, IOException {
     List<Container> distAllocContainers = request.getAllocatedContainers();
-    for (Container container : distAllocContainers) {
+    handleNewContainers(distAllocContainers, true);
+    AllocateResponse response = allocate(request.getAllocateRequest());
+    DistributedSchedulingAllocateResponse dsResp = recordFactory
+        .newRecordInstance(DistributedSchedulingAllocateResponse.class);
+    dsResp.setAllocateResponse(response);
+    dsResp.setNodesForScheduling(getLeastLoadedNodes());
+    return dsResp;
+  }
+
+  private void handleNewContainers(List<Container> allocContainers,
+                                   boolean isRemotelyAllocated) {
+    for (Container container : allocContainers) {
       // Create RMContainer
       SchedulerApplicationAttempt appAttempt =
           ((AbstractYarnScheduler) rmContext.getScheduler())
               .getCurrentAttemptForContainer(container.getId());
       RMContainer rmContainer = new RMContainerImpl(container,
           appAttempt.getApplicationAttemptId(), container.getNodeId(),
-          appAttempt.getUser(), rmContext, true);
+          appAttempt.getUser(), rmContext, isRemotelyAllocated);
       appAttempt.addRMContainer(container.getId(), rmContainer);
       rmContainer.handle(
           new RMContainerEvent(container.getId(),
               RMContainerEventType.LAUNCHED));
     }
-    AllocateResponse response = allocate(request.getAllocateRequest());
-    DistributedSchedulingAllocateResponse dsResp = recordFactory
-        .newRecordInstance(DistributedSchedulingAllocateResponse.class);
-    dsResp.setAllocateResponse(response);
-    dsResp.setNodesForScheduling(
-        this.nodeMonitor.selectLeastLoadedNodes(this.k));
-    return dsResp;
-  }
-
-  private void addToMapping(ConcurrentHashMap<String, Set<NodeId>> mapping,
-                            String rackName, NodeId nodeId) {
-    if (rackName != null) {
-      mapping.putIfAbsent(rackName, new HashSet<NodeId>());
-      Set<NodeId> nodeIds = mapping.get(rackName);
-      synchronized (nodeIds) {
-        nodeIds.add(nodeId);
-      }
-    }
-  }
-
-  private void removeFromMapping(ConcurrentHashMap<String, Set<NodeId>> mapping,
-                                 String rackName, NodeId nodeId) {
-    if (rackName != null) {
-      Set<NodeId> nodeIds = mapping.get(rackName);
-      synchronized (nodeIds) {
-        nodeIds.remove(nodeId);
-      }
-    }
   }
 
   @Override
@@ -303,10 +308,6 @@ public class OpportunisticContainerAllocatorAMService
       NodeAddedSchedulerEvent nodeAddedEvent = (NodeAddedSchedulerEvent) event;
       nodeMonitor.addNode(nodeAddedEvent.getContainerReports(),
           nodeAddedEvent.getAddedRMNode());
-      addToMapping(rackToNode, nodeAddedEvent.getAddedRMNode().getRackName(),
-          nodeAddedEvent.getAddedRMNode().getNodeID());
-      addToMapping(hostToNode, nodeAddedEvent.getAddedRMNode().getHostName(),
-          nodeAddedEvent.getAddedRMNode().getNodeID());
       break;
     case NODE_REMOVED:
       if (!(event instanceof NodeRemovedSchedulerEvent)) {
@@ -315,12 +316,6 @@ public class OpportunisticContainerAllocatorAMService
       NodeRemovedSchedulerEvent nodeRemovedEvent =
           (NodeRemovedSchedulerEvent) event;
       nodeMonitor.removeNode(nodeRemovedEvent.getRemovedRMNode());
-      removeFromMapping(rackToNode,
-          nodeRemovedEvent.getRemovedRMNode().getRackName(),
-          nodeRemovedEvent.getRemovedRMNode().getNodeID());
-      removeFromMapping(hostToNode,
-          nodeRemovedEvent.getRemovedRMNode().getHostName(),
-          nodeRemovedEvent.getRemovedRMNode().getNodeID());
       break;
     case NODE_UPDATE:
       if (!(event instanceof NodeUpdateSchedulerEvent)) {
@@ -364,4 +359,58 @@ public class OpportunisticContainerAllocatorAMService
   public QueueLimitCalculator getNodeManagerQueueLimitCalculator() {
     return nodeMonitor.getThresholdCalculator();
   }
+
+  private Resource createIncrContainerResource() {
+    return Resource.newInstance(
+        getConfig().getInt(
+            YarnConfiguration.OPPORTUNISTIC_CONTAINERS_INCR_MEMORY_MB,
+            YarnConfiguration.
+                OPPORTUNISTIC_CONTAINERS_INCR_MEMORY_MB_DEFAULT),
+        getConfig().getInt(
+            YarnConfiguration.OPPORTUNISTIC_CONTAINERS_INCR_VCORES,
+            YarnConfiguration.OPPORTUNISTIC_CONTAINERS_INCR_VCORES_DEFAULT)
+    );
+  }
+
+  private synchronized List<NodeId> getLeastLoadedNodes() {
+    long currTime = System.currentTimeMillis();
+    if ((currTime - lastCacheUpdateTime > cacheRefreshInterval)
+        || cachedNodeIds == null) {
+      cachedNodeIds = this.nodeMonitor.selectLeastLoadedNodes(this.k);
+      lastCacheUpdateTime = currTime;
+    }
+    return cachedNodeIds;
+  }
+
+  private Resource createMaxContainerResource() {
+    return Resource.newInstance(
+        getConfig().getInt(
+            YarnConfiguration.OPPORTUNISTIC_CONTAINERS_MAX_MEMORY_MB,
+            YarnConfiguration
+                .OPPORTUNISTIC_CONTAINERS_MAX_MEMORY_MB_DEFAULT),
+        getConfig().getInt(
+            YarnConfiguration.OPPORTUNISTIC_CONTAINERS_MAX_VCORES,
+            YarnConfiguration.OPPORTUNISTIC_CONTAINERS_MAX_VCORES_DEFAULT)
+    );
+  }
+
+  private Resource createMinContainerResource() {
+    return Resource.newInstance(
+        getConfig().getInt(
+            YarnConfiguration.OPPORTUNISTIC_CONTAINERS_MIN_MEMORY_MB,
+            YarnConfiguration.
+                OPPORTUNISTIC_CONTAINERS_MIN_MEMORY_MB_DEFAULT),
+        getConfig().getInt(
+            YarnConfiguration.OPPORTUNISTIC_CONTAINERS_MIN_VCORES,
+            YarnConfiguration.OPPORTUNISTIC_CONTAINERS_MIN_VCORES_DEFAULT)
+    );
+  }
+
+  private static ApplicationAttemptId getAppAttemptId() throws YarnException {
+    AMRMTokenIdentifier amrmTokenIdentifier =
+        YarnServerSecurityUtils.authorizeRequest();
+    ApplicationAttemptId applicationAttemptId =
+        amrmTokenIdentifier.getApplicationAttemptId();
+    return applicationAttemptId;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10be4598/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 5e9bece..d2d706d 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
@@ -1184,6 +1184,13 @@ public class ResourceManager extends CompositeService implements Recoverable {
     Configuration config = this.rmContext.getYarnConfiguration();
     if (YarnConfiguration.isOpportunisticContainerAllocationEnabled(config)
         || YarnConfiguration.isDistSchedulingEnabled(config)) {
+      if (YarnConfiguration.isDistSchedulingEnabled(config) &&
+          !YarnConfiguration
+              .isOpportunisticContainerAllocationEnabled(config)) {
+        throw new YarnRuntimeException(
+            "Invalid parameters: opportunistic container allocation has to " +
+                "be enabled when distributed scheduling is enabled.");
+      }
       OpportunisticContainerAllocatorAMService
           oppContainerAllocatingAMService =
           new OpportunisticContainerAllocatorAMService(this.rmContext,
@@ -1193,9 +1200,8 @@ public class ResourceManager extends CompositeService implements Recoverable {
               OpportunisticContainerAllocatorAMService.class.getName());
       // Add an event dispatcher for the
       // OpportunisticContainerAllocatorAMService to handle node
-      // updates/additions and removals.
-      // Since the SchedulerEvent is currently a super set of theses,
-      // we register interest for it..
+      // additions, updates and removals. Since the SchedulerEvent is currently
+      // a super set of theses, we register interest for it.
       addService(oppContainerAllocEventDispatcher);
       rmDispatcher.register(SchedulerEventType.class,
           oppContainerAllocEventDispatcher);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10be4598/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.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/scheduler/SchedulerApplicationAttempt.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
index adc3a97..9675fac 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
@@ -44,6 +44,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ExecutionType;
 import org.apache.hadoop.yarn.api.records.LogAggregationContext;
 import org.apache.hadoop.yarn.api.records.NMToken;
 import org.apache.hadoop.yarn.api.records.NodeId;
@@ -68,6 +69,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerUpda
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanContainerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.SchedulableEntity;
+
+import org.apache.hadoop.yarn.server.scheduler.OpportunisticContainerContext;
 import org.apache.hadoop.yarn.state.InvalidStateTransitionException;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
@@ -114,6 +117,9 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
   private boolean isAttemptRecovering;
 
   protected ResourceUsage attemptResourceUsage = new ResourceUsage();
+  /** Resource usage of opportunistic containers. */
+  protected ResourceUsage attemptOpportunisticResourceUsage =
+      new ResourceUsage();
   /** Scheduled by a remote scheduler. */
   protected ResourceUsage attemptResourceUsageAllocatedRemotely =
       new ResourceUsage();
@@ -132,6 +138,8 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
   // by NM should not be recovered.
   private Set<ContainerId> pendingRelease = null;
 
+  private OpportunisticContainerContext oppContainerContext;
+
   /**
    * Count how many times the application has been given an opportunity to
    * schedule a task at each priority. Each time the scheduler asks the
@@ -199,7 +207,17 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
     readLock = lock.readLock();
     writeLock = lock.writeLock();
   }
-  
+
+  public void setOpportunisticContainerContext(
+      OpportunisticContainerContext oppContext) {
+    this.oppContainerContext = oppContext;
+  }
+
+  public OpportunisticContainerContext
+      getOpportunisticContainerContext() {
+    return this.oppContainerContext;
+  }
+
   /**
    * Get the live containers of the application.
    * @return live containers of the application
@@ -331,6 +349,10 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
     try {
       writeLock.lock();
       liveContainers.put(id, rmContainer);
+      if (rmContainer.getExecutionType() == ExecutionType.OPPORTUNISTIC) {
+        this.attemptOpportunisticResourceUsage.incUsed(
+            rmContainer.getAllocatedResource());
+      }
       if (rmContainer.isRemotelyAllocated()) {
         this.attemptResourceUsageAllocatedRemotely.incUsed(
             rmContainer.getAllocatedResource());
@@ -344,9 +366,15 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
     try {
       writeLock.lock();
       RMContainer rmContainer = liveContainers.remove(containerId);
-      if (rmContainer != null && rmContainer.isRemotelyAllocated()) {
-        this.attemptResourceUsageAllocatedRemotely.decUsed(
-            rmContainer.getAllocatedResource());
+      if (rmContainer != null) {
+        if (rmContainer.getExecutionType() == ExecutionType.OPPORTUNISTIC) {
+          this.attemptOpportunisticResourceUsage
+              .decUsed(rmContainer.getAllocatedResource());
+        }
+        if (rmContainer.isRemotelyAllocated()) {
+          this.attemptResourceUsageAllocatedRemotely
+              .decUsed(rmContainer.getAllocatedResource());
+        }
       }
     } finally {
       writeLock.unlock();
@@ -612,12 +640,7 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
               container.getPriority(), rmContainer.getCreationTime(),
               this.logAggregationContext, rmContainer.getNodeLabelExpression(),
               containerType));
-      NMToken nmToken =
-          rmContext.getNMTokenSecretManager().createAndGetNMToken(getUser(),
-              getApplicationAttemptId(), container);
-      if (nmToken != null) {
-        updatedNMTokens.add(nmToken);
-      }
+      updateNMToken(container);
     } catch (IllegalArgumentException e) {
       // DNS might be down, skip returning this container.
       LOG.error("Error trying to assign container token and NM token to"
@@ -635,6 +658,21 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
     return container;
   }
 
+  public void updateNMTokens(Collection<Container> containers) {
+    for (Container container : containers) {
+      updateNMToken(container);
+    }
+  }
+
+  private void updateNMToken(Container container) {
+    NMToken nmToken =
+        rmContext.getNMTokenSecretManager().createAndGetNMToken(getUser(),
+            getApplicationAttemptId(), container);
+    if (nmToken != null) {
+      updatedNMTokens.add(nmToken);
+    }
+  }
+
   // Create container token and update NMToken altogether, if either of them fails for
   // some reason like DNS unavailable, do not return this container and keep it
   // in the newlyAllocatedContainers waiting to be refetched.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10be4598/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/distributed/NodeQueueLoadMonitor.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/scheduler/distributed/NodeQueueLoadMonitor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/distributed/NodeQueueLoadMonitor.java
index 017a256..b80a17c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/distributed/NodeQueueLoadMonitor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/distributed/NodeQueueLoadMonitor.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.Comparator;
 import java.util.List;
 import java.util.Map;
@@ -37,6 +38,7 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 /**
  * The NodeQueueLoadMonitor keeps track of load metrics (such as queue length
@@ -103,16 +105,23 @@ public class NodeQueueLoadMonitor implements ClusterMonitor {
       new ConcurrentHashMap<>();
   private final LoadComparator comparator;
   private QueueLimitCalculator thresholdCalculator;
+  private ReentrantReadWriteLock sortedNodesLock = new ReentrantReadWriteLock();
+  private ReentrantReadWriteLock clusterNodesLock =
+      new ReentrantReadWriteLock();
 
   Runnable computeTask = new Runnable() {
     @Override
     public void run() {
-      synchronized (sortedNodes) {
+      ReentrantReadWriteLock.WriteLock writeLock = sortedNodesLock.writeLock();
+      writeLock.lock();
+      try {
         sortedNodes.clear();
         sortedNodes.addAll(sortNodes());
         if (thresholdCalculator != null) {
           thresholdCalculator.update();
         }
+      } finally {
+        writeLock.unlock();
       }
     }
   };
@@ -166,9 +175,16 @@ public class NodeQueueLoadMonitor implements ClusterMonitor {
   @Override
   public void removeNode(RMNode removedRMNode) {
     LOG.debug("Node delete event for: " + removedRMNode.getNode().getName());
-    synchronized (this.clusterNodes) {
-      if (this.clusterNodes.containsKey(removedRMNode.getNodeID())) {
-        this.clusterNodes.remove(removedRMNode.getNodeID());
+    ReentrantReadWriteLock.WriteLock writeLock = clusterNodesLock.writeLock();
+    writeLock.lock();
+    ClusterNode node;
+    try {
+      node = this.clusterNodes.remove(removedRMNode.getNodeID());
+    } finally {
+      writeLock.unlock();
+    }
+    if (LOG.isDebugEnabled()) {
+      if (node != null) {
         LOG.debug("Delete ClusterNode: " + removedRMNode.getNodeID());
       } else {
         LOG.debug("Node not in list!");
@@ -186,7 +202,9 @@ public class NodeQueueLoadMonitor implements ClusterMonitor {
     int waitQueueLength = queuedContainersStatus.getWaitQueueLength();
     // Add nodes to clusterNodes. If estimatedQueueTime is -1, ignore node
     // UNLESS comparator is based on queue length.
-    synchronized (this.clusterNodes) {
+    ReentrantReadWriteLock.WriteLock writeLock = clusterNodesLock.writeLock();
+    writeLock.lock();
+    try {
       ClusterNode currentNode = this.clusterNodes.get(rmNode.getNodeID());
       if (currentNode == null) {
         if (estimatedQueueWaitTime != -1
@@ -222,6 +240,8 @@ public class NodeQueueLoadMonitor implements ClusterMonitor {
               "wait queue length [" + currentNode.queueLength + "]");
         }
       }
+    } finally {
+      writeLock.unlock();
     }
   }
 
@@ -245,15 +265,22 @@ public class NodeQueueLoadMonitor implements ClusterMonitor {
    * @return ordered list of nodes
    */
   public List<NodeId> selectLeastLoadedNodes(int k) {
-    synchronized (this.sortedNodes) {
-      return ((k < this.sortedNodes.size()) && (k >= 0)) ?
+    ReentrantReadWriteLock.ReadLock readLock = sortedNodesLock.readLock();
+    readLock.lock();
+    try {
+      List<NodeId> retVal = ((k < this.sortedNodes.size()) && (k >= 0)) ?
           new ArrayList<>(this.sortedNodes).subList(0, k) :
           new ArrayList<>(this.sortedNodes);
+      return Collections.unmodifiableList(retVal);
+    } finally {
+      readLock.unlock();
     }
   }
 
   private List<NodeId> sortNodes() {
-    synchronized (this.clusterNodes) {
+    ReentrantReadWriteLock.ReadLock readLock = clusterNodesLock.readLock();
+    readLock.lock();
+    try {
       ArrayList aList = new ArrayList<>(this.clusterNodes.values());
       List<NodeId> retList = new ArrayList<>();
       Object[] nodes = aList.toArray();
@@ -267,6 +294,8 @@ public class NodeQueueLoadMonitor implements ClusterMonitor {
         retList.add(((ClusterNode)nodes[j]).nodeId);
       }
       return retList;
+    } finally {
+      readLock.unlock();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10be4598/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestOpportunisticContainerAllocatorAMService.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/TestOpportunisticContainerAllocatorAMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestOpportunisticContainerAllocatorAMService.java
index 07c6b54..207f5ba 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestOpportunisticContainerAllocatorAMService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestOpportunisticContainerAllocatorAMService.java
@@ -62,6 +62,7 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.DistributedSche
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RegisterDistributedSchedulingAMResponsePBImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AMLivelinessMonitor;
 
+import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -97,6 +98,11 @@ public class TestOpportunisticContainerAllocatorAMService {
       public Configuration getYarnConfiguration() {
         return new YarnConfiguration();
       }
+
+      @Override
+      public RMContainerTokenSecretManager getContainerTokenSecretManager() {
+        return new RMContainerTokenSecretManager(conf);
+      }
     };
     Container c = factory.newRecordInstance(Container.class);
     c.setExecutionType(ExecutionType.OPPORTUNISTIC);
@@ -117,8 +123,8 @@ public class TestOpportunisticContainerAllocatorAMService {
     Server server = service.getServer(rpc, conf, addr, null);
     server.start();
 
-    // Verify that the DistrubutedSchedulingService can handle vanilla
-    // ApplicationMasterProtocol clients
+    // Verify that the OpportunisticContainerAllocatorAMSercvice can handle
+    // vanilla ApplicationMasterProtocol clients
     RPC.setProtocolEngine(conf, ApplicationMasterProtocolPB.class,
         ProtobufRpcEngine.class);
     ApplicationMasterProtocolPB ampProxy =


---------------------------------------------------------------------
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-5408. Compose Federation membership/application/policy APIs into an uber FederationStateStore API. (Ellen Hui via Subru).

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


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

Branch: refs/heads/YARN-2915
Commit: 75d47b09cb553921a8e15ad7b70f7be40989b93e
Parents: ddb5024
Author: Subru Krishnan <su...@apache.org>
Authored: Mon Aug 8 14:53:38 2016 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon Oct 3 14:15:36 2016 -0700

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


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

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

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

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

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

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


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


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

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


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

Branch: refs/heads/YARN-2915
Commit: a8c46adff3510bbca63e65524034dd366beab0c4
Parents: 0023e21
Author: Subru Krishnan <su...@apache.org>
Authored: Wed Aug 17 12:07:06 2016 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon Oct 3 14:15:36 2016 -0700

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


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

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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a8c46adf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/FederationStateStoreBaseTest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/FederationStateStoreBaseTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/FederationStateStoreBaseTest.java
index 414696b..63a5b65 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/FederationStateStoreBaseTest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/FederationStateStoreBaseTest.java
@@ -162,9 +162,9 @@ public abstract class FederationStateStoreBaseTest {
         SubClusterRegisterRequest.newInstance(subClusterInfo2));
 
     stateStore.subClusterHeartbeat(SubClusterHeartbeatRequest
-        .newInstance(subClusterId1, SubClusterState.SC_RUNNING, ""));
-    stateStore.subClusterHeartbeat(SubClusterHeartbeatRequest
-        .newInstance(subClusterId2, SubClusterState.SC_UNHEALTHY, ""));
+        .newInstance(subClusterId1, SubClusterState.SC_RUNNING, "capability"));
+    stateStore.subClusterHeartbeat(SubClusterHeartbeatRequest.newInstance(
+        subClusterId2, SubClusterState.SC_UNHEALTHY, "capability"));
 
     Assert.assertTrue(
         stateStore.getSubClusters(GetSubClustersInfoRequest.newInstance(true))


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


[23/50] [abbrv] hadoop git commit: YARN-5678. Log demand as demand in FSLeafQueue and FSParentQueue. (Yufei Gu via kasha)

Posted by su...@apache.org.
YARN-5678. Log demand as demand in FSLeafQueue and FSParentQueue. (Yufei Gu via kasha)


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

Branch: refs/heads/YARN-2915
Commit: 82857037b6e960dccdaf9e6b1b238411498a0dfe
Parents: fe9ebe2
Author: Karthik Kambatla <ka...@apache.org>
Authored: Sun Oct 2 22:09:43 2016 -0700
Committer: Karthik Kambatla <ka...@apache.org>
Committed: Sun Oct 2 22:09:43 2016 -0700

----------------------------------------------------------------------
 .../server/resourcemanager/scheduler/fair/FSLeafQueue.java     | 2 +-
 .../server/resourcemanager/scheduler/fair/FSParentQueue.java   | 6 +++---
 2 files changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/82857037/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.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/scheduler/fair/FSLeafQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java
index a6adb47..9d5bbe5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java
@@ -295,7 +295,7 @@ public class FSLeafQueue extends FSQueue {
     Resource toAdd = sched.getDemand();
     if (LOG.isDebugEnabled()) {
       LOG.debug("Counting resource from " + sched.getName() + " " + toAdd
-          + "; Total resource consumption for " + getName() + " now "
+          + "; Total resource demand for " + getName() + " now "
           + demand);
     }
     demand = Resources.add(demand, toAdd);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/82857037/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSParentQueue.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/scheduler/fair/FSParentQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSParentQueue.java
index e58c3f1..d05390b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSParentQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSParentQueue.java
@@ -158,13 +158,13 @@ public class FSParentQueue extends FSQueue {
       for (FSQueue childQueue : childQueues) {
         childQueue.updateDemand();
         Resource toAdd = childQueue.getDemand();
+        demand = Resources.add(demand, toAdd);
+        demand = Resources.componentwiseMin(demand, maxShare);
         if (LOG.isDebugEnabled()) {
           LOG.debug("Counting resource from " + childQueue.getName() + " " +
-              toAdd + "; Total resource consumption for " + getName() +
+              toAdd + "; Total resource demand for " + getName() +
               " now " + demand);
         }
-        demand = Resources.add(demand, toAdd);
-        demand = Resources.componentwiseMin(demand, maxShare);
         if (Resources.equals(demand, maxShare)) {
           break;
         }


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


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

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


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

Branch: refs/heads/YARN-2915
Commit: 911869a4327e47669e90d3a99cea011215e6f5cc
Parents: 24b9180
Author: Subru Krishnan <su...@apache.org>
Authored: Thu Sep 1 13:55:54 2016 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon Oct 3 14:15:36 2016 -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/911869a4/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/911869a4/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


[12/50] [abbrv] hadoop git commit: HADOOP-13671. Fix ClassFormatException in trunk build. Contributed by Kihwal Lee.

Posted by su...@apache.org.
HADOOP-13671. Fix ClassFormatException in trunk build. Contributed by Kihwal Lee.


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

Branch: refs/heads/YARN-2915
Commit: d6afcf364c3d5a8bdb7dbd99626fab70fe6dc47a
Parents: a0730aa
Author: Kihwal Lee <ki...@apache.org>
Authored: Fri Sep 30 13:26:28 2016 -0500
Committer: Kihwal Lee <ki...@apache.org>
Committed: Fri Sep 30 13:26:28 2016 -0500

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d6afcf36/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index d9a01a0..26ac16c 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -114,7 +114,7 @@
     <maven-jar-plugin.version>2.5</maven-jar-plugin.version>
     <maven-war-plugin.version>2.4</maven-war-plugin.version>
     <maven-source-plugin.version>2.3</maven-source-plugin.version>
-    <maven-project-info-reports-plugin.version>2.7</maven-project-info-reports-plugin.version>
+    <maven-project-info-reports-plugin.version>2.9</maven-project-info-reports-plugin.version>
     <maven-pdf-plugin.version>1.2</maven-pdf-plugin.version>
     <maven-remote-resources-plugin.version>1.5</maven-remote-resources-plugin.version>
     <build-helper-maven-plugin.version>1.9</build-helper-maven-plugin.version>


---------------------------------------------------------------------
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: HADOOP-13164 Optimize S3AFileSystem::deleteUnnecessaryFakeDirectories. Contributed by Rajesh Balamohan.

Posted by su...@apache.org.
HADOOP-13164 Optimize S3AFileSystem::deleteUnnecessaryFakeDirectories. Contributed by Rajesh Balamohan.


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

Branch: refs/heads/YARN-2915
Commit: ee0c722dc8fb81ec902cd1da5958ce5adb0ab08f
Parents: a1b8251
Author: Steve Loughran <st...@apache.org>
Authored: Thu Sep 29 16:59:33 2016 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Thu Sep 29 17:01:00 2016 +0100

----------------------------------------------------------------------
 .../fs/contract/AbstractFSContractTestBase.java |  2 +-
 .../org/apache/hadoop/fs/s3a/S3AFileSystem.java | 65 +++++++--------
 .../hadoop/fs/s3a/S3AInstrumentation.java       | 10 +++
 .../org/apache/hadoop/fs/s3a/Statistic.java     |  4 +
 .../fs/s3a/ITestS3AFileOperationCost.java       | 85 ++++++++++++++++++++
 .../org/apache/hadoop/fs/s3a/S3ATestUtils.java  | 13 ++-
 6 files changed, 144 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee0c722d/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractFSContractTestBase.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractFSContractTestBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractFSContractTestBase.java
index baea968..b2e68f5 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractFSContractTestBase.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractFSContractTestBase.java
@@ -359,7 +359,7 @@ public abstract class AbstractFSContractTestBase extends Assert
     assertEquals(text + " wrong read result " + result, -1, result);
   }
 
-  boolean rename(Path src, Path dst) throws IOException {
+  protected boolean rename(Path src, Path dst) throws IOException {
     return getFileSystem().rename(src, dst);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee0c722d/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
index dffef15..e3b2c63 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
@@ -668,7 +668,7 @@ public class S3AFileSystem extends FileSystem {
           copyFile(summary.getKey(), newDstKey, summary.getSize());
 
           if (keysToDelete.size() == MAX_ENTRIES_TO_DELETE) {
-            removeKeys(keysToDelete, true);
+            removeKeys(keysToDelete, true, false);
           }
         }
 
@@ -676,7 +676,7 @@ public class S3AFileSystem extends FileSystem {
           objects = continueListObjects(objects);
         } else {
           if (!keysToDelete.isEmpty()) {
-            removeKeys(keysToDelete, false);
+            removeKeys(keysToDelete, false, false);
           }
           break;
         }
@@ -924,17 +924,25 @@ public class S3AFileSystem extends FileSystem {
    * @param keysToDelete collection of keys to delete on the s3-backend
    * @param clearKeys clears the keysToDelete-list after processing the list
    *            when set to true
+   * @param deleteFakeDir indicates whether this is for deleting fake dirs
    */
   private void removeKeys(List<DeleteObjectsRequest.KeyVersion> keysToDelete,
-          boolean clearKeys) throws AmazonClientException {
+      boolean clearKeys, boolean deleteFakeDir) throws AmazonClientException {
+    if (keysToDelete.isEmpty()) {
+      // no keys
+      return;
+    }
     if (enableMultiObjectsDelete) {
       deleteObjects(new DeleteObjectsRequest(bucket).withKeys(keysToDelete));
-      instrumentation.fileDeleted(keysToDelete.size());
     } else {
       for (DeleteObjectsRequest.KeyVersion keyVersion : keysToDelete) {
         deleteObject(keyVersion.getKey());
       }
+    }
+    if (!deleteFakeDir) {
       instrumentation.fileDeleted(keysToDelete.size());
+    } else {
+      instrumentation.fakeDirsDeleted(keysToDelete.size());
     }
     if (clearKeys) {
       keysToDelete.clear();
@@ -1017,7 +1025,7 @@ public class S3AFileSystem extends FileSystem {
             LOG.debug("Got object to delete {}", summary.getKey());
 
             if (keys.size() == MAX_ENTRIES_TO_DELETE) {
-              removeKeys(keys, true);
+              removeKeys(keys, true, false);
             }
           }
 
@@ -1025,7 +1033,7 @@ public class S3AFileSystem extends FileSystem {
             objects = continueListObjects(objects);
           } else {
             if (!keys.isEmpty()) {
-              removeKeys(keys, false);
+              removeKeys(keys, false, false);
             }
             break;
           }
@@ -1504,37 +1512,30 @@ public class S3AFileSystem extends FileSystem {
   /**
    * Delete mock parent directories which are no longer needed.
    * This code swallows IO exceptions encountered
-   * @param f path
-   */
-  private void deleteUnnecessaryFakeDirectories(Path f) {
-    while (true) {
-      String key = "";
-      try {
-        key = pathToKey(f);
-        if (key.isEmpty()) {
-          break;
-        }
-
-        S3AFileStatus status = getFileStatus(f);
-
-        if (status.isDirectory() && status.isEmptyDirectory()) {
-          LOG.debug("Deleting fake directory {}/", key);
-          deleteObject(key + "/");
+   * @param path path
+   */
+  private void deleteUnnecessaryFakeDirectories(Path path) {
+    List<DeleteObjectsRequest.KeyVersion> keysToRemove = new ArrayList<>();
+    while (!path.isRoot()) {
+      String key = pathToKey(path);
+      key = (key.endsWith("/")) ? key : (key + "/");
+      keysToRemove.add(new DeleteObjectsRequest.KeyVersion(key));
+      path = path.getParent();
+    }
+    try {
+      removeKeys(keysToRemove, false, true);
+    } catch(AmazonClientException e) {
+      instrumentation.errorIgnored();
+      if (LOG.isDebugEnabled()) {
+        StringBuilder sb = new StringBuilder();
+        for(DeleteObjectsRequest.KeyVersion kv : keysToRemove) {
+          sb.append(kv.getKey()).append(",");
         }
-      } catch (IOException | AmazonClientException e) {
-        LOG.debug("While deleting key {} ", key, e);
-        instrumentation.errorIgnored();
+        LOG.debug("While deleting keys {} ", sb.toString(), e);
       }
-
-      if (f.isRoot()) {
-        break;
-      }
-
-      f = f.getParent();
     }
   }
 
-
   private void createFakeDirectory(final String objectName)
       throws AmazonClientException, AmazonServiceException,
       InterruptedIOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee0c722d/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
index b4c4063..26b5b51 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
@@ -75,6 +75,7 @@ public class S3AInstrumentation {
   private final MutableCounterLong numberOfFilesCopied;
   private final MutableCounterLong bytesOfFilesCopied;
   private final MutableCounterLong numberOfFilesDeleted;
+  private final MutableCounterLong numberOfFakeDirectoryDeletes;
   private final MutableCounterLong numberOfDirectoriesCreated;
   private final MutableCounterLong numberOfDirectoriesDeleted;
   private final Map<String, MutableCounterLong> streamMetrics =
@@ -135,6 +136,7 @@ public class S3AInstrumentation {
     numberOfFilesCopied = counter(FILES_COPIED);
     bytesOfFilesCopied = counter(FILES_COPIED_BYTES);
     numberOfFilesDeleted = counter(FILES_DELETED);
+    numberOfFakeDirectoryDeletes = counter(FAKE_DIRECTORIES_DELETED);
     numberOfDirectoriesCreated = counter(DIRECTORIES_CREATED);
     numberOfDirectoriesDeleted = counter(DIRECTORIES_DELETED);
     ignoredErrors = counter(IGNORED_ERRORS);
@@ -296,6 +298,14 @@ public class S3AInstrumentation {
   }
 
   /**
+   * Indicate that fake directory request was made.
+   * @param count number of directory entries included in the delete request.
+   */
+  public void fakeDirsDeleted(int count) {
+    numberOfFakeDirectoryDeletes.incr(count);
+  }
+
+  /**
    * Indicate that S3A created a directory.
    */
   public void directoryCreated() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee0c722d/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java
index cbc34d6..d84a355 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java
@@ -42,6 +42,10 @@ public enum Statistic {
       "Total number of files created through the object store."),
   FILES_DELETED("files_deleted",
       "Total number of files deleted from the object store."),
+  FAKE_DIRECTORIES_CREATED("fake_directories_created",
+      "Total number of fake directory entries created in the object store."),
+  FAKE_DIRECTORIES_DELETED("fake_directories_deleted",
+      "Total number of fake directory deletes submitted to object store."),
   IGNORED_ERRORS("ignored_errors", "Errors caught and ignored"),
   INVOCATION_COPY_FROM_LOCAL_FILE(CommonStatisticNames.OP_COPY_FROM_LOCAL_FILE,
       "Calls of copyFromLocalFile()"),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee0c722d/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileOperationCost.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileOperationCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileOperationCost.java
index 2a6ba0c..f19ea95 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileOperationCost.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileOperationCost.java
@@ -188,4 +188,89 @@ public class ITestS3AFileOperationCost extends AbstractFSContractTestBase {
       tmpFile.delete();
     }
   }
+
+  private void reset(MetricDiff... diffs) {
+    for (MetricDiff diff : diffs) {
+      diff.reset();
+    }
+  }
+
+  @Test
+  public void testFakeDirectoryDeletion() throws Throwable {
+    describe("Verify whether create file works after renaming a file. "
+        + "In S3, rename deletes any fake directories as a part of "
+        + "clean up activity");
+    S3AFileSystem fs = getFileSystem();
+    Path srcBaseDir = path("src");
+    mkdirs(srcBaseDir);
+    MetricDiff deleteRequests =
+        new MetricDiff(fs, Statistic.OBJECT_DELETE_REQUESTS);
+    MetricDiff directoriesDeleted =
+        new MetricDiff(fs, Statistic.DIRECTORIES_DELETED);
+    MetricDiff fakeDirectoriesDeleted =
+        new MetricDiff(fs, Statistic.FAKE_DIRECTORIES_DELETED);
+    MetricDiff directoriesCreated =
+        new MetricDiff(fs, Statistic.DIRECTORIES_CREATED);
+
+    Path srcDir = new Path(srcBaseDir, "1/2/3/4/5/6");
+    Path srcFilePath = new Path(srcDir, "source.txt");
+    int srcDirDepth = directoriesInPath(srcDir);
+    // one dir created, one removed
+    mkdirs(srcDir);
+    String state = "after mkdir(srcDir)";
+    directoriesCreated.assertDiffEquals(state, 1);
+/*  TODO: uncomment once HADOOP-13222 is in
+    deleteRequests.assertDiffEquals(state, 1);
+    directoriesDeleted.assertDiffEquals(state, 0);
+    fakeDirectoriesDeleted.assertDiffEquals(state, srcDirDepth);
+*/
+    reset(deleteRequests, directoriesCreated, directoriesDeleted,
+        fakeDirectoriesDeleted);
+
+    // creating a file should trigger demise of the src dir
+    touch(fs, srcFilePath);
+    state = "after touch(fs, srcFilePath)";
+    deleteRequests.assertDiffEquals(state, 1);
+    directoriesCreated.assertDiffEquals(state, 0);
+    directoriesDeleted.assertDiffEquals(state, 0);
+    fakeDirectoriesDeleted.assertDiffEquals(state, srcDirDepth);
+
+    reset(deleteRequests, directoriesCreated, directoriesDeleted,
+        fakeDirectoriesDeleted);
+
+    Path destBaseDir = path("dest");
+    Path destDir = new Path(destBaseDir, "1/2/3/4/5/6");
+    Path destFilePath = new Path(destDir, "dest.txt");
+    mkdirs(destDir);
+    state = "after mkdir(destDir)";
+
+    int destDirDepth = directoriesInPath(destDir);
+    directoriesCreated.assertDiffEquals(state, 1);
+/*  TODO: uncomment once HADOOP-13222 is in
+    deleteRequests.assertDiffEquals(state,1);
+    directoriesDeleted.assertDiffEquals(state,0);
+    fakeDirectoriesDeleted.assertDiffEquals(state,destDirDepth);
+*/
+    reset(deleteRequests, directoriesCreated, directoriesDeleted,
+        fakeDirectoriesDeleted);
+
+    fs.rename(srcFilePath, destFilePath);
+    state = "after rename(srcFilePath, destFilePath)";
+    directoriesCreated.assertDiffEquals(state, 1);
+    // one for the renamed file, one for the parent
+    deleteRequests.assertDiffEquals(state, 2);
+    directoriesDeleted.assertDiffEquals(state, 0);
+    fakeDirectoriesDeleted.assertDiffEquals(state, destDirDepth);
+
+    reset(deleteRequests, directoriesCreated, directoriesDeleted,
+        fakeDirectoriesDeleted);
+
+    assertIsFile(destFilePath);
+    assertIsDirectory(srcDir);
+  }
+
+  private int directoriesInPath(Path path) {
+    return path.isRoot() ? 0 : 1 + directoriesInPath(path.getParent());
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee0c722d/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
index e45db48..95f6d4b 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
@@ -298,14 +298,23 @@ public class S3ATestUtils {
 
     /**
      * Assert that the value of {@link #diff()} matches that expected.
+     * @param message message to print; metric name is appended
      * @param expected expected value.
      */
-    public void assertDiffEquals(long expected) {
-      Assert.assertEquals("Count of " + this,
+    public void assertDiffEquals(String message, long expected) {
+      Assert.assertEquals(message + ": " + statistic.getSymbol(),
           expected, diff());
     }
 
     /**
+     * Assert that the value of {@link #diff()} matches that expected.
+     * @param expected expected value.
+     */
+    public void assertDiffEquals(long expected) {
+      assertDiffEquals("Count of " + this, expected);
+    }
+
+    /**
      * Assert that the value of {@link #diff()} matches that of another
      * instance.
      * @param that the other metric diff instance.


---------------------------------------------------------------------
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: HADOOP-13317. Add logs to KMS server-side to improve supportability. Contributed by Suraj Acharya.

Posted by su...@apache.org.
HADOOP-13317. Add logs to KMS server-side to improve supportability. Contributed by Suraj Acharya.


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

Branch: refs/heads/YARN-2915
Commit: 89bd6d29a62afd7ed8ff87bcc29d17b1cb53dcb6
Parents: 2549ee9
Author: Xiao Chen <xi...@apache.org>
Authored: Fri Sep 30 17:51:39 2016 -0700
Committer: Xiao Chen <xi...@apache.org>
Committed: Fri Sep 30 17:51:39 2016 -0700

----------------------------------------------------------------------
 .../hadoop/crypto/key/kms/server/KMS.java       | 76 +++++++++++++++++---
 1 file changed, 66 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/89bd6d29/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMS.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMS.java b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMS.java
index f069fca..371f3f5 100644
--- a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMS.java
+++ b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMS.java
@@ -28,6 +28,8 @@ import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.crypto.key.kms.KMSClientProvider;
 import org.apache.hadoop.security.token.delegation.web.HttpUserGroupInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 
 import javax.ws.rs.Consumes;
@@ -68,6 +70,8 @@ public class KMS {
   private KeyProviderCryptoExtension provider;
   private KMSAudit kmsAudit;
 
+  private static final Logger LOG = LoggerFactory.getLogger(KMS.class);
+
   public KMS() throws Exception {
     provider = KMSWebApp.getKeyProvider();
     kmsAudit= KMSWebApp.getKMSAudit();
@@ -77,7 +81,7 @@ public class KMS {
       KMSOp operation) throws AccessControlException {
     KMSWebApp.getACLs().assertAccess(aclType, ugi, operation, null);
   }
-  
+
   private void assertAccess(KMSACLs.Type aclType, UserGroupInformation ugi,
       KMSOp operation, String key) throws AccessControlException {
     KMSWebApp.getACLs().assertAccess(aclType, ugi, operation, key);
@@ -100,6 +104,7 @@ public class KMS {
   @Produces(MediaType.APPLICATION_JSON)
   @SuppressWarnings("unchecked")
   public Response createKey(Map jsonKey) throws Exception {
+    LOG.trace("Entering createKey Method.");
     KMSWebApp.getAdminCallsMeter().mark();
     UserGroupInformation user = HttpUserGroupInformation.get();
     final String name = (String) jsonKey.get(KMSRESTConstants.NAME_FIELD);
@@ -111,6 +116,9 @@ public class KMS {
                  ? (Integer) jsonKey.get(KMSRESTConstants.LENGTH_FIELD) : 0;
     String description = (String)
         jsonKey.get(KMSRESTConstants.DESCRIPTION_FIELD);
+    LOG.debug("Creating key with name {}, cipher being used{}, " +
+            "length of key {}, description of key {}", name, cipher,
+             length, description);
     Map<String, String> attributes = (Map<String, String>)
         jsonKey.get(KMSRESTConstants.ATTRIBUTES_FIELD);
     if (material != null) {
@@ -151,6 +159,7 @@ public class KMS {
     String requestURL = KMSMDCFilter.getURL();
     int idx = requestURL.lastIndexOf(KMSRESTConstants.KEYS_RESOURCE);
     requestURL = requestURL.substring(0, idx);
+    LOG.trace("Exiting createKey Method.");
     return Response.created(getKeyURI(KMSRESTConstants.SERVICE_VERSION, name))
         .type(MediaType.APPLICATION_JSON)
         .header("Location", getKeyURI(requestURL, name)).entity(json).build();
@@ -160,11 +169,12 @@ public class KMS {
   @Path(KMSRESTConstants.KEY_RESOURCE + "/{name:.*}")
   public Response deleteKey(@PathParam("name") final String name)
       throws Exception {
+    LOG.trace("Entering deleteKey method.");
     KMSWebApp.getAdminCallsMeter().mark();
     UserGroupInformation user = HttpUserGroupInformation.get();
     assertAccess(KMSACLs.Type.DELETE, user, KMSOp.DELETE_KEY, name);
     KMSClientProvider.checkNotEmpty(name, "name");
-
+    LOG.debug("Deleting key with name {}.", name);
     user.doAs(new PrivilegedExceptionAction<Void>() {
       @Override
       public Void run() throws Exception {
@@ -175,7 +185,7 @@ public class KMS {
     });
 
     kmsAudit.ok(user, KMSOp.DELETE_KEY, name, "");
-
+    LOG.trace("Exiting deleteKey method.");
     return Response.ok().build();
   }
 
@@ -185,10 +195,12 @@ public class KMS {
   @Produces(MediaType.APPLICATION_JSON)
   public Response rolloverKey(@PathParam("name") final String name,
       Map jsonMaterial) throws Exception {
+    LOG.trace("Entering rolloverKey Method.");
     KMSWebApp.getAdminCallsMeter().mark();
     UserGroupInformation user = HttpUserGroupInformation.get();
     assertAccess(KMSACLs.Type.ROLLOVER, user, KMSOp.ROLL_NEW_VERSION, name);
     KMSClientProvider.checkNotEmpty(name, "name");
+    LOG.debug("Rolling key with name {}.", name);
     final String material = (String)
         jsonMaterial.get(KMSRESTConstants.MATERIAL_FIELD);
     if (material != null) {
@@ -216,6 +228,7 @@ public class KMS {
       keyVersion = removeKeyMaterial(keyVersion);
     }
     Map json = KMSServerJSONUtils.toJSON(keyVersion);
+    LOG.trace("Exiting rolloverKey Method.");
     return Response.ok().type(MediaType.APPLICATION_JSON).entity(json).build();
   }
 
@@ -224,6 +237,7 @@ public class KMS {
   @Produces(MediaType.APPLICATION_JSON)
   public Response getKeysMetadata(@QueryParam(KMSRESTConstants.KEY)
       List<String> keyNamesList) throws Exception {
+    LOG.trace("Entering getKeysMetadata method.");
     KMSWebApp.getAdminCallsMeter().mark();
     UserGroupInformation user = HttpUserGroupInformation.get();
     final String[] keyNames = keyNamesList.toArray(
@@ -241,6 +255,7 @@ public class KMS {
 
     Object json = KMSServerJSONUtils.toJSON(keyNames, keysMeta);
     kmsAudit.ok(user, KMSOp.GET_KEYS_METADATA, "");
+    LOG.trace("Exiting getKeysMetadata method.");
     return Response.ok().type(MediaType.APPLICATION_JSON).entity(json).build();
   }
 
@@ -248,6 +263,7 @@ public class KMS {
   @Path(KMSRESTConstants.KEYS_NAMES_RESOURCE)
   @Produces(MediaType.APPLICATION_JSON)
   public Response getKeyNames() throws Exception {
+    LOG.trace("Entering getKeyNames method.");
     KMSWebApp.getAdminCallsMeter().mark();
     UserGroupInformation user = HttpUserGroupInformation.get();
     assertAccess(KMSACLs.Type.GET_KEYS, user, KMSOp.GET_KEYS);
@@ -262,6 +278,7 @@ public class KMS {
     );
 
     kmsAudit.ok(user, KMSOp.GET_KEYS, "");
+    LOG.trace("Exiting getKeyNames method.");
     return Response.ok().type(MediaType.APPLICATION_JSON).entity(json).build();
   }
 
@@ -269,6 +286,9 @@ public class KMS {
   @Path(KMSRESTConstants.KEY_RESOURCE + "/{name:.*}")
   public Response getKey(@PathParam("name") String name)
       throws Exception {
+    LOG.trace("Entering getKey method.");
+    LOG.debug("Getting key information for key with name {}.", name);
+    LOG.trace("Exiting getKey method.");
     return getMetadata(name);
   }
 
@@ -278,10 +298,12 @@ public class KMS {
   @Produces(MediaType.APPLICATION_JSON)
   public Response getMetadata(@PathParam("name") final String name)
       throws Exception {
+    LOG.trace("Entering getMetadata method.");
     UserGroupInformation user = HttpUserGroupInformation.get();
     KMSClientProvider.checkNotEmpty(name, "name");
     KMSWebApp.getAdminCallsMeter().mark();
     assertAccess(KMSACLs.Type.GET_METADATA, user, KMSOp.GET_METADATA, name);
+    LOG.debug("Getting metadata for key with name {}.", name);
 
     KeyProvider.Metadata metadata = user.doAs(
         new PrivilegedExceptionAction<KeyProvider.Metadata>() {
@@ -294,6 +316,7 @@ public class KMS {
 
     Object json = KMSServerJSONUtils.toJSON(name, metadata);
     kmsAudit.ok(user, KMSOp.GET_METADATA, name, "");
+    LOG.trace("Exiting getMetadata method.");
     return Response.ok().type(MediaType.APPLICATION_JSON).entity(json).build();
   }
 
@@ -303,10 +326,12 @@ public class KMS {
   @Produces(MediaType.APPLICATION_JSON)
   public Response getCurrentVersion(@PathParam("name") final String name)
       throws Exception {
+    LOG.trace("Entering getCurrentVersion method.");
     UserGroupInformation user = HttpUserGroupInformation.get();
     KMSClientProvider.checkNotEmpty(name, "name");
     KMSWebApp.getKeyCallsMeter().mark();
     assertAccess(KMSACLs.Type.GET, user, KMSOp.GET_CURRENT_KEY, name);
+    LOG.debug("Getting key version for key with name {}.", name);
 
     KeyVersion keyVersion = user.doAs(
         new PrivilegedExceptionAction<KeyVersion>() {
@@ -319,6 +344,7 @@ public class KMS {
 
     Object json = KMSServerJSONUtils.toJSON(keyVersion);
     kmsAudit.ok(user, KMSOp.GET_CURRENT_KEY, name, "");
+    LOG.trace("Exiting getCurrentVersion method.");
     return Response.ok().type(MediaType.APPLICATION_JSON).entity(json).build();
   }
 
@@ -327,10 +353,12 @@ public class KMS {
   @Produces(MediaType.APPLICATION_JSON)
   public Response getKeyVersion(
       @PathParam("versionName") final String versionName) throws Exception {
+    LOG.trace("Entering getKeyVersion method.");
     UserGroupInformation user = HttpUserGroupInformation.get();
     KMSClientProvider.checkNotEmpty(versionName, "versionName");
     KMSWebApp.getKeyCallsMeter().mark();
     assertAccess(KMSACLs.Type.GET, user, KMSOp.GET_KEY_VERSION);
+    LOG.debug("Getting key with version name {}.", versionName);
 
     KeyVersion keyVersion = user.doAs(
         new PrivilegedExceptionAction<KeyVersion>() {
@@ -345,6 +373,7 @@ public class KMS {
       kmsAudit.ok(user, KMSOp.GET_KEY_VERSION, keyVersion.getName(), "");
     }
     Object json = KMSServerJSONUtils.toJSON(keyVersion);
+    LOG.trace("Exiting getKeyVersion method.");
     return Response.ok().type(MediaType.APPLICATION_JSON).entity(json).build();
   }
 
@@ -359,12 +388,16 @@ public class KMS {
           @DefaultValue("1")
           @QueryParam(KMSRESTConstants.EEK_NUM_KEYS) final int numKeys)
           throws Exception {
+    LOG.trace("Entering generateEncryptedKeys method.");
     UserGroupInformation user = HttpUserGroupInformation.get();
     KMSClientProvider.checkNotEmpty(name, "name");
     KMSClientProvider.checkNotNull(edekOp, "eekOp");
+    LOG.debug("Generating encrypted key with name {}," +
+            " the edek Operation is {}.", name, edekOp);
 
     Object retJSON;
     if (edekOp.equals(KMSRESTConstants.EEK_GENERATE)) {
+      LOG.debug("edek Operation is Generate.");
       assertAccess(KMSACLs.Type.GENERATE_EEK, user, KMSOp.GENERATE_EEK, name);
 
       final List<EncryptedKeyVersion> retEdeks =
@@ -375,6 +408,8 @@ public class KMS {
             new PrivilegedExceptionAction<Void>() {
               @Override
               public Void run() throws Exception {
+                  LOG.debug("Generated Encrypted key for {} number of keys.",
+                          numKeys);
                 for (int i = 0; i < numKeys; i++) {
                   retEdeks.add(provider.generateEncryptedKey(name));
                 }
@@ -384,6 +419,7 @@ public class KMS {
         );
 
       } catch (Exception e) {
+        LOG.error("Exception in generateEncryptedKeys:", e);
         throw new IOException(e);
       }
       kmsAudit.ok(user, KMSOp.GENERATE_EEK, name, "");
@@ -392,11 +428,18 @@ public class KMS {
         ((ArrayList)retJSON).add(KMSServerJSONUtils.toJSON(edek));
       }
     } else {
-      throw new IllegalArgumentException("Wrong " + KMSRESTConstants.EEK_OP +
-          " value, it must be " + KMSRESTConstants.EEK_GENERATE + " or " +
-          KMSRESTConstants.EEK_DECRYPT);
+      StringBuilder error;
+      error = new StringBuilder("IllegalArgumentException Wrong ");
+      error.append(KMSRESTConstants.EEK_OP);
+      error.append(" value, it must be ");
+      error.append(KMSRESTConstants.EEK_GENERATE);
+      error.append(" or ");
+      error.append(KMSRESTConstants.EEK_DECRYPT);
+      LOG.error(error.toString());
+      throw new IllegalArgumentException(error.toString());
     }
     KMSWebApp.getGenerateEEKCallsMeter().mark();
+    LOG.trace("Exiting generateEncryptedKeys method.");
     return Response.ok().type(MediaType.APPLICATION_JSON).entity(retJSON)
         .build();
   }
@@ -411,14 +454,17 @@ public class KMS {
       @QueryParam(KMSRESTConstants.EEK_OP) String eekOp,
       Map jsonPayload)
       throws Exception {
+    LOG.trace("Entering decryptEncryptedKey method.");
     UserGroupInformation user = HttpUserGroupInformation.get();
     KMSClientProvider.checkNotEmpty(versionName, "versionName");
     KMSClientProvider.checkNotNull(eekOp, "eekOp");
+    LOG.debug("Decrypting key for {}, the edek Operation is {}.",
+            versionName, eekOp);
 
     final String keyName = (String) jsonPayload.get(
         KMSRESTConstants.NAME_FIELD);
     String ivStr = (String) jsonPayload.get(KMSRESTConstants.IV_FIELD);
-    String encMaterialStr = 
+    String encMaterialStr =
         (String) jsonPayload.get(KMSRESTConstants.MATERIAL_FIELD);
     Object retJSON;
     if (eekOp.equals(KMSRESTConstants.EEK_DECRYPT)) {
@@ -445,11 +491,18 @@ public class KMS {
       retJSON = KMSServerJSONUtils.toJSON(retKeyVersion);
       kmsAudit.ok(user, KMSOp.DECRYPT_EEK, keyName, "");
     } else {
-      throw new IllegalArgumentException("Wrong " + KMSRESTConstants.EEK_OP +
-          " value, it must be " + KMSRESTConstants.EEK_GENERATE + " or " +
-          KMSRESTConstants.EEK_DECRYPT);
+      StringBuilder error;
+      error = new StringBuilder("IllegalArgumentException Wrong ");
+      error.append(KMSRESTConstants.EEK_OP);
+      error.append(" value, it must be ");
+      error.append(KMSRESTConstants.EEK_GENERATE);
+      error.append(" or ");
+      error.append(KMSRESTConstants.EEK_DECRYPT);
+      LOG.error(error.toString());
+      throw new IllegalArgumentException(error.toString());
     }
     KMSWebApp.getDecryptEEKCallsMeter().mark();
+    LOG.trace("Exiting decryptEncryptedKey method.");
     return Response.ok().type(MediaType.APPLICATION_JSON).entity(retJSON)
         .build();
   }
@@ -460,10 +513,12 @@ public class KMS {
   @Produces(MediaType.APPLICATION_JSON)
   public Response getKeyVersions(@PathParam("name") final String name)
       throws Exception {
+    LOG.trace("Entering getKeyVersions method.");
     UserGroupInformation user = HttpUserGroupInformation.get();
     KMSClientProvider.checkNotEmpty(name, "name");
     KMSWebApp.getKeyCallsMeter().mark();
     assertAccess(KMSACLs.Type.GET, user, KMSOp.GET_KEY_VERSIONS, name);
+    LOG.debug("Getting key versions for key {}", name);
 
     List<KeyVersion> ret = user.doAs(
         new PrivilegedExceptionAction<List<KeyVersion>>() {
@@ -476,6 +531,7 @@ public class KMS {
 
     Object json = KMSServerJSONUtils.toJSON(ret);
     kmsAudit.ok(user, KMSOp.GET_KEY_VERSIONS, name, "");
+    LOG.trace("Exiting getKeyVersions method.");
     return Response.ok().type(MediaType.APPLICATION_JSON).entity(json).build();
   }
 


---------------------------------------------------------------------
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-5693. Reduce loglevel to Debug in ContainerManagementProtocolProxy and AMRMClientImpl (yufeigu via rkanter)

Posted by su...@apache.org.
YARN-5693. Reduce loglevel to Debug in ContainerManagementProtocolProxy and AMRMClientImpl (yufeigu via rkanter)


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

Branch: refs/heads/YARN-2915
Commit: 2549ee9d4c4ddd3ebccdebb8623df30c0a8f27d2
Parents: 434c5ea
Author: Robert Kanter <rk...@apache.org>
Authored: Fri Sep 30 13:30:55 2016 -0700
Committer: Robert Kanter <rk...@apache.org>
Committed: Fri Sep 30 13:30:55 2016 -0700

----------------------------------------------------------------------
 .../hadoop/yarn/client/api/impl/AMRMClientImpl.java       | 10 ++++++----
 .../client/api/impl/ContainerManagementProtocolProxy.java |  7 +++++--
 2 files changed, 11 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2549ee9d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AMRMClientImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AMRMClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AMRMClientImpl.java
index 6f6bb85..3221661 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AMRMClientImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AMRMClientImpl.java
@@ -455,10 +455,12 @@ public class AMRMClientImpl<T extends ContainerRequest> extends AMRMClient<T> {
   protected void populateNMTokens(List<NMToken> nmTokens) {
     for (NMToken token : nmTokens) {
       String nodeId = token.getNodeId().toString();
-      if (getNMTokenCache().containsToken(nodeId)) {
-        LOG.info("Replacing token for : " + nodeId);
-      } else {
-        LOG.info("Received new token for : " + nodeId);
+      if (LOG.isDebugEnabled()) {
+        if (getNMTokenCache().containsToken(nodeId)) {
+          LOG.debug("Replacing token for : " + nodeId);
+        } else {
+          LOG.debug("Received new token for : " + nodeId);
+        }
       }
       getNMTokenCache().setToken(nodeId, token.getToken());
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2549ee9d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/ContainerManagementProtocolProxy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/ContainerManagementProtocolProxy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/ContainerManagementProtocolProxy.java
index b2bce22..c619e8a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/ContainerManagementProtocolProxy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/ContainerManagementProtocolProxy.java
@@ -78,8 +78,11 @@ public class ContainerManagementProtocolProxy {
           YarnConfiguration.NM_CLIENT_MAX_NM_PROXIES
               + " (" + maxConnectedNMs + ") can not be less than 0.");
     }
-    LOG.info(YarnConfiguration.NM_CLIENT_MAX_NM_PROXIES + " : "
-        + maxConnectedNMs);
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(YarnConfiguration.NM_CLIENT_MAX_NM_PROXIES + " : " +
+          maxConnectedNMs);
+    }
 
     if (maxConnectedNMs > 0) {
       cmProxy =


---------------------------------------------------------------------
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: Revert "HDFS-10923. Make InstrumentedLock require ReentrantLock."

Posted by su...@apache.org.
Revert "HDFS-10923. Make InstrumentedLock require ReentrantLock."

This reverts commit c7ce6fdc20fe053f0bb3bcf900ffc0e1db6feee5.


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

Branch: refs/heads/YARN-2915
Commit: fe9ebe20ab113567f0777c11cb48ce0d3ce587a8
Parents: c7ce6fd
Author: Arpit Agarwal <ar...@apache.org>
Authored: Fri Sep 30 23:11:51 2016 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Fri Sep 30 23:11:51 2016 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hdfs/InstrumentedLock.java    | 185 ++++++++++++++++++
 .../hadoop/hdfs/InstrumentedReentrantLock.java  | 195 -------------------
 .../datanode/fsdataset/impl/FsDatasetImpl.java  |   4 +-
 .../hadoop/hdfs/TestInstrumentedLock.java       | 166 ++++++++++++++++
 .../hdfs/TestInstrumentedReentrantLock.java     | 177 -----------------
 5 files changed, 353 insertions(+), 374 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fe9ebe20/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/InstrumentedLock.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/InstrumentedLock.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/InstrumentedLock.java
new file mode 100644
index 0000000..6279e95
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/InstrumentedLock.java
@@ -0,0 +1,185 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.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.hdfs;
+
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.commons.logging.Log;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.Timer;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * This is a debugging class that can be used by callers to track
+ * whether a specifc lock is being held for too long and periodically
+ * log a warning and stack trace, if so.
+ *
+ * The logged warnings are throttled so that logs are not spammed.
+ *
+ * A new instance of InstrumentedLock can be created for each object
+ * that needs to be instrumented.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class InstrumentedLock implements Lock {
+
+  private final Lock lock;
+  private final Log logger;
+  private final String name;
+  private final Timer clock;
+
+  /** Minimum gap between two lock warnings. */
+  private final long minLoggingGap;
+  /** Threshold for detecting long lock held time. */
+  private final long lockWarningThreshold;
+
+  // Tracking counters for lock statistics.
+  private volatile long lockAcquireTimestamp;
+  private final AtomicLong lastLogTimestamp;
+  private final AtomicLong warningsSuppressed = new AtomicLong(0);
+
+  /**
+   * Create a instrumented lock instance which logs a warning message
+   * when lock held time is above given threshold.
+   *
+   * @param name the identifier of the lock object
+   * @param logger this class does not have its own logger, will log to the
+   *               given logger instead
+   * @param minLoggingGapMs  the minimum time gap between two log messages,
+   *                         this is to avoid spamming to many logs
+   * @param lockWarningThresholdMs the time threshold to view lock held
+   *                               time as being "too long"
+   */
+  public InstrumentedLock(String name, Log logger, long minLoggingGapMs,
+      long lockWarningThresholdMs) {
+    this(name, logger, new ReentrantLock(),
+        minLoggingGapMs, lockWarningThresholdMs);
+  }
+
+  public InstrumentedLock(String name, Log logger, Lock lock,
+      long minLoggingGapMs, long lockWarningThresholdMs) {
+    this(name, logger, lock,
+        minLoggingGapMs, lockWarningThresholdMs, new Timer());
+  }
+
+  @VisibleForTesting
+  InstrumentedLock(String name, Log logger, Lock lock,
+      long minLoggingGapMs, long lockWarningThresholdMs, Timer clock) {
+    this.name = name;
+    this.lock = lock;
+    this.clock = clock;
+    this.logger = logger;
+    minLoggingGap = minLoggingGapMs;
+    lockWarningThreshold = lockWarningThresholdMs;
+    lastLogTimestamp = new AtomicLong(
+      clock.monotonicNow() - Math.max(minLoggingGap, lockWarningThreshold));
+  }
+
+  @Override
+  public void lock() {
+    lock.lock();
+    lockAcquireTimestamp = clock.monotonicNow();
+  }
+
+  @Override
+  public void lockInterruptibly() throws InterruptedException {
+    lock.lockInterruptibly();
+    lockAcquireTimestamp = clock.monotonicNow();
+  }
+
+  @Override
+  public boolean tryLock() {
+    if (lock.tryLock()) {
+      lockAcquireTimestamp = clock.monotonicNow();
+      return true;
+    }
+    return false;
+  }
+
+  @Override
+  public boolean tryLock(long time, TimeUnit unit) throws InterruptedException {
+    if (lock.tryLock(time, unit)) {
+      lockAcquireTimestamp = clock.monotonicNow();
+      return true;
+    }
+    return false;
+  }
+
+  @Override
+  public void unlock() {
+    long localLockReleaseTime = clock.monotonicNow();
+    long localLockAcquireTime = lockAcquireTimestamp;
+    lock.unlock();
+    check(localLockAcquireTime, localLockReleaseTime);
+  }
+
+  @Override
+  public Condition newCondition() {
+    return lock.newCondition();
+  }
+
+  @VisibleForTesting
+  void logWarning(long lockHeldTime, long suppressed) {
+    logger.warn(String.format("Lock held time above threshold: " +
+        "lock identifier: %s " +
+        "lockHeldTimeMs=%d ms. Suppressed %d lock warnings. " +
+        "The stack trace is: %s" ,
+        name, lockHeldTime, suppressed,
+        StringUtils.getStackTrace(Thread.currentThread())));
+  }
+
+  /**
+   * Log a warning if the lock was held for too long.
+   *
+   * Should be invoked by the caller immediately AFTER releasing the lock.
+   *
+   * @param acquireTime  - timestamp just after acquiring the lock.
+   * @param releaseTime - timestamp just before releasing the lock.
+   */
+  private void check(long acquireTime, long releaseTime) {
+    if (!logger.isWarnEnabled()) {
+      return;
+    }
+
+    final long lockHeldTime = releaseTime - acquireTime;
+    if (lockWarningThreshold - lockHeldTime < 0) {
+      long now;
+      long localLastLogTs;
+      do {
+        now = clock.monotonicNow();
+        localLastLogTs = lastLogTimestamp.get();
+        long deltaSinceLastLog = now - localLastLogTs;
+        // check should print log or not
+        if (deltaSinceLastLog - minLoggingGap < 0) {
+          warningsSuppressed.incrementAndGet();
+          return;
+        }
+      } while (!lastLogTimestamp.compareAndSet(localLastLogTs, now));
+      long suppressed = warningsSuppressed.getAndSet(0);
+      logWarning(lockHeldTime, suppressed);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fe9ebe20/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/InstrumentedReentrantLock.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/InstrumentedReentrantLock.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/InstrumentedReentrantLock.java
deleted file mode 100644
index 010571e..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/InstrumentedReentrantLock.java
+++ /dev/null
@@ -1,195 +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
- * <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.hdfs;
-
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.locks.Condition;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantLock;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.commons.logging.Log;
-import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.util.Timer;
-
-import com.google.common.annotations.VisibleForTesting;
-
-/**
- * This is a debugging class that can be used by callers to track
- * whether a specific lock is being held for too long and periodically
- * log a warning and stack trace, if so.
- *
- * The logged warnings are throttled so that logs are not spammed.
- *
- * A new instance of InstrumentedLock can be created for each object
- * that needs to be instrumented.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class InstrumentedReentrantLock implements Lock {
-
-  @VisibleForTesting
-  final ReentrantLock lock;
-  private final Log logger;
-  private final String name;
-  private final Timer clock;
-
-  /** Minimum gap between two lock warnings. */
-  private final long minLoggingGap;
-  /** Threshold for detecting long lock held time. */
-  private final long lockWarningThreshold;
-
-  // Tracking counters for lock statistics.
-  private volatile long lockAcquireTimestamp;
-  private final AtomicLong lastLogTimestamp;
-  private final AtomicLong warningsSuppressed = new AtomicLong(0);
-
-  /**
-   * Create a instrumented lock instance which logs a warning message
-   * when lock held time is above given threshold.
-   *
-   * @param name the identifier of the lock object
-   * @param logger this class does not have its own logger, will log to the
-   *               given logger instead
-   * @param minLoggingGapMs  the minimum time gap between two log messages,
-   *                         this is to avoid spamming to many logs
-   * @param lockWarningThresholdMs the time threshold to view lock held
-   *                               time as being "too long"
-   */
-  public InstrumentedReentrantLock(
-      String name, Log logger, long minLoggingGapMs,
-      long lockWarningThresholdMs) {
-    this(name, logger, new ReentrantLock(),
-        minLoggingGapMs, lockWarningThresholdMs);
-  }
-
-  public InstrumentedReentrantLock(
-      String name, Log logger, ReentrantLock lock,
-      long minLoggingGapMs, long lockWarningThresholdMs) {
-    this(name, logger, lock,
-        minLoggingGapMs, lockWarningThresholdMs, new Timer());
-  }
-
-  @VisibleForTesting
-  InstrumentedReentrantLock(
-      String name, Log logger, ReentrantLock lock,
-      long minLoggingGapMs, long lockWarningThresholdMs, Timer clock) {
-    this.name = name;
-    this.lock = lock;
-    this.clock = clock;
-    this.logger = logger;
-    minLoggingGap = minLoggingGapMs;
-    lockWarningThreshold = lockWarningThresholdMs;
-    lastLogTimestamp = new AtomicLong(
-      clock.monotonicNow() - Math.max(minLoggingGap, lockWarningThreshold));
-  }
-
-  @Override
-  public void lock() {
-    lock.lock();
-    if (lock.getHoldCount() == 1) {
-      lockAcquireTimestamp = clock.monotonicNow();
-    }
-  }
-
-  @Override
-  public void lockInterruptibly() throws InterruptedException {
-    lock.lockInterruptibly();
-    if (lock.getHoldCount() == 1) {
-      lockAcquireTimestamp = clock.monotonicNow();
-    }
-  }
-
-  @Override
-  public boolean tryLock() {
-    if (lock.tryLock() && lock.getHoldCount() == 1) {
-      lockAcquireTimestamp = clock.monotonicNow();
-      return true;
-    }
-    return false;
-  }
-
-  @Override
-  public boolean tryLock(long time, TimeUnit unit) throws InterruptedException {
-    if (lock.tryLock(time, unit) && lock.getHoldCount() == 1) {
-      lockAcquireTimestamp = clock.monotonicNow();
-      return true;
-    }
-    return false;
-  }
-
-  @Override
-  public void unlock() {
-    final boolean needReport = (lock.getHoldCount() == 1);
-    long localLockReleaseTime = clock.monotonicNow();
-    long localLockAcquireTime = lockAcquireTimestamp;
-    lock.unlock();
-    if (needReport) {
-      check(localLockAcquireTime, localLockReleaseTime);
-    }
-  }
-
-  @Override
-  public Condition newCondition() {
-    return lock.newCondition();
-  }
-
-  @VisibleForTesting
-  void logWarning(long lockHeldTime, long suppressed) {
-    logger.warn(String.format("Lock held time above threshold: " +
-        "lock identifier: %s " +
-        "lockHeldTimeMs=%d ms. Suppressed %d lock warnings. " +
-        "The stack trace is: %s" ,
-        name, lockHeldTime, suppressed,
-        StringUtils.getStackTrace(Thread.currentThread())));
-  }
-
-  /**
-   * Log a warning if the lock was held for too long.
-   *
-   * Should be invoked by the caller immediately AFTER releasing the lock.
-   *
-   * @param acquireTime  - timestamp just after acquiring the lock.
-   * @param releaseTime - timestamp just before releasing the lock.
-   */
-  private void check(long acquireTime, long releaseTime) {
-    if (!logger.isWarnEnabled()) {
-      return;
-    }
-
-    final long lockHeldTime = releaseTime - acquireTime;
-    if (lockWarningThreshold - lockHeldTime < 0) {
-      long now;
-      long localLastLogTs;
-      do {
-        now = clock.monotonicNow();
-        localLastLogTs = lastLogTimestamp.get();
-        long deltaSinceLastLog = now - localLastLogTs;
-        // check should print log or not
-        if (deltaSinceLastLog - minLoggingGap < 0) {
-          warningsSuppressed.incrementAndGet();
-          return;
-        }
-      } while (!lastLogTimestamp.compareAndSet(localLastLogTs, now));
-      long suppressed = warningsSuppressed.getAndSet(0);
-      logWarning(lockHeldTime, suppressed);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fe9ebe20/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
index ab31f25..26a2e9f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
@@ -58,7 +58,7 @@ import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.ExtendedBlockId;
-import org.apache.hadoop.hdfs.InstrumentedReentrantLock;
+import org.apache.hadoop.hdfs.InstrumentedLock;
 import org.apache.hadoop.util.AutoCloseableLock;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
@@ -266,7 +266,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     this.conf = conf;
     this.smallBufferSize = DFSUtilClient.getSmallBufferSize(conf);
     this.datasetLock = new AutoCloseableLock(
-        new InstrumentedReentrantLock(getClass().getName(), LOG,
+        new InstrumentedLock(getClass().getName(), LOG,
           conf.getTimeDuration(
             DFSConfigKeys.DFS_LOCK_SUPPRESS_WARNING_INTERVAL_KEY,
             DFSConfigKeys.DFS_LOCK_SUPPRESS_WARNING_INTERVAL_DEFAULT,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fe9ebe20/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestInstrumentedLock.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestInstrumentedLock.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestInstrumentedLock.java
new file mode 100644
index 0000000..f470688
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestInstrumentedLock.java
@@ -0,0 +1,166 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.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.hdfs;
+
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.locks.Lock;
+
+import org.apache.hadoop.util.AutoCloseableLock;
+import org.apache.hadoop.util.Timer;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+import static org.mockito.Mockito.*;
+import static org.junit.Assert.*;
+
+/**
+ * A test class for InstrumentedLock.
+ */
+public class TestInstrumentedLock {
+
+  static final Log LOG = LogFactory.getLog(TestInstrumentedLock.class);
+
+  @Rule public TestName name = new TestName();
+
+  /**
+   * Test exclusive access of the lock.
+   * @throws Exception
+   */
+  @Test(timeout=10000)
+  public void testMultipleThread() throws Exception {
+    String testname = name.getMethodName();
+    InstrumentedLock lock = new InstrumentedLock(testname, LOG, 0, 300);
+    lock.lock();
+    try {
+      Thread competingThread = new Thread() {
+        @Override
+        public void run() {
+          assertFalse(lock.tryLock());
+        }
+      };
+      competingThread.start();
+      competingThread.join();
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  /**
+   * Test the correctness with try-with-resource syntax.
+   * @throws Exception
+   */
+  @Test(timeout=10000)
+  public void testTryWithResourceSyntax() throws Exception {
+    String testname = name.getMethodName();
+    final AtomicReference<Thread> lockThread = new AtomicReference<>(null);
+    Lock lock = new InstrumentedLock(testname, LOG, 0, 300) {
+      @Override
+      public void lock() {
+        super.lock();
+        lockThread.set(Thread.currentThread());
+      }
+      @Override
+      public void unlock() {
+        super.unlock();
+        lockThread.set(null);
+      }
+    };
+    AutoCloseableLock acl = new AutoCloseableLock(lock);
+    try (AutoCloseable localLock = acl.acquire()) {
+      assertEquals(acl, localLock);
+      Thread competingThread = new Thread() {
+        @Override
+        public void run() {
+          assertNotEquals(Thread.currentThread(), lockThread.get());
+          assertFalse(lock.tryLock());
+        }
+      };
+      competingThread.start();
+      competingThread.join();
+      assertEquals(Thread.currentThread(), lockThread.get());
+    }
+    assertNull(lockThread.get());
+  }
+
+  /**
+   * Test the lock logs warning when lock held time is greater than threshold
+   * and not log warning otherwise.
+   * @throws Exception
+   */
+  @Test(timeout=10000)
+  public void testLockLongHoldingReport() throws Exception {
+    String testname = name.getMethodName();
+    final AtomicLong time = new AtomicLong(0);
+    Timer mclock = new Timer() {
+      @Override
+      public long monotonicNow() {
+        return time.get();
+      }
+    };
+    Lock mlock = mock(Lock.class);
+
+    final AtomicLong wlogged = new AtomicLong(0);
+    final AtomicLong wsuppresed = new AtomicLong(0);
+    InstrumentedLock lock = new InstrumentedLock(
+        testname, LOG, mlock, 2000, 300, mclock) {
+      @Override
+      void logWarning(long lockHeldTime, long suppressed) {
+        wlogged.incrementAndGet();
+        wsuppresed.set(suppressed);
+      }
+    };
+
+    // do not log warning when the lock held time is short
+    lock.lock();   // t = 0
+    time.set(200);
+    lock.unlock(); // t = 200
+    assertEquals(0, wlogged.get());
+    assertEquals(0, wsuppresed.get());
+
+    lock.lock();   // t = 200
+    time.set(700);
+    lock.unlock(); // t = 700
+    assertEquals(1, wlogged.get());
+    assertEquals(0, wsuppresed.get());
+
+    // despite the lock held time is greater than threshold
+    // suppress the log warning due to the logging gap
+    // (not recorded in wsuppressed until next log message)
+    lock.lock();   // t = 700
+    time.set(1100);
+    lock.unlock(); // t = 1100
+    assertEquals(1, wlogged.get());
+    assertEquals(0, wsuppresed.get());
+
+    // log a warning message when the lock held time is greater the threshold
+    // and the logging time gap is satisfied. Also should display suppressed
+    // previous warnings.
+    time.set(2400);
+    lock.lock();   // t = 2400
+    time.set(2800);
+    lock.unlock(); // t = 2800
+    assertEquals(2, wlogged.get());
+    assertEquals(1, wsuppresed.get());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fe9ebe20/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestInstrumentedReentrantLock.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestInstrumentedReentrantLock.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestInstrumentedReentrantLock.java
deleted file mode 100644
index 3374b8a..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestInstrumentedReentrantLock.java
+++ /dev/null
@@ -1,177 +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
- * <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.hdfs;
-
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantLock;
-
-import org.apache.hadoop.util.AutoCloseableLock;
-import org.apache.hadoop.util.FakeTimer;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-import static org.mockito.Mockito.*;
-import static org.junit.Assert.*;
-
-/**
- * A test class for {@link InstrumentedReentrantLock}.
- */
-public class TestInstrumentedReentrantLock {
-
-  static final Log LOG = LogFactory.getLog(TestInstrumentedReentrantLock.class);
-
-  @Rule public TestName name = new TestName();
-
-  /**
-   * Test exclusive access of the lock.
-   * @throws Exception
-   */
-  @Test(timeout=10000)
-  public void testMultipleThread() throws Exception {
-    String testname = name.getMethodName();
-    InstrumentedReentrantLock lock =
-        new InstrumentedReentrantLock(testname, LOG, 0, 300);
-    lock.lock();
-    try {
-      Thread competingThread = new Thread() {
-        @Override
-        public void run() {
-          assertFalse(lock.tryLock());
-        }
-      };
-      competingThread.start();
-      competingThread.join();
-    } finally {
-      lock.unlock();
-    }
-  }
-
-  /**
-   * Test the correctness with try-with-resource syntax.
-   * @throws Exception
-   */
-  @Test(timeout=10000)
-  public void testTryWithResourceSyntax() throws Exception {
-    String testname = name.getMethodName();
-    final AtomicReference<Thread> lockThread = new AtomicReference<>(null);
-    Lock lock = new InstrumentedReentrantLock(testname, LOG, 0, 300) {
-      @Override
-      public void lock() {
-        super.lock();
-        lockThread.set(Thread.currentThread());
-      }
-      @Override
-      public void unlock() {
-        super.unlock();
-        lockThread.set(null);
-      }
-    };
-    AutoCloseableLock acl = new AutoCloseableLock(lock);
-    try (AutoCloseable localLock = acl.acquire()) {
-      assertEquals(acl, localLock);
-      Thread competingThread = new Thread() {
-        @Override
-        public void run() {
-          assertNotEquals(Thread.currentThread(), lockThread.get());
-          assertFalse(lock.tryLock());
-        }
-      };
-      competingThread.start();
-      competingThread.join();
-      assertEquals(Thread.currentThread(), lockThread.get());
-    }
-    assertNull(lockThread.get());
-  }
-
-  /**
-   * Test the lock logs warning when lock held time is greater than threshold
-   * and not log warning otherwise.
-   * @throws Exception
-   */
-  @Test(timeout=10000)
-  public void testLockLongHoldingReport() throws Exception {
-    String testname = name.getMethodName();
-    FakeTimer mclock = new FakeTimer();
-    final int warningThreshold = 500;
-    final int minLoggingGap = warningThreshold * 10;
-
-    final AtomicLong wlogged = new AtomicLong(0);
-    final AtomicLong wsuppresed = new AtomicLong(0);
-    InstrumentedReentrantLock lock = new InstrumentedReentrantLock(
-        testname, LOG, new ReentrantLock(), minLoggingGap,
-        warningThreshold, mclock) {
-      @Override
-      void logWarning(long lockHeldTime, long suppressed) {
-        wlogged.incrementAndGet();
-        wsuppresed.set(suppressed);
-      }
-    };
-
-    // do not log warning when the lock held time is <= warningThreshold.
-    lock.lock();
-    mclock.advance(warningThreshold);
-    lock.unlock();
-    assertEquals(0, wlogged.get());
-    assertEquals(0, wsuppresed.get());
-
-    // log a warning when the lock held time exceeds the threshold.
-    lock.lock();
-    mclock.advance(warningThreshold + 1);
-    assertEquals(1, lock.lock.getHoldCount());
-    lock.unlock();
-    assertEquals(1, wlogged.get());
-    assertEquals(0, wsuppresed.get());
-
-    // despite the lock held time is greater than threshold
-    // suppress the log warning due to the logging gap
-    // (not recorded in wsuppressed until next log message)
-    lock.lock();
-    mclock.advance(warningThreshold + 1);
-    lock.unlock();
-    assertEquals(1, wlogged.get());
-    assertEquals(0, wsuppresed.get());
-
-    // log a warning message when the lock held time is greater the threshold
-    // and the logging time gap is satisfied. Also should display suppressed
-    // previous warnings.
-    lock.lock();
-    mclock.advance(minLoggingGap + 1);
-    lock.unlock(); // t = 2800
-    assertEquals(2, wlogged.get());
-    assertEquals(1, wsuppresed.get());
-
-    // Ensure that nested acquisitions do not log.
-    wlogged.set(0);
-    wsuppresed.set(0);
-    lock.lock();
-    lock.lock();
-    mclock.advance(minLoggingGap + 1);
-    lock.unlock();
-    assertEquals(0, wlogged.get());    // No warnings on nested release.
-    assertEquals(0, wsuppresed.get());
-    lock.unlock();
-    assertEquals(1, wlogged.get());    // Last release immediately logs.
-    assertEquals(0, wsuppresed.get());
-  }
-}


---------------------------------------------------------------------
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: HADOOP-13663 Index out of range in SysInfoWindows. Contributed by Inigo Goiri

Posted by su...@apache.org.
HADOOP-13663 Index out of range in SysInfoWindows. Contributed by Inigo Goiri


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

Branch: refs/heads/YARN-2915
Commit: 1518cb9532cbedeada1b3d880f4ef1059301e828
Parents: 47f8092
Author: Steve Loughran <st...@apache.org>
Authored: Thu Sep 29 11:35:00 2016 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Thu Sep 29 11:35:00 2016 +0100

----------------------------------------------------------------------
 .../org/apache/hadoop/util/SysInfoWindows.java  | 58 +++++++++++---------
 .../apache/hadoop/util/TestSysInfoWindows.java  |  7 ++-
 2 files changed, 37 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1518cb95/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SysInfoWindows.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SysInfoWindows.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SysInfoWindows.java
index 490c127..e21adac 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SysInfoWindows.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SysInfoWindows.java
@@ -100,36 +100,40 @@ public class SysInfoWindows extends SysInfo {
       String sysInfoStr = getSystemInfoInfoFromShell();
       if (sysInfoStr != null) {
         final int sysInfoSplitCount = 11;
-        String[] sysInfo = sysInfoStr.substring(0, sysInfoStr.indexOf("\r\n"))
-            .split(",");
-        if (sysInfo.length == sysInfoSplitCount) {
-          try {
-            vmemSize = Long.parseLong(sysInfo[0]);
-            memSize = Long.parseLong(sysInfo[1]);
-            vmemAvailable = Long.parseLong(sysInfo[2]);
-            memAvailable = Long.parseLong(sysInfo[3]);
-            numProcessors = Integer.parseInt(sysInfo[4]);
-            cpuFrequencyKhz = Long.parseLong(sysInfo[5]);
-            cumulativeCpuTimeMs = Long.parseLong(sysInfo[6]);
-            storageBytesRead = Long.parseLong(sysInfo[7]);
-            storageBytesWritten = Long.parseLong(sysInfo[8]);
-            netBytesRead = Long.parseLong(sysInfo[9]);
-            netBytesWritten = Long.parseLong(sysInfo[10]);
-            if (lastCumCpuTimeMs != -1) {
-              /**
-               * This number will be the aggregated usage across all cores in
-               * [0.0, 100.0]. For example, it will be 400.0 if there are 8
-               * cores and each of them is running at 50% utilization.
-               */
-              cpuUsage = (cumulativeCpuTimeMs - lastCumCpuTimeMs)
-                  * 100F / refreshInterval;
+        int index = sysInfoStr.indexOf("\r\n");
+        if (index >= 0) {
+          String[] sysInfo = sysInfoStr.substring(0, index).split(",");
+          if (sysInfo.length == sysInfoSplitCount) {
+            try {
+              vmemSize = Long.parseLong(sysInfo[0]);
+              memSize = Long.parseLong(sysInfo[1]);
+              vmemAvailable = Long.parseLong(sysInfo[2]);
+              memAvailable = Long.parseLong(sysInfo[3]);
+              numProcessors = Integer.parseInt(sysInfo[4]);
+              cpuFrequencyKhz = Long.parseLong(sysInfo[5]);
+              cumulativeCpuTimeMs = Long.parseLong(sysInfo[6]);
+              storageBytesRead = Long.parseLong(sysInfo[7]);
+              storageBytesWritten = Long.parseLong(sysInfo[8]);
+              netBytesRead = Long.parseLong(sysInfo[9]);
+              netBytesWritten = Long.parseLong(sysInfo[10]);
+              if (lastCumCpuTimeMs != -1) {
+                /**
+                 * This number will be the aggregated usage across all cores in
+                 * [0.0, 100.0]. For example, it will be 400.0 if there are 8
+                 * cores and each of them is running at 50% utilization.
+                 */
+                cpuUsage = (cumulativeCpuTimeMs - lastCumCpuTimeMs)
+                    * 100F / refreshInterval;
+              }
+            } catch (NumberFormatException nfe) {
+              LOG.warn("Error parsing sysInfo", nfe);
             }
-          } catch (NumberFormatException nfe) {
-            LOG.warn("Error parsing sysInfo", nfe);
+          } else {
+            LOG.warn("Expected split length of sysInfo to be "
+                + sysInfoSplitCount + ". Got " + sysInfo.length);
           }
         } else {
-          LOG.warn("Expected split length of sysInfo to be "
-              + sysInfoSplitCount + ". Got " + sysInfo.length);
+          LOG.warn("Wrong output from sysInfo: " + sysInfoStr);
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1518cb95/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestSysInfoWindows.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestSysInfoWindows.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestSysInfoWindows.java
index 5551576..fc99aeb 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestSysInfoWindows.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestSysInfoWindows.java
@@ -141,10 +141,15 @@ public class TestSysInfoWindows {
   @Test(timeout = 10000)
   public void errorInGetSystemInfo() {
     SysInfoWindowsMock tester = new SysInfoWindowsMock();
-    // info str derived from windows shell command has \r\n termination
+    // info str derived from windows shell command is null
     tester.setSysinfoString(null);
     // call a method to refresh values
     tester.getAvailablePhysicalMemorySize();
+
+    // info str derived from windows shell command with no \r\n termination
+    tester.setSysinfoString("");
+    // call a method to refresh values
+    tester.getAvailablePhysicalMemorySize();
   }
 
 }


---------------------------------------------------------------------
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: HDFS-10690. Optimize insertion/removal of replica in ShortCircuitCache. Contributed by Fenghua Hu.

Posted by su...@apache.org.
HDFS-10690. Optimize insertion/removal of replica in ShortCircuitCache. Contributed by Fenghua Hu.


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

Branch: refs/heads/YARN-2915
Commit: 607705c488fa5263d851cee578a2d319e6e52ecd
Parents: de7a0a9
Author: Xiaoyu Yao <xy...@apache.org>
Authored: Mon Oct 3 10:53:21 2016 -0700
Committer: Xiaoyu Yao <xy...@apache.org>
Committed: Mon Oct 3 10:53:21 2016 -0700

----------------------------------------------------------------------
 .../hdfs/shortcircuit/ShortCircuitCache.java    | 88 ++++++++++++--------
 .../hadoop/fs/TestEnhancedByteBufferAccess.java | 17 ++--
 .../shortcircuit/TestShortCircuitCache.java     |  9 +-
 3 files changed, 69 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/607705c4/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitCache.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitCache.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitCache.java
index 62ade70..bd02a97 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitCache.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitCache.java
@@ -26,13 +26,14 @@ import java.nio.MappedByteBuffer;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Map.Entry;
-import java.util.TreeMap;
+import java.util.NoSuchElementException;
 import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.ScheduledThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.locks.Condition;
 import java.util.concurrent.locks.ReentrantLock;
 
+import org.apache.commons.collections.map.LinkedMap;
 import org.apache.commons.lang.mutable.MutableBoolean;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.ExtendedBlockId;
@@ -107,16 +108,20 @@ public class ShortCircuitCache implements Closeable {
 
         int numDemoted = demoteOldEvictableMmaped(curMs);
         int numPurged = 0;
-        Long evictionTimeNs = (long) 0;
+        Long evictionTimeNs;
         while (true) {
-          Entry<Long, ShortCircuitReplica> entry =
-              evictable.ceilingEntry(evictionTimeNs);
-          if (entry == null) break;
-          evictionTimeNs = entry.getKey();
+          Object eldestKey;
+          try {
+            eldestKey = evictable.firstKey();
+          } catch (NoSuchElementException e) {
+            break;
+          }
+          evictionTimeNs = (Long)eldestKey;
           long evictionTimeMs =
               TimeUnit.MILLISECONDS.convert(evictionTimeNs, TimeUnit.NANOSECONDS);
           if (evictionTimeMs + maxNonMmappedEvictableLifespanMs >= curMs) break;
-          ShortCircuitReplica replica = entry.getValue();
+          ShortCircuitReplica replica = (ShortCircuitReplica)evictable.get(
+              eldestKey);
           if (LOG.isTraceEnabled()) {
             LOG.trace("CacheCleaner: purging " + replica + ": " +
                 StringUtils.getStackTrace(Thread.currentThread()));
@@ -263,11 +268,11 @@ public class ShortCircuitCache implements Closeable {
   private CacheCleaner cacheCleaner;
 
   /**
-   * Tree of evictable elements.
+   * LinkedMap of evictable elements.
    *
    * Maps (unique) insertion time in nanoseconds to the element.
    */
-  private final TreeMap<Long, ShortCircuitReplica> evictable = new TreeMap<>();
+  private final LinkedMap evictable = new LinkedMap();
 
   /**
    * Maximum total size of the cache, including both mmapped and
@@ -281,12 +286,11 @@ public class ShortCircuitCache implements Closeable {
   private long maxNonMmappedEvictableLifespanMs;
 
   /**
-   * Tree of mmaped evictable elements.
+   * LinkedMap of mmaped evictable elements.
    *
    * Maps (unique) insertion time in nanoseconds to the element.
    */
-  private final TreeMap<Long, ShortCircuitReplica> evictableMmapped =
-      new TreeMap<>();
+  private final LinkedMap evictableMmapped = new LinkedMap();
 
   /**
    * Maximum number of mmaped evictable elements.
@@ -482,13 +486,16 @@ public class ShortCircuitCache implements Closeable {
   private int demoteOldEvictableMmaped(long now) {
     int numDemoted = 0;
     boolean needMoreSpace = false;
-    Long evictionTimeNs = (long) 0;
+    Long evictionTimeNs;
 
     while (true) {
-      Entry<Long, ShortCircuitReplica> entry =
-          evictableMmapped.ceilingEntry(evictionTimeNs);
-      if (entry == null) break;
-      evictionTimeNs = entry.getKey();
+      Object eldestKey;
+      try {
+        eldestKey = evictableMmapped.firstKey();
+      } catch (NoSuchElementException e) {
+        break;
+      }
+      evictionTimeNs = (Long)eldestKey;
       long evictionTimeMs =
           TimeUnit.MILLISECONDS.convert(evictionTimeNs, TimeUnit.NANOSECONDS);
       if (evictionTimeMs + maxEvictableMmapedLifespanMs >= now) {
@@ -497,7 +504,8 @@ public class ShortCircuitCache implements Closeable {
         }
         needMoreSpace = true;
       }
-      ShortCircuitReplica replica = entry.getValue();
+      ShortCircuitReplica replica = (ShortCircuitReplica)evictableMmapped.get(
+          eldestKey);
       if (LOG.isTraceEnabled()) {
         String rationale = needMoreSpace ? "because we need more space" :
             "because it's too old";
@@ -527,10 +535,15 @@ public class ShortCircuitCache implements Closeable {
         return;
       }
       ShortCircuitReplica replica;
-      if (evictableSize == 0) {
-        replica = evictableMmapped.firstEntry().getValue();
-      } else {
-        replica = evictable.firstEntry().getValue();
+      try {
+        if (evictableSize == 0) {
+          replica = (ShortCircuitReplica)evictableMmapped.get(evictableMmapped
+              .firstKey());
+        } else {
+          replica = (ShortCircuitReplica)evictable.get(evictable.firstKey());
+        }
+      } catch (NoSuchElementException e) {
+        break;
       }
       if (LOG.isTraceEnabled()) {
         LOG.trace(this + ": trimEvictionMaps is purging " + replica +
@@ -573,10 +586,11 @@ public class ShortCircuitCache implements Closeable {
    * @param map       The map to remove it from.
    */
   private void removeEvictable(ShortCircuitReplica replica,
-      TreeMap<Long, ShortCircuitReplica> map) {
+      LinkedMap map) {
     Long evictableTimeNs = replica.getEvictableTimeNs();
     Preconditions.checkNotNull(evictableTimeNs);
-    ShortCircuitReplica removed = map.remove(evictableTimeNs);
+    ShortCircuitReplica removed = (ShortCircuitReplica)map.remove(
+        evictableTimeNs);
     Preconditions.checkState(removed == replica,
         "failed to make %s unevictable", replica);
     replica.setEvictableTimeNs(null);
@@ -593,7 +607,7 @@ public class ShortCircuitCache implements Closeable {
    * @param map              The map to insert it into.
    */
   private void insertEvictable(Long evictionTimeNs,
-      ShortCircuitReplica replica, TreeMap<Long, ShortCircuitReplica> map) {
+      ShortCircuitReplica replica, LinkedMap map) {
     while (map.containsKey(evictionTimeNs)) {
       evictionTimeNs++;
     }
@@ -861,14 +875,22 @@ public class ShortCircuitCache implements Closeable {
       IOUtilsClient.cleanup(LOG, cacheCleaner);
       // Purge all replicas.
       while (true) {
-        Entry<Long, ShortCircuitReplica> entry = evictable.firstEntry();
-        if (entry == null) break;
-        purge(entry.getValue());
+        Object eldestKey;
+        try {
+          eldestKey = evictable.firstKey();
+        } catch (NoSuchElementException e) {
+          break;
+        }
+        purge((ShortCircuitReplica)evictable.get(eldestKey));
       }
       while (true) {
-        Entry<Long, ShortCircuitReplica> entry = evictableMmapped.firstEntry();
-        if (entry == null) break;
-        purge(entry.getValue());
+        Object eldestKey;
+        try {
+          eldestKey = evictableMmapped.firstKey();
+        } catch (NoSuchElementException e) {
+          break;
+        }
+        purge((ShortCircuitReplica)evictableMmapped.get(eldestKey));
       }
     } finally {
       lock.unlock();
@@ -909,8 +931,8 @@ public class ShortCircuitCache implements Closeable {
     void visit(int numOutstandingMmaps,
         Map<ExtendedBlockId, ShortCircuitReplica> replicas,
         Map<ExtendedBlockId, InvalidToken> failedLoads,
-        Map<Long, ShortCircuitReplica> evictable,
-        Map<Long, ShortCircuitReplica> evictableMmapped);
+        LinkedMap evictable,
+        LinkedMap evictableMmapped);
   }
 
   @VisibleForTesting // ONLY for testing

http://git-wip-us.apache.org/repos/asf/hadoop/blob/607705c4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestEnhancedByteBufferAccess.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestEnhancedByteBufferAccess.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestEnhancedByteBufferAccess.java
index 0ccc07a..9cd46c1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestEnhancedByteBufferAccess.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestEnhancedByteBufferAccess.java
@@ -34,6 +34,7 @@ import java.util.Map;
 import java.util.Random;
 import java.util.concurrent.TimeoutException;
 
+import org.apache.commons.collections.map.LinkedMap;
 import org.apache.commons.lang.SystemUtils;
 import org.apache.commons.lang.mutable.MutableBoolean;
 import org.apache.commons.logging.Log;
@@ -307,8 +308,8 @@ public class TestEnhancedByteBufferAccess {
     public void visit(int numOutstandingMmaps,
         Map<ExtendedBlockId, ShortCircuitReplica> replicas,
         Map<ExtendedBlockId, InvalidToken> failedLoads,
-        Map<Long, ShortCircuitReplica> evictable,
-        Map<Long, ShortCircuitReplica> evictableMmapped) {
+        LinkedMap evictable,
+        LinkedMap evictableMmapped) {
       if (expectedNumOutstandingMmaps >= 0) {
         Assert.assertEquals(expectedNumOutstandingMmaps, numOutstandingMmaps);
       }
@@ -373,8 +374,8 @@ public class TestEnhancedByteBufferAccess {
       public void visit(int numOutstandingMmaps,
           Map<ExtendedBlockId, ShortCircuitReplica> replicas,
           Map<ExtendedBlockId, InvalidToken> failedLoads, 
-          Map<Long, ShortCircuitReplica> evictable,
-          Map<Long, ShortCircuitReplica> evictableMmapped) {
+          LinkedMap evictable,
+          LinkedMap evictableMmapped) {
         ShortCircuitReplica replica = replicas.get(
             new ExtendedBlockId(firstBlock.getBlockId(), firstBlock.getBlockPoolId()));
         Assert.assertNotNull(replica);
@@ -410,8 +411,8 @@ public class TestEnhancedByteBufferAccess {
           public void visit(int numOutstandingMmaps,
               Map<ExtendedBlockId, ShortCircuitReplica> replicas,
               Map<ExtendedBlockId, InvalidToken> failedLoads,
-              Map<Long, ShortCircuitReplica> evictable,
-              Map<Long, ShortCircuitReplica> evictableMmapped) {
+              LinkedMap evictable,
+              LinkedMap evictableMmapped) {
             finished.setValue(evictableMmapped.isEmpty());
           }
         });
@@ -685,8 +686,8 @@ public class TestEnhancedByteBufferAccess {
           public void visit(int numOutstandingMmaps,
               Map<ExtendedBlockId, ShortCircuitReplica> replicas,
               Map<ExtendedBlockId, InvalidToken> failedLoads,
-              Map<Long, ShortCircuitReplica> evictable,
-              Map<Long, ShortCircuitReplica> evictableMmapped) {
+              LinkedMap evictable,
+              LinkedMap evictableMmapped) {
             Assert.assertEquals(expectedOutstandingMmaps, numOutstandingMmaps);
             ShortCircuitReplica replica =
                 replicas.get(ExtendedBlockId.fromExtendedBlock(block));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/607705c4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java
index ac14438..8e217c2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java
@@ -34,6 +34,7 @@ import java.util.Iterator;
 import java.util.Map;
 import java.util.concurrent.TimeoutException;
 
+import org.apache.commons.collections.map.LinkedMap;
 import org.apache.commons.lang.mutable.MutableBoolean;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -502,8 +503,8 @@ public class TestShortCircuitCache {
       public void visit(int numOutstandingMmaps,
           Map<ExtendedBlockId, ShortCircuitReplica> replicas,
           Map<ExtendedBlockId, InvalidToken> failedLoads,
-          Map<Long, ShortCircuitReplica> evictable,
-          Map<Long, ShortCircuitReplica> evictableMmapped) {
+          LinkedMap evictable,
+          LinkedMap evictableMmapped) {
         ShortCircuitReplica replica = replicas.get(
             ExtendedBlockId.fromExtendedBlock(block));
         Assert.assertNotNull(replica);
@@ -518,8 +519,8 @@ public class TestShortCircuitCache {
       public void visit(int numOutstandingMmaps,
           Map<ExtendedBlockId, ShortCircuitReplica> replicas,
           Map<ExtendedBlockId, InvalidToken> failedLoads,
-          Map<Long, ShortCircuitReplica> evictable,
-          Map<Long, ShortCircuitReplica> evictableMmapped) {
+          LinkedMap evictable,
+          LinkedMap evictableMmapped) {
         ShortCircuitReplica replica = replicas.get(
             ExtendedBlockId.fromExtendedBlock(block));
         Assert.assertNotNull(replica);


---------------------------------------------------------------------
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-3672. Create Facade for Federation State and Policy Store. Contributed by Subru Krishnan

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


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

Branch: refs/heads/YARN-2915
Commit: 0023e219ba97997d125ab54e0e1cafae8a894a83
Parents: 760d0be
Author: Jian He <ji...@apache.org>
Authored: Wed Aug 17 11:13:19 2016 +0800
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon Oct 3 14:15:36 2016 -0700

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0023e219/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 26ac16c..e33cce7 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -90,6 +90,10 @@
     <guice.version>4.0</guice.version>
     <joda-time.version>2.9.4</joda-time.version>
 
+    <jcache.version>1.0.0</jcache.version>
+    <ehcache.version>3.0.3</ehcache.version>
+
+
     <!-- define the Java language version used by the compiler -->
     <javac.version>1.8</javac.version>
 
@@ -1163,6 +1167,16 @@
           <artifactId>kerb-simplekdc</artifactId>
           <version>1.0.0-RC2</version>
         </dependency>
+        <dependency>
+          <groupId>javax.cache</groupId>
+          <artifactId>cache-api</artifactId>
+          <version>${jcache.version}</version>
+        </dependency>
+        <dependency>
+          <groupId>org.ehcache</groupId>
+          <artifactId>ehcache</artifactId>
+          <version>${ehcache.version}</version>
+        </dependency>
     </dependencies>
   </dependencyManagement>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0023e219/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 a80bb2b..a3f9ad0 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
@@ -2484,6 +2484,19 @@ public class YarnConfiguration extends Configuration {
   ////////////////////////////////
 
   public static final String FEDERATION_PREFIX = YARN_PREFIX + "federation.";
+
+  public static final String FEDERATION_STATESTORE_CLIENT_CLASS =
+      FEDERATION_PREFIX + "state-store.class";
+
+  public static final String DEFAULT_FEDERATION_STATESTORE_CLIENT_CLASS =
+      "org.apache.hadoop.yarn.server.federation.store.impl.MemoryFederationStateStore";
+
+  public static final String FEDERATION_CACHE_TIME_TO_LIVE_SECS =
+      FEDERATION_PREFIX + "cache-ttl.secs";
+
+  // 5 minutes
+  public static final int DEFAULT_FEDERATION_CACHE_TIME_TO_LIVE_SECS = 5 * 60;
+
   public static final String FEDERATION_MACHINE_LIST =
       FEDERATION_PREFIX + "machine-list";
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0023e219/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 668821d..000f5de 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,10 @@ public class TestYarnConfigurationFields extends TestConfigurationFieldsBase {
         .add(YarnConfiguration.DEFAULT_AMRM_PROXY_INTERCEPTOR_CLASS_PIPELINE);
     configurationPropsToSkipCompare.add(YarnConfiguration.CURATOR_LEADER_ELECTOR);
 
+    // Federation default configs to be ignored
+    configurationPropsToSkipCompare
+        .add(YarnConfiguration.DEFAULT_FEDERATION_STATESTORE_CLIENT_CLASS);
+
     // Ignore blacklisting nodes for AM failures feature since it is still a
     // "work in progress"
     configurationPropsToSkipCompare.add(YarnConfiguration.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0023e219/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 9351519..910c296 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
@@ -2621,8 +2621,8 @@
     <description>The arguments to pass to the Node label script.</description>
     <name>yarn.nodemanager.node-labels.provider.script.opts</name>
   </property>
-  <!-- Other Configuration -->
 
+  <!-- Federation Configuration -->
   <property>
     <description>
       Machine list file to be loaded by the FederationSubCluster Resolver
@@ -2631,6 +2631,24 @@
   </property>
 
   <property>
+    <description>
+      Store class name for federation state store
+    </description>
+    <name>yarn.federation.state-store.class</name>
+    <value>org.apache.hadoop.yarn.server.federation.store.impl.MemoryFederationStateStore</value>
+  </property>
+
+  <property>
+   <description>
+    The time in seconds after which the federation state store local cache
+    will be refreshed periodically
+   </description>
+   <name>yarn.federation.cache-ttl.secs</name>
+   <value>300</value>
+  </property>
+
+  <!-- Other Configuration -->
+  <property>
     <description>The interval that the yarn client library uses to poll the
     completion status of the asynchronous API of application client protocol.
     </description>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0023e219/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 c16747a..b6fd0c5 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
@@ -109,6 +109,16 @@
       <groupId>org.fusesource.leveldbjni</groupId>
       <artifactId>leveldbjni-all</artifactId>
     </dependency>
+    <dependency>
+      <groupId>javax.cache</groupId>
+      <artifactId>cache-api</artifactId>
+      <version>${jcache.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.ehcache</groupId>
+      <artifactId>ehcache</artifactId>
+      <version>${ehcache.version}</version>
+    </dependency>
   </dependencies>
 
   <build>

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

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

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

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


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


[37/50] [abbrv] hadoop git commit: YARN-5307. Federation Application State Store internal APIs

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

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

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

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

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

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


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


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

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


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

Branch: refs/heads/YARN-2915
Commit: f43ad972eb0f205a4c6e828ca8ea56296129b76d
Parents: ff47882
Author: Subru Krishnan <su...@apache.org>
Authored: Thu Sep 22 17:06:57 2016 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon Oct 3 14:34:03 2016 -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/f43ad972/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/f43ad972/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/f43ad972/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/f43ad972/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/f43ad972/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/f43ad972/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/f43ad972/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/f43ad972/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/f43ad972/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/f43ad972/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/f43ad972/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/f43ad972/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/f43ad972/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/f43ad972/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/f43ad972/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/f43ad972/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/f43ad972/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/f43ad972/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