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/09/19 20:01:25 UTC

[01/44] hadoop git commit: HADOOP-13606 swift FS to add a service load metadata file. Contributed by Steve Loughran [Forced Update!]

Repository: hadoop
Updated Branches:
  refs/heads/YARN-2915 b8d9062e4 -> 9abc7dafd (forced update)


HADOOP-13606 swift FS to add a service load metadata file. 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/53a12fa7
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/53a12fa7
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/53a12fa7

Branch: refs/heads/YARN-2915
Commit: 53a12fa721bb431f7d481aac7d245c93efb56153
Parents: ea0c2b8
Author: Steve Loughran <st...@apache.org>
Authored: Wed Sep 14 15:44:20 2016 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Wed Sep 14 15:45:04 2016 +0100

----------------------------------------------------------------------
 .../src/main/resources/core-default.xml             |  6 ------
 .../services/org.apache.hadoop.fs.FileSystem        | 16 ++++++++++++++++
 2 files changed, 16 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/53a12fa7/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
index 0a12017..fda8c9b 100644
--- a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
+++ b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
@@ -799,12 +799,6 @@
 </property>
 
 <property>
-  <name>fs.swift.impl</name>
-  <value>org.apache.hadoop.fs.swift.snative.SwiftNativeFileSystem</value>
-  <description>The implementation class of the OpenStack Swift Filesystem</description>
-</property>
-
-<property>
   <name>fs.automatic.close</name>
   <value>true</value>
   <description>By default, FileSystem instances are automatically closed at program

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53a12fa7/hadoop-tools/hadoop-openstack/src/main/resources/META-INF/services/org.apache.hadoop.fs.FileSystem
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-openstack/src/main/resources/META-INF/services/org.apache.hadoop.fs.FileSystem b/hadoop-tools/hadoop-openstack/src/main/resources/META-INF/services/org.apache.hadoop.fs.FileSystem
new file mode 100644
index 0000000..649ea31
--- /dev/null
+++ b/hadoop-tools/hadoop-openstack/src/main/resources/META-INF/services/org.apache.hadoop.fs.FileSystem
@@ -0,0 +1,16 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+org.apache.hadoop.fs.swift.snative.SwiftNativeFileSystem


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


[38/44] 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/0c0de593
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/0c0de593
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/0c0de593

Branch: refs/heads/YARN-2915
Commit: 0c0de59388c4020a9585a625e215beffa967cac1
Parents: 9313ae1
Author: Subru Krishnan <su...@apache.org>
Authored: Mon Aug 8 14:53:38 2016 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon Sep 19 12:58:34 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/0c0de593/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/0c0de593/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/0c0de593/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/0c0de593/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/0c0de593/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/0c0de593/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


[03/44] hadoop git commit: HADOOP-13218. Migrate other Hadoop side tests to prepare for removing WritableRPCEngine. Contributed by Wei Zhou and Kai Zheng

Posted by su...@apache.org.
HADOOP-13218. Migrate other Hadoop side tests to prepare for removing WritableRPCEngine. Contributed by Wei Zhou and Kai Zheng


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

Branch: refs/heads/YARN-2915
Commit: ea0c2b8b051a2d14927e8f314245442f30748dc8
Parents: 696bc0e
Author: Kai Zheng <ka...@intel.com>
Authored: Thu Sep 15 10:46:00 2016 +0800
Committer: Kai Zheng <ka...@intel.com>
Committed: Thu Sep 15 10:46:00 2016 +0800

----------------------------------------------------------------------
 .../apache/hadoop/ipc/ProtobufRpcEngine.java    |   5 +-
 .../main/java/org/apache/hadoop/ipc/RPC.java    |  13 +-
 .../main/java/org/apache/hadoop/ipc/Server.java |   4 +-
 .../hadoop/security/UserGroupInformation.java   |   4 +-
 .../org/apache/hadoop/ipc/RPCCallBenchmark.java |  38 +--
 .../hadoop/ipc/TestMultipleProtocolServer.java  | 236 +--------------
 .../apache/hadoop/ipc/TestRPCCallBenchmark.java |  13 -
 .../apache/hadoop/ipc/TestRPCCompatibility.java | 242 ++-------------
 .../apache/hadoop/ipc/TestRPCWaitForProxy.java  |  44 ++-
 .../java/org/apache/hadoop/ipc/TestRpcBase.java |  50 +++-
 .../java/org/apache/hadoop/ipc/TestSaslRPC.java |  74 +++--
 .../hadoop/security/TestDoAsEffectiveUser.java  | 291 +++++++------------
 .../security/TestUserGroupInformation.java      |  28 +-
 .../hadoop-common/src/test/proto/test.proto     |   4 +-
 .../src/test/proto/test_rpc_service.proto       |   4 +-
 .../hdfs/server/namenode/NameNodeRpcServer.java |   3 -
 .../TestClientProtocolWithDelegationToken.java  | 119 --------
 .../mapreduce/v2/hs/server/HSAdminServer.java   |   3 -
 18 files changed, 300 insertions(+), 875 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea0c2b8b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
index 83e4b9e..e68bfd4 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
@@ -60,7 +60,7 @@ public class ProtobufRpcEngine implements RpcEngine {
   private static final ThreadLocal<AsyncGet<Message, Exception>>
       ASYNC_RETURN_MESSAGE = new ThreadLocal<>();
 
-  static { // Register the rpcRequest deserializer for WritableRpcEngine 
+  static { // Register the rpcRequest deserializer for ProtobufRpcEngine
     org.apache.hadoop.ipc.Server.registerProtocolEngine(
         RPC.RpcKind.RPC_PROTOCOL_BUFFER, RpcProtobufRequest.class,
         new Server.ProtoBufRpcInvoker());
@@ -194,7 +194,8 @@ public class ProtobufRpcEngine implements RpcEngine {
       }
       
       if (args.length != 2) { // RpcController + Message
-        throw new ServiceException("Too many parameters for request. Method: ["
+        throw new ServiceException(
+            "Too many or few parameters for request. Method: ["
             + method.getName() + "]" + ", Expected: 2, Actual: "
             + args.length);
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea0c2b8b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RPC.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RPC.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RPC.java
index 3f68d63..a62748e 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RPC.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RPC.java
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.ipc;
 
+import java.io.IOException;
+import java.io.InterruptedIOException;
 import java.lang.reflect.Field;
 import java.lang.reflect.InvocationHandler;
 import java.lang.reflect.Proxy;
@@ -26,7 +28,6 @@ import java.net.ConnectException;
 import java.net.InetSocketAddress;
 import java.net.NoRouteToHostException;
 import java.net.SocketTimeoutException;
-import java.io.*;
 import java.io.Closeable;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -37,11 +38,12 @@ import java.util.concurrent.atomic.AtomicBoolean;
 
 import javax.net.SocketFactory;
 
-import org.apache.commons.logging.*;
-
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.HadoopIllegalArgumentException;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
-import org.apache.hadoop.io.*;
+import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.retry.RetryPolicy;
 import org.apache.hadoop.ipc.Client.ConnectionId;
 import org.apache.hadoop.ipc.protobuf.ProtocolInfoProtos.ProtocolInfoService;
@@ -54,7 +56,6 @@ import org.apache.hadoop.security.token.SecretManager;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.*;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.Time;
 
@@ -87,7 +88,7 @@ public class RPC {
     RPC_WRITABLE ((short) 2),        // Use WritableRpcEngine 
     RPC_PROTOCOL_BUFFER ((short) 3); // Use ProtobufRpcEngine
     final static short MAX_INDEX = RPC_PROTOCOL_BUFFER.value; // used for array size
-    public final short value; //TODO make it private
+    private final short value;
 
     RpcKind(short val) {
       this.value = val;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea0c2b8b/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 f20ba94..531d574 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
@@ -237,14 +237,14 @@ public abstract class Server {
   static class RpcKindMapValue {
     final Class<? extends Writable> rpcRequestWrapperClass;
     final RpcInvoker rpcInvoker;
+
     RpcKindMapValue (Class<? extends Writable> rpcRequestWrapperClass,
           RpcInvoker rpcInvoker) {
       this.rpcInvoker = rpcInvoker;
       this.rpcRequestWrapperClass = rpcRequestWrapperClass;
     }   
   }
-  static Map<RPC.RpcKind, RpcKindMapValue> rpcKindMap = new
-      HashMap<RPC.RpcKind, RpcKindMapValue>(4);
+  static Map<RPC.RpcKind, RpcKindMapValue> rpcKindMap = new HashMap<>(4);
   
   
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea0c2b8b/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 0ad9abc..ed3a9d0 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
@@ -730,7 +730,7 @@ public class UserGroupInformation {
    * 
    * @param user                The principal name to load from the ticket
    *                            cache
-   * @param ticketCachePath     the path to the ticket cache file
+   * @param ticketCache     the path to the ticket cache file
    *
    * @throws IOException        if the kerberos login fails
    */
@@ -790,7 +790,7 @@ public class UserGroupInformation {
   /**
    * Create a UserGroupInformation from a Subject with Kerberos principal.
    *
-   * @param user                The KerberosPrincipal to use in UGI
+   * @param subject             The KerberosPrincipal to use in UGI
    *
    * @throws IOException        if the kerberos login fails
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea0c2b8b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/RPCCallBenchmark.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/RPCCallBenchmark.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/RPCCallBenchmark.java
index eb7b949..9356dab 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/RPCCallBenchmark.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/RPCCallBenchmark.java
@@ -17,13 +17,8 @@
  */
 package org.apache.hadoop.ipc;
 
-import java.io.IOException;
-import java.lang.management.ManagementFactory;
-import java.lang.management.ThreadMXBean;
-import java.net.InetSocketAddress;
-import java.security.PrivilegedExceptionAction;
-import java.util.concurrent.atomic.AtomicLong;
-
+import com.google.common.base.Joiner;
+import com.google.protobuf.BlockingService;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.CommandLineParser;
 import org.apache.commons.cli.GnuParser;
@@ -34,7 +29,6 @@ import org.apache.commons.cli.ParseException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.ipc.RPC.Server;
-import org.apache.hadoop.ipc.TestRPC.TestProtocol;
 import org.apache.hadoop.ipc.protobuf.TestProtos.EchoRequestProto;
 import org.apache.hadoop.ipc.protobuf.TestProtos.EchoResponseProto;
 import org.apache.hadoop.ipc.protobuf.TestRpcServiceProtos.TestProtobufRpcProto;
@@ -45,8 +39,12 @@ import org.apache.hadoop.test.MultithreadedTestUtil.TestContext;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 
-import com.google.common.base.Joiner;
-import com.google.protobuf.BlockingService;
+import java.io.IOException;
+import java.lang.management.ManagementFactory;
+import java.lang.management.ThreadMXBean;
+import java.net.InetSocketAddress;
+import java.security.PrivilegedExceptionAction;
+import java.util.concurrent.atomic.AtomicLong;
 
 /**
  * Benchmark for protobuf RPC.
@@ -68,7 +66,7 @@ public class RPCCallBenchmark extends TestRpcBase implements Tool {
     public int secondsToRun = 15;
     private int msgSize = 1024;
     public Class<? extends RpcEngine> rpcEngine =
-      WritableRpcEngine.class;
+        ProtobufRpcEngine.class;
     
     private MyOptions(String args[]) {
       try {
@@ -135,7 +133,7 @@ public class RPCCallBenchmark extends TestRpcBase implements Tool {
       
       opts.addOption(
           OptionBuilder.withLongOpt("engine").hasArg(true)
-          .withArgName("writable|protobuf")
+          .withArgName("protobuf")
           .withDescription("engine to use")
           .create('e'));
       
@@ -184,8 +182,6 @@ public class RPCCallBenchmark extends TestRpcBase implements Tool {
         String eng = line.getOptionValue('e');
         if ("protobuf".equals(eng)) {
           rpcEngine = ProtobufRpcEngine.class;
-        } else if ("writable".equals(eng)) {
-          rpcEngine = WritableRpcEngine.class;
         } else {
           throw new ParseException("invalid engine: " + eng);
         }
@@ -237,11 +233,6 @@ public class RPCCallBenchmark extends TestRpcBase implements Tool {
       server = new RPC.Builder(conf).setProtocol(TestRpcService.class)
           .setInstance(service).setBindAddress(opts.host).setPort(opts.getPort())
           .setNumHandlers(opts.serverThreads).setVerbose(false).build();
-    } else if (opts.rpcEngine == WritableRpcEngine.class) {
-      server = new RPC.Builder(conf).setProtocol(TestProtocol.class)
-          .setInstance(new TestRPC.TestImpl()).setBindAddress(opts.host)
-          .setPort(opts.getPort()).setNumHandlers(opts.serverThreads)
-          .setVerbose(false).build();
     } else {
       throw new RuntimeException("Bad engine: " + opts.rpcEngine);
     }
@@ -399,15 +390,6 @@ public class RPCCallBenchmark extends TestRpcBase implements Tool {
           return responseProto.getMessage();
         }
       };
-    } else if (opts.rpcEngine == WritableRpcEngine.class) {
-      final TestProtocol proxy = RPC.getProxy(
-          TestProtocol.class, TestProtocol.versionID, addr, conf);
-      return new RpcServiceWrapper() {
-        @Override
-        public String doEcho(String msg) throws Exception {
-          return proxy.echo(msg);
-        }
-      };
     } else {
       throw new RuntimeException("unsupported engine: " + opts.rpcEngine);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea0c2b8b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestMultipleProtocolServer.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestMultipleProtocolServer.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestMultipleProtocolServer.java
index 8b419e3..10e23ba 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestMultipleProtocolServer.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestMultipleProtocolServer.java
@@ -17,252 +17,28 @@
  */
 package org.apache.hadoop.ipc;
 
-import java.io.IOException;
-import java.net.InetSocketAddress;
-
-import org.junit.Assert;
-
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.ipc.protobuf.TestRpcServiceProtos.TestProtobufRpcProto;
-import org.apache.hadoop.net.NetUtils;
-import org.junit.Before;
 import org.junit.After;
+import org.junit.Before;
 import org.junit.Test;
-import com.google.protobuf.BlockingService;
 
 public class TestMultipleProtocolServer extends TestRpcBase {
-  private static InetSocketAddress addr;
-  private static RPC.Server server;
-
-  private static Configuration conf = new Configuration();
-  
-  
-  @ProtocolInfo(protocolName="Foo")
-  interface Foo0 extends VersionedProtocol {
-    public static final long versionID = 0L;
-    String ping() throws IOException;
-    
-  }
-  
-  @ProtocolInfo(protocolName="Foo")
-  interface Foo1 extends VersionedProtocol {
-    public static final long versionID = 1L;
-    String ping() throws IOException;
-    String ping2() throws IOException;
-  }
-  
-  @ProtocolInfo(protocolName="Foo")
-  interface FooUnimplemented extends VersionedProtocol {
-    public static final long versionID = 2L;
-    String ping() throws IOException;  
-  }
-  
-  interface Mixin extends VersionedProtocol{
-    public static final long versionID = 0L;
-    void hello() throws IOException;
-  }
-
-  interface Bar extends Mixin {
-    public static final long versionID = 0L;
-    int echo(int i) throws IOException;
-  }
-  
-  class Foo0Impl implements Foo0 {
-
-    @Override
-    public long getProtocolVersion(String protocol, long clientVersion)
-        throws IOException {
-      return Foo0.versionID;
-    }
-
-    @SuppressWarnings("unchecked")
-    @Override
-    public ProtocolSignature getProtocolSignature(String protocol,
-        long clientVersion, int clientMethodsHash) throws IOException {
-      Class<? extends VersionedProtocol> inter;
-      try {
-        inter = (Class<? extends VersionedProtocol>)getClass().
-                                          getGenericInterfaces()[0];
-      } catch (Exception e) {
-        throw new IOException(e);
-      }
-      return ProtocolSignature.getProtocolSignature(clientMethodsHash, 
-          getProtocolVersion(protocol, clientVersion), inter);
-    }
-
-    @Override
-    public String ping() {
-      return "Foo0";     
-    }
-    
-  }
-  
-  class Foo1Impl implements Foo1 {
-
-    @Override
-    public long getProtocolVersion(String protocol, long clientVersion)
-        throws IOException {
-      return Foo1.versionID;
-    }
-
-    @SuppressWarnings("unchecked")
-    @Override
-    public ProtocolSignature getProtocolSignature(String protocol,
-        long clientVersion, int clientMethodsHash) throws IOException {
-      Class<? extends VersionedProtocol> inter;
-      try {
-        inter = (Class<? extends VersionedProtocol>)getClass().
-                                        getGenericInterfaces()[0];
-      } catch (Exception e) {
-        throw new IOException(e);
-      }
-      return ProtocolSignature.getProtocolSignature(clientMethodsHash, 
-          getProtocolVersion(protocol, clientVersion), inter);
-    }
-
-    @Override
-    public String ping() {
-      return "Foo1";
-    }
 
-    @Override
-    public String ping2() {
-      return "Foo1";
-      
-    }
-    
-  }
-
-  
-  class BarImpl implements Bar {
-
-    @Override
-    public long getProtocolVersion(String protocol, long clientVersion)
-        throws IOException {
-      return Bar.versionID;
-    }
-
-    @SuppressWarnings("unchecked")
-    @Override
-    public ProtocolSignature getProtocolSignature(String protocol,
-        long clientVersion, int clientMethodsHash) throws IOException {
-      Class<? extends VersionedProtocol> inter;
-      try {
-        inter = (Class<? extends VersionedProtocol>)getClass().
-                                          getGenericInterfaces()[0];
-      } catch (Exception e) {
-        throw new IOException(e);
-      }
-      return ProtocolSignature.getProtocolSignature(clientMethodsHash, 
-          getProtocolVersion(protocol, clientVersion), inter);
-    }
-
-    @Override
-    public int echo(int i) {
-      return i;
-    }
-
-    @Override
-    public void hello() {
+  private static RPC.Server server;
 
-      
-    }
-  }
   @Before
   public void setUp() throws Exception {
-    // create a server with two handlers
-    server = new RPC.Builder(conf).setProtocol(Foo0.class)
-        .setInstance(new Foo0Impl()).setBindAddress(ADDRESS).setPort(0)
-        .setNumHandlers(2).setVerbose(false).build();
-    server.addProtocol(RPC.RpcKind.RPC_WRITABLE, Foo1.class, new Foo1Impl());
-    server.addProtocol(RPC.RpcKind.RPC_WRITABLE, Bar.class, new BarImpl());
-    server.addProtocol(RPC.RpcKind.RPC_WRITABLE, Mixin.class, new BarImpl());
-    
-    
-    // Add Protobuf server
-    // Create server side implementation
-    PBServerImpl pbServerImpl = new PBServerImpl();
-    BlockingService service = TestProtobufRpcProto
-        .newReflectiveBlockingService(pbServerImpl);
-    server.addProtocol(RPC.RpcKind.RPC_PROTOCOL_BUFFER, TestRpcService.class,
-        service);
-    server.start();
-    addr = NetUtils.getConnectAddress(server);
+    super.setupConf();
+
+    server = setupTestServer(conf, 2);
   }
-  
+
   @After
   public void tearDown() throws Exception {
     server.stop();
   }
 
-  @Test
-  public void test1() throws IOException {
-    ProtocolProxy<?> proxy;
-    proxy = RPC.getProtocolProxy(Foo0.class, Foo0.versionID, addr, conf);
 
-    Foo0 foo0 = (Foo0)proxy.getProxy(); 
-    Assert.assertEquals("Foo0", foo0.ping());
-    
-    
-    proxy = RPC.getProtocolProxy(Foo1.class, Foo1.versionID, addr, conf);
-    
-    
-    Foo1 foo1 = (Foo1)proxy.getProxy(); 
-    Assert.assertEquals("Foo1", foo1.ping());
-    Assert.assertEquals("Foo1", foo1.ping());
-    
-    
-    proxy = RPC.getProtocolProxy(Bar.class, Foo1.versionID, addr, conf);
-    
-    
-    Bar bar = (Bar)proxy.getProxy(); 
-    Assert.assertEquals(99, bar.echo(99));
-    
-    // Now test Mixin class method
-    
-    Mixin mixin = bar;
-    mixin.hello();
-  }
-  
-  
-  // Server does not implement the FooUnimplemented version of protocol Foo.
-  // See that calls to it fail.
-  @Test(expected=IOException.class)
-  public void testNonExistingProtocol() throws IOException {
-    ProtocolProxy<?> proxy;
-    proxy = RPC.getProtocolProxy(FooUnimplemented.class, 
-        FooUnimplemented.versionID, addr, conf);
-
-    FooUnimplemented foo = (FooUnimplemented)proxy.getProxy(); 
-    foo.ping();
-  }
-
-  /**
-   * getProtocolVersion of an unimplemented version should return highest version
-   * Similarly getProtocolSignature should work.
-   * @throws IOException
-   */
-  @Test
-  public void testNonExistingProtocol2() throws IOException {
-    ProtocolProxy<?> proxy;
-    proxy = RPC.getProtocolProxy(FooUnimplemented.class, 
-        FooUnimplemented.versionID, addr, conf);
-
-    FooUnimplemented foo = (FooUnimplemented)proxy.getProxy(); 
-    Assert.assertEquals(Foo1.versionID, 
-        foo.getProtocolVersion(RPC.getProtocolName(FooUnimplemented.class), 
-        FooUnimplemented.versionID));
-    foo.getProtocolSignature(RPC.getProtocolName(FooUnimplemented.class), 
-        FooUnimplemented.versionID, 0);
-  }
-  
-  @Test(expected=IOException.class)
-  public void testIncorrectServerCreation() throws IOException {
-    new RPC.Builder(conf).setProtocol(Foo1.class).setInstance(new Foo0Impl())
-        .setBindAddress(ADDRESS).setPort(0).setNumHandlers(2).setVerbose(false)
-        .build();
-  } 
-  
   // Now test a PB service - a server  hosts both PB and Writable Rpcs.
   @Test
   public void testPBService() throws Exception {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea0c2b8b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPCCallBenchmark.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPCCallBenchmark.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPCCallBenchmark.java
index 969f728..6d83d7d 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPCCallBenchmark.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPCCallBenchmark.java
@@ -26,19 +26,6 @@ import org.junit.Test;
 public class TestRPCCallBenchmark {
 
   @Test(timeout=20000)
-  public void testBenchmarkWithWritable() throws Exception {
-    int rc = ToolRunner.run(new RPCCallBenchmark(),
-        new String[] {
-      "--clientThreads", "30",
-      "--serverThreads", "30",
-      "--time", "5",
-      "--serverReaderThreads", "4",
-      "--messageSize", "1024",
-      "--engine", "writable"});
-    assertEquals(0, rc);
-  }
-  
-  @Test(timeout=20000)
   public void testBenchmarkWithProto() throws Exception {
     int rc = ToolRunner.run(new RPCCallBenchmark(),
         new String[] {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea0c2b8b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPCCompatibility.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPCCompatibility.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPCCompatibility.java
index 2ac2be9..a06d9fd 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPCCompatibility.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPCCompatibility.java
@@ -18,28 +18,20 @@
 
 package org.apache.hadoop.ipc;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.fail;
-
-import java.io.IOException;
-import java.lang.reflect.Method;
-import java.net.InetSocketAddress;
-
-import org.junit.Assert;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.ipc.protobuf.ProtocolInfoProtos.GetProtocolSignatureRequestProto;
-import org.apache.hadoop.ipc.protobuf.ProtocolInfoProtos.GetProtocolSignatureResponseProto;
-import org.apache.hadoop.ipc.protobuf.ProtocolInfoProtos.ProtocolSignatureProto;
-import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcResponseHeaderProto.RpcErrorCodeProto;
-import org.apache.hadoop.net.NetUtils;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
+import java.io.IOException;
+import java.lang.reflect.Method;
+import java.net.InetSocketAddress;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
 /** Unit test for supporting method-name based compatible RPCs. */
 public class TestRPCCompatibility {
   private static final String ADDRESS = "0.0.0.0";
@@ -49,7 +41,7 @@ public class TestRPCCompatibility {
 
   public static final Log LOG =
     LogFactory.getLog(TestRPCCompatibility.class);
-  
+
   private static Configuration conf = new Configuration();
 
   public interface TestProtocol0 extends VersionedProtocol {
@@ -120,6 +112,21 @@ public class TestRPCCompatibility {
   @Before
   public void setUp() {
     ProtocolSignature.resetCache();
+
+    RPC.setProtocolEngine(conf,
+        TestProtocol0.class, ProtobufRpcEngine.class);
+
+    RPC.setProtocolEngine(conf,
+        TestProtocol1.class, ProtobufRpcEngine.class);
+
+    RPC.setProtocolEngine(conf,
+        TestProtocol2.class, ProtobufRpcEngine.class);
+
+    RPC.setProtocolEngine(conf,
+        TestProtocol3.class, ProtobufRpcEngine.class);
+
+    RPC.setProtocolEngine(conf,
+        TestProtocol4.class, ProtobufRpcEngine.class);
   }
   
   @After
@@ -133,117 +140,7 @@ public class TestRPCCompatibility {
       server = null;
     }
   }
-  
-  @Test  // old client vs new server
-  public void testVersion0ClientVersion1Server() throws Exception {
-    // create a server with two handlers
-    TestImpl1 impl = new TestImpl1();
-    server = new RPC.Builder(conf).setProtocol(TestProtocol1.class)
-        .setInstance(impl).setBindAddress(ADDRESS).setPort(0).setNumHandlers(2)
-        .setVerbose(false).build();
-    server.addProtocol(RPC.RpcKind.RPC_WRITABLE, TestProtocol0.class, impl);
-    server.start();
-    addr = NetUtils.getConnectAddress(server);
-
-    proxy = RPC.getProtocolProxy(
-        TestProtocol0.class, TestProtocol0.versionID, addr, conf);
-
-    TestProtocol0 proxy0 = (TestProtocol0)proxy.getProxy();
-    proxy0.ping();
-  }
-  
-  @Test  // old client vs new server
-  public void testVersion1ClientVersion0Server() throws Exception {
-    // create a server with two handlers
-    server = new RPC.Builder(conf).setProtocol(TestProtocol0.class)
-        .setInstance(new TestImpl0()).setBindAddress(ADDRESS).setPort(0)
-        .setNumHandlers(2).setVerbose(false).build();
-    server.start();
-    addr = NetUtils.getConnectAddress(server);
-
-    proxy = RPC.getProtocolProxy(
-        TestProtocol1.class, TestProtocol1.versionID, addr, conf);
-
-    TestProtocol1 proxy1 = (TestProtocol1)proxy.getProxy();
-    proxy1.ping();
-    try {
-      proxy1.echo("hello");
-      fail("Echo should fail");
-    } catch(IOException e) {
-    }
-  }
-  
-  private class Version2Client {
 
-    private TestProtocol2 proxy2;
-    private ProtocolProxy<TestProtocol2> serverInfo;
-    
-    private Version2Client() throws IOException {
-      serverInfo =  RPC.getProtocolProxy(
-          TestProtocol2.class, TestProtocol2.versionID, addr, conf);
-      proxy2 = serverInfo.getProxy();
-    }
-    
-    public int echo(int value) throws IOException, NumberFormatException {
-      if (serverInfo.isMethodSupported("echo", int.class)) {
-System.out.println("echo int is supported");
-        return -value;  // use version 3 echo long
-      } else { // server is version 2
-System.out.println("echo int is NOT supported");
-        return Integer.parseInt(proxy2.echo(String.valueOf(value)));
-      }
-    }
-
-    public String echo(String value) throws IOException {
-      return proxy2.echo(value);
-    }
-
-    public void ping() throws IOException {
-      proxy2.ping();
-    }
-  }
-
-  @Test // Compatible new client & old server
-  public void testVersion2ClientVersion1Server() throws Exception {
-    // create a server with two handlers
-    TestImpl1 impl = new TestImpl1();
-    server = new RPC.Builder(conf).setProtocol(TestProtocol1.class)
-        .setInstance(impl).setBindAddress(ADDRESS).setPort(0).setNumHandlers(2)
-        .setVerbose(false).build();
-    server.addProtocol(RPC.RpcKind.RPC_WRITABLE, TestProtocol0.class, impl);
-    server.start();
-    addr = NetUtils.getConnectAddress(server);
-
-
-    Version2Client client = new Version2Client();
-    client.ping();
-    assertEquals("hello", client.echo("hello"));
-    
-    // echo(int) is not supported by server, so returning 3
-    // This verifies that echo(int) and echo(String)'s hash codes are different
-    assertEquals(3, client.echo(3));
-  }
-  
-  @Test // equal version client and server
-  public void testVersion2ClientVersion2Server() throws Exception {
-    // create a server with two handlers
-    TestImpl2 impl = new TestImpl2();
-    server = new RPC.Builder(conf).setProtocol(TestProtocol2.class)
-        .setInstance(impl).setBindAddress(ADDRESS).setPort(0).setNumHandlers(2)
-        .setVerbose(false).build();
-    server.addProtocol(RPC.RpcKind.RPC_WRITABLE, TestProtocol0.class, impl);
-    server.start();
-    addr = NetUtils.getConnectAddress(server);
-
-    Version2Client client = new Version2Client();
-
-    client.ping();
-    assertEquals("hello", client.echo("hello"));
-    
-    // now that echo(int) is supported by the server, echo(int) should return -3
-    assertEquals(-3, client.echo(3));
-  }
-  
   public interface TestProtocol3 {
     int echo(String value);
     int echo(int value);
@@ -297,97 +194,4 @@ System.out.println("echo int is NOT supported");
     @Override
     int echo(int value)  throws IOException;
   }
-  
-  @Test
-  public void testVersionMismatch() throws IOException {
-    server = new RPC.Builder(conf).setProtocol(TestProtocol2.class)
-        .setInstance(new TestImpl2()).setBindAddress(ADDRESS).setPort(0)
-        .setNumHandlers(2).setVerbose(false).build();
-    server.start();
-    addr = NetUtils.getConnectAddress(server);
-
-    TestProtocol4 proxy = RPC.getProxy(TestProtocol4.class,
-        TestProtocol4.versionID, addr, conf);
-    try {
-      proxy.echo(21);
-      fail("The call must throw VersionMismatch exception");
-    } catch (RemoteException ex) {
-      Assert.assertEquals(RPC.VersionMismatch.class.getName(), 
-          ex.getClassName());
-      Assert.assertTrue(ex.getErrorCode().equals(
-          RpcErrorCodeProto.ERROR_RPC_VERSION_MISMATCH));
-    }  catch (IOException ex) {
-      fail("Expected version mismatch but got " + ex);
-    }
-  }
-  
-  @Test
-  public void testIsMethodSupported() throws IOException {
-    server = new RPC.Builder(conf).setProtocol(TestProtocol2.class)
-        .setInstance(new TestImpl2()).setBindAddress(ADDRESS).setPort(0)
-        .setNumHandlers(2).setVerbose(false).build();
-    server.start();
-    addr = NetUtils.getConnectAddress(server);
-
-    TestProtocol2 proxy = RPC.getProxy(TestProtocol2.class,
-        TestProtocol2.versionID, addr, conf);
-    boolean supported = RpcClientUtil.isMethodSupported(proxy,
-        TestProtocol2.class, RPC.RpcKind.RPC_WRITABLE,
-        RPC.getProtocolVersion(TestProtocol2.class), "echo");
-    Assert.assertTrue(supported);
-    supported = RpcClientUtil.isMethodSupported(proxy,
-        TestProtocol2.class, RPC.RpcKind.RPC_PROTOCOL_BUFFER,
-        RPC.getProtocolVersion(TestProtocol2.class), "echo");
-    Assert.assertFalse(supported);
-  }
-
-  /**
-   * Verify that ProtocolMetaInfoServerSideTranslatorPB correctly looks up
-   * the server registry to extract protocol signatures and versions.
-   */
-  @Test
-  public void testProtocolMetaInfoSSTranslatorPB() throws Exception {
-    TestImpl1 impl = new TestImpl1();
-    server = new RPC.Builder(conf).setProtocol(TestProtocol1.class)
-        .setInstance(impl).setBindAddress(ADDRESS).setPort(0).setNumHandlers(2)
-        .setVerbose(false).build();
-    server.addProtocol(RPC.RpcKind.RPC_WRITABLE, TestProtocol0.class, impl);
-    server.start();
-
-    ProtocolMetaInfoServerSideTranslatorPB xlator = 
-        new ProtocolMetaInfoServerSideTranslatorPB(server);
-
-    GetProtocolSignatureResponseProto resp = xlator.getProtocolSignature(
-        null,
-        createGetProtocolSigRequestProto(TestProtocol1.class,
-            RPC.RpcKind.RPC_PROTOCOL_BUFFER));
-    //No signatures should be found
-    Assert.assertEquals(0, resp.getProtocolSignatureCount());
-    resp = xlator.getProtocolSignature(
-        null,
-        createGetProtocolSigRequestProto(TestProtocol1.class,
-            RPC.RpcKind.RPC_WRITABLE));
-    Assert.assertEquals(1, resp.getProtocolSignatureCount());
-    ProtocolSignatureProto sig = resp.getProtocolSignatureList().get(0);
-    Assert.assertEquals(TestProtocol1.versionID, sig.getVersion());
-    boolean found = false;
-    int expected = ProtocolSignature.getFingerprint(TestProtocol1.class
-        .getMethod("echo", String.class));
-    for (int m : sig.getMethodsList()) {
-      if (expected == m) {
-        found = true;
-        break;
-      }
-    }
-    Assert.assertTrue(found);
-  }
-  
-  private GetProtocolSignatureRequestProto createGetProtocolSigRequestProto(
-      Class<?> protocol, RPC.RpcKind rpcKind) {
-    GetProtocolSignatureRequestProto.Builder builder = 
-        GetProtocolSignatureRequestProto.newBuilder();
-    builder.setProtocol(protocol.getName());
-    builder.setRpcKind(rpcKind.toString());
-    return builder.build();
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea0c2b8b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPCWaitForProxy.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPCWaitForProxy.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPCWaitForProxy.java
index 5807998..d810fe3 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPCWaitForProxy.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPCWaitForProxy.java
@@ -18,9 +18,8 @@
 package org.apache.hadoop.ipc;
 
 import org.apache.hadoop.conf.Configuration;
-import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.*;
-import org.apache.hadoop.ipc.TestRPC.TestProtocol;
 import org.junit.Assert;
+import org.junit.Before;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -30,30 +29,39 @@ import java.net.ConnectException;
 import java.net.InetSocketAddress;
 import java.nio.channels.ClosedByInterruptException;
 
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SOCKET_TIMEOUTS_KEY;
+
 /**
  * tests that the proxy can be interrupted
  */
-public class TestRPCWaitForProxy extends Assert {
-  private static final String ADDRESS = "0.0.0.0";
+public class TestRPCWaitForProxy extends TestRpcBase {
   private static final Logger
       LOG = LoggerFactory.getLogger(TestRPCWaitForProxy.class);
 
   private static final Configuration conf = new Configuration();
 
+  @Before
+  public void setupProtocolEngine() {
+    RPC.setProtocolEngine(conf, TestRpcService.class,
+        ProtobufRpcEngine.class);
+  }
+
   /**
    * This tests that the time-bounded wait for a proxy operation works, and
    * times out.
    *
    * @throws Throwable any exception other than that which was expected
    */
-  @Test(timeout = 10000)
+  @Test(timeout = 50000)
   public void testWaitForProxy() throws Throwable {
     RpcThread worker = new RpcThread(0);
     worker.start();
     worker.join();
     Throwable caught = worker.getCaught();
-    assertNotNull("No exception was raised", caught);
-    if (!(caught instanceof ConnectException)) {
+    Throwable cause = caught.getCause();
+    Assert.assertNotNull("No exception was raised", cause);
+    if (!(cause instanceof ConnectException)) {
       throw caught;
     }
   }
@@ -69,11 +77,11 @@ public class TestRPCWaitForProxy extends Assert {
     RpcThread worker = new RpcThread(100);
     worker.start();
     Thread.sleep(1000);
-    assertTrue("worker hasn't started", worker.waitStarted);
+    Assert.assertTrue("worker hasn't started", worker.waitStarted);
     worker.interrupt();
     worker.join();
     Throwable caught = worker.getCaught();
-    assertNotNull("No exception was raised", caught);
+    Assert.assertNotNull("No exception was raised", caught);
     // looking for the root cause here, which can be wrapped
     // as part of the NetUtils work. Having this test look
     // a the type of exception there would be brittle to improvements
@@ -82,6 +90,8 @@ public class TestRPCWaitForProxy extends Assert {
     if (cause == null) {
       // no inner cause, use outer exception as root cause.
       cause = caught;
+    } else if (cause.getCause() != null) {
+      cause = cause.getCause();
     }
     if (!(cause instanceof InterruptedIOException)
         && !(cause instanceof ClosedByInterruptException)) {
@@ -112,12 +122,16 @@ public class TestRPCWaitForProxy extends Assert {
             IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SOCKET_TIMEOUTS_KEY,
             connectRetries);
         waitStarted = true;
-        TestProtocol proxy = RPC.waitForProxy(TestProtocol.class,
-            TestProtocol.versionID,
-            new InetSocketAddress(ADDRESS, 20),
-            config,
-            15000L);
-        proxy.echo("");
+
+        short invalidPort = 20;
+        InetSocketAddress invalidAddress = new InetSocketAddress(ADDRESS,
+            invalidPort);
+        TestRpcBase.TestRpcService proxy = RPC.getProxy(
+            TestRpcBase.TestRpcService.class,
+            1L, invalidAddress, conf);
+        // Test echo method
+        proxy.echo(null, newEchoRequest("hello"));
+
       } catch (Throwable throwable) {
         caught = throwable;
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea0c2b8b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRpcBase.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRpcBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRpcBase.java
index e991405..3c5885f 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRpcBase.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRpcBase.java
@@ -109,7 +109,8 @@ public class TestRpcBase {
     return setupTestServer(builder);
   }
 
-  protected static RPC.Server setupTestServer(RPC.Builder builder) throws IOException {
+  protected static RPC.Server setupTestServer(
+      RPC.Builder builder) throws IOException {
     RPC.Server server = builder.build();
 
     server.start();
@@ -190,17 +191,21 @@ public class TestRpcBase {
     public TestTokenIdentifier() {
       this(new Text(), new Text());
     }
+
     public TestTokenIdentifier(Text tokenid) {
       this(tokenid, new Text());
     }
+
     public TestTokenIdentifier(Text tokenid, Text realUser) {
       this.tokenid = tokenid == null ? new Text() : tokenid;
       this.realUser = realUser == null ? new Text() : realUser;
     }
+
     @Override
     public Text getKind() {
       return KIND_NAME;
     }
+
     @Override
     public UserGroupInformation getUser() {
       if (realUser.toString().isEmpty()) {
@@ -218,6 +223,7 @@ public class TestRpcBase {
       tokenid.readFields(in);
       realUser.readFields(in);
     }
+
     @Override
     public void write(DataOutput out) throws IOException {
       tokenid.write(out);
@@ -249,7 +255,7 @@ public class TestRpcBase {
     @SuppressWarnings("unchecked")
     @Override
     public Token<TestTokenIdentifier> selectToken(Text service,
-                                                  Collection<Token<? extends TokenIdentifier>> tokens) {
+                      Collection<Token<? extends TokenIdentifier>> tokens) {
       if (service == null) {
         return null;
       }
@@ -403,19 +409,17 @@ public class TestRpcBase {
     }
 
     @Override
-    public TestProtos.AuthUserResponseProto getAuthUser(
+    public TestProtos.UserResponseProto getAuthUser(
         RpcController controller, TestProtos.EmptyRequestProto request)
         throws ServiceException {
-      UserGroupInformation authUser = null;
+      UserGroupInformation authUser;
       try {
         authUser = UserGroupInformation.getCurrentUser();
       } catch (IOException e) {
         throw new ServiceException(e);
       }
 
-      return TestProtos.AuthUserResponseProto.newBuilder()
-          .setAuthUser(authUser.getUserName())
-          .build();
+      return newUserResponse(authUser.getUserName());
     }
 
     @Override
@@ -447,6 +451,34 @@ public class TestRpcBase {
 
       return TestProtos.EmptyResponseProto.newBuilder().build();
     }
+
+    @Override
+    public TestProtos.UserResponseProto getCurrentUser(
+        RpcController controller,
+        TestProtos.EmptyRequestProto request) throws ServiceException {
+      String user;
+      try {
+        user = UserGroupInformation.getCurrentUser().toString();
+      } catch (IOException e) {
+        throw new ServiceException("Failed to get current user", e);
+      }
+
+      return newUserResponse(user);
+    }
+
+    @Override
+    public TestProtos.UserResponseProto getServerRemoteUser(
+        RpcController controller,
+        TestProtos.EmptyRequestProto request) throws ServiceException {
+      String serverRemoteUser = Server.getRemoteUser().toString();
+      return newUserResponse(serverRemoteUser);
+    }
+
+    private TestProtos.UserResponseProto newUserResponse(String user) {
+      return TestProtos.UserResponseProto.newBuilder()
+          .setUser(user)
+          .build();
+    }
   }
 
   protected static TestProtos.EmptyRequestProto newEmptyRequest() {
@@ -493,8 +525,4 @@ public class TestRpcBase {
     }
     return null;
   }
-
-  protected static String convert(TestProtos.AuthUserResponseProto response) {
-    return response.getAuthUser();
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea0c2b8b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestSaslRPC.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestSaslRPC.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestSaslRPC.java
index 72371a7..c48ff2e 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestSaslRPC.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestSaslRPC.java
@@ -45,30 +45,55 @@ import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
 
-import javax.security.auth.callback.*;
-import javax.security.sasl.*;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.CallbackHandler;
+import javax.security.auth.callback.NameCallback;
+import javax.security.auth.callback.PasswordCallback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.sasl.AuthorizeCallback;
+import javax.security.sasl.Sasl;
+import javax.security.sasl.SaslClient;
+import javax.security.sasl.SaslException;
+import javax.security.sasl.SaslServer;
 import java.io.IOException;
 import java.lang.annotation.Annotation;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.security.PrivilegedExceptionAction;
 import java.security.Security;
-import java.util.*;
-import java.util.concurrent.*;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.regex.Pattern;
 
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_RPC_PROTECTION;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION;
-import static org.apache.hadoop.security.SaslRpcServer.AuthMethod.*;
-import static org.junit.Assert.*;
+import static org.apache.hadoop.security.SaslRpcServer.AuthMethod.KERBEROS;
+import static org.apache.hadoop.security.SaslRpcServer.AuthMethod.SIMPLE;
+import static org.apache.hadoop.security.SaslRpcServer.AuthMethod.TOKEN;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNotSame;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 /** Unit tests for using Sasl over RPC. */
 @RunWith(Parameterized.class)
 public class TestSaslRPC extends TestRpcBase {
   @Parameters
   public static Collection<Object[]> data() {
-    Collection<Object[]> params = new ArrayList<Object[]>();
+    Collection<Object[]> params = new ArrayList<>();
     for (QualityOfProtection qop : QualityOfProtection.values()) {
       params.add(new Object[]{ new QualityOfProtection[]{qop},qop, null });
     }
@@ -114,7 +139,7 @@ public class TestSaslRPC extends TestRpcBase {
     NONE(),
     VALID(),
     INVALID(),
-    OTHER();
+    OTHER()
   }
   
   @BeforeClass
@@ -230,7 +255,7 @@ public class TestSaslRPC extends TestRpcBase {
       final Server server = setupTestServer(conf, 5, sm);
       doDigestRpc(server, sm);
     } finally {
-      SecurityUtil.setSecurityInfoProviders(new SecurityInfo[0]);
+      SecurityUtil.setSecurityInfoProviders();
     }
   }
 
@@ -259,7 +284,7 @@ public class TestSaslRPC extends TestRpcBase {
     addr = NetUtils.getConnectAddress(server);
     TestTokenIdentifier tokenId = new TestTokenIdentifier(new Text(current
         .getUserName()));
-    Token<TestTokenIdentifier> token = new Token<TestTokenIdentifier>(tokenId, sm);
+    Token<TestTokenIdentifier> token = new Token<>(tokenId, sm);
     SecurityUtil.setTokenService(token, addr);
     current.addToken(token);
 
@@ -296,8 +321,8 @@ public class TestSaslRPC extends TestRpcBase {
 
     // set doPing to true
     newConf.setBoolean(CommonConfigurationKeys.IPC_CLIENT_PING_KEY, true);
-    ConnectionId remoteId = ConnectionId.getConnectionId(new InetSocketAddress(0),
-        TestRpcService.class, null, 0, null, newConf);
+    ConnectionId remoteId = ConnectionId.getConnectionId(
+        new InetSocketAddress(0), TestRpcService.class, null, 0, null, newConf);
     assertEquals(CommonConfigurationKeys.IPC_PING_INTERVAL_DEFAULT,
         remoteId.getPingInterval());
     // set doPing to false
@@ -806,13 +831,13 @@ public class TestSaslRPC extends TestRpcBase {
     final TestTokenSecretManager sm = new TestTokenSecretManager();
     boolean useSecretManager = (serverAuth != SIMPLE);
     if (enableSecretManager != null) {
-      useSecretManager &= enableSecretManager.booleanValue();
+      useSecretManager &= enableSecretManager;
     }
     if (forceSecretManager != null) {
-      useSecretManager |= forceSecretManager.booleanValue();
+      useSecretManager |= forceSecretManager;
     }
     final SecretManager<?> serverSm = useSecretManager ? sm : null;
-    
+
     Server server = serverUgi.doAs(new PrivilegedExceptionAction<Server>() {
       @Override
       public Server run() throws IOException {
@@ -867,13 +892,13 @@ public class TestSaslRPC extends TestRpcBase {
             proxy.ping(null, newEmptyRequest());
             // make sure the other side thinks we are who we said we are!!!
             assertEquals(clientUgi.getUserName(),
-                convert(proxy.getAuthUser(null, newEmptyRequest())));
+                proxy.getAuthUser(null, newEmptyRequest()).getUser());
             AuthMethod authMethod =
                 convert(proxy.getAuthMethod(null, newEmptyRequest()));
             // verify sasl completed with correct QOP
             assertEquals((authMethod != SIMPLE) ? expectedQop.saslQop : null,
-                RPC.getConnectionIdForProxy(proxy).getSaslQop());
-            return authMethod.toString();
+                         RPC.getConnectionIdForProxy(proxy).getSaslQop());
+            return authMethod != null ? authMethod.toString() : null;
           } catch (ServiceException se) {
             if (se.getCause() instanceof RemoteException) {
               throw (RemoteException) se.getCause();
@@ -898,21 +923,18 @@ public class TestSaslRPC extends TestRpcBase {
       String actual) {
     assertEquals(expect.toString(), actual);
   }
-  
-  private static void assertAuthEquals(Pattern expect,
-      String actual) {
+
+  private static void assertAuthEquals(Pattern expect, String actual) {
     // this allows us to see the regexp and the value it didn't match
     if (!expect.matcher(actual).matches()) {
-      assertEquals(expect, actual); // it failed
-    } else {
-      assertTrue(true); // it matched
+      fail(); // it failed
     }
   }
 
   /*
    * Class used to test overriding QOP values using SaslPropertiesResolver
    */
-  static class AuthSaslPropertiesResolver extends SaslPropertiesResolver{
+  static class AuthSaslPropertiesResolver extends SaslPropertiesResolver {
 
     @Override
     public Map<String, String> getServerProperties(InetAddress address) {
@@ -921,7 +943,7 @@ public class TestSaslRPC extends TestRpcBase {
       return newPropertes;
     }
   }
-  
+
   public static void main(String[] args) throws Exception {
     System.out.println("Testing Kerberos authentication over RPC");
     if (args.length != 2) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea0c2b8b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestDoAsEffectiveUser.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestDoAsEffectiveUser.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestDoAsEffectiveUser.java
index 50d389c..c4dbcac 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestDoAsEffectiveUser.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestDoAsEffectiveUser.java
@@ -17,40 +17,35 @@
  */
 package org.apache.hadoop.security;
 
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.net.NetworkInterface;
-import java.security.PrivilegedExceptionAction;
-import java.util.ArrayList;
-import java.util.Enumeration;
-
-import org.junit.Assert;
-
+import com.google.protobuf.ServiceException;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.ipc.ProtocolSignature;
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.Server;
-import org.apache.hadoop.ipc.VersionedProtocol;
-import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.ipc.TestRpcBase;
 import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
 import org.apache.hadoop.security.authorize.DefaultImpersonationProvider;
 import org.apache.hadoop.security.authorize.ProxyUsers;
 import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.security.token.TokenInfo;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
-import org.apache.hadoop.ipc.TestRpcBase.TestTokenSecretManager;
-import org.apache.hadoop.ipc.TestRpcBase.TestTokenIdentifier;
-import org.apache.hadoop.ipc.TestRpcBase.TestTokenSelector;
-import org.apache.commons.logging.*;
-import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.NetworkInterface;
+import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+import java.util.Enumeration;
 
 /**
- *
+ * Test do as effective user.
  */
-public class TestDoAsEffectiveUser {
+public class TestDoAsEffectiveUser extends TestRpcBase {
   final private static String REAL_USER_NAME = "realUser1@HADOOP.APACHE.ORG";
   final private static String REAL_USER_SHORT_NAME = "realUser1";
   final private static String PROXY_USER_NAME = "proxyUser";
@@ -58,8 +53,8 @@ public class TestDoAsEffectiveUser {
   final private static String GROUP2_NAME = "group2";
   final private static String[] GROUP_NAMES = new String[] { GROUP1_NAME,
       GROUP2_NAME };
-  private static final String ADDRESS = "0.0.0.0";
-  private TestProtocol proxy;
+
+  private TestRpcService client;
   private static final Configuration masterConf = new Configuration();
   
   
@@ -82,7 +77,7 @@ public class TestDoAsEffectiveUser {
 
   private void configureSuperUserIPAddresses(Configuration conf,
       String superUserShortName) throws IOException {
-    ArrayList<String> ipList = new ArrayList<String>();
+    ArrayList<String> ipList = new ArrayList<>();
     Enumeration<NetworkInterface> netInterfaceList = NetworkInterface
         .getNetworkInterfaces();
     while (netInterfaceList.hasMoreElements()) {
@@ -130,50 +125,19 @@ public class TestDoAsEffectiveUser {
         curUGI.toString());
   }
 
-  @TokenInfo(TestTokenSelector.class)
-  public interface TestProtocol extends VersionedProtocol {
-    public static final long versionID = 1L;
-
-    String aMethod() throws IOException;
-    String getServerRemoteUser() throws IOException;
-  }
-
-  public class TestImpl implements TestProtocol {
-
-    @Override
-    public String aMethod() throws IOException {
-      return UserGroupInformation.getCurrentUser().toString();
-    }
-
-    @Override
-    public String getServerRemoteUser() throws IOException {
-      return Server.getRemoteUser().toString();
-    }
-    
-    @Override
-    public long getProtocolVersion(String protocol, long clientVersion)
-        throws IOException {
-      return TestProtocol.versionID;
-    }
-
-    @Override
-    public ProtocolSignature getProtocolSignature(String protocol,
-        long clientVersion, int clientMethodsHash) throws IOException {
-      return new ProtocolSignature(TestProtocol.versionID, null);
-    }
-  }
-
-  private void checkRemoteUgi(final Server server,
-      final UserGroupInformation ugi, final Configuration conf)
-          throws Exception {
+  private void checkRemoteUgi(final UserGroupInformation ugi,
+                              final Configuration conf) throws Exception {
     ugi.doAs(new PrivilegedExceptionAction<Void>() {
       @Override
-      public Void run() throws IOException {
-        proxy = RPC.getProxy(
-            TestProtocol.class, TestProtocol.versionID,
-            NetUtils.getConnectAddress(server), conf);
-        Assert.assertEquals(ugi.toString(), proxy.aMethod());
-        Assert.assertEquals(ugi.toString(), proxy.getServerRemoteUser());
+      public Void run() throws ServiceException {
+        client = getClient(addr, conf);
+        String currentUser = client.getCurrentUser(null,
+            newEmptyRequest()).getUser();
+        String serverRemoteUser = client.getServerRemoteUser(null,
+            newEmptyRequest()).getUser();
+
+        Assert.assertEquals(ugi.toString(), currentUser);
+        Assert.assertEquals(ugi.toString(), serverRemoteUser);
         return null;
       }
     });    
@@ -185,29 +149,27 @@ public class TestDoAsEffectiveUser {
     conf.setStrings(DefaultImpersonationProvider.getTestProvider().
         getProxySuperuserGroupConfKey(REAL_USER_SHORT_NAME), "group1");
     configureSuperUserIPAddresses(conf, REAL_USER_SHORT_NAME);
-    Server server = new RPC.Builder(conf).setProtocol(TestProtocol.class)
-        .setInstance(new TestImpl()).setBindAddress(ADDRESS).setPort(0)
-        .setNumHandlers(5).setVerbose(true).build();
+    // Set RPC engine to protobuf RPC engine
+    RPC.setProtocolEngine(conf, TestRpcService.class,
+        ProtobufRpcEngine.class);
+    UserGroupInformation.setConfiguration(conf);
+    final Server server = setupTestServer(conf, 5);
 
     refreshConf(conf);
     try {
-      server.start();
-
       UserGroupInformation realUserUgi = UserGroupInformation
           .createRemoteUser(REAL_USER_NAME);
-      checkRemoteUgi(server, realUserUgi, conf);
+      checkRemoteUgi(realUserUgi, conf);
       
-      UserGroupInformation proxyUserUgi = UserGroupInformation.createProxyUserForTesting(
+      UserGroupInformation proxyUserUgi =
+          UserGroupInformation.createProxyUserForTesting(
           PROXY_USER_NAME, realUserUgi, GROUP_NAMES);
-      checkRemoteUgi(server, proxyUserUgi, conf);
+      checkRemoteUgi(proxyUserUgi, conf);
     } catch (Exception e) {
       e.printStackTrace();
       Assert.fail();
     } finally {
-      server.stop();
-      if (proxy != null) {
-        RPC.stopProxy(proxy);
-      }
+      stop(server, client);
     }
   }
 
@@ -218,29 +180,25 @@ public class TestDoAsEffectiveUser {
     conf.setStrings(DefaultImpersonationProvider.getTestProvider().
             getProxySuperuserGroupConfKey(REAL_USER_SHORT_NAME),
         "group1");
-    Server server = new RPC.Builder(conf).setProtocol(TestProtocol.class)
-        .setInstance(new TestImpl()).setBindAddress(ADDRESS).setPort(0)
-        .setNumHandlers(2).setVerbose(false).build();
+    RPC.setProtocolEngine(conf, TestRpcService.class,
+        ProtobufRpcEngine.class);
+    UserGroupInformation.setConfiguration(conf);
+    final Server server = setupTestServer(conf, 5);
 
     refreshConf(conf);
     try {
-      server.start();
-
       UserGroupInformation realUserUgi = UserGroupInformation
           .createRemoteUser(REAL_USER_NAME);
-      checkRemoteUgi(server, realUserUgi, conf);
+      checkRemoteUgi(realUserUgi, conf);
 
       UserGroupInformation proxyUserUgi = UserGroupInformation
           .createProxyUserForTesting(PROXY_USER_NAME, realUserUgi, GROUP_NAMES);
-      checkRemoteUgi(server, proxyUserUgi, conf);
+      checkRemoteUgi(proxyUserUgi, conf);
     } catch (Exception e) {
       e.printStackTrace();
       Assert.fail();
     } finally {
-      server.stop();
-      if (proxy != null) {
-        RPC.stopProxy(proxy);
-      }
+      stop(server, client);
     }
   }
 
@@ -256,17 +214,14 @@ public class TestDoAsEffectiveUser {
     conf.setStrings(DefaultImpersonationProvider.getTestProvider().
             getProxySuperuserGroupConfKey(REAL_USER_SHORT_NAME),
         "group1");
-    Server server = new RPC.Builder(conf).setProtocol(TestProtocol.class)
-        .setInstance(new TestImpl()).setBindAddress(ADDRESS).setPort(0)
-        .setNumHandlers(2).setVerbose(false).build();
+    RPC.setProtocolEngine(conf, TestRpcService.class,
+        ProtobufRpcEngine.class);
+    UserGroupInformation.setConfiguration(conf);
+    final Server server = setupTestServer(conf, 5);
 
     refreshConf(conf);
     
     try {
-      server.start();
-
-      final InetSocketAddress addr = NetUtils.getConnectAddress(server);
-
       UserGroupInformation realUserUgi = UserGroupInformation
           .createRemoteUser(REAL_USER_NAME);
 
@@ -275,11 +230,10 @@ public class TestDoAsEffectiveUser {
       String retVal = proxyUserUgi
           .doAs(new PrivilegedExceptionAction<String>() {
             @Override
-            public String run() throws IOException {
-              proxy = RPC.getProxy(TestProtocol.class,
-                  TestProtocol.versionID, addr, conf);
-              String ret = proxy.aMethod();
-              return ret;
+            public String run() throws ServiceException {
+              client = getClient(addr, conf);
+              return client.getCurrentUser(null,
+                  newEmptyRequest()).getUser();
             }
           });
 
@@ -287,10 +241,7 @@ public class TestDoAsEffectiveUser {
     } catch (Exception e) {
       e.printStackTrace();
     } finally {
-      server.stop();
-      if (proxy != null) {
-        RPC.stopProxy(proxy);
-      }
+      stop(server, client);
     }
   }
   
@@ -299,17 +250,14 @@ public class TestDoAsEffectiveUser {
     final Configuration conf = new Configuration();
     conf.setStrings(DefaultImpersonationProvider.getTestProvider().
         getProxySuperuserGroupConfKey(REAL_USER_SHORT_NAME), "group1");
-    Server server = new RPC.Builder(conf).setProtocol(TestProtocol.class)
-        .setInstance(new TestImpl()).setBindAddress(ADDRESS).setPort(0)
-        .setNumHandlers(2).setVerbose(false).build();
+    RPC.setProtocolEngine(conf, TestRpcService.class,
+        ProtobufRpcEngine.class);
+    UserGroupInformation.setConfiguration(conf);
+    final Server server = setupTestServer(conf, 2);
 
     refreshConf(conf);
 
     try {
-      server.start();
-
-      final InetSocketAddress addr = NetUtils.getConnectAddress(server);
-
       UserGroupInformation realUserUgi = UserGroupInformation
           .createRemoteUser(REAL_USER_NAME);
 
@@ -318,11 +266,10 @@ public class TestDoAsEffectiveUser {
       String retVal = proxyUserUgi
           .doAs(new PrivilegedExceptionAction<String>() {
             @Override
-            public String run() throws IOException {
-              proxy = RPC.getProxy(TestProtocol.class,
-                  TestProtocol.versionID, addr, conf);
-              String ret = proxy.aMethod();
-              return ret;
+            public String run() throws ServiceException {
+              client = getClient(addr, conf);
+              return client.getCurrentUser(null,
+                  newEmptyRequest()).getUser();
             }
           });
 
@@ -330,10 +277,7 @@ public class TestDoAsEffectiveUser {
     } catch (Exception e) {
       e.printStackTrace();
     } finally {
-      server.stop();
-      if (proxy != null) {
-        RPC.stopProxy(proxy);
-      }
+      stop(server, client);
     }
   }
 
@@ -341,15 +285,12 @@ public class TestDoAsEffectiveUser {
   public void testRealUserGroupNotSpecified() throws IOException {
     final Configuration conf = new Configuration();
     configureSuperUserIPAddresses(conf, REAL_USER_SHORT_NAME);
-    Server server = new RPC.Builder(conf).setProtocol(TestProtocol.class)
-        .setInstance(new TestImpl()).setBindAddress(ADDRESS).setPort(0)
-        .setNumHandlers(2).setVerbose(false).build();
+    RPC.setProtocolEngine(conf, TestRpcService.class,
+        ProtobufRpcEngine.class);
+    UserGroupInformation.setConfiguration(conf);
+    final Server server = setupTestServer(conf, 2);
 
     try {
-      server.start();
-
-      final InetSocketAddress addr = NetUtils.getConnectAddress(server);
-
       UserGroupInformation realUserUgi = UserGroupInformation
           .createRemoteUser(REAL_USER_NAME);
 
@@ -358,11 +299,10 @@ public class TestDoAsEffectiveUser {
       String retVal = proxyUserUgi
           .doAs(new PrivilegedExceptionAction<String>() {
             @Override
-            public String run() throws IOException {
-              proxy = (TestProtocol) RPC.getProxy(TestProtocol.class,
-                  TestProtocol.versionID, addr, conf);
-              String ret = proxy.aMethod();
-              return ret;
+            public String run() throws ServiceException {
+              client = getClient(addr, conf);
+              return client.getCurrentUser(null,
+                  newEmptyRequest()).getUser();
             }
           });
 
@@ -370,10 +310,7 @@ public class TestDoAsEffectiveUser {
     } catch (Exception e) {
       e.printStackTrace();
     } finally {
-      server.stop();
-      if (proxy != null) {
-        RPC.stopProxy(proxy);
-      }
+      stop(server, client);
     }
   }
   
@@ -384,17 +321,14 @@ public class TestDoAsEffectiveUser {
     conf.setStrings(DefaultImpersonationProvider.getTestProvider().
             getProxySuperuserGroupConfKey(REAL_USER_SHORT_NAME),
         "group3");
-    Server server = new RPC.Builder(conf).setProtocol(TestProtocol.class)
-        .setInstance(new TestImpl()).setBindAddress(ADDRESS).setPort(0)
-        .setNumHandlers(2).setVerbose(false).build();
+    RPC.setProtocolEngine(conf, TestRpcService.class,
+        ProtobufRpcEngine.class);
+    UserGroupInformation.setConfiguration(conf);
+    final Server server = setupTestServer(conf, 2);
     
     refreshConf(conf);
 
     try {
-      server.start();
-
-      final InetSocketAddress addr = NetUtils.getConnectAddress(server);
-
       UserGroupInformation realUserUgi = UserGroupInformation
           .createRemoteUser(REAL_USER_NAME);
 
@@ -403,11 +337,10 @@ public class TestDoAsEffectiveUser {
       String retVal = proxyUserUgi
           .doAs(new PrivilegedExceptionAction<String>() {
             @Override
-            public String run() throws IOException {
-              proxy = RPC.getProxy(TestProtocol.class,
-                  TestProtocol.versionID, addr, conf);
-              String ret = proxy.aMethod();
-              return ret;
+            public String run() throws ServiceException {
+              client = getClient(addr, conf);
+              return client.getCurrentUser(null,
+                  newEmptyRequest()).getUser();
             }
           });
 
@@ -415,10 +348,7 @@ public class TestDoAsEffectiveUser {
     } catch (Exception e) {
       e.printStackTrace();
     } finally {
-      server.stop();
-      if (proxy != null) {
-        RPC.stopProxy(proxy);
-      }
+      stop(server, client);
     }
   }
 
@@ -432,20 +362,17 @@ public class TestDoAsEffectiveUser {
     final Configuration conf = new Configuration(masterConf);
     TestTokenSecretManager sm = new TestTokenSecretManager();
     SecurityUtil.setAuthenticationMethod(AuthenticationMethod.KERBEROS, conf);
+    RPC.setProtocolEngine(conf, TestRpcService.class,
+        ProtobufRpcEngine.class);
     UserGroupInformation.setConfiguration(conf);
-    final Server server = new RPC.Builder(conf).setProtocol(TestProtocol.class)
-        .setInstance(new TestImpl()).setBindAddress(ADDRESS).setPort(0)
-        .setNumHandlers(5).setVerbose(true).setSecretManager(sm).build();
-
-    server.start();
+    final Server server = setupTestServer(conf, 5, sm);
 
     final UserGroupInformation current = UserGroupInformation
         .createRemoteUser(REAL_USER_NAME);    
-    
-    final InetSocketAddress addr = NetUtils.getConnectAddress(server);
+
     TestTokenIdentifier tokenId = new TestTokenIdentifier(new Text(current
         .getUserName()), new Text("SomeSuperUser"));
-    Token<TestTokenIdentifier> token = new Token<TestTokenIdentifier>(tokenId,
+    Token<TestTokenIdentifier> token = new Token<>(tokenId,
         sm);
     SecurityUtil.setTokenService(token, addr);
     UserGroupInformation proxyUserUgi = UserGroupInformation
@@ -453,23 +380,19 @@ public class TestDoAsEffectiveUser {
     proxyUserUgi.addToken(token);
     
     refreshConf(conf);
-    
+
     String retVal = proxyUserUgi.doAs(new PrivilegedExceptionAction<String>() {
       @Override
       public String run() throws Exception {
         try {
-          proxy = RPC.getProxy(TestProtocol.class,
-              TestProtocol.versionID, addr, conf);
-          String ret = proxy.aMethod();
-          return ret;
+          client = getClient(addr, conf);
+          return client.getCurrentUser(null,
+              newEmptyRequest()).getUser();
         } catch (Exception e) {
           e.printStackTrace();
           throw e;
         } finally {
-          server.stop();
-          if (proxy != null) {
-            RPC.stopProxy(proxy);
-          }
+          stop(server, client);
         }
       }
     });
@@ -486,42 +409,34 @@ public class TestDoAsEffectiveUser {
     TestTokenSecretManager sm = new TestTokenSecretManager();
     final Configuration newConf = new Configuration(masterConf);
     SecurityUtil.setAuthenticationMethod(AuthenticationMethod.KERBEROS, newConf);
+    // Set RPC engine to protobuf RPC engine
+    RPC.setProtocolEngine(newConf, TestRpcService.class,
+        ProtobufRpcEngine.class);
     UserGroupInformation.setConfiguration(newConf);
-    final Server server = new RPC.Builder(newConf)
-        .setProtocol(TestProtocol.class).setInstance(new TestImpl())
-        .setBindAddress(ADDRESS).setPort(0).setNumHandlers(5).setVerbose(true)
-        .setSecretManager(sm).build();
-
-    server.start();
+    final Server server = setupTestServer(newConf, 5, sm);
 
     final UserGroupInformation current = UserGroupInformation
         .createUserForTesting(REAL_USER_NAME, GROUP_NAMES);
     
     refreshConf(newConf);
-    
-    final InetSocketAddress addr = NetUtils.getConnectAddress(server);
+
     TestTokenIdentifier tokenId = new TestTokenIdentifier(new Text(current
         .getUserName()), new Text("SomeSuperUser"));
-    Token<TestTokenIdentifier> token = new Token<TestTokenIdentifier>(tokenId,
-        sm);
+    Token<TestTokenIdentifier> token = new Token<>(tokenId, sm);
     SecurityUtil.setTokenService(token, addr);
     current.addToken(token);
     String retVal = current.doAs(new PrivilegedExceptionAction<String>() {
       @Override
       public String run() throws Exception {
         try {
-          proxy = RPC.getProxy(TestProtocol.class,
-              TestProtocol.versionID, addr, newConf);
-          String ret = proxy.aMethod();
-          return ret;
+          client = getClient(addr, newConf);
+          return client.getCurrentUser(null,
+              newEmptyRequest()).getUser();
         } catch (Exception e) {
           e.printStackTrace();
           throw e;
         } finally {
-          server.stop();
-          if (proxy != null) {
-            RPC.stopProxy(proxy);
-          }
+          stop(server, client);
         }
       }
     });

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea0c2b8b/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 b3ea5f2..e45d70d 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
@@ -20,6 +20,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.ipc.TestRpcBase.TestTokenIdentifier;
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
 import org.apache.hadoop.security.SaslRpcServer.AuthMethod;
 import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
@@ -28,7 +29,11 @@ import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.util.StringUtils;
-import org.junit.*;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
 
 import javax.security.auth.Subject;
 import javax.security.auth.kerberos.KerberosPrincipal;
@@ -50,9 +55,22 @@ import java.util.Set;
 
 import static org.apache.hadoop.fs.CommonConfigurationKeys.HADOOP_USER_GROUP_METRICS_PERCENTILES_INTERVALS;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTH_TO_LOCAL;
-import static org.apache.hadoop.ipc.TestSaslRPC.*;
-import static org.apache.hadoop.test.MetricsAsserts.*;
-import static org.junit.Assert.*;
+import static org.apache.hadoop.test.MetricsAsserts.assertCounter;
+import static org.apache.hadoop.test.MetricsAsserts.assertCounterGt;
+import static org.apache.hadoop.test.MetricsAsserts.assertGaugeGt;
+import static org.apache.hadoop.test.MetricsAsserts.assertQuantileGauges;
+import static org.apache.hadoop.test.MetricsAsserts.getDoubleGauge;
+import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNotSame;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -109,7 +127,7 @@ public class TestUserGroupInformation {
     UserGroupInformation.setLoginUser(null);
   }
 
-  @Test (timeout = 30000)
+  @Test(timeout = 30000)
   public void testSimpleLogin() throws IOException {
     tryLoginAuthenticationMethod(AuthenticationMethod.SIMPLE, true);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea0c2b8b/hadoop-common-project/hadoop-common/src/test/proto/test.proto
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/proto/test.proto b/hadoop-common-project/hadoop-common/src/test/proto/test.proto
index 99cd93d..6411f97 100644
--- a/hadoop-common-project/hadoop-common/src/test/proto/test.proto
+++ b/hadoop-common-project/hadoop-common/src/test/proto/test.proto
@@ -88,6 +88,6 @@ message AuthMethodResponseProto {
   required string mechanismName = 2;
 }
 
-message AuthUserResponseProto {
-  required string authUser = 1;
+message UserResponseProto {
+  required string user = 1;
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea0c2b8b/hadoop-common-project/hadoop-common/src/test/proto/test_rpc_service.proto
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/proto/test_rpc_service.proto b/hadoop-common-project/hadoop-common/src/test/proto/test_rpc_service.proto
index 3292115..06f6c4f 100644
--- a/hadoop-common-project/hadoop-common/src/test/proto/test_rpc_service.proto
+++ b/hadoop-common-project/hadoop-common/src/test/proto/test_rpc_service.proto
@@ -40,9 +40,11 @@ service TestProtobufRpcProto {
   rpc exchange(ExchangeRequestProto) returns (ExchangeResponseProto);
   rpc sleep(SleepRequestProto) returns (EmptyResponseProto);
   rpc getAuthMethod(EmptyRequestProto) returns (AuthMethodResponseProto);
-  rpc getAuthUser(EmptyRequestProto) returns (AuthUserResponseProto);
+  rpc getAuthUser(EmptyRequestProto) returns (UserResponseProto);
   rpc echoPostponed(EchoRequestProto) returns (EchoResponseProto);
   rpc sendPostponed(EmptyRequestProto) returns (EmptyResponseProto);
+  rpc getCurrentUser(EmptyRequestProto) returns (UserResponseProto);
+  rpc getServerRemoteUser(EmptyRequestProto) returns (UserResponseProto);
 }
 
 service TestProtobufRpc2Proto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea0c2b8b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
index 6b52949..57f7cb1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
@@ -168,7 +168,6 @@ import org.apache.hadoop.ipc.RetryCache.CacheEntry;
 import org.apache.hadoop.ipc.RetryCache.CacheEntryWithPayload;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.ipc.StandbyException;
-import org.apache.hadoop.ipc.WritableRpcEngine;
 import org.apache.hadoop.ipc.RefreshRegistry;
 import org.apache.hadoop.ipc.RefreshResponse;
 import org.apache.hadoop.net.Node;
@@ -317,8 +316,6 @@ public class NameNodeRpcServer implements NamenodeProtocols {
         new TraceAdminProtocolServerSideTranslatorPB(this);
     BlockingService traceAdminService = TraceAdminService
         .newReflectiveBlockingService(traceAdminXlator);
-    
-    WritableRpcEngine.ensureInitialized();
 
     InetSocketAddress serviceRpcAddr = nn.getServiceRpcServerAddress(conf);
     if (serviceRpcAddr != null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea0c2b8b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestClientProtocolWithDelegationToken.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestClientProtocolWithDelegationToken.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestClientProtocolWithDelegationToken.java
deleted file mode 100644
index 0b7ee33..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestClientProtocolWithDelegationToken.java
+++ /dev/null
@@ -1,119 +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.hdfs.security;
-
-import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION;
-import static org.mockito.Mockito.mock;
-
-import java.net.InetSocketAddress;
-import java.security.PrivilegedExceptionAction;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.protocol.ClientProtocol;
-import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
-import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
-import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.ipc.Client;
-import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.ipc.Server;
-import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.SaslInputStream;
-import org.apache.hadoop.security.SaslRpcClient;
-import org.apache.hadoop.security.SaslRpcServer;
-import org.apache.hadoop.security.SecurityUtil;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.test.GenericTestUtils;
-import org.apache.log4j.Level;
-import org.junit.Test;
-
-/** Unit tests for using Delegation Token over RPC. */
-public class TestClientProtocolWithDelegationToken {
-  private static final String ADDRESS = "0.0.0.0";
-
-  public static final Log LOG = LogFactory
-      .getLog(TestClientProtocolWithDelegationToken.class);
-
-  private static final Configuration conf;
-  static {
-    conf = new Configuration();
-    conf.set(HADOOP_SECURITY_AUTHENTICATION, "kerberos");
-    UserGroupInformation.setConfiguration(conf);
-  }
-
-  static {
-    GenericTestUtils.setLogLevel(Client.LOG, Level.ALL);
-    GenericTestUtils.setLogLevel(Server.LOG, Level.ALL);
-    GenericTestUtils.setLogLevel(SaslRpcClient.LOG, Level.ALL);
-    GenericTestUtils.setLogLevel(SaslRpcServer.LOG, Level.ALL);
-    GenericTestUtils.setLogLevel(SaslInputStream.LOG, Level.ALL);
-  }
-
-  @Test
-  public void testDelegationTokenRpc() throws Exception {
-    ClientProtocol mockNN = mock(ClientProtocol.class);
-    FSNamesystem mockNameSys = mock(FSNamesystem.class);
-
-    DelegationTokenSecretManager sm = new DelegationTokenSecretManager(
-        DFSConfigKeys.DFS_NAMENODE_DELEGATION_KEY_UPDATE_INTERVAL_DEFAULT,
-        DFSConfigKeys.DFS_NAMENODE_DELEGATION_KEY_UPDATE_INTERVAL_DEFAULT,
-        DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_MAX_LIFETIME_DEFAULT,
-        3600000, mockNameSys);
-    sm.startThreads();
-    final Server server = new RPC.Builder(conf)
-        .setProtocol(ClientProtocol.class).setInstance(mockNN)
-        .setBindAddress(ADDRESS).setPort(0).setNumHandlers(5).setVerbose(true)
-        .setSecretManager(sm).build();
-    
-    server.start();
-
-    final UserGroupInformation current = UserGroupInformation.getCurrentUser();
-    final InetSocketAddress addr = NetUtils.getConnectAddress(server);
-    String user = current.getUserName();
-    Text owner = new Text(user);
-    DelegationTokenIdentifier dtId = new DelegationTokenIdentifier(owner, owner, null);
-    Token<DelegationTokenIdentifier> token = new Token<DelegationTokenIdentifier>(
-        dtId, sm);
-    SecurityUtil.setTokenService(token, addr);
-    LOG.info("Service for token is " + token.getService());
-    current.addToken(token);
-    current.doAs(new PrivilegedExceptionAction<Object>() {
-      @Override
-      public Object run() throws Exception {
-        ClientProtocol proxy = null;
-        try {
-          proxy = RPC.getProxy(ClientProtocol.class,
-              ClientProtocol.versionID, addr, conf);
-          proxy.getServerDefaults();
-        } finally {
-          server.stop();
-          if (proxy != null) {
-            RPC.stopProxy(proxy);
-          }
-        }
-        return null;
-      }
-    });
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea0c2b8b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/server/HSAdminServer.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/server/HSAdminServer.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/server/HSAdminServer.java
index 3fef5e2..729af0a 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/server/HSAdminServer.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/server/HSAdminServer.java
@@ -29,7 +29,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.ipc.WritableRpcEngine;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.Groups;
@@ -98,8 +97,6 @@ public class HSAdminServer extends AbstractService implements HSAdminProtocol {
     BlockingService refreshHSAdminProtocolService = HSAdminRefreshProtocolService
         .newReflectiveBlockingService(refreshHSAdminProtocolXlator);
 
-    WritableRpcEngine.ensureInitialized();
-
     clientRpcAddress = conf.getSocketAddr(
         JHAdminConfig.MR_HISTORY_BIND_HOST,
         JHAdminConfig.JHS_ADMIN_ADDRESS,


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


[04/44] hadoop git commit: YARN-5620. Core changes in NodeManager to support re-initialization of Containers with new launchContext. (asuresh)

Posted by su...@apache.org.
YARN-5620. Core changes in NodeManager to support re-initialization of Containers with new launchContext. (asuresh)


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

Branch: refs/heads/YARN-2915
Commit: 40b5a59b726733df456330a26f03d5174cc0bc1c
Parents: 2a8f55a
Author: Arun Suresh <as...@apache.org>
Authored: Thu Sep 15 07:15:11 2016 -0700
Committer: Arun Suresh <as...@apache.org>
Committed: Thu Sep 15 07:15:11 2016 -0700

----------------------------------------------------------------------
 .../nodemanager/DefaultContainerExecutor.java   |   2 +-
 .../containermanager/ContainerManagerImpl.java  |  51 +++-
 .../containermanager/container/Container.java   |   6 +
 .../container/ContainerEventType.java           |   3 +-
 .../container/ContainerImpl.java                | 274 ++++++++++++++++---
 .../container/ContainerReInitEvent.java         |  62 +++++
 .../container/ContainerState.java               |   2 +-
 .../launcher/ContainersLauncher.java            |   1 +
 .../launcher/ContainersLauncherEventType.java   |   1 +
 .../localizer/ResourceLocalizationService.java  |   3 +-
 .../containermanager/localizer/ResourceSet.java |  43 ++-
 .../ContainerLocalizationRequestEvent.java      |   4 +-
 .../TestContainerManagerWithLCE.java            |  36 +++
 .../BaseContainerManagerTest.java               |   6 +-
 .../containermanager/TestContainerManager.java  | 238 +++++++++++++++-
 .../nodemanager/webapp/MockContainer.java       |  15 +
 16 files changed, 682 insertions(+), 65 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/40b5a59b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DefaultContainerExecutor.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/DefaultContainerExecutor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DefaultContainerExecutor.java
index 9a0549d..59b69ac 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DefaultContainerExecutor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DefaultContainerExecutor.java
@@ -89,7 +89,7 @@ public class DefaultContainerExecutor extends ContainerExecutor {
   }
 
   protected void copyFile(Path src, Path dst, String owner) throws IOException {
-    lfs.util().copy(src, dst);
+    lfs.util().copy(src, dst, false, true);
   }
   
   protected void setScriptExecutable(Path script, String owner) throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40b5a59b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.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/containermanager/ContainerManagerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
index 52d8566..f909ca5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
@@ -110,11 +110,13 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Cont
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEventType;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerImpl;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerKillEvent;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerReInitEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainersLauncher;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainersLauncherEventType;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.SignalContainersLauncherEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.LocalResourceRequest;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceSet;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ContainerLocalizationRequestEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.LocalizationEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.LocalizationEventType;
@@ -163,6 +165,9 @@ import static org.apache.hadoop.service.Service.STATE.STARTED;
 public class ContainerManagerImpl extends CompositeService implements
     ContainerManager {
 
+  private enum ReinitOp {
+    UPGRADE, COMMIT, ROLLBACK, LOCALIZE;
+  }
   /**
    * Extra duration to wait for applications to be killed on shutdown.
    */
@@ -1529,18 +1534,8 @@ public class ContainerManagerImpl extends CompositeService implements
       ResourceLocalizationRequest request) throws YarnException, IOException {
 
     ContainerId containerId = request.getContainerId();
-    Container container = context.getContainers().get(containerId);
-    if (container == null) {
-      throw new YarnException("Specified " + containerId + " does not exist!");
-    }
-    if (!container.getContainerState()
-        .equals(org.apache.hadoop.yarn.server.nodemanager.
-            containermanager.container.ContainerState.RUNNING)) {
-      throw new YarnException(
-          containerId + " is at " + container.getContainerState()
-              + " state. Not able to localize new resources.");
-    }
-
+    Container container = preUpgradeOrLocalizeCheck(containerId,
+        ReinitOp.LOCALIZE);
     try {
       Map<LocalResourceVisibility, Collection<LocalResourceRequest>> req =
           container.getResourceSet().addResources(request.getLocalResources());
@@ -1556,6 +1551,38 @@ public class ContainerManagerImpl extends CompositeService implements
     return ResourceLocalizationResponse.newInstance();
   }
 
+  public void upgradeContainer(ContainerId containerId,
+      ContainerLaunchContext upgradeLaunchContext) throws YarnException {
+    Container container = preUpgradeOrLocalizeCheck(containerId,
+        ReinitOp.UPGRADE);
+    ResourceSet resourceSet = new ResourceSet();
+    try {
+      resourceSet.addResources(upgradeLaunchContext.getLocalResources());
+      dispatcher.getEventHandler().handle(
+          new ContainerReInitEvent(containerId, upgradeLaunchContext,
+              resourceSet));
+      container.setIsReInitializing(true);
+    } catch (URISyntaxException e) {
+      LOG.info("Error when parsing local resource URI for upgrade of" +
+          "Container [" + containerId + "]", e);
+      throw new YarnException(e);
+    }
+  }
+
+  private Container preUpgradeOrLocalizeCheck(ContainerId containerId,
+      ReinitOp op) throws YarnException {
+    Container container = context.getContainers().get(containerId);
+    if (container == null) {
+      throw new YarnException("Specified " + containerId + " does not exist!");
+    }
+    if (!container.isRunning() || container.isReInitializing()) {
+      throw new YarnException("Cannot perform " + op + " on [" + containerId
+          + "]. Current state is [" + container.getContainerState() + ", " +
+          "isReInitializing=" + container.isReInitializing() + "].");
+    }
+    return container;
+  }
+
   @SuppressWarnings("unchecked")
   private void internalSignalToContainer(SignalContainerRequest request,
       String sentBy) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40b5a59b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/Container.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/containermanager/container/Container.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/Container.java
index c4cea18..f6c27ab 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/Container.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/Container.java
@@ -76,4 +76,10 @@ public interface Container extends EventHandler<ContainerEvent> {
   Priority getPriority();
 
   ResourceSet getResourceSet();
+
+  boolean isRunning();
+
+  void setIsReInitializing(boolean isReInitializing);
+
+  boolean isReInitializing();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40b5a59b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerEventType.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/containermanager/container/ContainerEventType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerEventType.java
index 5622f8c..0b57505 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerEventType.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerEventType.java
@@ -25,6 +25,7 @@ public enum ContainerEventType {
   KILL_CONTAINER,
   UPDATE_DIAGNOSTICS_MSG,
   CONTAINER_DONE,
+  REINITIALIZE_CONTAINER,
 
   // DownloadManager
   CONTAINER_INITED,
@@ -36,5 +37,5 @@ public enum ContainerEventType {
   CONTAINER_LAUNCHED,
   CONTAINER_EXITED_WITH_SUCCESS,
   CONTAINER_EXITED_WITH_FAILURE,
-  CONTAINER_KILLED_ON_REQUEST,
+  CONTAINER_KILLED_ON_REQUEST
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40b5a59b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.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/containermanager/container/ContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
index ce9e581..12bbea9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
@@ -27,6 +27,7 @@ import java.util.Collection;
 import java.util.EnumSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
@@ -90,13 +91,24 @@ import org.apache.hadoop.yarn.util.resource.Resources;
 
 public class ContainerImpl implements Container {
 
+  private final static class ReInitializationContext {
+    private final ResourceSet resourceSet;
+    private final ContainerLaunchContext newLaunchContext;
+
+    private ReInitializationContext(ContainerLaunchContext newLaunchContext,
+        ResourceSet resourceSet) {
+      this.newLaunchContext = newLaunchContext;
+      this.resourceSet = resourceSet;
+    }
+  }
+
   private final Lock readLock;
   private final Lock writeLock;
   private final Dispatcher dispatcher;
   private final NMStateStoreService stateStore;
   private final Credentials credentials;
   private final NodeManagerMetrics metrics;
-  private final ContainerLaunchContext launchContext;
+  private volatile ContainerLaunchContext launchContext;
   private final ContainerTokenIdentifier containerTokenIdentifier;
   private final ContainerId containerId;
   private volatile Resource resource;
@@ -110,13 +122,15 @@ public class ContainerImpl implements Container {
   private long containerLaunchStartTime;
   private ContainerMetrics containerMetrics;
   private static Clock clock = SystemClock.getInstance();
-  private final ContainerRetryContext containerRetryContext;
+  private ContainerRetryContext containerRetryContext;
   // remaining retries to relaunch container if needed
   private int remainingRetryAttempts;
   private String workDir;
   private String logDir;
   private String host;
   private String ips;
+  private ReInitializationContext reInitContext;
+  private volatile boolean isReInitializing = false;
 
   /** The NM-wide configuration - not specific to this container */
   private final Configuration daemonConf;
@@ -141,23 +155,7 @@ public class ContainerImpl implements Container {
     this.stateStore = context.getNMStateStore();
     this.version = containerTokenIdentifier.getVersion();
     this.launchContext = launchContext;
-    if (launchContext != null
-        && launchContext.getContainerRetryContext() != null) {
-      this.containerRetryContext = launchContext.getContainerRetryContext();
-    } else {
-      this.containerRetryContext = ContainerRetryContext.NEVER_RETRY_CONTEXT;
-    }
-    this.remainingRetryAttempts = containerRetryContext.getMaxRetries();
-    int minimumRestartInterval = conf.getInt(
-        YarnConfiguration.NM_CONTAINER_RETRY_MINIMUM_INTERVAL_MS,
-        YarnConfiguration.DEFAULT_NM_CONTAINER_RETRY_MINIMUM_INTERVAL_MS);
-    if (containerRetryContext.getRetryPolicy()
-        != ContainerRetryPolicy.NEVER_RETRY
-        && containerRetryContext.getRetryInterval() < minimumRestartInterval) {
-      LOG.info("Set restart interval to minimum value " + minimumRestartInterval
-          + "ms for container " + containerTokenIdentifier.getContainerID());
-      this.containerRetryContext.setRetryInterval(minimumRestartInterval);
-    }
+
     this.diagnosticsMaxSize = conf.getInt(
         YarnConfiguration.NM_CONTAINER_DIAGNOSTICS_MAXIMUM_SIZE,
         YarnConfiguration.DEFAULT_NM_CONTAINER_DIAGNOSTICS_MAXIMUM_SIZE);
@@ -188,11 +186,37 @@ public class ContainerImpl implements Container {
       containerMetrics.recordStartTime(clock.getTime());
     }
 
+    // Configure the Retry Context
+    this.containerRetryContext =
+        configureRetryContext(conf, launchContext, this.containerId);
+    this.remainingRetryAttempts = this.containerRetryContext.getMaxRetries();
     stateMachine = stateMachineFactory.make(this);
     this.context = context;
     this.resourceSet = new ResourceSet();
   }
 
+  private static ContainerRetryContext configureRetryContext(
+      Configuration conf, ContainerLaunchContext launchContext,
+      ContainerId containerId) {
+    ContainerRetryContext context;
+    if (launchContext != null
+        && launchContext.getContainerRetryContext() != null) {
+      context = launchContext.getContainerRetryContext();
+    } else {
+      context = ContainerRetryContext.NEVER_RETRY_CONTEXT;
+    }
+    int minimumRestartInterval = conf.getInt(
+        YarnConfiguration.NM_CONTAINER_RETRY_MINIMUM_INTERVAL_MS,
+        YarnConfiguration.DEFAULT_NM_CONTAINER_RETRY_MINIMUM_INTERVAL_MS);
+    if (context.getRetryPolicy() != ContainerRetryPolicy.NEVER_RETRY
+        && context.getRetryInterval() < minimumRestartInterval) {
+      LOG.info("Set restart interval to minimum value " + minimumRestartInterval
+          + "ms for container " + containerId);
+      context.setRetryInterval(minimumRestartInterval);
+    }
+    return context;
+  }
+
   // constructor for a recovered container
   public ContainerImpl(Configuration conf, Dispatcher dispatcher,
       ContainerLaunchContext launchContext, Credentials creds,
@@ -299,6 +323,9 @@ public class ContainerImpl implements Container {
             ContainerState.EXITED_WITH_FAILURE),
         ContainerEventType.CONTAINER_EXITED_WITH_FAILURE,
         new RetryFailureTransition())
+    .addTransition(ContainerState.RUNNING, ContainerState.REINITIALIZING,
+        ContainerEventType.REINITIALIZE_CONTAINER,
+        new ReInitializeContainerTransition())
     .addTransition(ContainerState.RUNNING, ContainerState.RUNNING,
         ContainerEventType.RESOURCE_LOCALIZED,
         new ResourceLocalizedWhileRunningTransition())
@@ -310,10 +337,38 @@ public class ContainerImpl implements Container {
        UPDATE_DIAGNOSTICS_TRANSITION)
     .addTransition(ContainerState.RUNNING, ContainerState.KILLING,
         ContainerEventType.KILL_CONTAINER, new KillTransition())
-    .addTransition(ContainerState.RUNNING, ContainerState.EXITED_WITH_FAILURE,
+    .addTransition(ContainerState.RUNNING,
+        ContainerState.EXITED_WITH_FAILURE,
         ContainerEventType.CONTAINER_KILLED_ON_REQUEST,
         new KilledExternallyTransition())
 
+    // From REINITIALIZING State
+    .addTransition(ContainerState.REINITIALIZING,
+        ContainerState.EXITED_WITH_SUCCESS,
+        ContainerEventType.CONTAINER_EXITED_WITH_SUCCESS,
+        new ExitedWithSuccessTransition(true))
+    .addTransition(ContainerState.REINITIALIZING,
+        ContainerState.EXITED_WITH_FAILURE,
+        ContainerEventType.CONTAINER_EXITED_WITH_FAILURE,
+        new ExitedWithFailureTransition(true))
+    .addTransition(ContainerState.REINITIALIZING,
+        ContainerState.REINITIALIZING,
+        ContainerEventType.RESOURCE_LOCALIZED,
+        new ResourceLocalizedWhileReInitTransition())
+    .addTransition(ContainerState.REINITIALIZING, ContainerState.RUNNING,
+        ContainerEventType.RESOURCE_FAILED,
+        new ResourceLocalizationFailedWhileReInitTransition())
+    .addTransition(ContainerState.REINITIALIZING,
+        ContainerState.REINITIALIZING,
+        ContainerEventType.UPDATE_DIAGNOSTICS_MSG,
+        UPDATE_DIAGNOSTICS_TRANSITION)
+    .addTransition(ContainerState.REINITIALIZING, ContainerState.KILLING,
+        ContainerEventType.KILL_CONTAINER, new KillTransition())
+    .addTransition(ContainerState.REINITIALIZING,
+        ContainerState.LOCALIZED,
+        ContainerEventType.CONTAINER_KILLED_ON_REQUEST,
+        new KilledForReInitializationTransition())
+
     // From RELAUNCHING State
     .addTransition(ContainerState.RELAUNCHING, ContainerState.RUNNING,
         ContainerEventType.CONTAINER_LAUNCHED, new LaunchTransition())
@@ -458,7 +513,7 @@ public class ContainerImpl implements Container {
   }
 
   @Override
-  public Map<Path,List<String>> getLocalizedResources() {
+  public Map<Path, List<String>> getLocalizedResources() {
     this.readLock.lock();
     try {
       if (ContainerState.LOCALIZED == getContainerState()
@@ -775,7 +830,7 @@ public class ContainerImpl implements Container {
       ContainerResourceLocalizedEvent rsrcEvent = (ContainerResourceLocalizedEvent) event;
       LocalResourceRequest resourceRequest = rsrcEvent.getResource();
       Path location = rsrcEvent.getLocation();
-      List<String> syms =
+      Set<String> syms =
           container.resourceSet.resourceLocalized(resourceRequest, location);
       if (null == syms) {
         LOG.info("Localized resource " + resourceRequest +
@@ -822,17 +877,86 @@ public class ContainerImpl implements Container {
   }
 
   /**
-   * Resource is localized while the container is running - create symlinks
+   * Transition to start the Re-Initialization process.
+   */
+  static class ReInitializeContainerTransition extends ContainerTransition {
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public void transition(ContainerImpl container, ContainerEvent event) {
+      container.reInitContext = createReInitContext(event);
+      try {
+        Map<LocalResourceVisibility, Collection<LocalResourceRequest>>
+            pendingResources =
+            container.reInitContext.resourceSet.getAllResourcesByVisibility();
+        if (!pendingResources.isEmpty()) {
+          container.dispatcher.getEventHandler().handle(
+              new ContainerLocalizationRequestEvent(
+                  container, pendingResources));
+        } else {
+          // We are not waiting on any resources, so...
+          // Kill the current container.
+          container.dispatcher.getEventHandler().handle(
+              new ContainersLauncherEvent(container,
+                  ContainersLauncherEventType.CLEANUP_CONTAINER_FOR_REINIT));
+        }
+      } catch (Exception e) {
+        LOG.error("Container [" + container.getContainerId() + "]" +
+            " re-initialization failure..", e);
+        container.addDiagnostics("Error re-initializing due to" +
+            "[" + e.getMessage() + "]");
+      }
+    }
+
+    protected ReInitializationContext createReInitContext(
+        ContainerEvent event) {
+      ContainerReInitEvent rEvent = (ContainerReInitEvent)event;
+      return new ReInitializationContext(rEvent.getReInitLaunchContext(),
+          rEvent.getResourceSet());
+    }
+  }
+
+  /**
+   * Resource requested for Container Re-initialization has been localized.
+   * If all dependencies are met, then restart Container with new bits.
+   */
+  static class ResourceLocalizedWhileReInitTransition
+      extends ContainerTransition {
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public void transition(ContainerImpl container, ContainerEvent event) {
+      ContainerResourceLocalizedEvent rsrcEvent =
+          (ContainerResourceLocalizedEvent) event;
+      container.reInitContext.resourceSet.resourceLocalized(
+          rsrcEvent.getResource(), rsrcEvent.getLocation());
+      // Check if all ResourceLocalization has completed
+      if (container.reInitContext.resourceSet.getPendingResources()
+          .isEmpty()) {
+        // Kill the current container.
+        container.dispatcher.getEventHandler().handle(
+            new ContainersLauncherEvent(container,
+                ContainersLauncherEventType.CLEANUP_CONTAINER_FOR_REINIT));
+      }
+    }
+  }
+
+  /**
+   * Resource is localized while the container is running - create symlinks.
    */
   static class ResourceLocalizedWhileRunningTransition
       extends ContainerTransition {
 
+    @SuppressWarnings("unchecked")
     @Override
     public void transition(ContainerImpl container, ContainerEvent event) {
       ContainerResourceLocalizedEvent rsrcEvent =
           (ContainerResourceLocalizedEvent) event;
-      List<String> links = container.resourceSet
-          .resourceLocalized(rsrcEvent.getResource(), rsrcEvent.getLocation());
+      Set<String> links = container.resourceSet.resourceLocalized(
+          rsrcEvent.getResource(), rsrcEvent.getLocation());
+      if (links == null) {
+        return;
+      }
       // creating symlinks.
       for (String link : links) {
         try {
@@ -872,8 +996,29 @@ public class ContainerImpl implements Container {
   }
 
   /**
+   * Resource localization failed while the container is reinitializing.
+   */
+  static class ResourceLocalizationFailedWhileReInitTransition
+      extends ContainerTransition {
+
+    @Override
+    public void transition(ContainerImpl container, ContainerEvent event) {
+      ContainerResourceFailedEvent failedEvent =
+          (ContainerResourceFailedEvent) event;
+      container.resourceSet.resourceLocalizationFailed(
+          failedEvent.getResource());
+      container.addDiagnostics("Container aborting re-initialization.. "
+          + failedEvent.getDiagnosticMessage());
+      LOG.error("Container [" + container.getContainerId() + "] Re-init" +
+          " failed !! Resource [" + failedEvent.getResource() + "] could" +
+          " not be localized !!");
+      container.reInitContext = null;
+    }
+  }
+
+  /**
    * Transition from LOCALIZED state to RUNNING state upon receiving
-   * a CONTAINER_LAUNCHED event
+   * a CONTAINER_LAUNCHED event.
    */
   static class LaunchTransition extends ContainerTransition {
     @SuppressWarnings("unchecked")
@@ -883,6 +1028,12 @@ public class ContainerImpl implements Container {
       container.metrics.runningContainer();
       container.wasLaunched  = true;
 
+      if (container.reInitContext != null) {
+        container.reInitContext = null;
+        // Set rollback context here..
+        container.setIsReInitializing(false);
+      }
+
       if (container.recoveredAsKilled) {
         LOG.info("Killing " + container.containerId
             + " due to recovered as killed");
@@ -895,8 +1046,8 @@ public class ContainerImpl implements Container {
   }
 
   /**
-   * Transition from RUNNING or KILLING state to EXITED_WITH_SUCCESS state
-   * upon EXITED_WITH_SUCCESS message.
+   * Transition from RUNNING or KILLING state to
+   * EXITED_WITH_SUCCESS state upon EXITED_WITH_SUCCESS message.
    */
   @SuppressWarnings("unchecked")  // dispatcher not typed
   static class ExitedWithSuccessTransition extends ContainerTransition {
@@ -909,6 +1060,8 @@ public class ContainerImpl implements Container {
 
     @Override
     public void transition(ContainerImpl container, ContainerEvent event) {
+
+      container.setIsReInitializing(false);
       // Set exit code to 0 on success    	
       container.exitCode = 0;
     	
@@ -939,6 +1092,7 @@ public class ContainerImpl implements Container {
 
     @Override
     public void transition(ContainerImpl container, ContainerEvent event) {
+      container.setIsReInitializing(false);
       ContainerExitEvent exitEvent = (ContainerExitEvent) event;
       container.exitCode = exitEvent.getExitCode();
       if (exitEvent.getDiagnosticInfo() != null) {
@@ -959,7 +1113,7 @@ public class ContainerImpl implements Container {
   }
 
   /**
-   * Transition to EXITED_WITH_FAILURE or LOCALIZED state upon
+   * Transition to EXITED_WITH_FAILURE or RELAUNCHING state upon
    * CONTAINER_EXITED_WITH_FAILURE state.
    **/
   @SuppressWarnings("unchecked")  // dispatcher not typed
@@ -991,7 +1145,7 @@ public class ContainerImpl implements Container {
           } catch (IOException e) {
             LOG.warn(
                 "Unable to update remainingRetryAttempts in state store for "
-                + container.getContainerId(), e);
+                    + container.getContainerId(), e);
           }
         }
         LOG.info("Relaunching Container " + container.getContainerId()
@@ -1053,7 +1207,7 @@ public class ContainerImpl implements Container {
   }
 
   /**
-   * Transition to EXITED_WITH_FAILURE upon receiving KILLED_ON_REQUEST
+   * Transition to EXITED_WITH_FAILURE
    */
   static class KilledExternallyTransition extends ExitedWithFailureTransition {
     KilledExternallyTransition() {
@@ -1061,13 +1215,44 @@ public class ContainerImpl implements Container {
     }
 
     @Override
-    public void transition(ContainerImpl container, ContainerEvent event) {
+    public void transition(ContainerImpl container,
+        ContainerEvent event) {
       super.transition(container, event);
       container.addDiagnostics("Killed by external signal\n");
     }
   }
 
   /**
+   * Transition to LOCALIZED and wait for RE-LAUNCH
+   */
+  static class KilledForReInitializationTransition extends ContainerTransition {
+
+    @Override
+    public void transition(ContainerImpl container,
+        ContainerEvent event) {
+      LOG.info("Relaunching Container [" + container.getContainerId()
+          + "] for upgrade !!");
+      container.wasLaunched  = false;
+      container.metrics.endRunningContainer();
+
+      container.launchContext = container.reInitContext.newLaunchContext;
+
+      // Re configure the Retry Context
+      container.containerRetryContext =
+          configureRetryContext(container.context.getConf(),
+          container.launchContext, container.containerId);
+      // Reset the retry attempts since its a fresh start
+      container.remainingRetryAttempts =
+          container.containerRetryContext.getMaxRetries();
+
+      container.resourceSet = ResourceSet.merge(
+          container.resourceSet, container.reInitContext.resourceSet);
+
+      container.sendLaunchEvent();
+    }
+  }
+
+  /**
    * Transition from LOCALIZING to LOCALIZATION_FAILED upon receiving
    * RESOURCE_FAILED event.
    */
@@ -1122,16 +1307,20 @@ public class ContainerImpl implements Container {
   }
 
   /**
-   * Transitions upon receiving KILL_CONTAINER:
-   * - LOCALIZED -> KILLING
-   * - RUNNING -> KILLING
+   * Transitions upon receiving KILL_CONTAINER.
+   * - LOCALIZED -> KILLING.
+   * - RUNNING -> KILLING.
+   * - REINITIALIZING -> KILLING.
    */
   @SuppressWarnings("unchecked") // dispatcher not typed
   static class KillTransition implements
       SingleArcTransition<ContainerImpl, ContainerEvent> {
+
+    @SuppressWarnings("unchecked")
     @Override
     public void transition(ContainerImpl container, ContainerEvent event) {
       // Kill the process/process-grp
+      container.setIsReInitializing(false);
       container.dispatcher.getEventHandler().handle(
           new ContainersLauncherEvent(container,
               ContainersLauncherEventType.CLEANUP_CONTAINER));
@@ -1385,4 +1574,19 @@ public class ContainerImpl implements Container {
   public Priority getPriority() {
     return containerTokenIdentifier.getPriority();
   }
+
+  @Override
+  public boolean isRunning() {
+    return getContainerState() == ContainerState.RUNNING;
+  }
+
+  @Override
+  public void setIsReInitializing(boolean isReInitializing) {
+    this.isReInitializing = isReInitializing;
+  }
+
+  @Override
+  public boolean isReInitializing() {
+    return this.isReInitializing;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40b5a59b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerReInitEvent.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/containermanager/container/ContainerReInitEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerReInitEvent.java
new file mode 100644
index 0000000..2ccdbd7
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerReInitEvent.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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.nodemanager.containermanager.container;
+
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceSet;
+
+/**
+ * ContainerEvent sent by ContainerManager to ContainerImpl to
+ * re-initiate Container.
+ */
+public class ContainerReInitEvent extends ContainerEvent {
+
+  private final ContainerLaunchContext reInitLaunchContext;
+  private final ResourceSet resourceSet;
+
+  /**
+   * Container Re-Init Event.
+   * @param cID Container Id
+   * @param upgradeContext Upgrade context
+   * @param resourceSet Resource Set
+   */
+  public ContainerReInitEvent(ContainerId cID,
+      ContainerLaunchContext upgradeContext, ResourceSet resourceSet){
+    super(cID, ContainerEventType.REINITIALIZE_CONTAINER);
+    this.reInitLaunchContext = upgradeContext;
+    this.resourceSet = resourceSet;
+  }
+
+  /**
+   * Get the Launch Context to be used for upgrade.
+   * @return ContainerLaunchContext
+   */
+  public ContainerLaunchContext getReInitLaunchContext() {
+    return reInitLaunchContext;
+  }
+
+  /**
+   * Get the ResourceSet.
+   * @return ResourceSet.
+   */
+  public ResourceSet getResourceSet() {
+    return resourceSet;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40b5a59b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerState.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/containermanager/container/ContainerState.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerState.java
index 6b96204..70de90c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerState.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerState.java
@@ -20,6 +20,6 @@ package org.apache.hadoop.yarn.server.nodemanager.containermanager.container;
 
 public enum ContainerState {
   NEW, LOCALIZING, LOCALIZATION_FAILED, LOCALIZED, RUNNING, RELAUNCHING,
-  EXITED_WITH_SUCCESS, EXITED_WITH_FAILURE, KILLING,
+  REINITIALIZING, EXITED_WITH_SUCCESS, EXITED_WITH_FAILURE, KILLING,
   CONTAINER_CLEANEDUP_AFTER_KILL, CONTAINER_RESOURCES_CLEANINGUP, DONE
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40b5a59b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainersLauncher.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/containermanager/launcher/ContainersLauncher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainersLauncher.java
index e5fff00..d4a7bfd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainersLauncher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainersLauncher.java
@@ -137,6 +137,7 @@ public class ContainersLauncher extends AbstractService
         running.put(containerId, launch);
         break;
       case CLEANUP_CONTAINER:
+      case CLEANUP_CONTAINER_FOR_REINIT:
         ContainerLaunch launcher = running.remove(containerId);
         if (launcher == null) {
           // Container not launched. So nothing needs to be done.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40b5a59b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainersLauncherEventType.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/containermanager/launcher/ContainersLauncherEventType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainersLauncherEventType.java
index 2d7bc74..380a032 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainersLauncherEventType.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainersLauncherEventType.java
@@ -23,5 +23,6 @@ public enum ContainersLauncherEventType {
   RELAUNCH_CONTAINER,
   RECOVER_CONTAINER,
   CLEANUP_CONTAINER, // The process(grp) itself.
+  CLEANUP_CONTAINER_FOR_REINIT, // The process(grp) itself.
   SIGNAL_CONTAINER,
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40b5a59b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.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/containermanager/localizer/ResourceLocalizationService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
index b281ef5..2cf6ee9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
@@ -470,7 +470,8 @@ public class ResourceLocalizationService extends CompositeService
       ContainerLocalizationRequestEvent rsrcReqs) {
     Container c = rsrcReqs.getContainer();
     EnumSet<ContainerState> set =
-        EnumSet.of(ContainerState.LOCALIZING, ContainerState.RUNNING);
+        EnumSet.of(ContainerState.LOCALIZING,
+            ContainerState.RUNNING, ContainerState.REINITIALIZING);
     if (!set.contains(c.getContainerState())) {
       LOG.warn(c.getContainerId() + " is at " + c.getContainerState()
           + " state, do not localize resources.");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40b5a59b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceSet.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/containermanager/localizer/ResourceSet.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceSet.java
index a41ee20..5da3abc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceSet.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceSet.java
@@ -43,9 +43,9 @@ public class ResourceSet {
   private static final Log LOG = LogFactory.getLog(ResourceSet.class);
 
   // resources by localization state (localized, pending, failed)
-  private Map<Path, List<String>> localizedResources =
+  private Map<String, Path> localizedResources =
       new ConcurrentHashMap<>();
-  private Map<LocalResourceRequest, List<String>> pendingResources =
+  private Map<LocalResourceRequest, Set<String>> pendingResources =
       new ConcurrentHashMap<>();
   private Set<LocalResourceRequest> resourcesFailedToBeLocalized =
       new HashSet<>();
@@ -69,7 +69,7 @@ public class ResourceSet {
     if (localResourceMap == null || localResourceMap.isEmpty()) {
       return null;
     }
-    Map<LocalResourceRequest, List<String>> allResources = new HashMap<>();
+    Map<LocalResourceRequest, Set<String>> allResources = new HashMap<>();
     List<LocalResourceRequest> publicList = new ArrayList<>();
     List<LocalResourceRequest> privateList = new ArrayList<>();
     List<LocalResourceRequest> appList = new ArrayList<>();
@@ -77,7 +77,7 @@ public class ResourceSet {
     for (Map.Entry<String, LocalResource> rsrc : localResourceMap.entrySet()) {
       LocalResource resource = rsrc.getValue();
       LocalResourceRequest req = new LocalResourceRequest(rsrc.getValue());
-      allResources.putIfAbsent(req, new ArrayList<>());
+      allResources.putIfAbsent(req, new HashSet<>());
       allResources.get(req).add(rsrc.getKey());
       storeSharedCacheUploadPolicy(req,
           resource.getShouldBeUploadedToSharedCache());
@@ -121,13 +121,15 @@ public class ResourceSet {
    * @param location The path where the resource is localized
    * @return The list of symlinks for the localized resources.
    */
-  public List<String> resourceLocalized(LocalResourceRequest request,
+  public Set<String> resourceLocalized(LocalResourceRequest request,
       Path location) {
-    List<String> symlinks = pendingResources.remove(request);
+    Set<String> symlinks = pendingResources.remove(request);
     if (symlinks == null) {
       return null;
     } else {
-      localizedResources.put(location, symlinks);
+      for (String symlink : symlinks) {
+        localizedResources.put(symlink, location);
+      }
       return symlinks;
     }
   }
@@ -175,7 +177,12 @@ public class ResourceSet {
   }
 
   public Map<Path, List<String>> getLocalizedResources() {
-    return localizedResources;
+    Map<Path, List<String>> map = new HashMap<>();
+    for (Map.Entry<String, Path> entry : localizedResources.entrySet()) {
+      map.putIfAbsent(entry.getValue(), new ArrayList<>());
+      map.get(entry.getValue()).add(entry.getKey());
+    }
+    return map;
   }
 
   public Map<LocalResourceRequest, Path> getResourcesToBeUploaded() {
@@ -186,7 +193,25 @@ public class ResourceSet {
     return resourcesUploadPolicies;
   }
 
-  public Map<LocalResourceRequest, List<String>> getPendingResources() {
+  public Map<LocalResourceRequest, Set<String>> getPendingResources() {
     return pendingResources;
   }
+
+  public static ResourceSet merge(ResourceSet... resourceSets) {
+    ResourceSet merged = new ResourceSet();
+    for (ResourceSet rs : resourceSets) {
+      // This should overwrite existing symlinks
+      merged.localizedResources.putAll(rs.localizedResources);
+
+      merged.resourcesToBeUploaded.putAll(rs.resourcesToBeUploaded);
+      merged.resourcesUploadPolicies.putAll(rs.resourcesUploadPolicies);
+
+      // TODO : START : Should we de-dup here ?
+      merged.publicRsrcs.addAll(rs.publicRsrcs);
+      merged.privateRsrcs.addAll(rs.privateRsrcs);
+      merged.appRsrcs.addAll(rs.appRsrcs);
+      // TODO : END
+    }
+    return merged;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40b5a59b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/event/ContainerLocalizationRequestEvent.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/containermanager/localizer/event/ContainerLocalizationRequestEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/event/ContainerLocalizationRequestEvent.java
index 43a2f33..0344275 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/event/ContainerLocalizationRequestEvent.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/event/ContainerLocalizationRequestEvent.java
@@ -39,8 +39,8 @@ public class ContainerLocalizationRequestEvent extends
 
   /**
    * Event requesting the localization of the rsrc.
-   * @param c
-   * @param rsrc
+   * @param c Container
+   * @param rsrc LocalResourceRequests map
    */
   public ContainerLocalizationRequestEvent(Container c,
       Map<LocalResourceVisibility, Collection<LocalResourceRequest>> rsrc) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40b5a59b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestContainerManagerWithLCE.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestContainerManagerWithLCE.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestContainerManagerWithLCE.java
index aa0d975..8a27849 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestContainerManagerWithLCE.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestContainerManagerWithLCE.java
@@ -269,6 +269,42 @@ public class TestContainerManagerWithLCE extends TestContainerManager {
     super.testForcefulShutdownSignal();
   }
 
+  @Override
+  public void testContainerUpgradeSuccess() throws IOException,
+      InterruptedException, YarnException {
+    // Don't run the test if the binary is not available.
+    if (!shouldRunTest()) {
+      LOG.info("LCE binary path is not passed. Not running the test");
+      return;
+    }
+    LOG.info("Running testContainerUpgradeSuccess");
+    super.testContainerUpgradeSuccess();
+  }
+
+  @Override
+  public void testContainerUpgradeLocalizationFailure() throws IOException,
+      InterruptedException, YarnException {
+    // Don't run the test if the binary is not available.
+    if (!shouldRunTest()) {
+      LOG.info("LCE binary path is not passed. Not running the test");
+      return;
+    }
+    LOG.info("Running testContainerUpgradeLocalizationFailure");
+    super.testContainerUpgradeLocalizationFailure();
+  }
+
+  @Override
+  public void testContainerUpgradeProcessFailure() throws IOException,
+      InterruptedException, YarnException {
+    // Don't run the test if the binary is not available.
+    if (!shouldRunTest()) {
+      LOG.info("LCE binary path is not passed. Not running the test");
+      return;
+    }
+    LOG.info("Running testContainerUpgradeProcessFailure");
+    super.testContainerUpgradeProcessFailure();
+  }
+
   private boolean shouldRunTest() {
     return System
         .getProperty(YarnConfiguration.NM_LINUX_CONTAINER_EXECUTOR_PATH) != null;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40b5a59b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java
index ec38501..d359c3d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java
@@ -123,7 +123,11 @@ public abstract class BaseContainerManagerTest {
       conf) {
     public int getHttpPort() {
       return HTTP_PORT;
-    };
+    }
+    @Override
+    public ContainerExecutor getContainerExecutor() {
+      return exec;
+    }
   };
   protected ContainerExecutor exec;
   protected DeletionService delSrvc;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40b5a59b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
index 5785e1f..843dc2a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
@@ -25,6 +25,7 @@ import static org.mockito.Mockito.verify;
 
 import java.io.BufferedReader;
 import java.io.File;
+import java.io.FileNotFoundException;
 import java.io.FileReader;
 import java.io.IOException;
 import java.io.PrintWriter;
@@ -33,6 +34,7 @@ import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 
@@ -64,6 +66,8 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.hadoop.yarn.api.records.ContainerRetryContext;
+import org.apache.hadoop.yarn.api.records.ContainerRetryPolicy;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.LocalResource;
@@ -94,7 +98,6 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.Conta
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ContainerLocalizer;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService;
 import org.apache.hadoop.yarn.server.nodemanager.executor.ContainerSignalContext;
-import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
@@ -366,6 +369,237 @@ public class TestContainerManager extends BaseContainerManagerTest {
       DefaultContainerExecutor.containerIsAlive(pid));
   }
 
+  @Test
+  public void testContainerUpgradeSuccess() throws IOException,
+      InterruptedException, YarnException {
+    containerManager.start();
+    // ////// Construct the Container-id
+    ContainerId cId = createContainerId(0);
+    File oldStartFile = new File(tmpDir, "start_file_o.txt").getAbsoluteFile();
+
+    String pid = prepareInitialContainer(cId, oldStartFile);
+
+    File newStartFile = new File(tmpDir, "start_file_n.txt").getAbsoluteFile();
+
+    prepareContainerUpgrade(false, false, cId, newStartFile);
+
+    // Assert that the First process is not alive anymore
+    Assert.assertFalse("Process is still alive!",
+        DefaultContainerExecutor.containerIsAlive(pid));
+
+    BufferedReader reader =
+        new BufferedReader(new FileReader(newStartFile));
+    Assert.assertEquals("Upgrade World!", reader.readLine());
+
+    // Get the pid of the process
+    String newPid = reader.readLine().trim();
+    Assert.assertNotEquals("Old and New Pids must be different !", pid, newPid);
+    // No more lines
+    Assert.assertEquals(null, reader.readLine());
+
+    reader.close();
+
+    // Verify old file still exists and is accessible by
+    // the new process...
+    reader = new BufferedReader(new FileReader(oldStartFile));
+    Assert.assertEquals("Hello World!", reader.readLine());
+
+    // Assert that the New process is alive
+    Assert.assertTrue("New Process is not alive!",
+        DefaultContainerExecutor.containerIsAlive(newPid));
+  }
+
+  @Test
+  public void testContainerUpgradeLocalizationFailure() throws IOException,
+      InterruptedException, YarnException {
+    if (Shell.WINDOWS) {
+      return;
+    }
+    containerManager.start();
+    // ////// Construct the Container-id
+    ContainerId cId = createContainerId(0);
+    File oldStartFile = new File(tmpDir, "start_file_o.txt").getAbsoluteFile();
+
+    String pid = prepareInitialContainer(cId, oldStartFile);
+
+    File newStartFile = new File(tmpDir, "start_file_n.txt").getAbsoluteFile();
+
+    prepareContainerUpgrade(true, true, cId, newStartFile);
+
+    // Assert that the First process is STILL alive
+    // since upgrade was terminated..
+    Assert.assertTrue("Process is NOT alive!",
+        DefaultContainerExecutor.containerIsAlive(pid));
+  }
+
+  @Test
+  public void testContainerUpgradeProcessFailure() throws IOException,
+      InterruptedException, YarnException {
+    if (Shell.WINDOWS) {
+      return;
+    }
+    containerManager.start();
+    // ////// Construct the Container-id
+    ContainerId cId = createContainerId(0);
+    File oldStartFile = new File(tmpDir, "start_file_o.txt").getAbsoluteFile();
+
+    String pid = prepareInitialContainer(cId, oldStartFile);
+
+    File newStartFile = new File(tmpDir, "start_file_n.txt").getAbsoluteFile();
+
+    prepareContainerUpgrade(true, false, cId, newStartFile);
+
+    // Assert that the First process is not alive anymore
+    Assert.assertFalse("Process is still alive!",
+        DefaultContainerExecutor.containerIsAlive(pid));
+  }
+
+  /**
+   * Prepare a launch Context for container upgrade and request the
+   * Container Manager to re-initialize a running container using the
+   * new launch context.
+   * @param failCmd injects a start script that intentionally fails.
+   * @param failLoc injects a bad file Location that will fail localization.
+   */
+  private void prepareContainerUpgrade(boolean failCmd, boolean failLoc,
+      ContainerId cId, File startFile)
+      throws FileNotFoundException, YarnException, InterruptedException {
+    // Re-write scriptfile and processStartFile
+    File scriptFile = Shell.appendScriptExtension(tmpDir, "scriptFile_new");
+    PrintWriter fileWriter = new PrintWriter(scriptFile);
+
+    writeScriptFile(fileWriter, "Upgrade World!", startFile, cId, failCmd);
+
+    ContainerLaunchContext containerLaunchContext =
+        prepareContainerLaunchContext(scriptFile, "dest_file_new", failLoc);
+
+    containerManager.upgradeContainer(cId, containerLaunchContext);
+    try {
+      containerManager.upgradeContainer(cId, containerLaunchContext);
+    } catch (Exception e) {
+      Assert.assertTrue(e.getMessage().contains("Cannot perform UPGRADE"));
+    }
+    int timeoutSecs = 0;
+    int maxTimeToWait = failLoc ? 10 : 20;
+    // Wait for new processStartfile to be created
+    while (!startFile.exists() && timeoutSecs++ < maxTimeToWait) {
+      Thread.sleep(1000);
+      LOG.info("Waiting for New process start-file to be created");
+    }
+  }
+
+  /**
+   * Prepare and start an initial container. This container will be subsequently
+   * re-initialized for upgrade. It also waits for the container to start and
+   * returns the Pid of the running container.
+   */
+  private String prepareInitialContainer(ContainerId cId, File startFile)
+      throws IOException, YarnException, InterruptedException {
+    File scriptFileOld = Shell.appendScriptExtension(tmpDir, "scriptFile");
+    PrintWriter fileWriterOld = new PrintWriter(scriptFileOld);
+
+    writeScriptFile(fileWriterOld, "Hello World!", startFile, cId, false);
+
+    ContainerLaunchContext containerLaunchContext =
+        prepareContainerLaunchContext(scriptFileOld, "dest_file", false);
+
+    StartContainerRequest scRequest =
+        StartContainerRequest.newInstance(containerLaunchContext,
+            createContainerToken(cId,
+                DUMMY_RM_IDENTIFIER, context.getNodeId(), user,
+                context.getContainerTokenSecretManager()));
+    List<StartContainerRequest> list = new ArrayList<>();
+    list.add(scRequest);
+    StartContainersRequest allRequests =
+        StartContainersRequest.newInstance(list);
+    containerManager.startContainers(allRequests);
+
+    int timeoutSecs = 0;
+    while (!startFile.exists() && timeoutSecs++ < 20) {
+      Thread.sleep(1000);
+      LOG.info("Waiting for process start-file to be created");
+    }
+    Assert.assertTrue("ProcessStartFile doesn't exist!",
+        startFile.exists());
+
+    // Now verify the contents of the file
+    BufferedReader reader =
+        new BufferedReader(new FileReader(startFile));
+    Assert.assertEquals("Hello World!", reader.readLine());
+    // Get the pid of the process
+    String pid = reader.readLine().trim();
+    // No more lines
+    Assert.assertEquals(null, reader.readLine());
+
+    // Assert that the process is alive
+    Assert.assertTrue("Process is not alive!",
+        DefaultContainerExecutor.containerIsAlive(pid));
+    // Once more
+    Assert.assertTrue("Process is not alive!",
+        DefaultContainerExecutor.containerIsAlive(pid));
+    return pid;
+  }
+
+  private void writeScriptFile(PrintWriter fileWriter, String startLine,
+      File processStartFile, ContainerId cId, boolean isFailure) {
+    if (Shell.WINDOWS) {
+      fileWriter.println("@echo " + startLine + "> " + processStartFile);
+      fileWriter.println("@echo " + cId + ">> " + processStartFile);
+      fileWriter.println("@ping -n 100 127.0.0.1 >nul");
+    } else {
+      fileWriter.write("\numask 0"); // So that start file is readable by test
+      if (isFailure) {
+        // Echo PID and throw some error code
+        fileWriter.write("\necho $$ >> " + processStartFile);
+        fileWriter.write("\nexit 111");
+      } else {
+        fileWriter.write("\necho " + startLine + " > " + processStartFile);
+        fileWriter.write("\necho $$ >> " + processStartFile);
+        fileWriter.write("\nexec sleep 100");
+      }
+    }
+    fileWriter.close();
+  }
+
+  private ContainerLaunchContext prepareContainerLaunchContext(File scriptFile,
+      String destFName, boolean putBadFile) {
+    ContainerLaunchContext containerLaunchContext =
+        recordFactory.newRecordInstance(ContainerLaunchContext.class);
+    URL resourceAlpha = null;
+    if (putBadFile) {
+      File fileToDelete = new File(tmpDir, "fileToDelete")
+          .getAbsoluteFile();
+      resourceAlpha =
+          URL.fromPath(localFS
+              .makeQualified(new Path(fileToDelete.getAbsolutePath())));
+      fileToDelete.delete();
+    } else {
+      resourceAlpha =
+          URL.fromPath(localFS
+              .makeQualified(new Path(scriptFile.getAbsolutePath())));
+    }
+    LocalResource rsrcAlpha =
+        recordFactory.newRecordInstance(LocalResource.class);
+    rsrcAlpha.setResource(resourceAlpha);
+    rsrcAlpha.setSize(-1);
+    rsrcAlpha.setVisibility(LocalResourceVisibility.APPLICATION);
+    rsrcAlpha.setType(LocalResourceType.FILE);
+    rsrcAlpha.setTimestamp(scriptFile.lastModified());
+    Map<String, LocalResource> localResources = new HashMap<>();
+    localResources.put(destFName, rsrcAlpha);
+    containerLaunchContext.setLocalResources(localResources);
+
+    ContainerRetryContext containerRetryContext = ContainerRetryContext
+        .newInstance(
+            ContainerRetryPolicy.RETRY_ON_SPECIFIC_ERROR_CODES,
+            new HashSet<>(Arrays.asList(Integer.valueOf(111))), 4, 0);
+    containerLaunchContext.setContainerRetryContext(containerRetryContext);
+    List<String> commands = Arrays.asList(
+        Shell.getRunScriptCommand(scriptFile));
+    containerLaunchContext.setCommands(commands);
+    return containerLaunchContext;
+  }
+
   protected void testContainerLaunchAndExit(int exitCode) throws IOException,
       InterruptedException, YarnException {
 
@@ -556,7 +790,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
       Assert.fail();
     } catch (YarnException e) {
       Assert.assertTrue(
-          e.getMessage().contains("Not able to localize new resources"));
+          e.getMessage().contains("Cannot perform LOCALIZE"));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40b5a59b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockContainer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockContainer.java
index c176556..8c8bec7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockContainer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockContainer.java
@@ -190,4 +190,19 @@ public class MockContainer implements Container {
   public void setIpAndHost(String[] ipAndHost) {
 
   }
+
+  @Override
+  public boolean isRunning() {
+    return false;
+  }
+
+  @Override
+  public void setIsReInitializing(boolean isReInitializing) {
+
+  }
+
+  @Override
+  public boolean isReInitializing() {
+    return false;
+  }
 }


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


[23/44] hadoop git commit: YARN-3141. Improve locks in SchedulerApplicationAttempt/FSAppAttempt/FiCaSchedulerApp. Contributed by Wangda Tan

Posted by su...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/b8a30f2f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.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/FSAppAttempt.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/FSAppAttempt.java
index 9e5a807..3555faa 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/FSAppAttempt.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/FSAppAttempt.java
@@ -123,65 +123,72 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
     return queue.getMetrics();
   }
 
-  synchronized public void containerCompleted(RMContainer rmContainer,
+  public void containerCompleted(RMContainer rmContainer,
       ContainerStatus containerStatus, RMContainerEventType event) {
-    
-    Container container = rmContainer.getContainer();
-    ContainerId containerId = container.getId();
-    
-    // Remove from the list of newly allocated containers if found
-    newlyAllocatedContainers.remove(rmContainer);
-    
-    // Inform the container
-    rmContainer.handle(
-        new RMContainerFinishedEvent(
-            containerId,
-            containerStatus,
-            event)
-    );
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Completed container: " + rmContainer.getContainerId() +
-              " in state: " + rmContainer.getState() + " event:" + event);
-    }
+    try {
+      writeLock.lock();
+      Container container = rmContainer.getContainer();
+      ContainerId containerId = container.getId();
+
+      // Remove from the list of newly allocated containers if found
+      newlyAllocatedContainers.remove(rmContainer);
+
+      // Inform the container
+      rmContainer.handle(
+          new RMContainerFinishedEvent(containerId, containerStatus, event));
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Completed container: " + rmContainer.getContainerId()
+            + " in state: " + rmContainer.getState() + " event:" + event);
+      }
+
+      // Remove from the list of containers
+      liveContainers.remove(rmContainer.getContainerId());
 
-    // Remove from the list of containers
-    liveContainers.remove(rmContainer.getContainerId());
+      Resource containerResource = rmContainer.getContainer().getResource();
+      RMAuditLogger.logSuccess(getUser(), AuditConstants.RELEASE_CONTAINER,
+          "SchedulerApp", getApplicationId(), containerId, containerResource);
 
-    Resource containerResource = rmContainer.getContainer().getResource();
-    RMAuditLogger.logSuccess(getUser(), 
-        AuditConstants.RELEASE_CONTAINER, "SchedulerApp", 
-        getApplicationId(), containerId, containerResource);
-    
-    // Update usage metrics 
-    queue.getMetrics().releaseResources(getUser(), 1, containerResource);
-    this.attemptResourceUsage.decUsed(containerResource);
+      // Update usage metrics
+      queue.getMetrics().releaseResources(getUser(), 1, containerResource);
+      this.attemptResourceUsage.decUsed(containerResource);
 
-    // remove from preemption map if it is completed
-    preemptionMap.remove(rmContainer);
+      // remove from preemption map if it is completed
+      preemptionMap.remove(rmContainer);
 
-    // Clear resource utilization metrics cache.
-    lastMemoryAggregateAllocationUpdateTime = -1;
+      // Clear resource utilization metrics cache.
+      lastMemoryAggregateAllocationUpdateTime = -1;
+    } finally {
+      writeLock.unlock();
+    }
   }
 
-  private synchronized void unreserveInternal(
+  private void unreserveInternal(
       SchedulerRequestKey schedulerKey, FSSchedulerNode node) {
-    Map<NodeId, RMContainer> reservedContainers = 
-        this.reservedContainers.get(schedulerKey);
-    RMContainer reservedContainer = reservedContainers.remove(node.getNodeID());
-    if (reservedContainers.isEmpty()) {
-      this.reservedContainers.remove(schedulerKey);
-    }
-    
-    // Reset the re-reservation count
-    resetReReservations(schedulerKey);
+    try {
+      writeLock.lock();
+      Map<NodeId, RMContainer> reservedContainers = this.reservedContainers.get(
+          schedulerKey);
+      RMContainer reservedContainer = reservedContainers.remove(
+          node.getNodeID());
+      if (reservedContainers.isEmpty()) {
+        this.reservedContainers.remove(schedulerKey);
+      }
+
+      // Reset the re-reservation count
+      resetReReservations(schedulerKey);
 
-    Resource resource = reservedContainer.getContainer().getResource();
-    this.attemptResourceUsage.decReserved(resource);
+      Resource resource = reservedContainer.getContainer().getResource();
+      this.attemptResourceUsage.decReserved(resource);
 
-    LOG.info("Application " + getApplicationId() + " unreserved " + " on node "
-        + node + ", currently has " + reservedContainers.size()
-        + " at priority " + schedulerKey.getPriority() + "; currentReservation "
-        + this.attemptResourceUsage.getReserved());
+      LOG.info(
+          "Application " + getApplicationId() + " unreserved " + " on node "
+              + node + ", currently has " + reservedContainers.size()
+              + " at priority " + schedulerKey.getPriority()
+              + "; currentReservation " + this.attemptResourceUsage
+              .getReserved());
+    } finally {
+      writeLock.unlock();
+    }
   }
 
   private void subtractResourcesOnBlacklistedNodes(
@@ -239,17 +246,6 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
     return headroom;
   }
 
-  public synchronized float getLocalityWaitFactor(
-      SchedulerRequestKey schedulerKey, int clusterNodes) {
-    // Estimate: Required unique resources (i.e. hosts + racks)
-    int requiredResources = 
-        Math.max(this.getResourceRequests(schedulerKey).size() - 1, 0);
-    
-    // waitFactor can't be more than '1' 
-    // i.e. no point skipping more than clustersize opportunities
-    return Math.min(((float)requiredResources / clusterNodes), 1.0f);
-  }
-
   /**
    * Return the level at which we are allowed to schedule containers, given the
    * current size of the cluster and thresholds indicating how many nodes to
@@ -261,44 +257,56 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
    * @param rackLocalityThreshold rackLocalityThreshold
    * @return NodeType
    */
-  public synchronized NodeType getAllowedLocalityLevel(
+  NodeType getAllowedLocalityLevel(
       SchedulerRequestKey schedulerKey, int numNodes,
       double nodeLocalityThreshold, double rackLocalityThreshold) {
     // upper limit on threshold
-    if (nodeLocalityThreshold > 1.0) { nodeLocalityThreshold = 1.0; }
-    if (rackLocalityThreshold > 1.0) { rackLocalityThreshold = 1.0; }
+    if (nodeLocalityThreshold > 1.0) {
+      nodeLocalityThreshold = 1.0;
+    }
+    if (rackLocalityThreshold > 1.0) {
+      rackLocalityThreshold = 1.0;
+    }
 
     // If delay scheduling is not being used, can schedule anywhere
     if (nodeLocalityThreshold < 0.0 || rackLocalityThreshold < 0.0) {
       return NodeType.OFF_SWITCH;
     }
 
-    // Default level is NODE_LOCAL
-    if (!allowedLocalityLevel.containsKey(schedulerKey)) {
-      allowedLocalityLevel.put(schedulerKey, NodeType.NODE_LOCAL);
-      return NodeType.NODE_LOCAL;
-    }
-
-    NodeType allowed = allowedLocalityLevel.get(schedulerKey);
+    try {
+      writeLock.lock();
 
-    // If level is already most liberal, we're done
-    if (allowed.equals(NodeType.OFF_SWITCH)) return NodeType.OFF_SWITCH;
+      // Default level is NODE_LOCAL
+      if (!allowedLocalityLevel.containsKey(schedulerKey)) {
+        allowedLocalityLevel.put(schedulerKey, NodeType.NODE_LOCAL);
+        return NodeType.NODE_LOCAL;
+      }
 
-    double threshold = allowed.equals(NodeType.NODE_LOCAL) ? nodeLocalityThreshold :
-      rackLocalityThreshold;
+      NodeType allowed = allowedLocalityLevel.get(schedulerKey);
 
-    // Relax locality constraints once we've surpassed threshold.
-    if (getSchedulingOpportunities(schedulerKey) > (numNodes * threshold)) {
-      if (allowed.equals(NodeType.NODE_LOCAL)) {
-        allowedLocalityLevel.put(schedulerKey, NodeType.RACK_LOCAL);
-        resetSchedulingOpportunities(schedulerKey);
+      // If level is already most liberal, we're done
+      if (allowed.equals(NodeType.OFF_SWITCH)) {
+        return NodeType.OFF_SWITCH;
       }
-      else if (allowed.equals(NodeType.RACK_LOCAL)) {
-        allowedLocalityLevel.put(schedulerKey, NodeType.OFF_SWITCH);
-        resetSchedulingOpportunities(schedulerKey);
+
+      double threshold = allowed.equals(NodeType.NODE_LOCAL) ?
+          nodeLocalityThreshold :
+          rackLocalityThreshold;
+
+      // Relax locality constraints once we've surpassed threshold.
+      if (getSchedulingOpportunities(schedulerKey) > (numNodes * threshold)) {
+        if (allowed.equals(NodeType.NODE_LOCAL)) {
+          allowedLocalityLevel.put(schedulerKey, NodeType.RACK_LOCAL);
+          resetSchedulingOpportunities(schedulerKey);
+        } else if (allowed.equals(NodeType.RACK_LOCAL)) {
+          allowedLocalityLevel.put(schedulerKey, NodeType.OFF_SWITCH);
+          resetSchedulingOpportunities(schedulerKey);
+        }
       }
+      return allowedLocalityLevel.get(schedulerKey);
+    } finally {
+      writeLock.unlock();
     }
-    return allowedLocalityLevel.get(schedulerKey);
   }
 
   /**
@@ -311,119 +319,131 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
    * @param currentTimeMs currentTimeMs
    * @return NodeType
    */
-  public synchronized NodeType getAllowedLocalityLevelByTime(
+  NodeType getAllowedLocalityLevelByTime(
       SchedulerRequestKey schedulerKey, long nodeLocalityDelayMs,
       long rackLocalityDelayMs, long currentTimeMs) {
-
     // if not being used, can schedule anywhere
     if (nodeLocalityDelayMs < 0 || rackLocalityDelayMs < 0) {
       return NodeType.OFF_SWITCH;
     }
 
-    // default level is NODE_LOCAL
-    if (!allowedLocalityLevel.containsKey(schedulerKey)) {
-      // add the initial time of priority to prevent comparing with FsApp
-      // startTime and allowedLocalityLevel degrade
-      lastScheduledContainer.put(schedulerKey, currentTimeMs);
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Init the lastScheduledContainer time, priority: "
-            + schedulerKey.getPriority() + ", time: " + currentTimeMs);
+    try {
+      writeLock.lock();
+
+      // default level is NODE_LOCAL
+      if (!allowedLocalityLevel.containsKey(schedulerKey)) {
+        // add the initial time of priority to prevent comparing with FsApp
+        // startTime and allowedLocalityLevel degrade
+        lastScheduledContainer.put(schedulerKey, currentTimeMs);
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(
+              "Init the lastScheduledContainer time, priority: " + schedulerKey
+                  .getPriority() + ", time: " + currentTimeMs);
+        }
+        allowedLocalityLevel.put(schedulerKey, NodeType.NODE_LOCAL);
+        return NodeType.NODE_LOCAL;
       }
-      allowedLocalityLevel.put(schedulerKey, NodeType.NODE_LOCAL);
-      return NodeType.NODE_LOCAL;
-    }
 
-    NodeType allowed = allowedLocalityLevel.get(schedulerKey);
+      NodeType allowed = allowedLocalityLevel.get(schedulerKey);
 
-    // if level is already most liberal, we're done
-    if (allowed.equals(NodeType.OFF_SWITCH)) {
-      return NodeType.OFF_SWITCH;
-    }
-
-    // check waiting time
-    long waitTime = currentTimeMs;
-    if (lastScheduledContainer.containsKey(schedulerKey)) {
-      waitTime -= lastScheduledContainer.get(schedulerKey);
-    } else {
-      waitTime -= getStartTime();
-    }
+      // if level is already most liberal, we're done
+      if (allowed.equals(NodeType.OFF_SWITCH)) {
+        return NodeType.OFF_SWITCH;
+      }
 
-    long thresholdTime = allowed.equals(NodeType.NODE_LOCAL) ?
-            nodeLocalityDelayMs : rackLocalityDelayMs;
+      // check waiting time
+      long waitTime = currentTimeMs;
+      if (lastScheduledContainer.containsKey(schedulerKey)) {
+        waitTime -= lastScheduledContainer.get(schedulerKey);
+      } else{
+        waitTime -= getStartTime();
+      }
 
-    if (waitTime > thresholdTime) {
-      if (allowed.equals(NodeType.NODE_LOCAL)) {
-        allowedLocalityLevel.put(schedulerKey, NodeType.RACK_LOCAL);
-        resetSchedulingOpportunities(schedulerKey, currentTimeMs);
-      } else if (allowed.equals(NodeType.RACK_LOCAL)) {
-        allowedLocalityLevel.put(schedulerKey, NodeType.OFF_SWITCH);
-        resetSchedulingOpportunities(schedulerKey, currentTimeMs);
+      long thresholdTime = allowed.equals(NodeType.NODE_LOCAL) ?
+          nodeLocalityDelayMs :
+          rackLocalityDelayMs;
+
+      if (waitTime > thresholdTime) {
+        if (allowed.equals(NodeType.NODE_LOCAL)) {
+          allowedLocalityLevel.put(schedulerKey, NodeType.RACK_LOCAL);
+          resetSchedulingOpportunities(schedulerKey, currentTimeMs);
+        } else if (allowed.equals(NodeType.RACK_LOCAL)) {
+          allowedLocalityLevel.put(schedulerKey, NodeType.OFF_SWITCH);
+          resetSchedulingOpportunities(schedulerKey, currentTimeMs);
+        }
       }
+      return allowedLocalityLevel.get(schedulerKey);
+    } finally {
+      writeLock.unlock();
     }
-    return allowedLocalityLevel.get(schedulerKey);
   }
 
-  synchronized public RMContainer allocate(NodeType type, FSSchedulerNode node,
+  public RMContainer allocate(NodeType type, FSSchedulerNode node,
       SchedulerRequestKey schedulerKey, ResourceRequest request,
       Container reservedContainer) {
-    // Update allowed locality level
-    NodeType allowed = allowedLocalityLevel.get(schedulerKey);
-    if (allowed != null) {
-      if (allowed.equals(NodeType.OFF_SWITCH) &&
-          (type.equals(NodeType.NODE_LOCAL) ||
-              type.equals(NodeType.RACK_LOCAL))) {
-        this.resetAllowedLocalityLevel(schedulerKey, type);
+    RMContainer rmContainer;
+    Container container;
+
+    try {
+      writeLock.lock();
+      // Update allowed locality level
+      NodeType allowed = allowedLocalityLevel.get(schedulerKey);
+      if (allowed != null) {
+        if (allowed.equals(NodeType.OFF_SWITCH) && (type.equals(
+            NodeType.NODE_LOCAL) || type.equals(NodeType.RACK_LOCAL))) {
+          this.resetAllowedLocalityLevel(schedulerKey, type);
+        } else if (allowed.equals(NodeType.RACK_LOCAL) && type.equals(
+            NodeType.NODE_LOCAL)) {
+          this.resetAllowedLocalityLevel(schedulerKey, type);
+        }
       }
-      else if (allowed.equals(NodeType.RACK_LOCAL) &&
-          type.equals(NodeType.NODE_LOCAL)) {
-        this.resetAllowedLocalityLevel(schedulerKey, type);
+
+      // Required sanity check - AM can call 'allocate' to update resource
+      // request without locking the scheduler, hence we need to check
+      if (getTotalRequiredResources(schedulerKey) <= 0) {
+        return null;
       }
-    }
 
-    // Required sanity check - AM can call 'allocate' to update resource 
-    // request without locking the scheduler, hence we need to check
-    if (getTotalRequiredResources(schedulerKey) <= 0) {
-      return null;
-    }
+      container = reservedContainer;
+      if (container == null) {
+        container = createContainer(node, request.getCapability(),
+            schedulerKey);
+      }
 
-    Container container = reservedContainer;
-    if (container == null) {
-      container =
-          createContainer(node, request.getCapability(), schedulerKey);
-    }
-    
-    // Create RMContainer
-    RMContainer rmContainer = new RMContainerImpl(container,
-        getApplicationAttemptId(), node.getNodeID(),
-        appSchedulingInfo.getUser(), rmContext);
-    ((RMContainerImpl)rmContainer).setQueueName(this.getQueueName());
+      // Create RMContainer
+      rmContainer = new RMContainerImpl(container,
+          getApplicationAttemptId(), node.getNodeID(),
+          appSchedulingInfo.getUser(), rmContext);
+      ((RMContainerImpl) rmContainer).setQueueName(this.getQueueName());
 
-    // Add it to allContainers list.
-    newlyAllocatedContainers.add(rmContainer);
-    liveContainers.put(container.getId(), rmContainer);    
+      // Add it to allContainers list.
+      newlyAllocatedContainers.add(rmContainer);
+      liveContainers.put(container.getId(), rmContainer);
 
-    // Update consumption and track allocations
-    List<ResourceRequest> resourceRequestList = appSchedulingInfo.allocate(
-        type, node, schedulerKey, request, container);
-    this.attemptResourceUsage.incUsed(container.getResource());
+      // Update consumption and track allocations
+      List<ResourceRequest> resourceRequestList = appSchedulingInfo.allocate(
+          type, node, schedulerKey, request, container);
+      this.attemptResourceUsage.incUsed(container.getResource());
 
-    // Update resource requests related to "request" and store in RMContainer
-    ((RMContainerImpl) rmContainer).setResourceRequests(resourceRequestList);
+      // Update resource requests related to "request" and store in RMContainer
+      ((RMContainerImpl) rmContainer).setResourceRequests(resourceRequestList);
 
-    // Inform the container
-    rmContainer.handle(
-        new RMContainerEvent(container.getId(), RMContainerEventType.START));
+      // Inform the container
+      rmContainer.handle(
+          new RMContainerEvent(container.getId(), RMContainerEventType.START));
+
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("allocate: applicationAttemptId=" + container.getId()
+            .getApplicationAttemptId() + " container=" + container.getId()
+            + " host=" + container.getNodeId().getHost() + " type=" + type);
+      }
+      RMAuditLogger.logSuccess(getUser(), AuditConstants.ALLOC_CONTAINER,
+          "SchedulerApp", getApplicationId(), container.getId(),
+          container.getResource());
+    } finally {
+      writeLock.unlock();
+    }
 
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("allocate: applicationAttemptId=" 
-          + container.getId().getApplicationAttemptId() 
-          + " container=" + container.getId() + " host="
-          + container.getNodeId().getHost() + " type=" + type);
-    }
-    RMAuditLogger.logSuccess(getUser(), 
-        AuditConstants.ALLOC_CONTAINER, "SchedulerApp", 
-        getApplicationId(), container.getId(), container.getResource());
-    
     return rmContainer;
   }
 
@@ -434,19 +454,30 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
    * @param schedulerKey Scheduler Key
    * @param level NodeType
    */
-  public synchronized void resetAllowedLocalityLevel(
+  public void resetAllowedLocalityLevel(
       SchedulerRequestKey schedulerKey, NodeType level) {
-    NodeType old = allowedLocalityLevel.get(schedulerKey);
-    LOG.info("Raising locality level from " + old + " to " + level + " at " +
-        " priority " + schedulerKey.getPriority());
-    allowedLocalityLevel.put(schedulerKey, level);
+    NodeType old;
+    try {
+      writeLock.lock();
+      old = allowedLocalityLevel.put(schedulerKey, level);
+    } finally {
+      writeLock.unlock();
+    }
+
+    LOG.info("Raising locality level from " + old + " to " + level + " at "
+        + " priority " + schedulerKey.getPriority());
   }
 
   // related methods
   public void addPreemption(RMContainer container, long time) {
     assert preemptionMap.get(container) == null;
-    preemptionMap.put(container, time);
-    Resources.addTo(preemptedResources, container.getAllocatedResource());
+    try {
+      writeLock.lock();
+      preemptionMap.put(container, time);
+      Resources.addTo(preemptedResources, container.getAllocatedResource());
+    } finally {
+      writeLock.unlock();
+    }
   }
 
   public Long getContainerPreemptionTime(RMContainer container) {
@@ -584,21 +615,35 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
         getUser(), rmContainer.getContainer().getResource());
   }
 
-  private synchronized void setReservation(SchedulerNode node) {
-    String rackName = node.getRackName() == null ? "NULL" : node.getRackName();
-    Set<String> rackReservations = reservations.get(rackName);
-    if (rackReservations == null) {
-      rackReservations = new HashSet<>();
-      reservations.put(rackName, rackReservations);
+  private void setReservation(SchedulerNode node) {
+    String rackName =
+        node.getRackName() == null ? "NULL" : node.getRackName();
+
+    try {
+      writeLock.lock();
+      Set<String> rackReservations = reservations.get(rackName);
+      if (rackReservations == null) {
+        rackReservations = new HashSet<>();
+        reservations.put(rackName, rackReservations);
+      }
+      rackReservations.add(node.getNodeName());
+    } finally {
+      writeLock.unlock();
     }
-    rackReservations.add(node.getNodeName());
   }
 
-  private synchronized void clearReservation(SchedulerNode node) {
-    String rackName = node.getRackName() == null ? "NULL" : node.getRackName();
-    Set<String> rackReservations = reservations.get(rackName);
-    if (rackReservations != null) {
-      rackReservations.remove(node.getNodeName());
+  private void clearReservation(SchedulerNode node) {
+    String rackName =
+        node.getRackName() == null ? "NULL" : node.getRackName();
+
+    try {
+      writeLock.lock();
+      Set<String> rackReservations = reservations.get(rackName);
+      if (rackReservations != null) {
+        rackReservations.remove(node.getNodeName());
+      }
+    } finally {
+      writeLock.unlock();
     }
   }
 
@@ -737,7 +782,8 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
     // For each priority, see if we can schedule a node local, rack local
     // or off-switch request. Rack of off-switch requests may be delayed
     // (not scheduled) in order to promote better locality.
-    synchronized (this) {
+    try {
+      writeLock.lock();
       for (SchedulerRequestKey schedulerKey : keysToTry) {
         // Skip it for reserved container, since
         // we already check it in isValidReservation.
@@ -772,8 +818,8 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
 
         if (rackLocalRequest != null && rackLocalRequest.getNumContainers() != 0
             && localRequest != null && localRequest.getNumContainers() != 0) {
-          return assignContainer(node, localRequest,
-              NodeType.NODE_LOCAL, reserved, schedulerKey);
+          return assignContainer(node, localRequest, NodeType.NODE_LOCAL,
+              reserved, schedulerKey);
         }
 
         if (rackLocalRequest != null && !rackLocalRequest.getRelaxLocality()) {
@@ -781,29 +827,31 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
         }
 
         if (rackLocalRequest != null && rackLocalRequest.getNumContainers() != 0
-            && (allowedLocality.equals(NodeType.RACK_LOCAL) ||
-            allowedLocality.equals(NodeType.OFF_SWITCH))) {
-          return assignContainer(node, rackLocalRequest,
-              NodeType.RACK_LOCAL, reserved, schedulerKey);
+            && (allowedLocality.equals(NodeType.RACK_LOCAL) || allowedLocality
+            .equals(NodeType.OFF_SWITCH))) {
+          return assignContainer(node, rackLocalRequest, NodeType.RACK_LOCAL,
+              reserved, schedulerKey);
         }
 
-        ResourceRequest offSwitchRequest =
-            getResourceRequest(schedulerKey, ResourceRequest.ANY);
+        ResourceRequest offSwitchRequest = getResourceRequest(schedulerKey,
+            ResourceRequest.ANY);
         if (offSwitchRequest != null && !offSwitchRequest.getRelaxLocality()) {
           continue;
         }
 
-        if (offSwitchRequest != null &&
-            offSwitchRequest.getNumContainers() != 0) {
-          if (!hasNodeOrRackLocalRequests(schedulerKey) ||
-              allowedLocality.equals(NodeType.OFF_SWITCH)) {
-            return assignContainer(
-                node, offSwitchRequest, NodeType.OFF_SWITCH, reserved,
-                schedulerKey);
+        if (offSwitchRequest != null
+            && offSwitchRequest.getNumContainers() != 0) {
+          if (!hasNodeOrRackLocalRequests(schedulerKey) || allowedLocality
+              .equals(NodeType.OFF_SWITCH)) {
+            return assignContainer(node, offSwitchRequest, NodeType.OFF_SWITCH,
+                reserved, schedulerKey);
           }
         }
       }
+    } finally {
+      writeLock.unlock();
     }
+
     return Resources.none();
   }
 
@@ -963,14 +1011,17 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
     Resources.addTo(demand, getCurrentConsumption());
 
     // Add up outstanding resource requests
-    synchronized (this) {
+    try {
+      writeLock.lock();
       for (SchedulerRequestKey k : getSchedulerKeys()) {
         ResourceRequest r = getResourceRequest(k, ResourceRequest.ANY);
         if (r != null) {
-          Resources.multiplyAndAddTo(demand,
-              r.getCapability(), r.getNumContainers());
+          Resources.multiplyAndAddTo(demand, r.getCapability(),
+              r.getNumContainers());
         }
       }
+    } finally {
+      writeLock.unlock();
     }
   }
 


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


[34/44] 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/888c1500/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/44] hadoop git commit: YARN-3141. Improve locks in SchedulerApplicationAttempt/FSAppAttempt/FiCaSchedulerApp. Contributed by Wangda Tan

Posted by su...@apache.org.
YARN-3141. Improve locks in SchedulerApplicationAttempt/FSAppAttempt/FiCaSchedulerApp. Contributed by Wangda Tan


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

Branch: refs/heads/YARN-2915
Commit: b8a30f2f170ffbd590e7366c3c944ab4919e40df
Parents: ea29e3b
Author: Jian He <ji...@apache.org>
Authored: Mon Sep 19 16:58:39 2016 +0800
Committer: Jian He <ji...@apache.org>
Committed: Mon Sep 19 17:08:01 2016 +0800

----------------------------------------------------------------------
 .../scheduler/SchedulerApplicationAttempt.java  | 744 +++++++++++--------
 .../allocator/RegularContainerAllocator.java    |   2 +-
 .../scheduler/common/fica/FiCaSchedulerApp.java | 418 ++++++-----
 .../scheduler/fair/FSAppAttempt.java            | 465 ++++++------
 4 files changed, 922 insertions(+), 707 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b8a30f2f/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 97d29cf..adc3a97 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
@@ -26,8 +26,11 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
 
+import com.google.common.collect.ConcurrentHashMultiset;
 import org.apache.commons.lang.time.DateUtils;
 import org.apache.commons.lang.time.FastDateFormat;
 import org.apache.commons.logging.Log;
@@ -71,8 +74,6 @@ import org.apache.hadoop.yarn.util.resource.Resources;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
-import com.google.common.collect.HashMultiset;
-import com.google.common.collect.Multiset;
 
 /**
  * Represents an application attempt from the viewpoint of the scheduler.
@@ -97,14 +98,14 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
   protected final AppSchedulingInfo appSchedulingInfo;
   protected ApplicationAttemptId attemptId;
   protected Map<ContainerId, RMContainer> liveContainers =
-      new HashMap<ContainerId, RMContainer>();
+      new ConcurrentHashMap<>();
   protected final Map<SchedulerRequestKey, Map<NodeId, RMContainer>>
       reservedContainers = new HashMap<>();
 
-  private final Multiset<SchedulerRequestKey> reReservations =
-      HashMultiset.create();
+  private final ConcurrentHashMultiset<SchedulerRequestKey> reReservations =
+      ConcurrentHashMultiset.create();
   
-  private Resource resourceLimit = Resource.newInstance(0, 0);
+  private volatile Resource resourceLimit = Resource.newInstance(0, 0);
   private boolean unmanagedAM = true;
   private boolean amRunning = false;
   private LogAggregationContext logAggregationContext;
@@ -138,8 +139,9 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
    * the application successfully schedules a task (at rack or node local), it
    * is reset to 0.
    */
-  Multiset<SchedulerRequestKey> schedulingOpportunities = HashMultiset.create();
-  
+  private ConcurrentHashMultiset<SchedulerRequestKey> schedulingOpportunities =
+      ConcurrentHashMultiset.create();
+
   /**
    * Count how many times the application has been given an opportunity to
    * schedule a non-partitioned resource request at each priority. Each time the
@@ -147,15 +149,16 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
    * incremented, and each time the application successfully schedules a task,
    * it is reset to 0 when schedule any task at corresponding priority.
    */
-  Multiset<SchedulerRequestKey> missedNonPartitionedReqSchedulingOpportunity =
-      HashMultiset.create();
+  private ConcurrentHashMultiset<SchedulerRequestKey>
+      missedNonPartitionedReqSchedulingOpportunity =
+      ConcurrentHashMultiset.create();
   
   // Time of the last container scheduled at the current allowed level
   protected Map<SchedulerRequestKey, Long> lastScheduledContainer =
-      new HashMap<>();
+      new ConcurrentHashMap<>();
 
-  protected Queue queue;
-  protected boolean isStopped = false;
+  protected volatile Queue queue;
+  protected volatile boolean isStopped = false;
 
   protected String appAMNodePartitionName = CommonNodeLabelsManager.NO_LABEL;
 
@@ -163,6 +166,9 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
 
   private RMAppAttempt appAttempt;
 
+  protected ReentrantReadWriteLock.ReadLock readLock;
+  protected ReentrantReadWriteLock.WriteLock writeLock;
+
   public SchedulerApplicationAttempt(ApplicationAttemptId applicationAttemptId, 
       String user, Queue queue, ActiveUsersManager activeUsersManager,
       RMContext rmContext) {
@@ -188,14 +194,23 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
             appSubmissionContext.getLogAggregationContext();
       }
     }
+
+    ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+    readLock = lock.readLock();
+    writeLock = lock.writeLock();
   }
   
   /**
    * Get the live containers of the application.
    * @return live containers of the application
    */
-  public synchronized Collection<RMContainer> getLiveContainers() {
-    return new ArrayList<RMContainer>(liveContainers.values());
+  public Collection<RMContainer> getLiveContainers() {
+    try {
+      readLock.lock();
+      return new ArrayList<>(liveContainers.values());
+    } finally {
+      readLock.unlock();
+    }
   }
 
   public AppSchedulingInfo getAppSchedulingInfo() {
@@ -243,20 +258,36 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
     return appSchedulingInfo.getSchedulerKeys();
   }
   
-  public synchronized ResourceRequest getResourceRequest(
+  public ResourceRequest getResourceRequest(
       SchedulerRequestKey schedulerKey, String resourceName) {
-    return appSchedulingInfo.getResourceRequest(schedulerKey, resourceName);
+    try {
+      readLock.lock();
+      return appSchedulingInfo.getResourceRequest(schedulerKey, resourceName);
+    } finally {
+      readLock.unlock();
+    }
+
   }
 
-  public synchronized int getTotalRequiredResources(
+  public int getTotalRequiredResources(
       SchedulerRequestKey schedulerKey) {
-    ResourceRequest request =
-        getResourceRequest(schedulerKey, ResourceRequest.ANY);
-    return request == null ? 0 : request.getNumContainers();
+    try {
+      readLock.lock();
+      ResourceRequest request =
+          getResourceRequest(schedulerKey, ResourceRequest.ANY);
+      return request == null ? 0 : request.getNumContainers();
+    } finally {
+      readLock.unlock();
+    }
   }
 
-  public synchronized Resource getResource(SchedulerRequestKey schedulerKey) {
-    return appSchedulingInfo.getResource(schedulerKey);
+  public Resource getResource(SchedulerRequestKey schedulerKey) {
+    try {
+      readLock.lock();
+      return appSchedulingInfo.getResource(schedulerKey);
+    } finally {
+      readLock.unlock();
+    }
   }
 
   public String getQueueName() {
@@ -291,38 +322,48 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
     return unmanagedAM;
   }
 
-  public synchronized RMContainer getRMContainer(ContainerId id) {
+  public RMContainer getRMContainer(ContainerId id) {
     return liveContainers.get(id);
   }
 
-  public synchronized void addRMContainer(
+  public void addRMContainer(
       ContainerId id, RMContainer rmContainer) {
-    liveContainers.put(id, rmContainer);
-    if (rmContainer.isRemotelyAllocated()) {
-      this.attemptResourceUsageAllocatedRemotely.incUsed(
-          rmContainer.getAllocatedResource());
+    try {
+      writeLock.lock();
+      liveContainers.put(id, rmContainer);
+      if (rmContainer.isRemotelyAllocated()) {
+        this.attemptResourceUsageAllocatedRemotely.incUsed(
+            rmContainer.getAllocatedResource());
+      }
+    } finally {
+      writeLock.unlock();
     }
   }
 
-  public synchronized void removeRMContainer(ContainerId containerId) {
-    RMContainer rmContainer = liveContainers.remove(containerId);
-    if (rmContainer != null && rmContainer.isRemotelyAllocated()) {
-      this.attemptResourceUsageAllocatedRemotely.decUsed(
-          rmContainer.getAllocatedResource());
+  public void removeRMContainer(ContainerId containerId) {
+    try {
+      writeLock.lock();
+      RMContainer rmContainer = liveContainers.remove(containerId);
+      if (rmContainer != null && rmContainer.isRemotelyAllocated()) {
+        this.attemptResourceUsageAllocatedRemotely.decUsed(
+            rmContainer.getAllocatedResource());
+      }
+    } finally {
+      writeLock.unlock();
     }
   }
 
-  protected synchronized void resetReReservations(
+  protected void resetReReservations(
       SchedulerRequestKey schedulerKey) {
     reReservations.setCount(schedulerKey, 0);
   }
 
-  protected synchronized void addReReservation(
+  protected void addReReservation(
       SchedulerRequestKey schedulerKey) {
     reReservations.add(schedulerKey);
   }
 
-  public synchronized int getReReservations(SchedulerRequestKey schedulerKey) {
+  public int getReReservations(SchedulerRequestKey schedulerKey) {
     return reReservations.count(schedulerKey);
   }
 
@@ -333,7 +374,7 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
    */
   @Stable
   @Private
-  public synchronized Resource getCurrentReservation() {
+  public Resource getCurrentReservation() {
     return attemptResourceUsage.getReserved();
   }
   
@@ -341,28 +382,43 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
     return queue;
   }
   
-  public synchronized boolean updateResourceRequests(
+  public boolean updateResourceRequests(
       List<ResourceRequest> requests) {
-    if (!isStopped) {
-      return appSchedulingInfo.updateResourceRequests(requests, false);
+    try {
+      writeLock.lock();
+      if (!isStopped) {
+        return appSchedulingInfo.updateResourceRequests(requests, false);
+      }
+      return false;
+    } finally {
+      writeLock.unlock();
     }
-    return false;
   }
   
-  public synchronized void recoverResourceRequestsForContainer(
+  public void recoverResourceRequestsForContainer(
       List<ResourceRequest> requests) {
-    if (!isStopped) {
-      appSchedulingInfo.updateResourceRequests(requests, true);
+    try {
+      writeLock.lock();
+      if (!isStopped) {
+        appSchedulingInfo.updateResourceRequests(requests, true);
+      }
+    } finally {
+      writeLock.unlock();
     }
   }
   
-  public synchronized void stop(RMAppAttemptState rmAppAttemptFinalState) {
-    // Cleanup all scheduling information
-    isStopped = true;
-    appSchedulingInfo.stop();
+  public void stop(RMAppAttemptState rmAppAttemptFinalState) {
+    try {
+      writeLock.lock();
+      // Cleanup all scheduling information
+      isStopped = true;
+      appSchedulingInfo.stop();
+    } finally {
+      writeLock.unlock();
+    }
   }
 
-  public synchronized boolean isStopped() {
+  public boolean isStopped() {
     return isStopped;
   }
 
@@ -370,29 +426,40 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
    * Get the list of reserved containers
    * @return All of the reserved containers.
    */
-  public synchronized List<RMContainer> getReservedContainers() {
-    List<RMContainer> reservedContainers = new ArrayList<RMContainer>();
-    for (Map.Entry<SchedulerRequestKey, Map<NodeId, RMContainer>> e :
-      this.reservedContainers.entrySet()) {
-      reservedContainers.addAll(e.getValue().values());
+  public List<RMContainer> getReservedContainers() {
+    List<RMContainer> list = new ArrayList<>();
+    try {
+      readLock.lock();
+      for (Entry<SchedulerRequestKey, Map<NodeId, RMContainer>> e :
+          this.reservedContainers.entrySet()) {
+        list.addAll(e.getValue().values());
+      }
+      return list;
+    } finally {
+      readLock.unlock();
     }
-    return reservedContainers;
+
   }
   
-  public synchronized boolean reserveIncreasedContainer(SchedulerNode node,
+  public boolean reserveIncreasedContainer(SchedulerNode node,
       SchedulerRequestKey schedulerKey, RMContainer rmContainer,
       Resource reservedResource) {
-    if (commonReserve(node, schedulerKey, rmContainer, reservedResource)) {
-      attemptResourceUsage.incReserved(node.getPartition(),
-          reservedResource);
-      // succeeded
-      return true;
+    try {
+      writeLock.lock();
+      if (commonReserve(node, schedulerKey, rmContainer, reservedResource)) {
+        attemptResourceUsage.incReserved(node.getPartition(), reservedResource);
+        // succeeded
+        return true;
+      }
+
+      return false;
+    } finally {
+      writeLock.unlock();
     }
-    
-    return false;
+
   }
   
-  private synchronized boolean commonReserve(SchedulerNode node,
+  private boolean commonReserve(SchedulerNode node,
       SchedulerRequestKey schedulerKey, RMContainer rmContainer,
       Resource reservedResource) {
     try {
@@ -423,101 +490,100 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
     return true;
   }
   
-  public synchronized RMContainer reserve(SchedulerNode node,
+  public RMContainer reserve(SchedulerNode node,
       SchedulerRequestKey schedulerKey, RMContainer rmContainer,
       Container container) {
-    // Create RMContainer if necessary
-    if (rmContainer == null) {
-      rmContainer =
-          new RMContainerImpl(container, getApplicationAttemptId(),
-              node.getNodeID(), appSchedulingInfo.getUser(), rmContext);
-      attemptResourceUsage.incReserved(node.getPartition(),
-          container.getResource());
-      ((RMContainerImpl)rmContainer).setQueueName(this.getQueueName());
-
-      // Reset the re-reservation count
-      resetReReservations(schedulerKey);
-    } else {
-      // Note down the re-reservation
-      addReReservation(schedulerKey);
-    }
-    
-    commonReserve(node, schedulerKey, rmContainer, container.getResource());
+    try {
+      writeLock.lock();
+      // Create RMContainer if necessary
+      if (rmContainer == null) {
+        rmContainer = new RMContainerImpl(container, getApplicationAttemptId(),
+            node.getNodeID(), appSchedulingInfo.getUser(), rmContext);
+        attemptResourceUsage.incReserved(node.getPartition(),
+            container.getResource());
+        ((RMContainerImpl) rmContainer).setQueueName(this.getQueueName());
+
+        // Reset the re-reservation count
+        resetReReservations(schedulerKey);
+      } else{
+        // Note down the re-reservation
+        addReReservation(schedulerKey);
+      }
 
-    return rmContainer;
-  }
-  
-  /**
-   * Has the application reserved the given <code>node</code> at the
-   * given <code>priority</code>?
-   * @param node node to be checked
-   * @param schedulerKey scheduler key  of reserved container
-   * @return true is reserved, false if not
-   */
-  public synchronized boolean isReserved(SchedulerNode node,
-      SchedulerRequestKey schedulerKey) {
-    Map<NodeId, RMContainer> reservedContainers = 
-        this.reservedContainers.get(schedulerKey);
-    if (reservedContainers != null) {
-      return reservedContainers.containsKey(node.getNodeID());
+      commonReserve(node, schedulerKey, rmContainer, container.getResource());
+
+      return rmContainer;
+    } finally {
+      writeLock.unlock();
     }
-    return false;
+
   }
-  
-  public synchronized void setHeadroom(Resource globalLimit) {
-    this.resourceLimit = globalLimit; 
+
+  public void setHeadroom(Resource globalLimit) {
+    this.resourceLimit = Resources.componentwiseMax(globalLimit,
+        Resources.none());
   }
 
   /**
    * Get available headroom in terms of resources for the application's user.
    * @return available resource headroom
    */
-  public synchronized Resource getHeadroom() {
-    // Corner case to deal with applications being slightly over-limit
-    if (resourceLimit.getMemorySize() < 0) {
-      resourceLimit.setMemorySize(0);
-    }
-    
+  public Resource getHeadroom() {
     return resourceLimit;
   }
   
-  public synchronized int getNumReservedContainers(
+  public int getNumReservedContainers(
       SchedulerRequestKey schedulerKey) {
-    Map<NodeId, RMContainer> reservedContainers = 
-        this.reservedContainers.get(schedulerKey);
-    return (reservedContainers == null) ? 0 : reservedContainers.size();
+    try {
+      readLock.lock();
+      Map<NodeId, RMContainer> map = this.reservedContainers.get(
+          schedulerKey);
+      return (map == null) ? 0 : map.size();
+    } finally {
+      readLock.unlock();
+    }
   }
   
   @SuppressWarnings("unchecked")
-  public synchronized void containerLaunchedOnNode(ContainerId containerId,
+  public void containerLaunchedOnNode(ContainerId containerId,
       NodeId nodeId) {
-    // Inform the container
-    RMContainer rmContainer = getRMContainer(containerId);
-    if (rmContainer == null) {
-      // Some unknown container sneaked into the system. Kill it.
-      rmContext.getDispatcher().getEventHandler()
-        .handle(new RMNodeCleanContainerEvent(nodeId, containerId));
-      return;
-    }
+    try {
+      writeLock.lock();
+      // Inform the container
+      RMContainer rmContainer = getRMContainer(containerId);
+      if (rmContainer == null) {
+        // Some unknown container sneaked into the system. Kill it.
+        rmContext.getDispatcher().getEventHandler().handle(
+            new RMNodeCleanContainerEvent(nodeId, containerId));
+        return;
+      }
 
-    rmContainer.handle(new RMContainerEvent(containerId,
-        RMContainerEventType.LAUNCHED));
+      rmContainer.handle(
+          new RMContainerEvent(containerId, RMContainerEventType.LAUNCHED));
+    } finally {
+      writeLock.unlock();
+    }
   }
   
-  public synchronized void showRequests() {
+  public void showRequests() {
     if (LOG.isDebugEnabled()) {
-      for (SchedulerRequestKey schedulerKey : getSchedulerKeys()) {
-        Map<String, ResourceRequest> requests =
-            getResourceRequests(schedulerKey);
-        if (requests != null) {
-          LOG.debug("showRequests:" + " application=" + getApplicationId()
-              + " headRoom=" + getHeadroom() + " currentConsumption="
-              + attemptResourceUsage.getUsed().getMemorySize());
-          for (ResourceRequest request : requests.values()) {
+      try {
+        readLock.lock();
+        for (SchedulerRequestKey schedulerKey : getSchedulerKeys()) {
+          Map<String, ResourceRequest> requests = getResourceRequests(
+              schedulerKey);
+          if (requests != null) {
             LOG.debug("showRequests:" + " application=" + getApplicationId()
-                + " request=" + request);
+                + " headRoom=" + getHeadroom() + " currentConsumption="
+                + attemptResourceUsage.getUsed().getMemorySize());
+            for (ResourceRequest request : requests.values()) {
+              LOG.debug("showRequests:" + " application=" + getApplicationId()
+                  + " request=" + request);
+            }
           }
         }
+      } finally {
+        readLock.unlock();
       }
     }
   }
@@ -572,54 +638,75 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
   // 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.
-  public synchronized List<Container> pullNewlyAllocatedContainers() {
-    List<Container> returnContainerList =
-        new ArrayList<Container>(newlyAllocatedContainers.size());
-    for (Iterator<RMContainer> i = newlyAllocatedContainers.iterator(); i
-        .hasNext();) {
-      RMContainer rmContainer = i.next();
-      Container updatedContainer =
-          updateContainerAndNMToken(rmContainer, true, false);
-      // Only add container to return list when it's not null. updatedContainer
-      // could be null when generate token failed, it can be caused by DNS
-      // resolving failed.
-      if (updatedContainer != null) {
-        returnContainerList.add(updatedContainer);
-        i.remove();
+  public List<Container> pullNewlyAllocatedContainers() {
+    try {
+      writeLock.lock();
+      List<Container> returnContainerList = new ArrayList<Container>(
+          newlyAllocatedContainers.size());
+
+      Iterator<RMContainer> i = newlyAllocatedContainers.iterator();
+      while (i.hasNext()) {
+        RMContainer rmContainer = i.next();
+        Container updatedContainer = updateContainerAndNMToken(rmContainer,
+            true, false);
+        // Only add container to return list when it's not null.
+        // updatedContainer could be null when generate token failed, it can be
+        // caused by DNS resolving failed.
+        if (updatedContainer != null) {
+          returnContainerList.add(updatedContainer);
+          i.remove();
+        }
       }
+      return returnContainerList;
+    } finally {
+      writeLock.unlock();
     }
-    return returnContainerList;
+
   }
   
-  private synchronized List<Container> pullNewlyUpdatedContainers(
+  private List<Container> pullNewlyUpdatedContainers(
       Map<ContainerId, RMContainer> updatedContainerMap, boolean increase) {
-    List<Container> returnContainerList =
-        new ArrayList<Container>(updatedContainerMap.size());
-    for (Iterator<Entry<ContainerId, RMContainer>> i =
-        updatedContainerMap.entrySet().iterator(); i.hasNext();) {
-      RMContainer rmContainer = i.next().getValue();
-      Container updatedContainer =
-          updateContainerAndNMToken(rmContainer, false, increase);
-      if (updatedContainer != null) {
-        returnContainerList.add(updatedContainer);
-        i.remove();
+    try {
+      writeLock.lock();
+      List <Container> returnContainerList = new ArrayList <Container>(
+          updatedContainerMap.size());
+
+      Iterator<Entry<ContainerId, RMContainer>> i =
+          updatedContainerMap.entrySet().iterator();
+      while (i.hasNext()) {
+        RMContainer rmContainer = i.next().getValue();
+        Container updatedContainer = updateContainerAndNMToken(rmContainer,
+            false, increase);
+        if (updatedContainer != null) {
+          returnContainerList.add(updatedContainer);
+          i.remove();
+        }
       }
+      return returnContainerList;
+    } finally {
+      writeLock.unlock();
     }
-    return returnContainerList;
+
   }
 
-  public synchronized List<Container> pullNewlyIncreasedContainers() {
+  public List<Container> pullNewlyIncreasedContainers() {
     return pullNewlyUpdatedContainers(newlyIncreasedContainers, true);
   }
   
-  public synchronized List<Container> pullNewlyDecreasedContainers() {
+  public List<Container> pullNewlyDecreasedContainers() {
     return pullNewlyUpdatedContainers(newlyDecreasedContainers, false);
   }
   
-  public synchronized List<NMToken> pullUpdatedNMTokens() {
-    List<NMToken> returnList = new ArrayList<NMToken>(updatedNMTokens);
-    updatedNMTokens.clear();
-    return returnList;
+  public List<NMToken> pullUpdatedNMTokens() {
+    try {
+      writeLock.lock();
+      List <NMToken> returnList = new ArrayList<>(updatedNMTokens);
+      updatedNMTokens.clear();
+      return returnList;
+    } finally {
+      writeLock.unlock();
+    }
+
   }
 
   public boolean isWaitingForAMContainer() {
@@ -628,53 +715,63 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
     return (!unmanagedAM && appAttempt.getMasterContainer() == null);
   }
 
-  public synchronized void updateBlacklist(List<String> blacklistAdditions,
+  public void updateBlacklist(List<String> blacklistAdditions,
       List<String> blacklistRemovals) {
-    if (!isStopped) {
-      if (isWaitingForAMContainer()) {
-        // The request is for the AM-container, and the AM-container is launched
-        // by the system. So, update the places that are blacklisted by system
-        // (as opposed to those blacklisted by the application).
-        this.appSchedulingInfo.updatePlacesBlacklistedBySystem(
-            blacklistAdditions, blacklistRemovals);
-      } else {
-        this.appSchedulingInfo.updatePlacesBlacklistedByApp(blacklistAdditions,
-            blacklistRemovals);
+    try {
+      writeLock.lock();
+      if (!isStopped) {
+        if (isWaitingForAMContainer()) {
+          // The request is for the AM-container, and the AM-container is
+          // launched by the system. So, update the places that are blacklisted
+          // by system (as opposed to those blacklisted by the application).
+          this.appSchedulingInfo.updatePlacesBlacklistedBySystem(
+              blacklistAdditions, blacklistRemovals);
+        } else{
+          this.appSchedulingInfo.updatePlacesBlacklistedByApp(
+              blacklistAdditions, blacklistRemovals);
+        }
       }
+    } finally {
+      writeLock.unlock();
     }
   }
 
   public boolean isPlaceBlacklisted(String resourceName) {
-    boolean forAMContainer = isWaitingForAMContainer();
-    return this.appSchedulingInfo.isPlaceBlacklisted(resourceName,
-      forAMContainer);
+    try {
+      readLock.lock();
+      boolean forAMContainer = isWaitingForAMContainer();
+      return this.appSchedulingInfo.isPlaceBlacklisted(resourceName,
+          forAMContainer);
+    } finally {
+      readLock.unlock();
+    }
   }
 
-  public synchronized int addMissedNonPartitionedRequestSchedulingOpportunity(
+  public int addMissedNonPartitionedRequestSchedulingOpportunity(
       SchedulerRequestKey schedulerKey) {
-    missedNonPartitionedReqSchedulingOpportunity.add(schedulerKey);
-    return missedNonPartitionedReqSchedulingOpportunity.count(schedulerKey);
+    return missedNonPartitionedReqSchedulingOpportunity.add(
+        schedulerKey, 1) + 1;
   }
 
-  public synchronized void
+  public void
       resetMissedNonPartitionedRequestSchedulingOpportunity(
       SchedulerRequestKey schedulerKey) {
     missedNonPartitionedReqSchedulingOpportunity.setCount(schedulerKey, 0);
   }
 
   
-  public synchronized void addSchedulingOpportunity(
+  public void addSchedulingOpportunity(
       SchedulerRequestKey schedulerKey) {
-    int count = schedulingOpportunities.count(schedulerKey);
-    if (count < Integer.MAX_VALUE) {
-      schedulingOpportunities.setCount(schedulerKey, count + 1);
+    try {
+      schedulingOpportunities.add(schedulerKey, 1);
+    } catch (IllegalArgumentException e) {
+      // This happens when count = MAX_INT, ignore the exception
     }
   }
   
-  public synchronized void subtractSchedulingOpportunity(
+  public void subtractSchedulingOpportunity(
       SchedulerRequestKey schedulerKey) {
-    int count = schedulingOpportunities.count(schedulerKey) - 1;
-    this.schedulingOpportunities.setCount(schedulerKey, Math.max(count,  0));
+    this.schedulingOpportunities.removeExactly(schedulerKey, 1);
   }
 
   /**
@@ -684,7 +781,7 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
    * @param schedulerKey Scheduler Key
    * @return number of scheduling opportunities
    */
-  public synchronized int getSchedulingOpportunities(
+  public int getSchedulingOpportunities(
       SchedulerRequestKey schedulerKey) {
     return schedulingOpportunities.count(schedulerKey);
   }
@@ -696,16 +793,22 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
    *
    * @param schedulerKey The priority of the container scheduled.
    */
-  public synchronized void resetSchedulingOpportunities(
+  public void resetSchedulingOpportunities(
       SchedulerRequestKey schedulerKey) {
     resetSchedulingOpportunities(schedulerKey, System.currentTimeMillis());
   }
 
   // used for continuous scheduling
-  public synchronized void resetSchedulingOpportunities(
+  public void resetSchedulingOpportunities(
       SchedulerRequestKey schedulerKey, long currentTimeMs) {
-    lastScheduledContainer.put(schedulerKey, currentTimeMs);
-    schedulingOpportunities.setCount(schedulerKey, 0);
+    try {
+      writeLock.lock();
+      lastScheduledContainer.put(schedulerKey, currentTimeMs);
+      schedulingOpportunities.setCount(schedulerKey, 0);
+    } finally {
+      writeLock.unlock();
+    }
+
   }
 
   @VisibleForTesting
@@ -713,7 +816,7 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
     schedulingOpportunities.setCount(schedulerKey, count);
   }
 
-  synchronized AggregateAppResourceUsage getRunningAggregateAppResourceUsage() {
+  private AggregateAppResourceUsage getRunningAggregateAppResourceUsage() {
     long currentTimeMillis = System.currentTimeMillis();
     // Don't walk the whole container list if the resources were computed
     // recently.
@@ -737,101 +840,120 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
     return new AggregateAppResourceUsage(lastMemorySeconds, lastVcoreSeconds);
   }
 
-  public synchronized ApplicationResourceUsageReport getResourceUsageReport() {
-    AggregateAppResourceUsage runningResourceUsage =
-        getRunningAggregateAppResourceUsage();
-    Resource usedResourceClone =
-        Resources.clone(attemptResourceUsage.getAllUsed());
-    Resource reservedResourceClone =
-        Resources.clone(attemptResourceUsage.getReserved());
-    Resource cluster = rmContext.getScheduler().getClusterResource();
-    ResourceCalculator calc = rmContext.getScheduler().getResourceCalculator();
-    float queueUsagePerc = 0.0f;
-    float clusterUsagePerc = 0.0f;
-    if (!calc.isInvalidDivisor(cluster)) {
-      queueUsagePerc =
-          calc.divide(cluster, usedResourceClone, Resources.multiply(cluster,
-              queue.getQueueInfo(false, false).getCapacity())) * 100;
-      clusterUsagePerc = calc.divide(cluster, usedResourceClone, cluster) * 100;
+  public ApplicationResourceUsageReport getResourceUsageReport() {
+    try {
+      writeLock.lock();
+      AggregateAppResourceUsage runningResourceUsage =
+          getRunningAggregateAppResourceUsage();
+      Resource usedResourceClone = Resources.clone(
+          attemptResourceUsage.getAllUsed());
+      Resource reservedResourceClone = Resources.clone(
+          attemptResourceUsage.getReserved());
+      Resource cluster = rmContext.getScheduler().getClusterResource();
+      ResourceCalculator calc =
+          rmContext.getScheduler().getResourceCalculator();
+      float queueUsagePerc = 0.0f;
+      float clusterUsagePerc = 0.0f;
+      if (!calc.isInvalidDivisor(cluster)) {
+        queueUsagePerc = calc.divide(cluster, usedResourceClone, Resources
+            .multiply(cluster, queue.getQueueInfo(false, false).getCapacity()))
+            * 100;
+        clusterUsagePerc = calc.divide(cluster, usedResourceClone, cluster)
+            * 100;
+      }
+      return ApplicationResourceUsageReport.newInstance(liveContainers.size(),
+          reservedContainers.size(), usedResourceClone, reservedResourceClone,
+          Resources.add(usedResourceClone, reservedResourceClone),
+          runningResourceUsage.getMemorySeconds(),
+          runningResourceUsage.getVcoreSeconds(), queueUsagePerc,
+          clusterUsagePerc);
+    } finally {
+      writeLock.unlock();
     }
-    return ApplicationResourceUsageReport.newInstance(liveContainers.size(),
-        reservedContainers.size(), usedResourceClone, reservedResourceClone,
-        Resources.add(usedResourceClone, reservedResourceClone),
-        runningResourceUsage.getMemorySeconds(),
-        runningResourceUsage.getVcoreSeconds(), queueUsagePerc,
-        clusterUsagePerc);
   }
 
-  public synchronized Map<ContainerId, RMContainer> getLiveContainersMap() {
+  @VisibleForTesting
+  public Map<ContainerId, RMContainer> getLiveContainersMap() {
     return this.liveContainers;
   }
 
-  public synchronized Resource getResourceLimit() {
-    return this.resourceLimit;
-  }
-
-  public synchronized Map<SchedulerRequestKey, Long>
+  public Map<SchedulerRequestKey, Long>
       getLastScheduledContainer() {
     return this.lastScheduledContainer;
   }
 
-  public synchronized void transferStateFromPreviousAttempt(
+  public void transferStateFromPreviousAttempt(
       SchedulerApplicationAttempt appAttempt) {
-    this.liveContainers = appAttempt.getLiveContainersMap();
-    // this.reReservations = appAttempt.reReservations;
-    this.attemptResourceUsage.copyAllUsed(appAttempt.attemptResourceUsage);
-    this.resourceLimit = appAttempt.getResourceLimit();
-    // this.currentReservation = appAttempt.currentReservation;
-    // this.newlyAllocatedContainers = appAttempt.newlyAllocatedContainers;
-    // this.schedulingOpportunities = appAttempt.schedulingOpportunities;
-    this.lastScheduledContainer = appAttempt.getLastScheduledContainer();
-    this.appSchedulingInfo
-      .transferStateFromPreviousAppSchedulingInfo(appAttempt.appSchedulingInfo);
+    try {
+      writeLock.lock();
+      this.liveContainers = appAttempt.getLiveContainersMap();
+      // this.reReservations = appAttempt.reReservations;
+      this.attemptResourceUsage.copyAllUsed(appAttempt.attemptResourceUsage);
+      this.setHeadroom(appAttempt.resourceLimit);
+      // this.currentReservation = appAttempt.currentReservation;
+      // this.newlyAllocatedContainers = appAttempt.newlyAllocatedContainers;
+      // this.schedulingOpportunities = appAttempt.schedulingOpportunities;
+      this.lastScheduledContainer = appAttempt.getLastScheduledContainer();
+      this.appSchedulingInfo.transferStateFromPreviousAppSchedulingInfo(
+          appAttempt.appSchedulingInfo);
+    } finally {
+      writeLock.unlock();
+    }
   }
   
-  public synchronized void move(Queue newQueue) {
-    QueueMetrics oldMetrics = queue.getMetrics();
-    QueueMetrics newMetrics = newQueue.getMetrics();
-    String newQueueName = newQueue.getQueueName();
-    String user = getUser();
-    for (RMContainer liveContainer : liveContainers.values()) {
-      Resource resource = liveContainer.getContainer().getResource();
-      ((RMContainerImpl)liveContainer).setQueueName(newQueueName);
-      oldMetrics.releaseResources(user, 1, resource);
-      newMetrics.allocateResources(user, 1, resource, false);
-    }
-    for (Map<NodeId, RMContainer> map : reservedContainers.values()) {
-      for (RMContainer reservedContainer : map.values()) {
-        ((RMContainerImpl)reservedContainer).setQueueName(newQueueName);
-        Resource resource = reservedContainer.getReservedResource();
-        oldMetrics.unreserveResource(user, resource);
-        newMetrics.reserveResource(user, resource);
+  public void move(Queue newQueue) {
+    try {
+      writeLock.lock();
+      QueueMetrics oldMetrics = queue.getMetrics();
+      QueueMetrics newMetrics = newQueue.getMetrics();
+      String newQueueName = newQueue.getQueueName();
+      String user = getUser();
+      for (RMContainer liveContainer : liveContainers.values()) {
+        Resource resource = liveContainer.getContainer().getResource();
+        ((RMContainerImpl) liveContainer).setQueueName(newQueueName);
+        oldMetrics.releaseResources(user, 1, resource);
+        newMetrics.allocateResources(user, 1, resource, false);
+      }
+      for (Map<NodeId, RMContainer> map : reservedContainers.values()) {
+        for (RMContainer reservedContainer : map.values()) {
+          ((RMContainerImpl) reservedContainer).setQueueName(newQueueName);
+          Resource resource = reservedContainer.getReservedResource();
+          oldMetrics.unreserveResource(user, resource);
+          newMetrics.reserveResource(user, resource);
+        }
       }
-    }
 
-    appSchedulingInfo.move(newQueue);
-    this.queue = newQueue;
+      appSchedulingInfo.move(newQueue);
+      this.queue = newQueue;
+    } finally {
+      writeLock.unlock();
+    }
   }
 
-  public synchronized void recoverContainer(SchedulerNode node,
+  public void recoverContainer(SchedulerNode node,
       RMContainer rmContainer) {
-    // recover app scheduling info
-    appSchedulingInfo.recoverContainer(rmContainer);
+    try {
+      writeLock.lock();
+      // recover app scheduling info
+      appSchedulingInfo.recoverContainer(rmContainer);
 
-    if (rmContainer.getState().equals(RMContainerState.COMPLETED)) {
-      return;
+      if (rmContainer.getState().equals(RMContainerState.COMPLETED)) {
+        return;
+      }
+      LOG.info("SchedulerAttempt " + getApplicationAttemptId()
+          + " is recovering container " + rmContainer.getContainerId());
+      liveContainers.put(rmContainer.getContainerId(), rmContainer);
+      attemptResourceUsage.incUsed(node.getPartition(),
+          rmContainer.getContainer().getResource());
+
+      // resourceLimit: updated when LeafQueue#recoverContainer#allocateResource
+      // is called.
+      // newlyAllocatedContainers.add(rmContainer);
+      // schedulingOpportunities
+      // lastScheduledContainer
+    } finally {
+      writeLock.unlock();
     }
-    LOG.info("SchedulerAttempt " + getApplicationAttemptId()
-      + " is recovering container " + rmContainer.getContainerId());
-    liveContainers.put(rmContainer.getContainerId(), rmContainer);
-    attemptResourceUsage.incUsed(node.getPartition(), rmContainer
-        .getContainer().getResource());
-    
-    // resourceLimit: updated when LeafQueue#recoverContainer#allocateResource
-    // is called.
-    // newlyAllocatedContainers.add(rmContainer);
-    // schedulingOpportunities
-    // lastScheduledContainer
   }
 
   public void incNumAllocatedContainers(NodeType containerType,
@@ -915,49 +1037,64 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
     return attemptResourceUsage;
   }
   
-  public synchronized boolean removeIncreaseRequest(NodeId nodeId,
+  public boolean removeIncreaseRequest(NodeId nodeId,
       SchedulerRequestKey schedulerKey, ContainerId containerId) {
-    return appSchedulingInfo.removeIncreaseRequest(nodeId, schedulerKey,
-        containerId);
+    try {
+      writeLock.lock();
+      return appSchedulingInfo.removeIncreaseRequest(nodeId, schedulerKey,
+          containerId);
+    } finally {
+      writeLock.unlock();
+    }
   }
   
-  public synchronized boolean updateIncreaseRequests(
+  public boolean updateIncreaseRequests(
       List<SchedContainerChangeRequest> increaseRequests) {
-    return appSchedulingInfo.updateIncreaseRequests(increaseRequests);
+    try {
+      writeLock.lock();
+      return appSchedulingInfo.updateIncreaseRequests(increaseRequests);
+    } finally {
+      writeLock.unlock();
+    }
   }
   
-  private synchronized void changeContainerResource(
+  private void changeContainerResource(
       SchedContainerChangeRequest changeRequest, boolean increase) {
-    if (increase) {
-      appSchedulingInfo.increaseContainer(changeRequest);
-    } else {
-      appSchedulingInfo.decreaseContainer(changeRequest);
-    }
+    try {
+      writeLock.lock();
+      if (increase) {
+        appSchedulingInfo.increaseContainer(changeRequest);
+      } else{
+        appSchedulingInfo.decreaseContainer(changeRequest);
+      }
 
-    RMContainer changedRMContainer = changeRequest.getRMContainer(); 
-    changedRMContainer.handle(
-        new RMContainerChangeResourceEvent(changeRequest.getContainerId(),
-            changeRequest.getTargetCapacity(), increase));
-
-    // remove pending and not pulled by AM newly-increased/decreased-containers
-    // and add the new one
-    if (increase) {
-      newlyDecreasedContainers.remove(changeRequest.getContainerId());
-      newlyIncreasedContainers.put(changeRequest.getContainerId(),
-          changedRMContainer);
-    } else {
-      newlyIncreasedContainers.remove(changeRequest.getContainerId());
-      newlyDecreasedContainers.put(changeRequest.getContainerId(),
-          changedRMContainer);
+      RMContainer changedRMContainer = changeRequest.getRMContainer();
+      changedRMContainer.handle(
+          new RMContainerChangeResourceEvent(changeRequest.getContainerId(),
+              changeRequest.getTargetCapacity(), increase));
+
+      // remove pending and not pulled by AM newly-increased or
+      // decreased-containers and add the new one
+      if (increase) {
+        newlyDecreasedContainers.remove(changeRequest.getContainerId());
+        newlyIncreasedContainers.put(changeRequest.getContainerId(),
+            changedRMContainer);
+      } else{
+        newlyIncreasedContainers.remove(changeRequest.getContainerId());
+        newlyDecreasedContainers.put(changeRequest.getContainerId(),
+            changedRMContainer);
+      }
+    } finally {
+      writeLock.unlock();
     }
   }
   
-  public synchronized void decreaseContainer(
+  public void decreaseContainer(
       SchedContainerChangeRequest decreaseRequest) {
     changeContainerResource(decreaseRequest, false);
   }
   
-  public synchronized void increaseContainer(
+  public void increaseContainer(
       SchedContainerChangeRequest increaseRequest) {
     changeContainerResource(increaseRequest, true);
   }
@@ -1025,7 +1162,10 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
     this.isAttemptRecovering = isRecovering;
   }
 
-  public static enum AMState {
+  /**
+   * Different state for Application Master, user can see this state from web UI
+   */
+  public enum AMState {
     UNMANAGED("User launched the Application Master, since it's unmanaged. "),
     INACTIVATED("Application is added to the scheduler and is not yet activated. "),
     ACTIVATED("Application is Activated, waiting for resources to be assigned for AM. "),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b8a30f2f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.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/capacity/allocator/RegularContainerAllocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java
index 8d4042c..1a3f71f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java
@@ -251,7 +251,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
     return result;
   }
   
-  public synchronized float getLocalityWaitFactor(
+  public float getLocalityWaitFactor(
       SchedulerRequestKey schedulerKey, int clusterNodes) {
     // Estimate: Required unique resources (i.e. hosts + racks)
     int requiredResources = 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b8a30f2f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.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/common/fica/FiCaSchedulerApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
index 9c84a23..f40ecd7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
@@ -63,7 +63,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSAMCont
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSAssignment;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityHeadroomProvider;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerContext;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueueCapacities;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode;
@@ -99,7 +98,6 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
    * to hold the message if its app doesn't not get container from a node
    */
   private String appSkipNodeDiagnostics;
-  private CapacitySchedulerContext capacitySchedulerContext;
 
   public FiCaSchedulerApp(ApplicationAttemptId applicationAttemptId, 
       String user, Queue queue, ActiveUsersManager activeUsersManager,
@@ -153,118 +151,128 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
 
     containerAllocator = new ContainerAllocator(this, rc, rmContext,
         activitiesManager);
-
-    if (scheduler instanceof CapacityScheduler) {
-      capacitySchedulerContext = (CapacitySchedulerContext) scheduler;
-    }
   }
 
-  public synchronized boolean containerCompleted(RMContainer rmContainer,
+  public boolean containerCompleted(RMContainer rmContainer,
       ContainerStatus containerStatus, RMContainerEventType event,
       String partition) {
-    ContainerId containerId = rmContainer.getContainerId();
+    try {
+      writeLock.lock();
+      ContainerId containerId = rmContainer.getContainerId();
 
-    // Remove from the list of containers
-    if (null == liveContainers.remove(containerId)) {
-      return false;
-    }
+      // Remove from the list of containers
+      if (null == liveContainers.remove(containerId)) {
+        return false;
+      }
 
-    // Remove from the list of newly allocated containers if found
-    newlyAllocatedContainers.remove(rmContainer);
+      // Remove from the list of newly allocated containers if found
+      newlyAllocatedContainers.remove(rmContainer);
 
-    // Inform the container
-    rmContainer.handle(
-        new RMContainerFinishedEvent(containerId, containerStatus, event));
+      // Inform the container
+      rmContainer.handle(
+          new RMContainerFinishedEvent(containerId, containerStatus, event));
 
-    containersToPreempt.remove(containerId);
+      containersToPreempt.remove(containerId);
 
-    Resource containerResource = rmContainer.getContainer().getResource();
-    RMAuditLogger.logSuccess(getUser(),
-        AuditConstants.RELEASE_CONTAINER, "SchedulerApp",
-        getApplicationId(), containerId, containerResource);
-    
-    // Update usage metrics 
-    queue.getMetrics().releaseResources(getUser(), 1, containerResource);
-    attemptResourceUsage.decUsed(partition, containerResource);
+      Resource containerResource = rmContainer.getContainer().getResource();
+      RMAuditLogger.logSuccess(getUser(), AuditConstants.RELEASE_CONTAINER,
+          "SchedulerApp", getApplicationId(), containerId, containerResource);
+
+      // Update usage metrics
+      queue.getMetrics().releaseResources(getUser(), 1, containerResource);
+      attemptResourceUsage.decUsed(partition, containerResource);
 
-    // Clear resource utilization metrics cache.
-    lastMemoryAggregateAllocationUpdateTime = -1;
+      // Clear resource utilization metrics cache.
+      lastMemoryAggregateAllocationUpdateTime = -1;
 
-    return true;
+      return true;
+    } finally {
+      writeLock.unlock();
+    }
   }
 
-  public synchronized RMContainer allocate(NodeType type, FiCaSchedulerNode node,
+  public RMContainer allocate(NodeType type, FiCaSchedulerNode node,
       SchedulerRequestKey schedulerKey, ResourceRequest request,
       Container container) {
+    try {
+      writeLock.lock();
 
-    if (isStopped) {
-      return null;
-    }
-    
-    // Required sanity check - AM can call 'allocate' to update resource
-    // request without locking the scheduler, hence we need to check
-    if (getTotalRequiredResources(schedulerKey) <= 0) {
-      return null;
-    }
+      if (isStopped) {
+        return null;
+      }
+
+      // Required sanity check - AM can call 'allocate' to update resource
+      // request without locking the scheduler, hence we need to check
+      if (getTotalRequiredResources(schedulerKey) <= 0) {
+        return null;
+      }
 
-    // Create RMContainer
-    RMContainer rmContainer =
-        new RMContainerImpl(container, this.getApplicationAttemptId(),
-            node.getNodeID(), appSchedulingInfo.getUser(), this.rmContext,
-            request.getNodeLabelExpression());
-    ((RMContainerImpl)rmContainer).setQueueName(this.getQueueName());
+      // Create RMContainer
+      RMContainer rmContainer = new RMContainerImpl(container,
+          this.getApplicationAttemptId(), node.getNodeID(),
+          appSchedulingInfo.getUser(), this.rmContext,
+          request.getNodeLabelExpression());
+      ((RMContainerImpl) rmContainer).setQueueName(this.getQueueName());
 
-    updateAMContainerDiagnostics(AMState.ASSIGNED, null);
+      updateAMContainerDiagnostics(AMState.ASSIGNED, null);
 
-    // Add it to allContainers list.
-    newlyAllocatedContainers.add(rmContainer);
+      // Add it to allContainers list.
+      newlyAllocatedContainers.add(rmContainer);
 
-    ContainerId containerId = container.getId();
-    liveContainers.put(containerId, rmContainer);
+      ContainerId containerId = container.getId();
+      liveContainers.put(containerId, rmContainer);
 
-    // Update consumption and track allocations
-    List<ResourceRequest> resourceRequestList = appSchedulingInfo.allocate(
-        type, node, schedulerKey, request, container);
+      // Update consumption and track allocations
+      List<ResourceRequest> resourceRequestList = appSchedulingInfo.allocate(
+          type, node, schedulerKey, request, container);
 
-    attemptResourceUsage.incUsed(node.getPartition(), container.getResource());
+      attemptResourceUsage.incUsed(node.getPartition(),
+          container.getResource());
 
-    // Update resource requests related to "request" and store in RMContainer
-    ((RMContainerImpl)rmContainer).setResourceRequests(resourceRequestList);
+      // Update resource requests related to "request" and store in RMContainer
+      ((RMContainerImpl) rmContainer).setResourceRequests(resourceRequestList);
 
-    // Inform the container
-    rmContainer.handle(
-        new RMContainerEvent(containerId, RMContainerEventType.START));
+      // Inform the container
+      rmContainer.handle(
+          new RMContainerEvent(containerId, RMContainerEventType.START));
 
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("allocate: applicationAttemptId=" 
-          + containerId.getApplicationAttemptId()
-          + " container=" + containerId + " host="
-          + container.getNodeId().getHost() + " type=" + type);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("allocate: applicationAttemptId=" + containerId
+            .getApplicationAttemptId() + " container=" + containerId + " host="
+            + container.getNodeId().getHost() + " type=" + type);
+      }
+      RMAuditLogger.logSuccess(getUser(), AuditConstants.ALLOC_CONTAINER,
+          "SchedulerApp", getApplicationId(), containerId,
+          container.getResource());
+
+      return rmContainer;
+    } finally {
+      writeLock.unlock();
     }
-    RMAuditLogger.logSuccess(getUser(),
-        AuditConstants.ALLOC_CONTAINER, "SchedulerApp",
-        getApplicationId(), containerId, container.getResource());
-    
-    return rmContainer;
   }
 
-  public synchronized boolean unreserve(SchedulerRequestKey schedulerKey,
+  public boolean unreserve(SchedulerRequestKey schedulerKey,
       FiCaSchedulerNode node, RMContainer rmContainer) {
-    // Cancel increase request (if it has reserved increase request 
-    rmContainer.cancelIncreaseReservation();
-    
-    // Done with the reservation?
-    if (internalUnreserve(node, schedulerKey)) {
-      node.unreserveResource(this);
-
-      // Update reserved metrics
-      queue.getMetrics().unreserveResource(getUser(),
-          rmContainer.getReservedResource());
-      queue.decReservedResource(node.getPartition(),
-          rmContainer.getReservedResource());
-      return true;
+    try {
+      writeLock.lock();
+      // Cancel increase request (if it has reserved increase request
+      rmContainer.cancelIncreaseReservation();
+
+      // Done with the reservation?
+      if (internalUnreserve(node, schedulerKey)) {
+        node.unreserveResource(this);
+
+        // Update reserved metrics
+        queue.getMetrics().unreserveResource(getUser(),
+            rmContainer.getReservedResource());
+        queue.decReservedResource(node.getPartition(),
+            rmContainer.getReservedResource());
+        return true;
+      }
+      return false;
+    } finally {
+      writeLock.unlock();
     }
-    return false;
   }
 
   private boolean internalUnreserve(FiCaSchedulerNode node,
@@ -303,33 +311,15 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
     return false;
   }
 
-  public synchronized float getLocalityWaitFactor(
-      SchedulerRequestKey schedulerKey, int clusterNodes) {
-    // Estimate: Required unique resources (i.e. hosts + racks)
-    int requiredResources = 
-        Math.max(this.getResourceRequests(schedulerKey).size() - 1, 0);
-    
-    // waitFactor can't be more than '1' 
-    // i.e. no point skipping more than clustersize opportunities
-    return Math.min(((float)requiredResources / clusterNodes), 1.0f);
-  }
-
-  public synchronized Resource getTotalPendingRequests() {
-    Resource ret = Resource.newInstance(0, 0);
-    for (ResourceRequest rr : appSchedulingInfo.getAllResourceRequests()) {
-      // to avoid double counting we count only "ANY" resource requests
-      if (ResourceRequest.isAnyLocation(rr.getResourceName())){
-        Resources.addTo(ret,
-            Resources.multiply(rr.getCapability(), rr.getNumContainers()));
+  public void markContainerForPreemption(ContainerId cont) {
+    try {
+      writeLock.lock();
+      // ignore already completed containers
+      if (liveContainers.containsKey(cont)) {
+        containersToPreempt.add(cont);
       }
-    }
-    return ret;
-  }
-
-  public synchronized void markContainerForPreemption(ContainerId cont) {
-    // ignore already completed containers
-    if (liveContainers.containsKey(cont)) {
-      containersToPreempt.add(cont);
+    } finally {
+      writeLock.unlock();
     }
   }
 
@@ -343,94 +333,115 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
    * @param minimumAllocation
    * @return an allocation
    */
-  public synchronized Allocation getAllocation(ResourceCalculator rc,
+  public Allocation getAllocation(ResourceCalculator resourceCalculator,
       Resource clusterResource, Resource minimumAllocation) {
-
-    Set<ContainerId> currentContPreemption = Collections.unmodifiableSet(
-        new HashSet<ContainerId>(containersToPreempt));
-    containersToPreempt.clear();
-    Resource tot = Resource.newInstance(0, 0);
-    for(ContainerId c : currentContPreemption){
-      Resources.addTo(tot,
-          liveContainers.get(c).getContainer().getResource());
+    try {
+      writeLock.lock();
+      Set<ContainerId> currentContPreemption = Collections.unmodifiableSet(
+          new HashSet<ContainerId>(containersToPreempt));
+      containersToPreempt.clear();
+      Resource tot = Resource.newInstance(0, 0);
+      for (ContainerId c : currentContPreemption) {
+        Resources.addTo(tot, liveContainers.get(c).getContainer()
+            .getResource());
+      }
+      int numCont = (int) Math.ceil(
+          Resources.divide(rc, clusterResource, tot, minimumAllocation));
+      ResourceRequest rr = ResourceRequest.newInstance(Priority.UNDEFINED,
+          ResourceRequest.ANY, minimumAllocation, numCont);
+      List<Container> newlyAllocatedContainers = pullNewlyAllocatedContainers();
+      List<Container> newlyIncreasedContainers = pullNewlyIncreasedContainers();
+      List<Container> newlyDecreasedContainers = pullNewlyDecreasedContainers();
+      List<NMToken> updatedNMTokens = pullUpdatedNMTokens();
+      Resource headroom = getHeadroom();
+      setApplicationHeadroomForMetrics(headroom);
+      return new Allocation(newlyAllocatedContainers, headroom, null,
+          currentContPreemption, Collections.singletonList(rr), updatedNMTokens,
+          newlyIncreasedContainers, newlyDecreasedContainers);
+    } finally {
+      writeLock.unlock();
     }
-    int numCont = (int) Math.ceil(
-        Resources.divide(rc, clusterResource, tot, minimumAllocation));
-    ResourceRequest rr = ResourceRequest.newInstance(
-        Priority.UNDEFINED, ResourceRequest.ANY,
-        minimumAllocation, numCont);
-    List<Container> newlyAllocatedContainers = pullNewlyAllocatedContainers();
-    List<Container> newlyIncreasedContainers = pullNewlyIncreasedContainers();
-    List<Container> newlyDecreasedContainers = pullNewlyDecreasedContainers();
-    List<NMToken> updatedNMTokens = pullUpdatedNMTokens();
-    Resource headroom = getHeadroom();
-    setApplicationHeadroomForMetrics(headroom);
-    return new Allocation(newlyAllocatedContainers, headroom, null,
-        currentContPreemption, Collections.singletonList(rr), updatedNMTokens,
-        newlyIncreasedContainers, newlyDecreasedContainers);
   }
-  
-  synchronized public NodeId getNodeIdToUnreserve(
+
+  @VisibleForTesting
+  public NodeId getNodeIdToUnreserve(
       SchedulerRequestKey schedulerKey, Resource resourceNeedUnreserve,
       ResourceCalculator rc, Resource clusterResource) {
+    try {
+      writeLock.lock();
+      // first go around make this algorithm simple and just grab first
+      // reservation that has enough resources
+      Map<NodeId, RMContainer> reservedContainers = this.reservedContainers.get(
+          schedulerKey);
+
+      if ((reservedContainers != null) && (!reservedContainers.isEmpty())) {
+        for (Map.Entry<NodeId, RMContainer> entry : reservedContainers
+            .entrySet()) {
+          NodeId nodeId = entry.getKey();
+          RMContainer reservedContainer = entry.getValue();
+          if (reservedContainer.hasIncreaseReservation()) {
+            // Currently, only regular container allocation supports continuous
+            // reservation looking, we don't support canceling increase request
+            // reservation when allocating regular container.
+            continue;
+          }
 
-    // first go around make this algorithm simple and just grab first
-    // reservation that has enough resources
-    Map<NodeId, RMContainer> reservedContainers = this.reservedContainers
-        .get(schedulerKey);
-
-    if ((reservedContainers != null) && (!reservedContainers.isEmpty())) {
-      for (Map.Entry<NodeId, RMContainer> entry : reservedContainers.entrySet()) {
-        NodeId nodeId = entry.getKey();
-        RMContainer reservedContainer = entry.getValue();
-        if (reservedContainer.hasIncreaseReservation()) {
-          // Currently, only regular container allocation supports continuous
-          // reservation looking, we don't support canceling increase request
-          // reservation when allocating regular container.
-          continue;
-        }
-        
-        Resource reservedResource = reservedContainer.getReservedResource();
-        
-        // make sure we unreserve one with at least the same amount of
-        // resources, otherwise could affect capacity limits
-        if (Resources.fitsIn(rc, clusterResource, resourceNeedUnreserve,
-            reservedResource)) {
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("unreserving node with reservation size: "
-                + reservedResource
-                + " in order to allocate container with size: " + resourceNeedUnreserve);
+          Resource reservedResource = reservedContainer.getReservedResource();
+
+          // make sure we unreserve one with at least the same amount of
+          // resources, otherwise could affect capacity limits
+          if (Resources.fitsIn(rc, clusterResource, resourceNeedUnreserve,
+              reservedResource)) {
+            if (LOG.isDebugEnabled()) {
+              LOG.debug(
+                  "unreserving node with reservation size: " + reservedResource
+                      + " in order to allocate container with size: "
+                      + resourceNeedUnreserve);
+            }
+            return nodeId;
           }
-          return nodeId;
         }
       }
+      return null;
+    } finally {
+      writeLock.unlock();
     }
-    return null;
   }
   
-  public synchronized void setHeadroomProvider(
+  public void setHeadroomProvider(
     CapacityHeadroomProvider headroomProvider) {
-    this.headroomProvider = headroomProvider;
-  }
-
-  public synchronized CapacityHeadroomProvider getHeadroomProvider() {
-    return headroomProvider;
+    try {
+      writeLock.lock();
+      this.headroomProvider = headroomProvider;
+    } finally {
+      writeLock.unlock();
+    }
   }
   
   @Override
-  public synchronized Resource getHeadroom() {
-    if (headroomProvider != null) {
-      return headroomProvider.getHeadroom();
+  public Resource getHeadroom() {
+    try {
+      readLock.lock();
+      if (headroomProvider != null) {
+        return headroomProvider.getHeadroom();
+      }
+      return super.getHeadroom();
+    } finally {
+      readLock.unlock();
     }
-    return super.getHeadroom();
+
   }
   
   @Override
-  public synchronized void transferStateFromPreviousAttempt(
+  public void transferStateFromPreviousAttempt(
       SchedulerApplicationAttempt appAttempt) {
-    super.transferStateFromPreviousAttempt(appAttempt);
-    this.headroomProvider = 
-      ((FiCaSchedulerApp) appAttempt).getHeadroomProvider();
+    try {
+      writeLock.lock();
+      super.transferStateFromPreviousAttempt(appAttempt);
+      this.headroomProvider = ((FiCaSchedulerApp) appAttempt).headroomProvider;
+    } finally {
+      writeLock.unlock();
+    }
   }
   
   public boolean reserveIncreasedContainer(SchedulerRequestKey schedulerKey,
@@ -444,11 +455,11 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
 
       // Update the node
       node.reserveResource(this, schedulerKey, rmContainer);
-      
+
       // Succeeded
       return true;
     }
-    
+
     return false;
   }
 
@@ -515,9 +526,12 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
       showRequests();
     }
 
-    synchronized (this) {
+    try {
+      writeLock.lock();
       return containerAllocator.assignContainers(clusterResource, node,
           schedulingMode, currentResourceLimits, reservedContainer);
+    } finally {
+      writeLock.unlock();
     }
   }
 
@@ -625,23 +639,33 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
    * Capacity Scheduler.
    */
   @Override
-  public synchronized ApplicationResourceUsageReport getResourceUsageReport() {
-    ApplicationResourceUsageReport report = super.getResourceUsageReport();
-    Resource cluster = rmContext.getScheduler().getClusterResource();
-    Resource totalPartitionRes =
-        rmContext.getNodeLabelManager()
-          .getResourceByLabel(getAppAMNodePartitionName(), cluster);
-    ResourceCalculator calc = rmContext.getScheduler().getResourceCalculator();
-    if (!calc.isInvalidDivisor(totalPartitionRes)) {
-      float queueAbsMaxCapPerPartition =
-          ((AbstractCSQueue)getQueue()).getQueueCapacities()
-            .getAbsoluteCapacity(getAppAMNodePartitionName());
-      float queueUsagePerc =
-          calc.divide(totalPartitionRes, report.getUsedResources(),
-              Resources.multiply(totalPartitionRes,
-                  queueAbsMaxCapPerPartition)) * 100;
-      report.setQueueUsagePercentage(queueUsagePerc);
+  public ApplicationResourceUsageReport getResourceUsageReport() {
+    try {
+      // Use write lock here because
+      // SchedulerApplicationAttempt#getResourceUsageReport updated fields
+      // TODO: improve this
+      writeLock.lock();
+      ApplicationResourceUsageReport report = super.getResourceUsageReport();
+      Resource cluster = rmContext.getScheduler().getClusterResource();
+      Resource totalPartitionRes =
+          rmContext.getNodeLabelManager().getResourceByLabel(
+              getAppAMNodePartitionName(), cluster);
+      ResourceCalculator calc =
+          rmContext.getScheduler().getResourceCalculator();
+      if (!calc.isInvalidDivisor(totalPartitionRes)) {
+        float queueAbsMaxCapPerPartition =
+            ((AbstractCSQueue) getQueue()).getQueueCapacities()
+                .getAbsoluteCapacity(getAppAMNodePartitionName());
+        float queueUsagePerc = calc.divide(totalPartitionRes,
+            report.getUsedResources(),
+            Resources.multiply(totalPartitionRes, queueAbsMaxCapPerPartition))
+            * 100;
+        report.setQueueUsagePercentage(queueUsagePerc);
+      }
+      return report;
+    } finally {
+      writeLock.unlock();
     }
-    return report;
+
   }
 }


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


[11/44] hadoop git commit: HADOOP-13580. If user is unauthorized, log "unauthorized" instead of "Invalid signed text:". Contributed by Wei-Chiu Chuang.

Posted by su...@apache.org.
HADOOP-13580. If user is unauthorized, log "unauthorized" instead of "Invalid signed text:". Contributed by Wei-Chiu Chuang.


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

Branch: refs/heads/YARN-2915
Commit: f6f3a447bf3b2900a2e9a0615ad9877f9310e062
Parents: cd099a6
Author: Wei-Chiu Chuang <we...@apache.org>
Authored: Fri Sep 16 14:53:09 2016 -0700
Committer: Wei-Chiu Chuang <we...@apache.org>
Committed: Fri Sep 16 14:53:30 2016 -0700

----------------------------------------------------------------------
 .../security/authentication/server/AuthenticationFilter.java      | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f6f3a447/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationFilter.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationFilter.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationFilter.java
index 0a9b8b5..5262fdc 100644
--- a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationFilter.java
+++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationFilter.java
@@ -438,6 +438,9 @@ public class AuthenticationFilter implements Filter {
       for (Cookie cookie : cookies) {
         if (cookie.getName().equals(AuthenticatedURL.AUTH_COOKIE)) {
           tokenStr = cookie.getValue();
+          if (tokenStr.isEmpty()) {
+            throw new AuthenticationException("Unauthorized access");
+          }
           try {
             tokenStr = signer.verifyAndExtract(tokenStr);
           } catch (SignerException ex) {


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


[12/44] hadoop git commit: HDFS-10823. Implement HttpFSFileSystem#listStatusIterator.

Posted by su...@apache.org.
HDFS-10823. Implement HttpFSFileSystem#listStatusIterator.


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

Branch: refs/heads/YARN-2915
Commit: 8a40953058d50d421d62b71067a13b626b3cba1f
Parents: f6f3a44
Author: Andrew Wang <wa...@apache.org>
Authored: Fri Sep 16 15:37:36 2016 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Fri Sep 16 15:37:36 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/fs/FileSystem.java   | 125 ++++++++++++++++---
 .../apache/hadoop/fs/TestFilterFileSystem.java  |   1 +
 .../org/apache/hadoop/fs/TestHarFileSystem.java |   1 +
 .../hadoop/hdfs/web/WebHdfsFileSystem.java      |  68 ++++------
 .../hadoop/fs/http/client/HttpFSFileSystem.java |  56 +++++++--
 .../hadoop/fs/http/client/HttpFSUtils.java      |   2 +
 .../hadoop/fs/http/server/FSOperations.java     |  62 +++++++++
 .../http/server/HttpFSParametersProvider.java   |  20 +++
 .../hadoop/fs/http/server/HttpFSServer.java     |  17 +++
 .../service/hadoop/FileSystemAccessService.java |   4 +-
 .../fs/http/client/BaseTestHttpFSWith.java      |  62 ++++++++-
 11 files changed, 340 insertions(+), 78 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a409530/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
index 9bde29d..5939f97 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
@@ -73,6 +73,7 @@ import org.apache.hadoop.util.StringUtils;
 import org.apache.htrace.core.Tracer;
 import org.apache.htrace.core.TraceScope;
 
+import com.google.common.base.Preconditions;
 import com.google.common.annotations.VisibleForTesting;
 
 import static com.google.common.base.Preconditions.checkArgument;
@@ -1530,7 +1531,68 @@ public abstract class FileSystem extends Configured implements Closeable {
    */
   public abstract FileStatus[] listStatus(Path f) throws FileNotFoundException, 
                                                          IOException;
-    
+
+  /**
+   * Represents a batch of directory entries when iteratively listing a
+   * directory. This is a private API not meant for use by end users.
+   * <p>
+   * For internal use by FileSystem subclasses that override
+   * {@link FileSystem#listStatusBatch(Path, byte[])} to implement iterative
+   * listing.
+   */
+  @InterfaceAudience.Private
+  public static class DirectoryEntries {
+    private final FileStatus[] entries;
+    private final byte[] token;
+    private final boolean hasMore;
+
+    public DirectoryEntries(FileStatus[] entries, byte[] token, boolean
+        hasMore) {
+      this.entries = entries;
+      if (token != null) {
+        this.token = token.clone();
+      } else {
+        this.token = null;
+      }
+      this.hasMore = hasMore;
+    }
+
+    public FileStatus[] getEntries() {
+      return entries;
+    }
+
+    public byte[] getToken() {
+      return token;
+    }
+
+    public boolean hasMore() {
+      return hasMore;
+    }
+  }
+
+  /**
+   * Given an opaque iteration token, return the next batch of entries in a
+   * directory. This is a private API not meant for use by end users.
+   * <p>
+   * This method should be overridden by FileSystem subclasses that want to
+   * use the generic {@link FileSystem#listStatusIterator(Path)} implementation.
+   * @param f Path to list
+   * @param token opaque iteration token returned by previous call, or null
+   *              if this is the first call.
+   * @return
+   * @throws FileNotFoundException
+   * @throws IOException
+   */
+  @InterfaceAudience.Private
+  protected DirectoryEntries listStatusBatch(Path f, byte[] token) throws
+      FileNotFoundException, IOException {
+    // The default implementation returns the entire listing as a single batch.
+    // Thus, there is never a second batch, and no need to respect the passed
+    // token or set a token in the returned DirectoryEntries.
+    FileStatus[] listing = listStatus(f);
+    return new DirectoryEntries(listing, null, false);
+  }
+
   /*
    * Filter files/directories in the given path using the user-supplied path
    * filter. Results are added to the given array <code>results</code>.
@@ -1767,6 +1829,49 @@ public abstract class FileSystem extends Configured implements Closeable {
   }
 
   /**
+   * Generic iterator for implementing {@link #listStatusIterator(Path)}.
+   */
+  private class DirListingIterator<T extends FileStatus> implements
+      RemoteIterator<T> {
+
+    private final Path path;
+    private DirectoryEntries entries;
+    private int i = 0;
+
+    DirListingIterator(Path path) {
+      this.path = path;
+    }
+
+    @Override
+    public boolean hasNext() throws IOException {
+      if (entries == null) {
+        fetchMore();
+      }
+      return i < entries.getEntries().length ||
+          entries.hasMore();
+    }
+
+    private void fetchMore() throws IOException {
+      byte[] token = null;
+      if (entries != null) {
+        token = entries.getToken();
+      }
+      entries = listStatusBatch(path, token);
+      i = 0;
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public T next() throws IOException {
+      Preconditions.checkState(hasNext(), "No more items in iterator");
+      if (i == entries.getEntries().length) {
+        fetchMore();
+      }
+      return (T)entries.getEntries()[i++];
+    }
+  }
+
+  /**
    * Returns a remote iterator so that followup calls are made on demand
    * while consuming the entries. Each file system implementation should
    * override this method and provide a more efficient implementation, if
@@ -1779,23 +1884,7 @@ public abstract class FileSystem extends Configured implements Closeable {
    */
   public RemoteIterator<FileStatus> listStatusIterator(final Path p)
   throws FileNotFoundException, IOException {
-    return new RemoteIterator<FileStatus>() {
-      private final FileStatus[] stats = listStatus(p);
-      private int i = 0;
-
-      @Override
-      public boolean hasNext() {
-        return i<stats.length;
-      }
-
-      @Override
-      public FileStatus next() throws IOException {
-        if (!hasNext()) {
-          throw new NoSuchElementException("No more entry in " + p);
-        }
-        return stats[i++];
-      }
-    };
+    return new DirListingIterator<>(p);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a409530/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFilterFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFilterFileSystem.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFilterFileSystem.java
index e1312bc..24f3dc8 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFilterFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFilterFileSystem.java
@@ -103,6 +103,7 @@ public class TestFilterFileSystem {
     public void processDeleteOnExit();
     public FsStatus getStatus();
     public FileStatus[] listStatus(Path f, PathFilter filter);
+    public FileStatus[] listStatusBatch(Path f, byte[] token);
     public FileStatus[] listStatus(Path[] files);
     public FileStatus[] listStatus(Path[] files, PathFilter filter);
     public FileStatus[] globStatus(Path pathPattern);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a409530/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystem.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystem.java
index d2020b9..bacdbb7 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystem.java
@@ -115,6 +115,7 @@ public class TestHarFileSystem {
     public QuotaUsage getQuotaUsage(Path f);
     public FsStatus getStatus();
     public FileStatus[] listStatus(Path f, PathFilter filter);
+    public FileStatus[] listStatusBatch(Path f, byte[] token);
     public FileStatus[] listStatus(Path[] files);
     public FileStatus[] listStatus(Path[] files, PathFilter filter);
     public FileStatus[] globStatus(Path pathPattern);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a409530/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
index 9a9edc8..19de5b5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
@@ -64,7 +64,6 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.GlobalStorageStatistics;
 import org.apache.hadoop.fs.GlobalStorageStatistics.StorageStatisticsProvider;
-import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.StorageStatistics;
 import org.apache.hadoop.fs.permission.FsCreateModes;
 import org.apache.hadoop.hdfs.DFSOpsCountStatistics;
@@ -1504,55 +1503,30 @@ public class WebHdfsFileSystem extends FileSystem
   }
 
   private static final byte[] EMPTY_ARRAY = new byte[] {};
-  private class DirListingIterator<T extends FileStatus> implements
-      RemoteIterator<T> {
 
-    private final Path path;
-    private DirectoryListing thisListing;
-    private int i = 0;
-    private byte[] prevKey = EMPTY_ARRAY;
-
-    DirListingIterator(Path path) {
-      this.path = path;
-    }
-
-    @Override
-    public boolean hasNext() throws IOException {
-      if (thisListing == null) {
-        fetchMore();
-      }
-      return i < thisListing.getPartialListing().length ||
-          thisListing.hasMore();
-    }
-
-    private void fetchMore() throws IOException {
-      thisListing = new FsPathResponseRunner<DirectoryListing>(
-          GetOpParam.Op.LISTSTATUS_BATCH,
-          path, new StartAfterParam(new String(prevKey, Charsets.UTF_8))) {
-        @Override
-        DirectoryListing decodeResponse(Map<?, ?> json) throws IOException {
-          return JsonUtilClient.toDirectoryListing(json);
-        }
-      }.run();
-      i = 0;
-      prevKey = thisListing.getLastName();
+  @Override
+  public DirectoryEntries listStatusBatch(Path f, byte[] token) throws
+      FileNotFoundException, IOException {
+    byte[] prevKey = EMPTY_ARRAY;
+    if (token != null) {
+      prevKey = token;
     }
-
-    @Override
-    @SuppressWarnings("unchecked")
-    public T next() throws IOException {
-      Preconditions.checkState(hasNext(), "No more items in iterator");
-      if (i == thisListing.getPartialListing().length) {
-        fetchMore();
+    DirectoryListing listing = new FsPathResponseRunner<DirectoryListing>(
+        GetOpParam.Op.LISTSTATUS_BATCH,
+        f, new StartAfterParam(new String(prevKey, Charsets.UTF_8))) {
+      @Override
+      DirectoryListing decodeResponse(Map<?, ?> json) throws IOException {
+        return JsonUtilClient.toDirectoryListing(json);
       }
-      return (T)makeQualified(thisListing.getPartialListing()[i++], path);
-    }
-  }
-
-  @Override
-  public RemoteIterator<FileStatus> listStatusIterator(final Path f)
-      throws FileNotFoundException, IOException {
-    return new DirListingIterator<>(f);
+    }.run();
+    // Qualify the returned FileStatus array
+    final HdfsFileStatus[] statuses = listing.getPartialListing();
+    FileStatus[] qualified = new FileStatus[statuses.length];
+    for (int i = 0; i < statuses.length; i++) {
+      qualified[i] = makeQualified(statuses[i], f);
+    }
+    return new DirectoryEntries(qualified, listing.getLastName(),
+        listing.hasMore());
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a409530/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java
index 0f97d90..77e3323 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java
@@ -20,6 +20,8 @@ package org.apache.hadoop.fs.http.client;
 import java.util.ArrayList;
 import java.util.EnumSet;
 import java.util.List;
+
+import com.google.common.base.Charsets;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.ContentSummary;
@@ -111,6 +113,7 @@ public class HttpFSFileSystem extends FileSystem
   public static final String XATTR_SET_FLAG_PARAM = "flag";
   public static final String XATTR_ENCODING_PARAM = "encoding";
   public static final String NEW_LENGTH_PARAM = "newlength";
+  public static final String START_AFTER_PARAM = "startAfter";
 
   public static final Short DEFAULT_PERMISSION = 0755;
   public static final String ACLSPEC_DEFAULT = "";
@@ -184,6 +187,10 @@ public class HttpFSFileSystem extends FileSystem
 
   public static final String ENC_BIT_JSON = "encBit";
 
+  public static final String DIRECTORY_LISTING_JSON = "DirectoryListing";
+  public static final String PARTIAL_LISTING_JSON = "partialListing";
+  public static final String REMAINING_ENTRIES_JSON = "remainingEntries";
+
   public static final int HTTP_TEMPORARY_REDIRECT = 307;
 
   private static final String HTTP_GET = "GET";
@@ -203,7 +210,7 @@ public class HttpFSFileSystem extends FileSystem
     MODIFYACLENTRIES(HTTP_PUT), REMOVEACLENTRIES(HTTP_PUT),
     REMOVEDEFAULTACL(HTTP_PUT), REMOVEACL(HTTP_PUT), SETACL(HTTP_PUT),
     DELETE(HTTP_DELETE), SETXATTR(HTTP_PUT), GETXATTRS(HTTP_GET),
-    REMOVEXATTR(HTTP_PUT), LISTXATTRS(HTTP_GET);
+    REMOVEXATTR(HTTP_PUT), LISTXATTRS(HTTP_GET), LISTSTATUS_BATCH(HTTP_GET);
 
     private String httpMethod;
 
@@ -666,6 +673,17 @@ public class HttpFSFileSystem extends FileSystem
     return (Boolean) json.get(DELETE_JSON);
   }
 
+  private FileStatus[] toFileStatuses(JSONObject json, Path f) {
+    json = (JSONObject) json.get(FILE_STATUSES_JSON);
+    JSONArray jsonArray = (JSONArray) json.get(FILE_STATUS_JSON);
+    FileStatus[] array = new FileStatus[jsonArray.size()];
+    f = makeQualified(f);
+    for (int i = 0; i < jsonArray.size(); i++) {
+      array[i] = createFileStatus(f, (JSONObject) jsonArray.get(i));
+    }
+    return array;
+  }
+
   /**
    * List the statuses of the files/directories in the given path if the path is
    * a directory.
@@ -684,14 +702,36 @@ public class HttpFSFileSystem extends FileSystem
                                            params, f, true);
     HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
     JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
-    json = (JSONObject) json.get(FILE_STATUSES_JSON);
-    JSONArray jsonArray = (JSONArray) json.get(FILE_STATUS_JSON);
-    FileStatus[] array = new FileStatus[jsonArray.size()];
-    f = makeQualified(f);
-    for (int i = 0; i < jsonArray.size(); i++) {
-      array[i] = createFileStatus(f, (JSONObject) jsonArray.get(i));
+    return toFileStatuses(json, f);
+  }
+
+  @Override
+  public DirectoryEntries listStatusBatch(Path f, byte[] token) throws
+      FileNotFoundException, IOException {
+    Map<String, String> params = new HashMap<String, String>();
+    params.put(OP_PARAM, Operation.LISTSTATUS_BATCH.toString());
+    if (token != null) {
+      params.put(START_AFTER_PARAM, new String(token, Charsets.UTF_8));
     }
-    return array;
+    HttpURLConnection conn = getConnection(
+        Operation.LISTSTATUS_BATCH.getMethod(),
+        params, f, true);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    // Parse the FileStatus array
+    JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
+    JSONObject listing = (JSONObject) json.get(DIRECTORY_LISTING_JSON);
+    FileStatus[] statuses = toFileStatuses(
+        (JSONObject) listing.get(PARTIAL_LISTING_JSON), f);
+    // New token is the last FileStatus entry
+    byte[] newToken = null;
+    if (statuses.length > 0) {
+      newToken = statuses[statuses.length - 1].getPath().getName().toString()
+          .getBytes(Charsets.UTF_8);
+    }
+    // Parse the remainingEntries boolean into hasMore
+    final long remainingEntries = (Long) listing.get(REMAINING_ENTRIES_JSON);
+    final boolean hasMore = remainingEntries > 0 ? true : false;
+    return new DirectoryEntries(statuses, newToken, hasMore);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a409530/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSUtils.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSUtils.java
index 95e26d7..fcc7bab 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSUtils.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSUtils.java
@@ -43,6 +43,8 @@ public class HttpFSUtils {
 
   public static final String SERVICE_VERSION = "/v1";
 
+  public static final byte[] EMPTY_BYTES = {};
+
   private static final String SERVICE_PATH = SERVICE_NAME + SERVICE_VERSION;
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a409530/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java
index 39597eb..46948f9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java
@@ -22,6 +22,7 @@ import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FileChecksum;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FilterFileSystem;
 import org.apache.hadoop.fs.GlobFilter;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
@@ -37,6 +38,7 @@ import org.apache.hadoop.util.StringUtils;
 import org.json.simple.JSONArray;
 import org.json.simple.JSONObject;
 
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
@@ -108,6 +110,27 @@ public class FSOperations {
     return json;
   }
 
+  /**
+   * Serializes a DirectoryEntries object into the JSON for a
+   * WebHDFS {@link org.apache.hadoop.hdfs.protocol.DirectoryListing}.
+   * <p>
+   * These two classes are slightly different, due to the impedance
+   * mismatches between the WebHDFS and FileSystem APIs.
+   * @param entries
+   * @return json
+   */
+  private static Map<String, Object> toJson(FileSystem.DirectoryEntries
+      entries) {
+    Map<String, Object> json = new LinkedHashMap<>();
+    Map<String, Object> inner = new LinkedHashMap<>();
+    Map<String, Object> fileStatuses = toJson(entries.getEntries());
+    inner.put(HttpFSFileSystem.PARTIAL_LISTING_JSON, fileStatuses);
+    inner.put(HttpFSFileSystem.REMAINING_ENTRIES_JSON, entries.hasMore() ? 1
+        : 0);
+    json.put(HttpFSFileSystem.DIRECTORY_LISTING_JSON, inner);
+    return json;
+  }
+
   /** Converts an <code>AclStatus</code> object into a JSON object.
    *
    * @param aclStatus AclStatus object
@@ -625,6 +648,45 @@ public class FSOperations {
   }
 
   /**
+   * Executor that performs a batched directory listing.
+   */
+  @InterfaceAudience.Private
+  public static class FSListStatusBatch implements FileSystemAccess
+      .FileSystemExecutor<Map> {
+    private final Path path;
+    private final byte[] token;
+
+    public FSListStatusBatch(String path, byte[] token) throws IOException {
+      this.path = new Path(path);
+      this.token = token.clone();
+    }
+
+    /**
+     * Simple wrapper filesystem that exposes the protected batched
+     * listStatus API so we can use it.
+     */
+    private static class WrappedFileSystem extends FilterFileSystem {
+      public WrappedFileSystem(FileSystem f) {
+        super(f);
+      }
+
+      @Override
+      public DirectoryEntries listStatusBatch(Path f, byte[] token) throws
+          FileNotFoundException, IOException {
+        return super.listStatusBatch(f, token);
+      }
+    }
+
+    @Override
+    public Map execute(FileSystem fs) throws IOException {
+      WrappedFileSystem wrappedFS = new WrappedFileSystem(fs);
+      FileSystem.DirectoryEntries entries =
+          wrappedFS.listStatusBatch(path, token);
+      return toJson(entries);
+    }
+  }
+
+  /**
    * Executor that performs a mkdirs FileSystemAccess files system operation.
    */
   @InterfaceAudience.Private

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a409530/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSParametersProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSParametersProvider.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSParametersProvider.java
index 5c4204a..25585c5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSParametersProvider.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSParametersProvider.java
@@ -91,6 +91,8 @@ public class HttpFSParametersProvider extends ParametersProvider {
     PARAMS_DEF.put(Operation.GETXATTRS, 
         new Class[]{XAttrNameParam.class, XAttrEncodingParam.class});
     PARAMS_DEF.put(Operation.LISTXATTRS, new Class[]{});
+    PARAMS_DEF.put(Operation.LISTSTATUS_BATCH,
+        new Class[]{StartAfterParam.class});
   }
 
   public HttpFSParametersProvider() {
@@ -520,4 +522,22 @@ public class HttpFSParametersProvider extends ParametersProvider {
       super(NAME, XAttrCodec.class, null);
     }
   }
+
+  /**
+   * Class for startafter parameter.
+   */
+  @InterfaceAudience.Private
+  public static class StartAfterParam extends StringParam {
+    /**
+     * Parameter name.
+     */
+    public static final String NAME = HttpFSFileSystem.START_AFTER_PARAM;
+
+    /**
+     * Constructor.
+     */
+    public StartAfterParam() {
+      super(NAME, null);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a409530/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java
index db4692a..a4db124 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java
@@ -18,12 +18,14 @@
 
 package org.apache.hadoop.fs.http.server;
 
+import com.google.common.base.Charsets;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.XAttrCodec;
 import org.apache.hadoop.fs.XAttrSetFlag;
 import org.apache.hadoop.fs.http.client.HttpFSFileSystem;
+import org.apache.hadoop.fs.http.client.HttpFSUtils;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.AccessTimeParam;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.AclPermissionParam;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.BlockSizeParam;
@@ -320,6 +322,21 @@ public class HttpFSServer {
       response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
       break;
     }
+    case LISTSTATUS_BATCH: {
+      String startAfter = params.get(
+          HttpFSParametersProvider.StartAfterParam.NAME,
+          HttpFSParametersProvider.StartAfterParam.class);
+      byte[] token = HttpFSUtils.EMPTY_BYTES;
+      if (startAfter != null) {
+        token = startAfter.getBytes(Charsets.UTF_8);
+      }
+      FSOperations.FSListStatusBatch command = new FSOperations
+          .FSListStatusBatch(path, token);
+      @SuppressWarnings("rawtypes") Map json = fsExecute(user, command);
+      AUDIT_LOG.info("[{}] token [{}]", path, token);
+      response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
+      break;
+    }
     default: {
       throw new IOException(
           MessageFormat.format("Invalid HTTP GET operation [{0}]", op.value()));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a409530/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/hadoop/FileSystemAccessService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/hadoop/FileSystemAccessService.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/hadoop/FileSystemAccessService.java
index 88780cb..0b767be 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/hadoop/FileSystemAccessService.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/hadoop/FileSystemAccessService.java
@@ -84,7 +84,7 @@ public class FileSystemAccessService extends BaseService implements FileSystemAc
       count = 0;
     }
 
-    synchronized FileSystem getFileSytem(Configuration conf)
+    synchronized FileSystem getFileSystem(Configuration conf)
       throws IOException {
       if (fs == null) {
         fs = FileSystem.get(conf);
@@ -290,7 +290,7 @@ public class FileSystemAccessService extends BaseService implements FileSystemAc
     }
     Configuration conf = new Configuration(namenodeConf);
     conf.set(HTTPFS_FS_USER, user);
-    return cachedFS.getFileSytem(conf);
+    return cachedFS.getFileSystem(conf);
   }
 
   protected void closeFileSystem(FileSystem fs) throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8a409530/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java
index aea6cf8..e475803 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystemTestHelper;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.http.server.HttpFSServerWebApp;
 import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclStatus;
@@ -44,6 +45,7 @@ import org.apache.hadoop.test.TestHdfsHelper;
 import org.apache.hadoop.test.TestJetty;
 import org.apache.hadoop.test.TestJettyHelper;
 import org.junit.Assert;
+import org.junit.Assume;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
@@ -62,6 +64,7 @@ import java.io.Writer;
 import java.net.URI;
 import java.net.URL;
 import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.List;
@@ -136,14 +139,19 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
     return "webhdfs";
   }
 
-  protected FileSystem getHttpFSFileSystem() throws Exception {
-    Configuration conf = new Configuration();
+  protected FileSystem getHttpFSFileSystem(Configuration conf) throws
+      Exception {
     conf.set("fs.webhdfs.impl", getFileSystemClass().getName());
     URI uri = new URI(getScheme() + "://" +
                       TestJettyHelper.getJettyURL().toURI().getAuthority());
     return FileSystem.get(uri, conf);
   }
 
+  protected FileSystem getHttpFSFileSystem() throws Exception {
+    Configuration conf = new Configuration();
+    return getHttpFSFileSystem(conf);
+  }
+
   protected void testGet() throws Exception {
     FileSystem fs = getHttpFSFileSystem();
     Assert.assertNotNull(fs);
@@ -355,6 +363,51 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
     assertEquals(stati[0].getPath().getName(), path.getName());
   }
 
+  private static void assertSameListing(FileSystem expected, FileSystem
+      actual, Path p) throws IOException {
+    // Consume all the entries from both iterators
+    RemoteIterator<FileStatus> exIt = expected.listStatusIterator(p);
+    List<FileStatus> exStatuses = new ArrayList<>();
+    while (exIt.hasNext()) {
+      exStatuses.add(exIt.next());
+    }
+    RemoteIterator<FileStatus> acIt = actual.listStatusIterator(p);
+    List<FileStatus> acStatuses = new ArrayList<>();
+    while (acIt.hasNext()) {
+      acStatuses.add(acIt.next());
+    }
+    assertEquals(exStatuses.size(), acStatuses.size());
+    for (int i = 0; i < exStatuses.size(); i++) {
+      FileStatus expectedStatus = exStatuses.get(i);
+      FileStatus actualStatus = acStatuses.get(i);
+      // Path URIs are fully qualified, so compare just the path component
+      assertEquals(expectedStatus.getPath().toUri().getPath(),
+          actualStatus.getPath().toUri().getPath());
+    }
+  }
+
+  private void testListStatusBatch() throws Exception {
+    // LocalFileSystem writes checksum files next to the data files, which
+    // show up when listing via LFS. This makes the listings not compare
+    // properly.
+    Assume.assumeFalse(isLocalFS());
+
+    FileSystem proxyFs = FileSystem.get(getProxiedFSConf());
+    Configuration conf = new Configuration();
+    conf.setInt(DFSConfigKeys.DFS_LIST_LIMIT, 2);
+    FileSystem httpFs = getHttpFSFileSystem(conf);
+
+    // Test an empty directory
+    Path dir = new Path(getProxiedFSTestDir(), "dir");
+    proxyFs.mkdirs(dir);
+    assertSameListing(proxyFs, httpFs, dir);
+    // Create and test in a loop
+    for (int i = 0; i < 10; i++) {
+      proxyFs.create(new Path(dir, "file" + i)).close();
+      assertSameListing(proxyFs, httpFs, dir);
+    }
+  }
+
   private void testWorkingdirectory() throws Exception {
     FileSystem fs = FileSystem.get(getProxiedFSConf());
     Path workingDir = fs.getWorkingDirectory();
@@ -863,7 +916,7 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
     GET, OPEN, CREATE, APPEND, TRUNCATE, CONCAT, RENAME, DELETE, LIST_STATUS, 
     WORKING_DIRECTORY, MKDIRS, SET_TIMES, SET_PERMISSION, SET_OWNER, 
     SET_REPLICATION, CHECKSUM, CONTENT_SUMMARY, FILEACLS, DIRACLS, SET_XATTR,
-    GET_XATTRS, REMOVE_XATTR, LIST_XATTRS, ENCRYPTION
+    GET_XATTRS, REMOVE_XATTR, LIST_XATTRS, ENCRYPTION, LIST_STATUS_BATCH
   }
 
   private void operation(Operation op) throws Exception {
@@ -940,6 +993,9 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
       case ENCRYPTION:
         testEncryption();
         break;
+      case LIST_STATUS_BATCH:
+        testListStatusBatch();
+        break;
     }
   }
 


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


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

Branch: refs/heads/YARN-2915
Commit: 377f1ea09d645561f10a6da95512ea837a1939bc
Parents: b627de2
Author: Jian He <ji...@apache.org>
Authored: Fri Sep 2 12:23:57 2016 +0800
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon Sep 19 12:58:34 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/377f1ea0/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 7dabc8e..55000e7 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/377f1ea0/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 50dc154..fae8b79 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/377f1ea0/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/377f1ea0/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 15330bc..40621a3 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
@@ -737,6 +737,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/377f1ea0/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/377f1ea0/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/377f1ea0/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/377f1ea0/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/377f1ea0/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 9e05f6d..cdb7133 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
@@ -337,7 +337,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
       byte[] data = getData(epochNodePath);

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


[18/44] hadoop git commit: MAPREDUCE-6774. Add support for HDFS erasure code policy to TestDFSIO. Contributed by Sammi Chen

Posted by su...@apache.org.
MAPREDUCE-6774. Add support for HDFS erasure code policy to TestDFSIO. Contributed by Sammi Chen


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

Branch: refs/heads/YARN-2915
Commit: 501a77856d6b6edfb261547117e719da7a9cd221
Parents: 58bae35
Author: Kai Zheng <ka...@intel.com>
Authored: Sun Sep 18 09:03:15 2016 +0800
Committer: Kai Zheng <ka...@intel.com>
Committed: Sun Sep 18 09:03:15 2016 +0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/fs/TestDFSIO.java    | 159 +++++++++++++++----
 1 file changed, 124 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/501a7785/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/TestDFSIO.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/TestDFSIO.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/TestDFSIO.java
index 05d4d77..d218169 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/TestDFSIO.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/TestDFSIO.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.SequenceFile.CompressionType;
@@ -106,9 +107,14 @@ public class TestDFSIO implements Tool {
                     " [-nrFiles N]" +
                     " [-size Size[B|KB|MB|GB|TB]]" +
                     " [-resFile resultFileName] [-bufferSize Bytes]" +
-                    " [-storagePolicy storagePolicyName]";
+                    " [-storagePolicy storagePolicyName]" +
+                    " [-erasureCodePolicy erasureCodePolicyName]";
 
   private Configuration config;
+  private static final String STORAGE_POLICY_NAME_KEY =
+      "test.io.block.storage.policy";
+  private static final String ERASURE_CODE_POLICY_NAME_KEY =
+      "test.io.erasure.code.policy";
 
   static{
     Configuration.addDefaultResource("hdfs-default.xml");
@@ -211,9 +217,9 @@ public class TestDFSIO implements Tool {
     bench = new TestDFSIO();
     bench.getConf().setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
     cluster = new MiniDFSCluster.Builder(bench.getConf())
-                                .numDataNodes(2)
-                                .format(true)
-                                .build();
+        .numDataNodes(2)
+        .format(true)
+        .build();
     FileSystem fs = cluster.getFileSystem();
     bench.createControlFile(fs, DEFAULT_NR_BYTES, DEFAULT_NR_FILES);
 
@@ -356,7 +362,7 @@ public class TestDFSIO implements Tool {
             ReflectionUtils.newInstance(codec, getConf());
       }
 
-      blockStoragePolicy = getConf().get("test.io.block.storage.policy", null);
+      blockStoragePolicy = getConf().get(STORAGE_POLICY_NAME_KEY, null);
     }
 
     @Override // IOMapperBase
@@ -388,9 +394,10 @@ public class TestDFSIO implements Tool {
    */
   public static class WriteMapper extends IOStatMapper {
 
-    public WriteMapper() { 
-      for(int i=0; i < bufferSize; i++)
-        buffer[i] = (byte)('0' + i % 50);
+    public WriteMapper() {
+      for (int i = 0; i < bufferSize; i++) {
+        buffer[i] = (byte) ('0' + i % 50);
+      }
     }
 
     @Override // IOMapperBase
@@ -431,6 +438,9 @@ public class TestDFSIO implements Tool {
     fs.delete(getDataDir(config), true);
     fs.delete(writeDir, true);
     long tStart = System.currentTimeMillis();
+    if (isECEnabled()) {
+      createAndEnableECOnPath(fs, getDataDir(config));
+    }
     runIOTest(WriteMapper.class, writeDir);
     long execTime = System.currentTimeMillis() - tStart;
     return execTime;
@@ -734,6 +744,7 @@ public class TestDFSIO implements Tool {
     TestType testType = null;
     int bufferSize = DEFAULT_BUFFER_SIZE;
     long nrBytes = 1*MEGA;
+    String erasureCodePolicyName = null;
     int nrFiles = 1;
     long skipSize = 0;
     String resFileName = DEFAULT_RES_FILE_NAME;
@@ -785,26 +796,31 @@ public class TestDFSIO implements Tool {
         resFileName = args[++i];
       } else if (args[i].equalsIgnoreCase("-storagePolicy")) {
         storagePolicy = args[++i];
+      } else if (args[i].equalsIgnoreCase("-erasureCodePolicy")) {
+        erasureCodePolicyName = args[++i];
       } else {
         System.err.println("Illegal argument: " + args[i]);
         return -1;
       }
     }
-    if(testType == null)
+    if (testType == null) {
       return -1;
-    if(testType == TestType.TEST_TYPE_READ_BACKWARD)
+    }
+    if (testType == TestType.TEST_TYPE_READ_BACKWARD) {
       skipSize = -bufferSize;
-    else if(testType == TestType.TEST_TYPE_READ_SKIP && skipSize == 0)
+    } else if (testType == TestType.TEST_TYPE_READ_SKIP && skipSize == 0) {
       skipSize = bufferSize;
+    }
 
     LOG.info("nrFiles = " + nrFiles);
     LOG.info("nrBytes (MB) = " + toMB(nrBytes));
     LOG.info("bufferSize = " + bufferSize);
-    if(skipSize > 0)
+    if (skipSize > 0) {
       LOG.info("skipSize = " + skipSize);
+    }
     LOG.info("baseDir = " + getBaseDir(config));
     
-    if(compressionClass != null) {
+    if (compressionClass != null) {
       config.set("test.io.compression.class", compressionClass);
       LOG.info("compressionClass = " + compressionClass);
     }
@@ -813,31 +829,16 @@ public class TestDFSIO implements Tool {
     config.setLong("test.io.skip.size", skipSize);
     FileSystem fs = FileSystem.get(config);
 
-    if (storagePolicy != null) {
-      boolean isValid = false;
-      Collection<BlockStoragePolicy> storagePolicies =
-          ((DistributedFileSystem) fs).getAllStoragePolicies();
-      try {
-        for (BlockStoragePolicy policy : storagePolicies) {
-          if (policy.getName().equals(storagePolicy)) {
-            isValid = true;
-            break;
-          }
-        }
-      } catch (Exception e) {
-        throw new IOException("Get block storage policies error: ", e);
-      }
-      if (!isValid) {
-        System.out.println("Invalid block storage policy: " + storagePolicy);
-        System.out.println("Current supported storage policy list: ");
-        for (BlockStoragePolicy policy : storagePolicies) {
-          System.out.println(policy.getName());
-        }
+    if (erasureCodePolicyName != null) {
+      if (!checkErasureCodePolicy(erasureCodePolicyName, fs, testType)) {
         return -1;
       }
+    }
 
-      config.set("test.io.block.storage.policy", storagePolicy);
-      LOG.info("storagePolicy = " + storagePolicy);
+    if (storagePolicy != null) {
+      if (!checkStoragePolicy(storagePolicy, fs)) {
+        return -1;
+      }
     }
 
     if (isSequential) {
@@ -908,6 +909,94 @@ public class TestDFSIO implements Tool {
     return ((float)bytes)/MEGA;
   }
 
+  private boolean checkErasureCodePolicy(String erasureCodePolicyName,
+      FileSystem fs, TestType testType) throws IOException {
+    Collection<ErasureCodingPolicy> list =
+        ((DistributedFileSystem) fs).getAllErasureCodingPolicies();
+    boolean isValid = false;
+    for (ErasureCodingPolicy ec : list) {
+      if (erasureCodePolicyName.equals(ec.getName())) {
+        isValid = true;
+        break;
+      }
+    }
+
+    if (!isValid) {
+      System.out.println("Invalid erasure code policy: " +
+          erasureCodePolicyName);
+      System.out.println("Current supported erasure code policy list: ");
+      for (ErasureCodingPolicy ec : list) {
+        System.out.println(ec.getName());
+      }
+      return false;
+    }
+
+    if (testType == TestType.TEST_TYPE_APPEND ||
+        testType == TestType.TEST_TYPE_TRUNCATE) {
+      System.out.println("So far append or truncate operation" +
+          " does not support erasureCodePolicy");
+      return false;
+    }
+
+    config.set(ERASURE_CODE_POLICY_NAME_KEY, erasureCodePolicyName);
+    LOG.info("erasureCodePolicy = " + erasureCodePolicyName);
+    return true;
+  }
+
+  private boolean checkStoragePolicy(String storagePolicy, FileSystem fs)
+      throws IOException {
+    boolean isValid = false;
+    Collection<BlockStoragePolicy> storagePolicies =
+        ((DistributedFileSystem) fs).getAllStoragePolicies();
+    try {
+      for (BlockStoragePolicy policy : storagePolicies) {
+        if (policy.getName().equals(storagePolicy)) {
+          isValid = true;
+          break;
+        }
+      }
+    } catch (Exception e) {
+      throw new IOException("Get block storage policies error: ", e);
+    }
+
+    if (!isValid) {
+      System.out.println("Invalid block storage policy: " + storagePolicy);
+      System.out.println("Current supported storage policy list: ");
+      for (BlockStoragePolicy policy : storagePolicies) {
+        System.out.println(policy.getName());
+      }
+      return false;
+    }
+
+    config.set(STORAGE_POLICY_NAME_KEY, storagePolicy);
+    LOG.info("storagePolicy = " + storagePolicy);
+    return true;
+  }
+
+  private boolean isECEnabled() {
+    String erasureCodePolicyName =
+        getConf().get(ERASURE_CODE_POLICY_NAME_KEY, null);
+    return erasureCodePolicyName != null ? true : false;
+  }
+
+  void createAndEnableECOnPath(FileSystem fs, Path path)
+      throws IOException {
+    String erasureCodePolicyName =
+        getConf().get(ERASURE_CODE_POLICY_NAME_KEY, null);
+
+    fs.mkdirs(path);
+    Collection<ErasureCodingPolicy> list =
+        ((DistributedFileSystem) fs).getAllErasureCodingPolicies();
+    for (ErasureCodingPolicy ec : list) {
+      if (erasureCodePolicyName.equals(ec.getName())) {
+        ((DistributedFileSystem) fs).setErasureCodingPolicy(path, ec);
+        LOG.info("enable erasureCodePolicy = " + erasureCodePolicyName  +
+            " on " + path.toString());
+        break;
+      }
+    }
+  }
+
   private void analyzeResult( FileSystem fs,
                               TestType testType,
                               long execTime,


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


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

Branch: refs/heads/YARN-2915
Commit: 888c150048b0ca3fe57d29306fce465a213f81f4
Parents: aaedc83
Author: Subru Krishnan <su...@apache.org>
Authored: Wed Aug 17 12:07:06 2016 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon Sep 19 12:58:34 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/888c1500/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/888c1500/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/888c1500/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/888c1500/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/888c1500/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/888c1500/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/888c1500/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


[06/44] hadoop git commit: Revert "HADOOP-13410. RunJar adds the content of the jar twice to the classpath (Yuanbo Liu via sjlee)"

Posted by su...@apache.org.
Revert "HADOOP-13410. RunJar adds the content of the jar twice to the classpath (Yuanbo Liu via sjlee)"

This reverts commit 4d3ea92f4fe4be2a0ee9849c65cb1c91b0c5711b.


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

Branch: refs/heads/YARN-2915
Commit: fcbac00212e99c91bd1368e002823d1260b3879f
Parents: 7cad7b7
Author: Sangjin Lee <sj...@apache.org>
Authored: Thu Sep 15 10:18:56 2016 -0700
Committer: Sangjin Lee <sj...@apache.org>
Committed: Thu Sep 15 10:18:56 2016 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/util/RunJar.java           | 6 ++++--
 1 file changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fcbac002/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/RunJar.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/RunJar.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/RunJar.java
index d91a78b..19b51ad 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/RunJar.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/RunJar.java
@@ -226,7 +226,7 @@ public class RunJar {
 
     unJar(file, workDir);
 
-    ClassLoader loader = createClassLoader(workDir);
+    ClassLoader loader = createClassLoader(file, workDir);
 
     Thread.currentThread().setContextClassLoader(loader);
     Class<?> mainClass = Class.forName(mainClassName, true, loader);
@@ -250,13 +250,14 @@ public class RunJar {
    * the user jar as well as the HADOOP_CLASSPATH. Otherwise, it creates a
    * classloader that simply adds the user jar to the classpath.
    */
-  private ClassLoader createClassLoader(final File workDir)
+  private ClassLoader createClassLoader(File file, final File workDir)
       throws MalformedURLException {
     ClassLoader loader;
     // see if the client classloader is enabled
     if (useClientClassLoader()) {
       StringBuilder sb = new StringBuilder();
       sb.append(workDir).append("/").
+          append(File.pathSeparator).append(file).
           append(File.pathSeparator).append(workDir).append("/classes/").
           append(File.pathSeparator).append(workDir).append("/lib/*");
       // HADOOP_CLASSPATH is added to the client classpath
@@ -276,6 +277,7 @@ public class RunJar {
     } else {
       List<URL> classPath = new ArrayList<>();
       classPath.add(new File(workDir + "/").toURI().toURL());
+      classPath.add(file.toURI().toURL());
       classPath.add(new File(workDir, "classes/").toURI().toURL());
       File[] libs = new File(workDir, "lib").listFiles();
       if (libs != null) {


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


[09/44] hadoop git commit: HADOOP-13411. Checkstyle suppression by annotation or comment. Contributed by John Zhuge.

Posted by su...@apache.org.
HADOOP-13411. Checkstyle suppression by annotation or comment. Contributed by John Zhuge.


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

Branch: refs/heads/YARN-2915
Commit: 0e68e14d5506288bbda02cfc755babf1e9875187
Parents: b09a03c
Author: Wei-Chiu Chuang <we...@apache.org>
Authored: Fri Sep 16 14:16:25 2016 -0700
Committer: Wei-Chiu Chuang <we...@apache.org>
Committed: Fri Sep 16 14:16:25 2016 -0700

----------------------------------------------------------------------
 .../src/main/resources/checkstyle/checkstyle.xml          | 10 +++++++++-
 1 file changed, 9 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0e68e14d/hadoop-build-tools/src/main/resources/checkstyle/checkstyle.xml
----------------------------------------------------------------------
diff --git a/hadoop-build-tools/src/main/resources/checkstyle/checkstyle.xml b/hadoop-build-tools/src/main/resources/checkstyle/checkstyle.xml
index 3aaf176..033529f 100644
--- a/hadoop-build-tools/src/main/resources/checkstyle/checkstyle.xml
+++ b/hadoop-build-tools/src/main/resources/checkstyle/checkstyle.xml
@@ -49,6 +49,10 @@
 
 <module name="Checker">
 
+    <module name="SuppressWarningsFilter"/>
+    <module name="SuppressionCommentFilter"/>
+    <module name="SuppressWithNearbyCommentFilter"/>
+
     <!-- Checks that a package.html file exists for each package.     -->
     <!-- See http://checkstyle.sf.net/config_javadoc.html#PackageHtml -->
     <module name="JavadocPackage"/>
@@ -67,6 +71,10 @@
 
     <module name="TreeWalker">
 
+        <module name="SuppressWarningsHolder"/>
+        <module name="FileContentsHolder"/>
+
+
         <!-- Checks for Javadoc comments.                     -->
         <!-- See http://checkstyle.sf.net/config_javadoc.html -->
         <module name="JavadocType">
@@ -180,7 +188,7 @@
         <module name="Indentation">
             <property name="basicOffset" value="2" />
             <property name="caseIndent" value="0" />
-        </module> 
+        </module>
         <!--<module name="TodoComment"/>-->
         <module name="UpperEll"/>
 


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


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

Branch: refs/heads/YARN-2915
Commit: b5af206d698b8f0340d6d10f6fbb8ee5089d6231
Parents: 888c150
Author: Jian He <ji...@apache.org>
Authored: Mon Aug 22 14:43:07 2016 +0800
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon Sep 19 12:58:34 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/b5af206d/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/b5af206d/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 1e53bf4..293df71 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
@@ -2487,6 +2487,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/b5af206d/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/b5af206d/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/b5af206d/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/b5af206d/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/b5af206d/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 4fe916f..5461535 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
@@ -2632,6 +2632,13 @@
   <!-- 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/b5af206d/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/b5af206d/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/b5af206d/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/b5af206d/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/b5af206d/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


[02/44] hadoop git commit: HDFS-10805. Reduce runtime for append test. Contributed by Gergely Novak.

Posted by su...@apache.org.
HDFS-10805. Reduce runtime for append test. Contributed by Gergely Novak.


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

Branch: refs/heads/YARN-2915
Commit: 2a8f55a0cf147b567d70c0c07f84cf27fd413c19
Parents: 53a12fa
Author: Arpit Agarwal <ar...@apache.org>
Authored: Wed Sep 14 09:31:17 2016 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Wed Sep 14 09:31:17 2016 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/AppendTestUtil.java    | 18 +++++++++++++++---
 1 file changed, 15 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2a8f55a0/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/AppendTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/AppendTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/AppendTestUtil.java
index de4da5f..268bdf9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/AppendTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/AppendTestUtil.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs;
 
+import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
@@ -238,7 +239,8 @@ public class AppendTestUtil {
   }
 
   public static void testAppend(FileSystem fs, Path p) throws IOException {
-    final byte[] bytes = new byte[1000];
+    final int size = 1000;
+    final byte[] bytes = randomBytes(seed, size);
 
     { //create file
       final FSDataOutputStream out = fs.create(p, (short)1);
@@ -247,12 +249,22 @@ public class AppendTestUtil {
       assertEquals(bytes.length, fs.getFileStatus(p).getLen());
     }
 
-    for(int i = 2; i < 500; i++) {
+    final int appends = 50;
+    for (int i = 2; i < appends; i++) {
       //append
       final FSDataOutputStream out = fs.append(p);
       out.write(bytes);
       out.close();
-      assertEquals(i*bytes.length, fs.getFileStatus(p).getLen());
+      assertEquals(i * bytes.length, fs.getFileStatus(p).getLen());
     }
+
+    // Check the appended content
+    final FSDataInputStream in = fs.open(p);
+    for (int i = 0; i < appends - 1; i++) {
+      byte[] read = new byte[size];
+      in.read(i * bytes.length, read, 0, size);
+      assertArrayEquals(bytes, read);
+    }
+    in.close();
   }
 }
\ 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/44] hadoop git commit: YARN-4232. TopCLI console support for HA mode. Contributed by Bibin A Chundatt

Posted by su...@apache.org.
YARN-4232. TopCLI console support for HA mode. Contributed by Bibin A Chundatt


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

Branch: refs/heads/YARN-2915
Commit: ade7c2bc9ccf09d843ccb3dfa56c1453a9e87318
Parents: 501a778
Author: Naganarasimha <na...@apache.org>
Authored: Sat Sep 17 09:52:39 2016 +0530
Committer: Naganarasimha <na...@apache.org>
Committed: Sat Sep 17 09:52:39 2016 +0530

----------------------------------------------------------------------
 .../apache/hadoop/yarn/client/cli/TopCLI.java   | 114 ++++++++++++++++---
 .../hadoop/yarn/client/cli/TestTopCLI.java      | 106 +++++++++++++++++
 .../hadoop/yarn/webapp/util/WebAppUtils.java    |  35 +++---
 3 files changed, 224 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ade7c2bc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/TopCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/TopCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/TopCLI.java
index eca1266..f99dd48 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/TopCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/TopCLI.java
@@ -20,10 +20,13 @@ package org.apache.hadoop.yarn.client.cli;
 
 import java.io.IOException;
 import java.io.InputStream;
+import java.net.ConnectException;
+import java.net.MalformedURLException;
 import java.net.URL;
 import java.net.URLConnection;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.EnumMap;
@@ -37,6 +40,10 @@ import java.util.Set;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 
+import javax.net.ssl.HttpsURLConnection;
+import javax.net.ssl.SSLSocketFactory;
+
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.cache.Cache;
 import com.google.common.cache.CacheBuilder;
 import org.apache.commons.cli.CommandLine;
@@ -50,7 +57,12 @@ import org.apache.commons.lang.time.DateFormatUtils;
 import org.apache.commons.lang.time.DurationFormatUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.http.HttpConfig.Policy;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
+import org.apache.hadoop.security.authentication.client.KerberosAuthenticator;
+import org.apache.hadoop.security.ssl.SSLFactory;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest;
@@ -60,12 +72,17 @@ import org.apache.hadoop.yarn.api.records.QueueInfo;
 import org.apache.hadoop.yarn.api.records.QueueStatistics;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.api.records.YarnClusterMetrics;
+import org.apache.hadoop.yarn.conf.HAUtil;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
+import org.codehaus.jettison.json.JSONException;
 import org.codehaus.jettison.json.JSONObject;
 
 public class TopCLI extends YarnCLI {
 
+  private static final String CLUSTER_INFO_URL = "/ws/v1/cluster/info";
+
   private static final Log LOG = LogFactory.getLog(TopCLI.class);
   private String CLEAR = "\u001b[2J";
   private String CLEAR_LINE = "\u001b[2K";
@@ -742,18 +759,12 @@ public class TopCLI extends YarnCLI {
 
   long getRMStartTime() {
     try {
-      URL url =
-          new URL("http://"
-              + client.getConfig().get(YarnConfiguration.RM_WEBAPP_ADDRESS)
-              + "/ws/v1/cluster/info");
-      URLConnection conn = url.openConnection();
-      conn.connect();
-      InputStream in = conn.getInputStream();
-      String encoding = conn.getContentEncoding();
-      encoding = encoding == null ? "UTF-8" : encoding;
-      String body = IOUtils.toString(in, encoding);
-      JSONObject obj = new JSONObject(body);
-      JSONObject clusterInfo = obj.getJSONObject("clusterInfo");
+      // connect with url
+      URL url = getClusterUrl();
+      if (null == url) {
+        return -1;
+      }
+      JSONObject clusterInfo = getJSONObject(connect(url));
       return clusterInfo.getLong("startedOn");
     } catch (Exception e) {
       LOG.error("Could not fetch RM start time", e);
@@ -761,6 +772,80 @@ public class TopCLI extends YarnCLI {
     return -1;
   }
 
+  private JSONObject getJSONObject(URLConnection conn)
+      throws IOException, JSONException {
+    InputStream in = conn.getInputStream();
+    String encoding = conn.getContentEncoding();
+    encoding = encoding == null ? "UTF-8" : encoding;
+    String body = IOUtils.toString(in, encoding);
+    JSONObject obj = new JSONObject(body);
+    JSONObject clusterInfo = obj.getJSONObject("clusterInfo");
+    return clusterInfo;
+  }
+
+  private URL getClusterUrl() throws Exception {
+    URL url = null;
+    Configuration conf = getConf();
+    if (HAUtil.isHAEnabled(conf)) {
+      Collection<String> haids = HAUtil.getRMHAIds(conf);
+      for (String rmhid : haids) {
+        try {
+          url = getHAClusterUrl(conf, rmhid);
+          if (isActive(url)) {
+            break;
+          }
+        } catch (ConnectException e) {
+          // ignore and try second one when one of RM is down
+        }
+      }
+    } else {
+      url = new URL(
+          WebAppUtils.getRMWebAppURLWithScheme(conf) + CLUSTER_INFO_URL);
+    }
+    return url;
+  }
+
+  private boolean isActive(URL url) throws Exception {
+    URLConnection connect = connect(url);
+    JSONObject clusterInfo = getJSONObject(connect);
+    return clusterInfo.getString("haState").equals("ACTIVE");
+  }
+
+  @VisibleForTesting
+  public URL getHAClusterUrl(Configuration conf, String rmhid)
+      throws MalformedURLException {
+    return new URL(WebAppUtils.getHttpSchemePrefix(conf)
+        + WebAppUtils.getResolvedRemoteRMWebAppURLWithoutScheme(conf,
+            YarnConfiguration.useHttps(conf) ? Policy.HTTPS_ONLY
+                : Policy.HTTP_ONLY,
+            rmhid)
+        + CLUSTER_INFO_URL);
+  }
+
+  private URLConnection connect(URL url) throws Exception {
+    AuthenticatedURL.Token token = new AuthenticatedURL.Token();
+    AuthenticatedURL authUrl;
+    SSLFactory clientSslFactory;
+    URLConnection connection;
+    // If https is chosen, configures SSL client.
+    if (YarnConfiguration.useHttps(getConf())) {
+      clientSslFactory = new SSLFactory(SSLFactory.Mode.CLIENT, getConf());
+      clientSslFactory.init();
+      SSLSocketFactory sslSocktFact = clientSslFactory.createSSLSocketFactory();
+
+      authUrl =
+          new AuthenticatedURL(new KerberosAuthenticator(), clientSslFactory);
+      connection = authUrl.openConnection(url, token);
+      HttpsURLConnection httpsConn = (HttpsURLConnection) connection;
+      httpsConn.setSSLSocketFactory(sslSocktFact);
+    } else {
+      authUrl = new AuthenticatedURL(new KerberosAuthenticator());
+      connection = authUrl.openConnection(url, token);
+    }
+    connection.connect();
+    return connection;
+  }
+
   String getHeader(QueueMetrics queueMetrics, NodesInformation nodes) {
     StringBuilder ret = new StringBuilder();
     String queue = "root";
@@ -768,7 +853,10 @@ public class TopCLI extends YarnCLI {
       queue = StringUtils.join(queues, ",");
     }
     long now = Time.now();
-    long uptime = now - rmStartTime;
+    long uptime = 0L;
+    if (rmStartTime != -1) {
+      uptime = now - rmStartTime;
+    }
     long days = TimeUnit.MILLISECONDS.toDays(uptime);
     long hours =
         TimeUnit.MILLISECONDS.toHours(uptime)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ade7c2bc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestTopCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestTopCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestTopCLI.java
new file mode 100644
index 0000000..706400f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestTopCLI.java
@@ -0,0 +1,106 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.cli;
+
+import java.io.IOException;
+import java.net.URL;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Test class for TopCli.
+ *
+ */
+public class TestTopCLI {
+
+  private static final String RM1_NODE_ID = "rm1";
+  private static final String RM2_NODE_ID = "rm2";
+
+  private static List<String> dummyHostNames =
+      Arrays.asList("host1", "host2", "host3");
+
+  private static Map<String, String> savedStaticResolution = new HashMap<>();
+
+  @BeforeClass
+  public static void initializeDummyHostnameResolution() throws Exception {
+    String previousIpAddress;
+    for (String hostName : dummyHostNames) {
+      previousIpAddress = NetUtils.getStaticResolution(hostName);
+      if (null != previousIpAddress) {
+        savedStaticResolution.put(hostName, previousIpAddress);
+      }
+      NetUtils.addStaticResolution(hostName, "10.20.30.40");
+    }
+  }
+
+  @AfterClass
+  public static void restoreDummyHostnameResolution() throws Exception {
+    for (Map.Entry<String, String> hostnameToIpEntry : savedStaticResolution
+        .entrySet()) {
+      NetUtils.addStaticResolution(hostnameToIpEntry.getKey(),
+          hostnameToIpEntry.getValue());
+    }
+  }
+
+  @Test
+  public void testHAClusterInfoURL() throws IOException, InterruptedException {
+    TopCLI topcli = new TopCLI();
+    // http
+    String rm1Address = "host2:8088";
+    String rm2Address = "host3:8088";
+    Configuration conf = topcli.getConf();
+    conf.set(YarnConfiguration.RM_WEBAPP_ADDRESS + "." + RM1_NODE_ID,
+        rm1Address);
+    topcli.getConf().set(
+        YarnConfiguration.RM_WEBAPP_ADDRESS + "." + RM2_NODE_ID, rm2Address);
+    topcli.getConf().setBoolean(YarnConfiguration.RM_HA_ENABLED, true);
+    topcli.getConf().set(YarnConfiguration.RM_HA_IDS,
+        RM1_NODE_ID + "," + RM2_NODE_ID);
+    URL clusterUrl = topcli.getHAClusterUrl(conf, RM1_NODE_ID);
+    Assert.assertEquals("http", clusterUrl.getProtocol());
+    Assert.assertEquals(rm1Address, clusterUrl.getAuthority());
+    clusterUrl = topcli.getHAClusterUrl(conf, RM2_NODE_ID);
+    Assert.assertEquals("http", clusterUrl.getProtocol());
+    Assert.assertEquals(rm2Address, clusterUrl.getAuthority());
+    // https
+    rm1Address = "host2:9088";
+    rm2Address = "host3:9088";
+    conf = topcli.getConf();
+    conf.set(YarnConfiguration.RM_WEBAPP_HTTPS_ADDRESS + "." + RM1_NODE_ID,
+        rm1Address);
+    conf.set(YarnConfiguration.RM_WEBAPP_HTTPS_ADDRESS + "." + RM2_NODE_ID,
+        rm2Address);
+    conf.setBoolean(YarnConfiguration.RM_HA_ENABLED, true);
+    conf.set(YarnConfiguration.RM_HA_IDS, RM1_NODE_ID + "," + RM2_NODE_ID);
+    conf.set(YarnConfiguration.YARN_HTTP_POLICY_KEY, "HTTPS_ONLY");
+    clusterUrl = topcli.getHAClusterUrl(conf, RM1_NODE_ID);
+    Assert.assertEquals("https", clusterUrl.getProtocol());
+    Assert.assertEquals(rm1Address, clusterUrl.getAuthority());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ade7c2bc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/util/WebAppUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/util/WebAppUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/util/WebAppUtils.java
index 0139500..89f0551 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/util/WebAppUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/util/WebAppUtils.java
@@ -40,7 +40,6 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.conf.HAUtil;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
-import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.util.RMHAUtils;
 import org.apache.hadoop.yarn.webapp.BadRequestException;
 import org.apache.hadoop.yarn.webapp.NotFoundException;
@@ -183,32 +182,32 @@ public class WebAppUtils {
 
   public static String getResolvedRemoteRMWebAppURLWithoutScheme(Configuration conf,
       Policy httpPolicy) {
-    InetSocketAddress address = null;
     String rmId = null;
     if (HAUtil.isHAEnabled(conf)) {
       // If HA enabled, pick one of the RM-IDs and rely on redirect to go to
       // the Active RM
       rmId = (String) HAUtil.getRMHAIds(conf).toArray()[0];
     }
+    return getResolvedRemoteRMWebAppURLWithoutScheme(conf, httpPolicy, rmId);
+  }
+
+  public static String getResolvedRemoteRMWebAppURLWithoutScheme(
+      Configuration conf, Policy httpPolicy, String rmId) {
+    InetSocketAddress address = null;
 
     if (httpPolicy == Policy.HTTPS_ONLY) {
-      address =
-          conf.getSocketAddr(
-              rmId == null
-                  ? YarnConfiguration.RM_WEBAPP_HTTPS_ADDRESS
-                  : HAUtil.addSuffix(
-                  YarnConfiguration.RM_WEBAPP_HTTPS_ADDRESS, rmId),
-              YarnConfiguration.DEFAULT_RM_WEBAPP_HTTPS_ADDRESS,
-              YarnConfiguration.DEFAULT_RM_WEBAPP_HTTPS_PORT);
+      address = conf.getSocketAddr(
+          rmId == null ? YarnConfiguration.RM_WEBAPP_HTTPS_ADDRESS
+              : HAUtil.addSuffix(YarnConfiguration.RM_WEBAPP_HTTPS_ADDRESS,
+                  rmId),
+          YarnConfiguration.DEFAULT_RM_WEBAPP_HTTPS_ADDRESS,
+          YarnConfiguration.DEFAULT_RM_WEBAPP_HTTPS_PORT);
     } else {
-      address =
-          conf.getSocketAddr(
-              rmId == null
-                  ? YarnConfiguration.RM_WEBAPP_ADDRESS
-                  : HAUtil.addSuffix(
-                  YarnConfiguration.RM_WEBAPP_ADDRESS, rmId),
-              YarnConfiguration.DEFAULT_RM_WEBAPP_ADDRESS,
-              YarnConfiguration.DEFAULT_RM_WEBAPP_PORT);
+      address = conf.getSocketAddr(
+          rmId == null ? YarnConfiguration.RM_WEBAPP_ADDRESS
+              : HAUtil.addSuffix(YarnConfiguration.RM_WEBAPP_ADDRESS, rmId),
+          YarnConfiguration.DEFAULT_RM_WEBAPP_ADDRESS,
+          YarnConfiguration.DEFAULT_RM_WEBAPP_PORT);
     }
     return getResolvedAddress(address);
   }


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


[31/44] 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/5ae955b3
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/5ae955b3
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/5ae955b3

Branch: refs/heads/YARN-2915
Commit: 5ae955b3501816807fa7b801d4ef2a31ba7e8684
Parents: b5af206
Author: Jian He <ji...@apache.org>
Authored: Tue Aug 30 12:20:52 2016 +0800
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon Sep 19 12:58:34 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/5ae955b3/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 293df71..50dc154 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
@@ -2494,9 +2494,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";
 
@@ -2509,6 +2506,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/5ae955b3/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/5ae955b3/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/5ae955b3/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/5ae955b3/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/5ae955b3/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/5ae955b3/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 bf72fc1..15330bc 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;
@@ -175,6 +176,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;
@@ -467,6 +469,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) &&
@@ -689,6 +695,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);
@@ -1262,6 +1282,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/5ae955b3/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/5ae955b3/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/5ae955b3/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/5ae955b3/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/5ae955b3/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


[25/44] hadoop git commit: HDFS-10868. Remove stray references to DFS_HDFS_BLOCKS_METADATA_ENABLED.

Posted by su...@apache.org.
HDFS-10868. Remove stray references to DFS_HDFS_BLOCKS_METADATA_ENABLED.


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

Branch: refs/heads/YARN-2915
Commit: c54f6ef30fbd5fbb9663e182b76bafb55ef567ad
Parents: b8a30f2
Author: Andrew Wang <wa...@apache.org>
Authored: Mon Sep 19 11:17:03 2016 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Mon Sep 19 11:17:03 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java | 3 ---
 .../src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java      | 4 ----
 .../test/java/org/apache/hadoop/tools/TestHdfsConfigFields.java  | 2 --
 3 files changed, 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c54f6ef3/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
index 642d4c8..4c754d9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
@@ -131,9 +131,6 @@ public interface HdfsClientConfigKeys {
           "dfs.client.key.provider.cache.expiry";
   long    DFS_CLIENT_KEY_PROVIDER_CACHE_EXPIRY_DEFAULT =
               TimeUnit.DAYS.toMillis(10); // 10 days
-  String  DFS_HDFS_BLOCKS_METADATA_ENABLED =
-      "dfs.datanode.hdfs-blocks-metadata.enabled";
-  boolean DFS_HDFS_BLOCKS_METADATA_ENABLED_DEFAULT = false;
 
   String  DFS_DATANODE_KERBEROS_PRINCIPAL_KEY =
       "dfs.datanode.kerberos.principal";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c54f6ef3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index 3532d25..df45e2a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
@@ -58,10 +58,6 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
       .DFS_CHECKSUM_TYPE_KEY;
   public static final String  DFS_CHECKSUM_TYPE_DEFAULT =
       HdfsClientConfigKeys.DFS_CHECKSUM_TYPE_DEFAULT;
-  public static final String  DFS_HDFS_BLOCKS_METADATA_ENABLED =
-      HdfsClientConfigKeys.DFS_HDFS_BLOCKS_METADATA_ENABLED;
-  public static final boolean DFS_HDFS_BLOCKS_METADATA_ENABLED_DEFAULT =
-      HdfsClientConfigKeys.DFS_HDFS_BLOCKS_METADATA_ENABLED_DEFAULT;
   public static final String DFS_WEBHDFS_ACL_PERMISSION_PATTERN_DEFAULT =
       HdfsClientConfigKeys.DFS_WEBHDFS_ACL_PERMISSION_PATTERN_DEFAULT;
   public static final String  DFS_WEBHDFS_NETTY_LOW_WATERMARK =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c54f6ef3/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestHdfsConfigFields.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestHdfsConfigFields.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestHdfsConfigFields.java
index 46420f1..bf29428 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestHdfsConfigFields.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestHdfsConfigFields.java
@@ -76,8 +76,6 @@ public class TestHdfsConfigFields extends TestConfigurationFieldsBase {
     configurationPropsToSkipCompare
         .add("dfs.corruptfilesreturned.max");
     configurationPropsToSkipCompare
-        .add("dfs.datanode.hdfs-blocks-metadata.enabled");
-    configurationPropsToSkipCompare
         .add("dfs.metrics.session-id");
     configurationPropsToSkipCompare
         .add("dfs.datanode.synconclose");


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


[05/44] hadoop git commit: HADOOP-13616. Broken code snippet area in Hadoop Benchmarking. Contributed by Kai Sasaki.

Posted by su...@apache.org.
HADOOP-13616. Broken code snippet area in Hadoop Benchmarking. Contributed by Kai Sasaki.


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

Branch: refs/heads/YARN-2915
Commit: 7cad7b704644f260688eb0d4d92413f3a8630f25
Parents: 40b5a59
Author: Akira Ajisaka <aa...@apache.org>
Authored: Thu Sep 15 23:40:00 2016 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Thu Sep 15 23:42:34 2016 +0900

----------------------------------------------------------------------
 .../hadoop-common/src/site/markdown/Benchmarking.md                 | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7cad7b70/hadoop-common-project/hadoop-common/src/site/markdown/Benchmarking.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/Benchmarking.md b/hadoop-common-project/hadoop-common/src/site/markdown/Benchmarking.md
index 678dcee..ebd7086 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/Benchmarking.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/Benchmarking.md
@@ -91,6 +91,7 @@ When running benchmarks with the above operation(s), please provide operation-sp
 The benchmark measures the number of operations performed by the name-node per second. Specifically, for each operation tested, it reports the total running time in seconds (_Elapsed Time_), operation throughput (_Ops per sec_), and average time for the operations (_Average Time_). The higher, the better.
 
 Following is a sample reports by running following commands that opens 100K files with 1K threads against a remote name-node. See [HDFS scalability: the limits to growth](https://www.usenix.org/legacy/publications/login/2010-04/openpdfs/shvachko.pdf) for real-world benchmark stats.
+
 ```
 $ hadoop org.apache.hadoop.hdfs.server.namenode.NNThroughputBenchmark -fs hdfs://nameservice:9000 -op open -threads 1000 -files 100000
 


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


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

Branch: refs/heads/YARN-2915
Commit: ab3ceda31665b9b5832e048ceead979317a2af57
Parents: 7d8b114
Author: Subru Krishnan <su...@apache.org>
Authored: Thu Aug 4 15:54:38 2016 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon Sep 19 12:58:34 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/ab3ceda3/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/ab3ceda3/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/ab3ceda3/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/ab3ceda3/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/ab3ceda3/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/ab3ceda3/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


[15/44] hadoop git commit: YARN-5642. Typos in 9 log messages. Contributed by Mehran Hassani

Posted by su...@apache.org.
YARN-5642. Typos in 9 log messages. Contributed by Mehran Hassani


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

Branch: refs/heads/YARN-2915
Commit: 4174b9756c8c7877797545c4356b1f40df603ec5
Parents: 7d21c28
Author: Naganarasimha <na...@apache.org>
Authored: Sat Sep 17 10:35:39 2016 +0530
Committer: Naganarasimha <na...@apache.org>
Committed: Sat Sep 17 10:35:39 2016 +0530

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/yarn/event/AsyncDispatcher.java  | 2 +-
 .../apache/hadoop/yarn/security/YarnAuthorizationProvider.java   | 2 +-
 .../FileSystemApplicationHistoryStore.java                       | 2 +-
 .../containermanager/monitor/ContainersMonitorImpl.java          | 2 +-
 .../resourcemanager/reservation/AbstractReservationSystem.java   | 2 +-
 .../server/resourcemanager/scheduler/capacity/LeafQueue.java     | 2 +-
 .../server/resourcemanager/scheduler/capacity/ParentQueue.java   | 4 ++--
 .../server/resourcemanager/scheduler/fair/FairScheduler.java     | 2 +-
 8 files changed, 9 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4174b975/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/AsyncDispatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/AsyncDispatcher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/AsyncDispatcher.java
index 89b5861..df542ed 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/AsyncDispatcher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/AsyncDispatcher.java
@@ -142,7 +142,7 @@ public class AsyncDispatcher extends AbstractService implements Dispatcher {
   protected void serviceStop() throws Exception {
     if (drainEventsOnStop) {
       blockNewEvents = true;
-      LOG.info("AsyncDispatcher is draining to stop, igonring any new events.");
+      LOG.info("AsyncDispatcher is draining to stop, ignoring any new events.");
       long endTime = System.currentTimeMillis() + getConfig()
           .getLong(YarnConfiguration.DISPATCHER_DRAIN_EVENTS_TIMEOUT,
               YarnConfiguration.DEFAULT_DISPATCHER_DRAIN_EVENTS_TIMEOUT);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4174b975/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/YarnAuthorizationProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/YarnAuthorizationProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/YarnAuthorizationProvider.java
index dd81ebd..4b43ea1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/YarnAuthorizationProvider.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/YarnAuthorizationProvider.java
@@ -54,7 +54,7 @@ public abstract class YarnAuthorizationProvider {
             (YarnAuthorizationProvider) ReflectionUtils.newInstance(
               authorizerClass, conf);
         authorizer.init(conf);
-        LOG.info(authorizerClass.getName() + " is instiantiated.");
+        LOG.info(authorizerClass.getName() + " is instantiated.");
       }
     }
     return authorizer;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4174b975/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/FileSystemApplicationHistoryStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/FileSystemApplicationHistoryStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/FileSystemApplicationHistoryStore.java
index 295b8ab..bb52b55 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/FileSystemApplicationHistoryStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/FileSystemApplicationHistoryStore.java
@@ -405,7 +405,7 @@ public class FileSystemApplicationHistoryStore extends AbstractService
           }
         }
       }
-      LOG.info("Completed reading history information of all conatiners"
+      LOG.info("Completed reading history information of all containers"
           + " of application attempt " + appAttemptId);
     } catch (IOException e) {
       LOG.info("Error when reading history information of some containers"

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4174b975/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.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/containermanager/monitor/ContainersMonitorImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java
index fb8cfc4..c456bde 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java
@@ -216,7 +216,7 @@ public class ContainersMonitorImpl extends AbstractService implements
             return false;
     }
     if (!(isPmemCheckEnabled() || isVmemCheckEnabled())) {
-      LOG.info("Neither virutal-memory nor physical-memory monitoring is " +
+      LOG.info("Neither virtual-memory nor physical-memory monitoring is " +
           "needed. Not running the monitor-thread");
       return false;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4174b975/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractReservationSystem.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/reservation/AbstractReservationSystem.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractReservationSystem.java
index 601a2a7..8769ca1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractReservationSystem.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractReservationSystem.java
@@ -415,7 +415,7 @@ public abstract class AbstractReservationSystem extends AbstractService
             minAllocation, maxAllocation, planQueueName,
             getReplanner(planQueuePath), getReservationSchedulerConfiguration()
             .getMoveOnExpiry(planQueuePath), rmContext);
-    LOG.info("Intialized plan {} based on reservable queue {}",
+    LOG.info("Initialized plan {} based on reservable queue {}",
         plan.toString(), planQueueName);
     return plan;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4174b975/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.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/capacity/LeafQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
index 1ca69be..922d711 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
@@ -230,7 +230,7 @@ public class LeafQueue extends AbstractCSQueue {
     LOG.info("Initializing " + queueName + "\n" +
         "capacity = " + queueCapacities.getCapacity() +
         " [= (float) configuredCapacity / 100 ]" + "\n" + 
-        "asboluteCapacity = " + queueCapacities.getAbsoluteCapacity() +
+        "absoluteCapacity = " + queueCapacities.getAbsoluteCapacity() +
         " [= parentAbsoluteCapacity * capacity ]" + "\n" +
         "maxCapacity = " + queueCapacities.getMaximumCapacity() +
         " [= configuredMaxCapacity ]" + "\n" +

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4174b975/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.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/capacity/ParentQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java
index a245e3b..3e9785f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java
@@ -125,9 +125,9 @@ public class ParentQueue extends AbstractCSQueue {
 
     LOG.info(queueName +
         ", capacity=" + this.queueCapacities.getCapacity() +
-        ", asboluteCapacity=" + this.queueCapacities.getAbsoluteCapacity() +
+        ", absoluteCapacity=" + this.queueCapacities.getAbsoluteCapacity() +
         ", maxCapacity=" + this.queueCapacities.getMaximumCapacity() +
-        ", asboluteMaxCapacity=" + this.queueCapacities.getAbsoluteMaximumCapacity() + 
+        ", absoluteMaxCapacity=" + this.queueCapacities.getAbsoluteMaximumCapacity() +
         ", state=" + state +
         ", acls=" + aclsString + 
         ", labels=" + labelStrBuilder.toString() + "\n" +

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4174b975/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 15e2318..310f2f9 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
@@ -944,7 +944,7 @@ public class FairScheduler extends
     FSAppAttempt application = getSchedulerApp(appAttemptId);
     if (application == null) {
       LOG.info("Calling allocate on removed " +
-          "or non existant application " + appAttemptId);
+          "or non existent application " + appAttemptId);
       return EMPTY_ALLOCATION;
     }
 


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


[22/44] hadoop git commit: YARN-5577. [Atsv2] Document object passing in infofilters with an example (Rohith Sharma K S via Varun Saxena)

Posted by su...@apache.org.
YARN-5577. [Atsv2] Document object passing in infofilters with an example (Rohith Sharma K S via Varun Saxena)


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

Branch: refs/heads/YARN-2915
Commit: ea29e3bc27f15516f4346d1312eef703bcd3d032
Parents: 3552c2b
Author: Varun Saxena <va...@apache.org>
Authored: Mon Sep 19 14:33:06 2016 +0530
Committer: Varun Saxena <va...@apache.org>
Committed: Mon Sep 19 14:33:06 2016 +0530

----------------------------------------------------------------------
 .../hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md    | 6 ++++++
 1 file changed, 6 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea29e3bc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md
index b6a0da4..6b7bd08 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md
@@ -712,6 +712,8 @@ none of the apps match the predicates, an empty list will be returned.
   "eq" means equals, "ne" means not equals and existence of key is not required for a match and "ene" means not equals but existence of key is
   required. We can combine any number of ANDs' and ORs' to create complex expressions.  Brackets can be used to club expressions together.<br/>
   _For example_ : infofilters can be "(((infokey1 eq value1) AND (infokey2 ne value1)) OR (infokey1 ene value3))".<br/>
+  Note : If value is an object then value can be given in the form of JSON format without any space.<br/>
+  _For example_ : infofilters can be (infokey1 eq {"&lt;key&gt;":"&lt;value&gt;","&lt;key&gt;":"&lt;value&gt;"...}).<br/>
   Please note that URL unsafe characters such as spaces will have to be suitably encoded.
 1. `conffilters` - If specified, matched applications must have exact matches to the given config name and must be either equal or not equal
   to the given config value. Both the config name and value must be strings. conffilters are represented in the same form as infofilters.
@@ -837,6 +839,8 @@ match the predicates, an empty list will be returned.
   "eq" means equals, "ne" means not equals and existence of key is not required for a match and "ene" means not equals but existence of key is
   required. We can combine any number of ANDs' and ORs' to create complex expressions.  Brackets can be used to club expressions together.<br/>
   _For example_ : infofilters can be "(((infokey1 eq value1) AND (infokey2 ne value1)) OR (infokey1 ene value3))".<br/>
+  Note : If value is an object then value can be given in the form of JSON format without any space.<br/>
+  _For example_ : infofilters can be (infokey1 eq {"&lt;key&gt;":"&lt;value&gt;","&lt;key&gt;":"&lt;value&gt;"...}).<br/>
   Please note that URL unsafe characters such as spaces will have to be suitably encoded.
 1. `conffilters` - If specified, matched applications must have exact matches to the given config name and must be either equal or not equal
   to the given config value. Both the config name and value must be strings. conffilters are represented in the same form as infofilters.
@@ -1035,6 +1039,8 @@ If none of the entities match the predicates, an empty list will be returned.
   "eq" means equals, "ne" means not equals and existence of key is not required for a match and "ene" means not equals but existence of key is
   required. We can combine any number of ANDs' and ORs' to create complex expressions.  Brackets can be used to club expressions together.<br/>
   _For example_ : infofilters can be "(((infokey1 eq value1) AND (infokey2 ne value1)) OR (infokey1 ene value3))".<br/>
+  Note : If value is an object then value can be given in the form of JSON format without any space.<br/>
+  _For example_ : infofilters can be (infokey1 eq {"&lt;key&gt;":"&lt;value&gt;","&lt;key&gt;":"&lt;value&gt;"...}).<br/>
   Please note that URL unsafe characters such as spaces will have to be suitably encoded.
 1. `conffilters` - If specified, matched entities must have exact matches to the given config name and must be either equal or not equal
   to the given config value. Both the config name and value must be strings. conffilters are represented in the same form as infofilters.


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


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

Branch: refs/heads/YARN-2915
Commit: 7d8b114671c159ffbefee5a3acc3909b3e39881a
Parents: ea8d4d5
Author: Subru Krishnan <su...@apache.org>
Authored: Fri Jul 29 16:53:40 2016 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon Sep 19 12:58:34 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/7d8b1146/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/7d8b1146/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 11bf56b..a4ae51f 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;
@@ -338,127 +320,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)) {
-      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 {
@@ -522,118 +389,6 @@ public class TestPBImplRecords {
     generateByNewInstance(ResourceUtilization.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/7d8b1146/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/7d8b1146/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/7d8b1146/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/7d8b1146/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/7d8b1146/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/7d8b1146/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/7d8b1146/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/7d8b1146/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/7d8b1146/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/7d8b1146/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/7d8b1146/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/7d8b1146/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/7d8b1146/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/7d8b1146/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/7d8b1146/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/7d8b1146/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


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

Branch: refs/heads/YARN-2915
Commit: aaedc83151dcf61ff5a6ec86495879cfc2676821
Parents: f874e28
Author: Jian He <ji...@apache.org>
Authored: Wed Aug 17 11:13:19 2016 +0800
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon Sep 19 12:58:34 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/aaedc831/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index d9a01a0..8c8ce2e 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/aaedc831/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 473ee0f..1e53bf4 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
@@ -2486,6 +2486,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/aaedc831/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/aaedc831/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 218ac79..4fe916f 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
@@ -2628,8 +2628,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
@@ -2638,6 +2638,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/aaedc831/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/aaedc831/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/aaedc831/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/aaedc831/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/aaedc831/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


[08/44] hadoop git commit: HDFS-10862. Typos in 4 log messages. Contributed by Mehran Hassani.

Posted by su...@apache.org.
HDFS-10862. Typos in 4 log messages. Contributed by Mehran Hassani.


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

Branch: refs/heads/YARN-2915
Commit: b09a03cd7d26cf96ec26a81ba11f00778241eb3e
Parents: ec3ea18
Author: Akira Ajisaka <aa...@apache.org>
Authored: Fri Sep 16 16:08:47 2016 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Fri Sep 16 16:08:47 2016 +0900

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java  | 2 +-
 .../hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeList.java   | 2 +-
 .../java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java   | 2 +-
 .../java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java     | 2 +-
 4 files changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b09a03cd/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java
index 0282524..77555f8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java
@@ -814,7 +814,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
       attrs = Nfs3Utils.getFileAttr(dfsClient, Nfs3Utils.getFileIdPath(handle),
           iug);
       if (readCount < count) {
-        LOG.info("Partical read. Asked offset: " + offset + " count: " + count
+        LOG.info("Partial read. Asked offset: " + offset + " count: " + count
             + " and read back: " + readCount + " file size: "
             + attrs.getSize());
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b09a03cd/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeList.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeList.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeList.java
index 80d3736..f869008 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeList.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeList.java
@@ -201,7 +201,7 @@ class FsVolumeList {
                 + " " + bpid + " on volume " + v + ": " + timeTaken + "ms");
           } catch (ClosedChannelException e) {
             FsDatasetImpl.LOG.info("The volume " + v + " is closed while " +
-                "addng replicas, ignored.");
+                "adding replicas, ignored.");
           } catch (IOException ioe) {
             FsDatasetImpl.LOG.info("Caught exception while adding replicas " +
                 "from " + v + ". Will throw later.", ioe);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b09a03cd/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 340d491..c913c23 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
@@ -326,7 +326,7 @@ public class FSDirectory implements Closeable {
     int threshold = conf.getInt(
         DFSConfigKeys.DFS_NAMENODE_NAME_CACHE_THRESHOLD_KEY,
         DFSConfigKeys.DFS_NAMENODE_NAME_CACHE_THRESHOLD_DEFAULT);
-    NameNode.LOG.info("Caching file names occuring more than " + threshold
+    NameNode.LOG.info("Caching file names occurring more than " + threshold
         + " times");
     nameCache = new NameCache<ByteArray>(threshold);
     namesystem = ns;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b09a03cd/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
index ac7d146..5135838 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
@@ -245,7 +245,7 @@ public class NNStorage extends Storage implements Closeable,
     /* We don't want more than one thread trying to restore at a time */
     synchronized (this.restorationLock) {
       LOG.info("NNStorage.attemptRestoreRemovedStorage: check removed(failed) "+
-               "storarge. removedStorages size = " + removedStorageDirs.size());
+               "storage. removedStorages size = " + removedStorageDirs.size());
       for (StorageDirectory sd : this.removedStorageDirs) {
         File root = sd.getRoot();
         LOG.info("currently disabled dir " + root.getAbsolutePath() +


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


[37/44] 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/9abc7daf
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/9abc7daf
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/9abc7daf

Branch: refs/heads/YARN-2915
Commit: 9abc7dafde05e788a525af47ec015c2672d9c111
Parents: 377f1ea
Author: Subru Krishnan <su...@apache.org>
Authored: Wed Sep 7 17:33:34 2016 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon Sep 19 12:58:34 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/9abc7daf/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/9abc7daf/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/9abc7daf/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/9abc7daf/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/9abc7daf/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/9abc7daf/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/9abc7daf/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/9abc7daf/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/9abc7daf/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/9abc7daf/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/9abc7daf/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/9abc7daf/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/9abc7daf/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/9abc7daf/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/9abc7daf/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/9abc7daf/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/9abc7daf/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


[16/44] hadoop git commit: YARN-5657. Fix TestDefaultContainerExecutor. (asuresh)

Posted by su...@apache.org.
YARN-5657. Fix TestDefaultContainerExecutor. (asuresh)


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

Branch: refs/heads/YARN-2915
Commit: f67237cbe7bc48a1b9088e990800b37529f1db2a
Parents: 4174b97
Author: Arun Suresh <as...@apache.org>
Authored: Sat Sep 17 09:29:03 2016 -0700
Committer: Arun Suresh <as...@apache.org>
Committed: Sat Sep 17 09:32:05 2016 -0700

----------------------------------------------------------------------
 .../server/nodemanager/TestDefaultContainerExecutor.java     | 8 +++++---
 1 file changed, 5 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f67237cb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestDefaultContainerExecutor.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestDefaultContainerExecutor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestDefaultContainerExecutor.java
index e2ec693..3bb8008 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestDefaultContainerExecutor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestDefaultContainerExecutor.java
@@ -22,6 +22,7 @@ import static org.apache.hadoop.fs.CreateFlag.CREATE;
 import static org.apache.hadoop.fs.CreateFlag.OVERWRITE;
 import static org.junit.Assert.assertTrue;
 import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyBoolean;
 import static org.mockito.Matchers.isA;
 import static org.mockito.Mockito.doAnswer;
 import static org.mockito.Mockito.mock;
@@ -386,8 +387,8 @@ public class TestDefaultContainerExecutor {
           }
         }
         return null;
-      }
-    }).when(mockUtil).copy(any(Path.class), any(Path.class));
+      }}).when(mockUtil).copy(any(Path.class), any(Path.class),
+        anyBoolean(), anyBoolean());
 
     doAnswer(new Answer() {
       @Override
@@ -478,7 +479,8 @@ public class TestDefaultContainerExecutor {
     }
 
     // Verify that the calls happen the expected number of times
-    verify(mockUtil, times(1)).copy(any(Path.class), any(Path.class));
+    verify(mockUtil, times(1)).copy(any(Path.class), any(Path.class),
+        anyBoolean(), anyBoolean());
     verify(mockLfs, times(2)).getFsStatus(any(Path.class));
   }
 


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


[14/44] hadoop git commit: MAPREDUCE-6777. Typos in 4 log messages. Contributed by Mehran Hassani

Posted by su...@apache.org.
MAPREDUCE-6777. Typos in 4 log messages. Contributed by Mehran Hassani


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

Branch: refs/heads/YARN-2915
Commit: 7d21c280a82b2f02675bf0048f0e965d99a05ae7
Parents: ade7c2b
Author: Naganarasimha <na...@apache.org>
Authored: Sat Sep 17 10:19:59 2016 +0530
Committer: Naganarasimha <na...@apache.org>
Committed: Sat Sep 17 10:19:59 2016 +0530

----------------------------------------------------------------------
 .../java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java     | 2 +-
 .../apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java | 2 +-
 .../src/main/java/org/apache/hadoop/mapred/Task.java               | 2 +-
 .../src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java     | 2 +-
 4 files changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7d21c280/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java
index 49a00c5..2c0ea2b 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java
@@ -260,7 +260,7 @@ public class TaskAttemptListenerImpl extends CompositeService
 
   @Override
   public void done(TaskAttemptID taskAttemptID) throws IOException {
-    LOG.info("Done acknowledgement from " + taskAttemptID.toString());
+    LOG.info("Done acknowledgment from " + taskAttemptID.toString());
 
     org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId attemptID =
         TypeConverter.toYarn(taskAttemptID);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7d21c280/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java
index 53d8a36..e61bbaa 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java
@@ -400,7 +400,7 @@ public class JobHistoryEventHandler extends AbstractService
         }
         mi.shutDownTimer();
       } catch (IOException e) {
-        LOG.info("Exception while cancelling delayed flush timer. "
+        LOG.info("Exception while canceling delayed flush timer. "
             + "Likely caused by a failed flush " + e.getMessage());
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7d21c280/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Task.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Task.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Task.java
index 3c58a67..d20e9bd 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Task.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Task.java
@@ -1312,7 +1312,7 @@ abstract public class Task implements Writable, Configurable {
     setPhase(TaskStatus.Phase.CLEANUP);
     getProgress().setStatus("cleanup");
     statusUpdate(umbilical);
-    LOG.info("Runnning cleanup for the task");
+    LOG.info("Running cleanup for the task");
     // do the cleanup
     committer.abortTask(taskContext);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7d21c280/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java
index 7818c81..558ee38 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java
@@ -672,7 +672,7 @@ public class ShuffleHandler extends AuxiliaryService {
       return;
     }
     if (loadedVersion.isCompatibleTo(getCurrentVersion())) {
-      LOG.info("Storing state DB schedma version info " + getCurrentVersion());
+      LOG.info("Storing state DB schema version info " + getCurrentVersion());
       storeVersion();
     } else {
       throw new IOException(


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


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

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


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


[21/44] hadoop git commit: YARN-5637. Changes in NodeManager to support Container rollback and commit. (asuresh)

Posted by su...@apache.org.
YARN-5637. Changes in NodeManager to support Container rollback and commit. (asuresh)


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

Branch: refs/heads/YARN-2915
Commit: 3552c2b99dff4f21489ff284f9dcba40e897a1e5
Parents: ea839bd
Author: Arun Suresh <as...@apache.org>
Authored: Fri Sep 16 16:53:18 2016 -0700
Committer: Arun Suresh <as...@apache.org>
Committed: Sun Sep 18 10:55:18 2016 -0700

----------------------------------------------------------------------
 .../containermanager/ContainerManagerImpl.java  |  68 ++++++-
 .../containermanager/container/Container.java   |   4 +
 .../container/ContainerEventType.java           |   1 +
 .../container/ContainerImpl.java                | 188 ++++++++++++++-----
 .../container/ContainerReInitEvent.java         |  20 +-
 .../TestContainerManagerWithLCE.java            |  42 ++++-
 .../containermanager/TestContainerManager.java  | 152 +++++++++++++--
 .../nodemanager/webapp/MockContainer.java       |  10 +
 8 files changed, 401 insertions(+), 84 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3552c2b9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.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/containermanager/ContainerManagerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
index f909ca5..8a9ad99 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
@@ -165,8 +165,8 @@ import static org.apache.hadoop.service.Service.STATE.STARTED;
 public class ContainerManagerImpl extends CompositeService implements
     ContainerManager {
 
-  private enum ReinitOp {
-    UPGRADE, COMMIT, ROLLBACK, LOCALIZE;
+  private enum ReInitOp {
+    RE_INIT, COMMIT, ROLLBACK, LOCALIZE;
   }
   /**
    * Extra duration to wait for applications to be killed on shutdown.
@@ -1535,7 +1535,7 @@ public class ContainerManagerImpl extends CompositeService implements
 
     ContainerId containerId = request.getContainerId();
     Container container = preUpgradeOrLocalizeCheck(containerId,
-        ReinitOp.LOCALIZE);
+        ReInitOp.LOCALIZE);
     try {
       Map<LocalResourceVisibility, Collection<LocalResourceRequest>> req =
           container.getResourceSet().addResources(request.getLocalResources());
@@ -1551,16 +1551,31 @@ public class ContainerManagerImpl extends CompositeService implements
     return ResourceLocalizationResponse.newInstance();
   }
 
-  public void upgradeContainer(ContainerId containerId,
-      ContainerLaunchContext upgradeLaunchContext) throws YarnException {
+  /**
+   * ReInitialize a container using a new Launch Context. If the
+   * retryFailureContext is not provided, The container is
+   * terminated on Failure.
+   *
+   * NOTE: Auto-Commit is true by default. This also means that the rollback
+   *       context is purged as soon as the command to start the new process
+   *       is sent. (The Container moves to RUNNING state)
+   *
+   * @param containerId Container Id.
+   * @param autoCommit Auto Commit flag.
+   * @param reInitLaunchContext Target Launch Context.
+   * @throws YarnException Yarn Exception.
+   */
+  public void reInitializeContainer(ContainerId containerId,
+      ContainerLaunchContext reInitLaunchContext, boolean autoCommit)
+      throws YarnException {
     Container container = preUpgradeOrLocalizeCheck(containerId,
-        ReinitOp.UPGRADE);
+        ReInitOp.RE_INIT);
     ResourceSet resourceSet = new ResourceSet();
     try {
-      resourceSet.addResources(upgradeLaunchContext.getLocalResources());
+      resourceSet.addResources(reInitLaunchContext.getLocalResources());
       dispatcher.getEventHandler().handle(
-          new ContainerReInitEvent(containerId, upgradeLaunchContext,
-              resourceSet));
+          new ContainerReInitEvent(containerId, reInitLaunchContext,
+              resourceSet, autoCommit));
       container.setIsReInitializing(true);
     } catch (URISyntaxException e) {
       LOG.info("Error when parsing local resource URI for upgrade of" +
@@ -1569,8 +1584,41 @@ public class ContainerManagerImpl extends CompositeService implements
     }
   }
 
+  /**
+   * Rollback the last reInitialization, if possible.
+   * @param containerId Container ID.
+   * @throws YarnException Yarn Exception.
+   */
+  public void rollbackReInitialization(ContainerId containerId)
+      throws YarnException {
+    Container container = preUpgradeOrLocalizeCheck(containerId,
+        ReInitOp.ROLLBACK);
+    if (container.canRollback()) {
+      dispatcher.getEventHandler().handle(
+          new ContainerEvent(containerId, ContainerEventType.ROLLBACK_REINIT));
+    } else {
+      throw new YarnException("Nothing to rollback to !!");
+    }
+  }
+
+  /**
+   * Commit last reInitialization after which no rollback will be possible.
+   * @param containerId Container ID.
+   * @throws YarnException Yarn Exception.
+   */
+  public void commitReInitialization(ContainerId containerId)
+      throws YarnException {
+    Container container = preUpgradeOrLocalizeCheck(containerId,
+        ReInitOp.COMMIT);
+    if (container.canRollback()) {
+      container.commitUpgrade();
+    } else {
+      throw new YarnException("Nothing to Commit !!");
+    }
+  }
+
   private Container preUpgradeOrLocalizeCheck(ContainerId containerId,
-      ReinitOp op) throws YarnException {
+      ReInitOp op) throws YarnException {
     Container container = context.getContainers().get(containerId);
     if (container == null) {
       throw new YarnException("Specified " + containerId + " does not exist!");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3552c2b9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/Container.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/containermanager/container/Container.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/Container.java
index f6c27ab..78c240a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/Container.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/Container.java
@@ -82,4 +82,8 @@ public interface Container extends EventHandler<ContainerEvent> {
   void setIsReInitializing(boolean isReInitializing);
 
   boolean isReInitializing();
+
+  boolean canRollback();
+
+  void commitUpgrade();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3552c2b9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerEventType.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/containermanager/container/ContainerEventType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerEventType.java
index 0b57505..afea0e6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerEventType.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerEventType.java
@@ -26,6 +26,7 @@ public enum ContainerEventType {
   UPDATE_DIAGNOSTICS_MSG,
   CONTAINER_DONE,
   REINITIALIZE_CONTAINER,
+  ROLLBACK_REINIT,
 
   // DownloadManager
   CONTAINER_INITED,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3552c2b9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.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/containermanager/container/ContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
index 12bbea9..0707df0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
@@ -91,14 +91,42 @@ import org.apache.hadoop.yarn.util.resource.Resources;
 
 public class ContainerImpl implements Container {
 
-  private final static class ReInitializationContext {
-    private final ResourceSet resourceSet;
+  private static final class ReInitializationContext {
     private final ContainerLaunchContext newLaunchContext;
+    private final ResourceSet newResourceSet;
+
+    // Rollback state
+    private final ContainerLaunchContext oldLaunchContext;
+    private final ResourceSet oldResourceSet;
 
     private ReInitializationContext(ContainerLaunchContext newLaunchContext,
-        ResourceSet resourceSet) {
+        ResourceSet newResourceSet,
+        ContainerLaunchContext oldLaunchContext,
+        ResourceSet oldResourceSet) {
       this.newLaunchContext = newLaunchContext;
-      this.resourceSet = resourceSet;
+      this.newResourceSet = newResourceSet;
+      this.oldLaunchContext = oldLaunchContext;
+      this.oldResourceSet = oldResourceSet;
+    }
+
+    private boolean canRollback() {
+      return (oldLaunchContext != null);
+    }
+
+    private ResourceSet mergedResourceSet() {
+      if (oldLaunchContext == null) {
+        return newResourceSet;
+      }
+      return ResourceSet.merge(oldResourceSet, newResourceSet);
+    }
+
+    private ReInitializationContext createContextForRollback() {
+      if (oldLaunchContext == null) {
+        return null;
+      } else {
+        return new ReInitializationContext(
+            oldLaunchContext, oldResourceSet, null, null);
+      }
     }
   }
 
@@ -129,7 +157,7 @@ public class ContainerImpl implements Container {
   private String logDir;
   private String host;
   private String ips;
-  private ReInitializationContext reInitContext;
+  private volatile ReInitializationContext reInitContext;
   private volatile boolean isReInitializing = false;
 
   /** The NM-wide configuration - not specific to this container */
@@ -187,8 +215,8 @@ public class ContainerImpl implements Container {
     }
 
     // Configure the Retry Context
-    this.containerRetryContext =
-        configureRetryContext(conf, launchContext, this.containerId);
+    this.containerRetryContext = configureRetryContext(
+        conf, launchContext, this.containerId);
     this.remainingRetryAttempts = this.containerRetryContext.getMaxRetries();
     stateMachine = stateMachineFactory.make(this);
     this.context = context;
@@ -320,12 +348,16 @@ public class ContainerImpl implements Container {
         new ExitedWithSuccessTransition(true))
     .addTransition(ContainerState.RUNNING,
         EnumSet.of(ContainerState.RELAUNCHING,
+            ContainerState.LOCALIZED,
             ContainerState.EXITED_WITH_FAILURE),
         ContainerEventType.CONTAINER_EXITED_WITH_FAILURE,
         new RetryFailureTransition())
     .addTransition(ContainerState.RUNNING, ContainerState.REINITIALIZING,
         ContainerEventType.REINITIALIZE_CONTAINER,
         new ReInitializeContainerTransition())
+    .addTransition(ContainerState.RUNNING, ContainerState.REINITIALIZING,
+        ContainerEventType.ROLLBACK_REINIT,
+        new RollbackContainerTransition())
     .addTransition(ContainerState.RUNNING, ContainerState.RUNNING,
         ContainerEventType.RESOURCE_LOCALIZED,
         new ResourceLocalizedWhileRunningTransition())
@@ -884,15 +916,15 @@ public class ContainerImpl implements Container {
     @SuppressWarnings("unchecked")
     @Override
     public void transition(ContainerImpl container, ContainerEvent event) {
-      container.reInitContext = createReInitContext(event);
+      container.reInitContext = createReInitContext(container, event);
       try {
         Map<LocalResourceVisibility, Collection<LocalResourceRequest>>
-            pendingResources =
-            container.reInitContext.resourceSet.getAllResourcesByVisibility();
-        if (!pendingResources.isEmpty()) {
+            resByVisibility = container.reInitContext.newResourceSet
+            .getAllResourcesByVisibility();
+        if (!resByVisibility.isEmpty()) {
           container.dispatcher.getEventHandler().handle(
               new ContainerLocalizationRequestEvent(
-                  container, pendingResources));
+                  container, resByVisibility));
         } else {
           // We are not waiting on any resources, so...
           // Kill the current container.
@@ -909,10 +941,30 @@ public class ContainerImpl implements Container {
     }
 
     protected ReInitializationContext createReInitContext(
-        ContainerEvent event) {
-      ContainerReInitEvent rEvent = (ContainerReInitEvent)event;
-      return new ReInitializationContext(rEvent.getReInitLaunchContext(),
-          rEvent.getResourceSet());
+        ContainerImpl container, ContainerEvent event) {
+      ContainerReInitEvent reInitEvent = (ContainerReInitEvent)event;
+      return new ReInitializationContext(
+          reInitEvent.getReInitLaunchContext(),
+          reInitEvent.getResourceSet(),
+          // If AutoCommit is turned on, then no rollback can happen...
+          // So don't need to store the previous context.
+          (reInitEvent.isAutoCommit() ? null : container.launchContext),
+          (reInitEvent.isAutoCommit() ? null : container.resourceSet));
+    }
+  }
+
+  /**
+   * Transition to start the Rollback process.
+   */
+  static class RollbackContainerTransition extends
+      ReInitializeContainerTransition {
+
+    @Override
+    protected ReInitializationContext createReInitContext(ContainerImpl
+        container, ContainerEvent event) {
+      LOG.warn("Container [" + container.getContainerId() + "]" +
+          " about to be explicitly Rolledback !!");
+      return container.reInitContext.createContextForRollback();
     }
   }
 
@@ -928,10 +980,10 @@ public class ContainerImpl implements Container {
     public void transition(ContainerImpl container, ContainerEvent event) {
       ContainerResourceLocalizedEvent rsrcEvent =
           (ContainerResourceLocalizedEvent) event;
-      container.reInitContext.resourceSet.resourceLocalized(
+      container.reInitContext.newResourceSet.resourceLocalized(
           rsrcEvent.getResource(), rsrcEvent.getLocation());
       // Check if all ResourceLocalization has completed
-      if (container.reInitContext.resourceSet.getPendingResources()
+      if (container.reInitContext.newResourceSet.getPendingResources()
           .isEmpty()) {
         // Kill the current container.
         container.dispatcher.getEventHandler().handle(
@@ -1028,10 +1080,13 @@ public class ContainerImpl implements Container {
       container.metrics.runningContainer();
       container.wasLaunched  = true;
 
-      if (container.reInitContext != null) {
+      container.setIsReInitializing(false);
+      // Check if this launch was due to a re-initialization.
+      // If autocommit == true, then wipe the re-init context. This ensures
+      // that any subsequent failures do not trigger a rollback.
+      if (container.reInitContext != null
+          && !container.reInitContext.canRollback()) {
         container.reInitContext = null;
-        // Set rollback context here..
-        container.setIsReInitializing(false);
       }
 
       if (container.recoveredAsKilled) {
@@ -1148,36 +1203,50 @@ public class ContainerImpl implements Container {
                     + container.getContainerId(), e);
           }
         }
-        LOG.info("Relaunching Container " + container.getContainerId()
-            + ". Remaining retry attempts(after relaunch) : "
-            + container.remainingRetryAttempts
-            + ". Interval between retries is "
-            + container.containerRetryContext.getRetryInterval() + "ms");
-        container.wasLaunched  = false;
-        container.metrics.endRunningContainer();
-        if (container.containerRetryContext.getRetryInterval() == 0) {
-          container.sendRelaunchEvent();
-        } else {
-          // wait for some time, then send launch event
-          new Thread() {
-            @Override
-            public void run() {
-              try {
-                Thread.sleep(
-                    container.containerRetryContext.getRetryInterval());
-                container.sendRelaunchEvent();
-              } catch (InterruptedException e) {
-                return;
-              }
-            }
-          }.start();
-        }
+        doRelaunch(container, container.remainingRetryAttempts,
+            container.containerRetryContext.getRetryInterval());
         return ContainerState.RELAUNCHING;
+      } else if (container.canRollback()) {
+        // Rollback is possible only if the previous launch context is
+        // available.
+        container.addDiagnostics("Container Re-init Auto Rolled-Back.");
+        LOG.info("Rolling back Container reInitialization for [" +
+            container.getContainerId() + "] !!");
+        container.reInitContext =
+            container.reInitContext.createContextForRollback();
+        new KilledForReInitializationTransition().transition(container, event);
+        return ContainerState.LOCALIZED;
       } else {
         new ExitedWithFailureTransition(true).transition(container, event);
         return ContainerState.EXITED_WITH_FAILURE;
       }
     }
+
+    private void doRelaunch(final ContainerImpl container,
+        int remainingRetryAttempts, final int retryInterval) {
+      LOG.info("Relaunching Container " + container.getContainerId()
+          + ". Remaining retry attempts(after relaunch) : "
+          + remainingRetryAttempts + ". Interval between retries is "
+          + retryInterval + "ms");
+      container.wasLaunched  = false;
+      container.metrics.endRunningContainer();
+      if (retryInterval == 0) {
+        container.sendRelaunchEvent();
+      } else {
+        // wait for some time, then send launch event
+        new Thread() {
+          @Override
+          public void run() {
+            try {
+              Thread.sleep(retryInterval);
+              container.sendRelaunchEvent();
+            } catch (InterruptedException e) {
+              return;
+            }
+          }
+        }.start();
+      }
+    }
   }
 
   @Override
@@ -1188,24 +1257,29 @@ public class ContainerImpl implements Container {
 
   @Override
   public boolean shouldRetry(int errorCode) {
+    return shouldRetry(errorCode, containerRetryContext,
+        remainingRetryAttempts);
+  }
+
+  public static boolean shouldRetry(int errorCode,
+      ContainerRetryContext retryContext, int remainingRetryAttempts) {
     if (errorCode == ExitCode.SUCCESS.getExitCode()
         || errorCode == ExitCode.FORCE_KILLED.getExitCode()
         || errorCode == ExitCode.TERMINATED.getExitCode()) {
       return false;
     }
 
-    ContainerRetryPolicy retryPolicy = containerRetryContext.getRetryPolicy();
+    ContainerRetryPolicy retryPolicy = retryContext.getRetryPolicy();
     if (retryPolicy == ContainerRetryPolicy.RETRY_ON_ALL_ERRORS
         || (retryPolicy == ContainerRetryPolicy.RETRY_ON_SPECIFIC_ERROR_CODES
-            && containerRetryContext.getErrorCodes() != null
-            && containerRetryContext.getErrorCodes().contains(errorCode))) {
+        && retryContext.getErrorCodes() != null
+        && retryContext.getErrorCodes().contains(errorCode))) {
       return remainingRetryAttempts > 0
           || remainingRetryAttempts == ContainerRetryContext.RETRY_FOREVER;
     }
 
     return false;
   }
-
   /**
    * Transition to EXITED_WITH_FAILURE
    */
@@ -1240,13 +1314,12 @@ public class ContainerImpl implements Container {
       // Re configure the Retry Context
       container.containerRetryContext =
           configureRetryContext(container.context.getConf(),
-          container.launchContext, container.containerId);
+              container.launchContext, container.containerId);
       // Reset the retry attempts since its a fresh start
       container.remainingRetryAttempts =
           container.containerRetryContext.getMaxRetries();
 
-      container.resourceSet = ResourceSet.merge(
-          container.resourceSet, container.reInitContext.resourceSet);
+      container.resourceSet = container.reInitContext.mergedResourceSet();
 
       container.sendLaunchEvent();
     }
@@ -1589,4 +1662,15 @@ public class ContainerImpl implements Container {
   public boolean isReInitializing() {
     return this.isReInitializing;
   }
+
+  @Override
+  public boolean canRollback() {
+    return (this.reInitContext != null)
+        && (this.reInitContext.canRollback());
+  }
+
+  @Override
+  public void commitUpgrade() {
+    this.reInitContext = null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3552c2b9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerReInitEvent.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/containermanager/container/ContainerReInitEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerReInitEvent.java
index 2ccdbd7..46eba03 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerReInitEvent.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerReInitEvent.java
@@ -30,18 +30,22 @@ public class ContainerReInitEvent extends ContainerEvent {
 
   private final ContainerLaunchContext reInitLaunchContext;
   private final ResourceSet resourceSet;
+  private final boolean autoCommit;
 
   /**
    * Container Re-Init Event.
-   * @param cID Container Id
-   * @param upgradeContext Upgrade context
-   * @param resourceSet Resource Set
+   * @param cID Container Id.
+   * @param upgradeContext Upgrade Context.
+   * @param resourceSet Resource Set.
+   * @param autoCommit Auto Commit.
    */
   public ContainerReInitEvent(ContainerId cID,
-      ContainerLaunchContext upgradeContext, ResourceSet resourceSet){
+      ContainerLaunchContext upgradeContext,
+      ResourceSet resourceSet, boolean autoCommit){
     super(cID, ContainerEventType.REINITIALIZE_CONTAINER);
     this.reInitLaunchContext = upgradeContext;
     this.resourceSet = resourceSet;
+    this.autoCommit = autoCommit;
   }
 
   /**
@@ -59,4 +63,12 @@ public class ContainerReInitEvent extends ContainerEvent {
   public ResourceSet getResourceSet() {
     return resourceSet;
   }
+
+  /**
+   * Should this re-Initialization be auto-committed.
+   * @return AutoCommit.
+   */
+  public boolean isAutoCommit() {
+    return autoCommit;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3552c2b9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestContainerManagerWithLCE.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestContainerManagerWithLCE.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestContainerManagerWithLCE.java
index 8a27849..79182ce 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestContainerManagerWithLCE.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestContainerManagerWithLCE.java
@@ -270,15 +270,15 @@ public class TestContainerManagerWithLCE extends TestContainerManager {
   }
 
   @Override
-  public void testContainerUpgradeSuccess() throws IOException,
+  public void testContainerUpgradeSuccessAutoCommit() throws IOException,
       InterruptedException, YarnException {
     // Don't run the test if the binary is not available.
     if (!shouldRunTest()) {
       LOG.info("LCE binary path is not passed. Not running the test");
       return;
     }
-    LOG.info("Running testContainerUpgradeSuccess");
-    super.testContainerUpgradeSuccess();
+    LOG.info("Running testContainerUpgradeSuccessAutoCommit");
+    super.testContainerUpgradeSuccessAutoCommit();
   }
 
   @Override
@@ -294,6 +294,42 @@ public class TestContainerManagerWithLCE extends TestContainerManager {
   }
 
   @Override
+  public void testContainerUpgradeSuccessExplicitCommit() throws IOException,
+    InterruptedException, YarnException {
+    // Don't run the test if the binary is not available.
+    if (!shouldRunTest()) {
+      LOG.info("LCE binary path is not passed. Not running the test");
+      return;
+    }
+    LOG.info("Running testContainerUpgradeSuccessExplicitCommit");
+    super.testContainerUpgradeSuccessExplicitCommit();
+  }
+
+  @Override
+  public void testContainerUpgradeSuccessExplicitRollback() throws IOException,
+      InterruptedException, YarnException {
+    // Don't run the test if the binary is not available.
+    if (!shouldRunTest()) {
+      LOG.info("LCE binary path is not passed. Not running the test");
+      return;
+    }
+    LOG.info("Running testContainerUpgradeSuccessExplicitRollback");
+    super.testContainerUpgradeSuccessExplicitRollback();
+  }
+
+  @Override
+  public void testContainerUpgradeRollbackDueToFailure() throws IOException,
+      InterruptedException, YarnException {
+    // Don't run the test if the binary is not available.
+    if (!shouldRunTest()) {
+      LOG.info("LCE binary path is not passed. Not running the test");
+      return;
+    }
+    LOG.info("Running testContainerUpgradeRollbackDueToFailure");
+    super.testContainerUpgradeRollbackDueToFailure();
+  }
+
+  @Override
   public void testContainerUpgradeProcessFailure() throws IOException,
       InterruptedException, YarnException {
     // Don't run the test if the binary is not available.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3552c2b9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
index 843dc2a..72049e6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
@@ -369,9 +369,8 @@ public class TestContainerManager extends BaseContainerManagerTest {
       DefaultContainerExecutor.containerIsAlive(pid));
   }
 
-  @Test
-  public void testContainerUpgradeSuccess() throws IOException,
-      InterruptedException, YarnException {
+  private String[] testContainerUpgradeSuccess(boolean autoCommit)
+      throws IOException, InterruptedException, YarnException {
     containerManager.start();
     // ////// Construct the Container-id
     ContainerId cId = createContainerId(0);
@@ -381,7 +380,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
 
     File newStartFile = new File(tmpDir, "start_file_n.txt").getAbsoluteFile();
 
-    prepareContainerUpgrade(false, false, cId, newStartFile);
+    prepareContainerUpgrade(autoCommit, false, false, cId, newStartFile);
 
     // Assert that the First process is not alive anymore
     Assert.assertFalse("Process is still alive!",
@@ -407,6 +406,80 @@ public class TestContainerManager extends BaseContainerManagerTest {
     // Assert that the New process is alive
     Assert.assertTrue("New Process is not alive!",
         DefaultContainerExecutor.containerIsAlive(newPid));
+    return new String[]{pid, newPid};
+  }
+
+  @Test
+  public void testContainerUpgradeSuccessAutoCommit() throws IOException,
+      InterruptedException, YarnException {
+    testContainerUpgradeSuccess(true);
+    // Should not be able to Commit (since already auto committed)
+    try {
+      containerManager.commitReInitialization(createContainerId(0));
+      Assert.fail();
+    } catch (Exception e) {
+      Assert.assertTrue(e.getMessage().contains("Nothing to Commit"));
+    }
+  }
+
+  @Test
+  public void testContainerUpgradeSuccessExplicitCommit() throws IOException,
+      InterruptedException, YarnException {
+    testContainerUpgradeSuccess(false);
+    ContainerId cId = createContainerId(0);
+    containerManager.commitReInitialization(cId);
+    // Should not be able to Rollback once committed
+    try {
+      containerManager.rollbackReInitialization(cId);
+      Assert.fail();
+    } catch (Exception e) {
+      Assert.assertTrue(e.getMessage().contains("Nothing to rollback to"));
+    }
+  }
+
+  @Test
+  public void testContainerUpgradeSuccessExplicitRollback() throws IOException,
+      InterruptedException, YarnException {
+    String[] pids = testContainerUpgradeSuccess(false);
+
+    // Delete the old start File..
+    File oldStartFile = new File(tmpDir, "start_file_o.txt").getAbsoluteFile();
+    oldStartFile.delete();
+
+    ContainerId cId = createContainerId(0);
+    // Explicit Rollback
+    containerManager.rollbackReInitialization(cId);
+
+    // Original should be dead anyway
+    Assert.assertFalse("Original Process is still alive!",
+        DefaultContainerExecutor.containerIsAlive(pids[0]));
+
+    // Wait for upgraded process to die
+    int timeoutSecs = 0;
+    while (!DefaultContainerExecutor.containerIsAlive(pids[1])
+        && timeoutSecs++ < 20) {
+      Thread.sleep(1000);
+      LOG.info("Waiting for Upgraded process to die..");
+    }
+
+    timeoutSecs = 0;
+    // Wait for new processStartfile to be created
+    while (!oldStartFile.exists() && timeoutSecs++ < 20) {
+      Thread.sleep(1000);
+      LOG.info("Waiting for New process start-file to be created");
+    }
+
+    // Now verify the contents of the file
+    BufferedReader reader =
+        new BufferedReader(new FileReader(oldStartFile));
+    Assert.assertEquals("Hello World!", reader.readLine());
+    // Get the pid of the process
+    String rolledBackPid = reader.readLine().trim();
+    // No more lines
+    Assert.assertEquals(null, reader.readLine());
+
+    Assert.assertNotEquals("The Rolled-back process should be a different pid",
+        pids[0], rolledBackPid);
   }
 
   @Test
@@ -424,7 +497,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
 
     File newStartFile = new File(tmpDir, "start_file_n.txt").getAbsoluteFile();
 
-    prepareContainerUpgrade(true, true, cId, newStartFile);
+    prepareContainerUpgrade(false, true, true, cId, newStartFile);
 
     // Assert that the First process is STILL alive
     // since upgrade was terminated..
@@ -447,22 +520,69 @@ public class TestContainerManager extends BaseContainerManagerTest {
 
     File newStartFile = new File(tmpDir, "start_file_n.txt").getAbsoluteFile();
 
-    prepareContainerUpgrade(true, false, cId, newStartFile);
+    // Since Autocommit is true, there is also no rollback context...
+    // which implies that if the new process fails, since there is no
+    // rollback, it is terminated.
+    prepareContainerUpgrade(true, true, false, cId, newStartFile);
 
     // Assert that the First process is not alive anymore
     Assert.assertFalse("Process is still alive!",
         DefaultContainerExecutor.containerIsAlive(pid));
   }
 
+  @Test
+  public void testContainerUpgradeRollbackDueToFailure() throws IOException,
+      InterruptedException, YarnException {
+    if (Shell.WINDOWS) {
+      return;
+    }
+    containerManager.start();
+    // ////// Construct the Container-id
+    ContainerId cId = createContainerId(0);
+    File oldStartFile = new File(tmpDir, "start_file_o.txt").getAbsoluteFile();
+
+    String pid = prepareInitialContainer(cId, oldStartFile);
+
+    File newStartFile = new File(tmpDir, "start_file_n.txt").getAbsoluteFile();
+
+    prepareContainerUpgrade(false, true, false, cId, newStartFile);
+
+    // Assert that the First process is not alive anymore
+    Assert.assertFalse("Original Process is still alive!",
+        DefaultContainerExecutor.containerIsAlive(pid));
+
+    int timeoutSecs = 0;
+    // Wait for oldStartFile to be created
+    while (!oldStartFile.exists() && timeoutSecs++ < 20) {
+      System.out.println("\nFiles: " +
+          Arrays.toString(oldStartFile.getParentFile().list()));
+      Thread.sleep(1000);
+      LOG.info("Waiting for New process start-file to be created");
+    }
+
+    // Now verify the contents of the file
+    BufferedReader reader =
+        new BufferedReader(new FileReader(oldStartFile));
+    Assert.assertEquals("Hello World!", reader.readLine());
+    // Get the pid of the process
+    String rolledBackPid = reader.readLine().trim();
+    // No more lines
+    Assert.assertEquals(null, reader.readLine());
+
+    Assert.assertNotEquals("The Rolled-back process should be a different pid",
+        pid, rolledBackPid);
+  }
+
   /**
    * Prepare a launch Context for container upgrade and request the
    * Container Manager to re-initialize a running container using the
    * new launch context.
+   * @param autoCommit Enable autoCommit.
    * @param failCmd injects a start script that intentionally fails.
    * @param failLoc injects a bad file Location that will fail localization.
    */
-  private void prepareContainerUpgrade(boolean failCmd, boolean failLoc,
-      ContainerId cId, File startFile)
+  private void prepareContainerUpgrade(boolean autoCommit, boolean failCmd,
+      boolean failLoc, ContainerId cId, File startFile)
       throws FileNotFoundException, YarnException, InterruptedException {
     // Re-write scriptfile and processStartFile
     File scriptFile = Shell.appendScriptExtension(tmpDir, "scriptFile_new");
@@ -471,13 +591,15 @@ public class TestContainerManager extends BaseContainerManagerTest {
     writeScriptFile(fileWriter, "Upgrade World!", startFile, cId, failCmd);
 
     ContainerLaunchContext containerLaunchContext =
-        prepareContainerLaunchContext(scriptFile, "dest_file_new", failLoc);
+        prepareContainerLaunchContext(scriptFile, "dest_file_new", failLoc, 0);
 
-    containerManager.upgradeContainer(cId, containerLaunchContext);
+    containerManager.reInitializeContainer(cId, containerLaunchContext,
+        autoCommit);
     try {
-      containerManager.upgradeContainer(cId, containerLaunchContext);
+      containerManager.reInitializeContainer(cId, containerLaunchContext,
+          autoCommit);
     } catch (Exception e) {
-      Assert.assertTrue(e.getMessage().contains("Cannot perform UPGRADE"));
+      Assert.assertTrue(e.getMessage().contains("Cannot perform RE_INIT"));
     }
     int timeoutSecs = 0;
     int maxTimeToWait = failLoc ? 10 : 20;
@@ -501,7 +623,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
     writeScriptFile(fileWriterOld, "Hello World!", startFile, cId, false);
 
     ContainerLaunchContext containerLaunchContext =
-        prepareContainerLaunchContext(scriptFileOld, "dest_file", false);
+        prepareContainerLaunchContext(scriptFileOld, "dest_file", false, 4);
 
     StartContainerRequest scRequest =
         StartContainerRequest.newInstance(containerLaunchContext,
@@ -562,7 +684,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
   }
 
   private ContainerLaunchContext prepareContainerLaunchContext(File scriptFile,
-      String destFName, boolean putBadFile) {
+      String destFName, boolean putBadFile, int numRetries) {
     ContainerLaunchContext containerLaunchContext =
         recordFactory.newRecordInstance(ContainerLaunchContext.class);
     URL resourceAlpha = null;
@@ -592,7 +714,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
     ContainerRetryContext containerRetryContext = ContainerRetryContext
         .newInstance(
             ContainerRetryPolicy.RETRY_ON_SPECIFIC_ERROR_CODES,
-            new HashSet<>(Arrays.asList(Integer.valueOf(111))), 4, 0);
+            new HashSet<>(Arrays.asList(Integer.valueOf(111))), numRetries, 0);
     containerLaunchContext.setContainerRetryContext(containerRetryContext);
     List<String> commands = Arrays.asList(
         Shell.getRunScriptCommand(scriptFile));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3552c2b9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockContainer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockContainer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockContainer.java
index 8c8bec7..164488d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockContainer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockContainer.java
@@ -205,4 +205,14 @@ public class MockContainer implements Container {
   public boolean isReInitializing() {
     return false;
   }
+
+  @Override
+  public boolean canRollback() {
+    return false;
+  }
+
+  @Override
+  public void commitUpgrade() {
+
+  }
 }


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


[19/44] hadoop git commit: HADOOP-13621. s3:// should have been fully cut off from trunk. Contributed by Mingliang Liu.

Posted by su...@apache.org.
HADOOP-13621. s3:// should have been fully cut off from trunk. Contributed by Mingliang Liu.


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

Branch: refs/heads/YARN-2915
Commit: 96142efa2dded3212d07e7e14e251066865fb7d2
Parents: f67237c
Author: Mingliang Liu <li...@apache.org>
Authored: Fri Sep 16 18:36:26 2016 -0700
Committer: Mingliang Liu <li...@apache.org>
Committed: Sat Sep 17 22:07:46 2016 -0700

----------------------------------------------------------------------
 .../src/site/markdown/tools/hadoop-aws/index.md | 44 +++-----------------
 1 file changed, 6 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/96142efa/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
index 7fcadb9..160aa46 100644
--- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
+++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
@@ -28,7 +28,7 @@ HADOOP_OPTIONAL_TOOLS in hadoop-env.sh has 'hadoop-aws' in the list.
 
 ### Features
 
-**NOTE: `s3:` is being phased out. Use `s3n:` or `s3a:` instead.**
+**NOTE: `s3:` has been phased out. Use `s3n:` or `s3a:` instead.**
 
 1. The second-generation, `s3n:` filesystem, making it easy to share
 data between hadoop and other applications via the S3 object store.
@@ -86,38 +86,6 @@ these instructions \u2014and be aware that all issues related to S3 integration
 in EMR can only be addressed by Amazon themselves: please raise your issues
 with them.
 
-## S3
-
-The `s3://` filesystem is the original S3 store in the Hadoop codebase.
-It implements an inode-style filesystem atop S3, and was written to
-provide scaleability when S3 had significant limits on the size of blobs.
-It is incompatible with any other application's use of data in S3.
-
-It is now deprecated and will be removed in Hadoop 3. Please do not use,
-and migrate off data which is on it.
-
-### Dependencies
-
-* `jets3t` jar
-* `commons-codec` jar
-* `commons-logging` jar
-* `httpclient` jar
-* `httpcore` jar
-* `java-xmlbuilder` jar
-
-### Authentication properties
-
-    <property>
-      <name>fs.s3.awsAccessKeyId</name>
-      <description>AWS access key ID</description>
-    </property>
-
-    <property>
-      <name>fs.s3.awsSecretAccessKey</name>
-      <description>AWS secret key</description>
-    </property>
-
-
 ## S3N
 
 S3N was the first S3 Filesystem client which used "native" S3 objects, hence
@@ -171,16 +139,16 @@ it should be used wherever possible.
 ### Other properties
 
     <property>
-      <name>fs.s3.buffer.dir</name>
+      <name>fs.s3n.buffer.dir</name>
       <value>${hadoop.tmp.dir}/s3</value>
-      <description>Determines where on the local filesystem the s3:/s3n: filesystem
+      <description>Determines where on the local filesystem the s3n: filesystem
       should store files before sending them to S3
       (or after retrieving them from S3).
       </description>
     </property>
 
     <property>
-      <name>fs.s3.maxRetries</name>
+      <name>fs.s3n.maxRetries</name>
       <value>4</value>
       <description>The maximum number of retries for reading or writing files to
         S3, before we signal failure to the application.
@@ -188,7 +156,7 @@ it should be used wherever possible.
     </property>
 
     <property>
-      <name>fs.s3.sleepTimeSeconds</name>
+      <name>fs.s3n.sleepTimeSeconds</name>
       <value>10</value>
       <description>The number of seconds to sleep between each S3 retry.
       </description>
@@ -1011,7 +979,7 @@ includes `distcp`.
 
 ### `ClassNotFoundException: org.apache.hadoop.fs.s3a.S3AFileSystem`
 
-(or `org.apache.hadoop.fs.s3native.NativeS3FileSystem`, `org.apache.hadoop.fs.s3.S3FileSystem`).
+(or `org.apache.hadoop.fs.s3native.NativeS3FileSystem`).
 
 These are the Hadoop classes, found in the `hadoop-aws` JAR. An exception
 reporting one of these classes is missing means that this JAR is not on


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


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

Branch: refs/heads/YARN-2915
Commit: ea8d4d501228b2a4ea46cbee8493dab6bed206e5
Parents: c54f6ef
Author: Subru Krishnan <su...@apache.org>
Authored: Tue Jul 19 15:08:25 2016 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon Sep 19 12:58:34 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/ea8d4d50/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 a5c0f71..7dabc8e 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/ea8d4d50/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 3353e33..b276801 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


[20/44] hadoop git commit: HDFS-10489. Deprecate dfs.encryption.key.provider.uri for HDFS encryption zones. Contributed by Xiao Chen.

Posted by su...@apache.org.
HDFS-10489. Deprecate dfs.encryption.key.provider.uri for HDFS encryption zones. Contributed by Xiao Chen.


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

Branch: refs/heads/YARN-2915
Commit: ea839bd48e4478fc7b6d0a69e0eaeae2de5e0f0d
Parents: 96142ef
Author: Xiao Chen <xi...@apache.org>
Authored: Sat Sep 17 22:25:39 2016 -0700
Committer: Xiao Chen <xi...@apache.org>
Committed: Sat Sep 17 22:25:39 2016 -0700

----------------------------------------------------------------------
 .../hadoop/crypto/key/KeyProviderFactory.java      |  3 ++-
 .../hadoop/fs/CommonConfigurationKeysPublic.java   |  8 ++++++++
 .../src/main/resources/core-default.xml            |  8 ++++++++
 .../src/site/markdown/DeprecatedProperties.md      |  1 +
 .../hadoop-kms/src/site/markdown/index.md.vm       | 10 +++++-----
 .../java/org/apache/hadoop/hdfs/DFSUtilClient.java | 13 +++++++------
 .../org/apache/hadoop/hdfs/HdfsConfiguration.java  |  3 +++
 .../org/apache/hadoop/hdfs/KeyProviderCache.java   |  6 +++---
 .../hadoop/hdfs/client/HdfsClientConfigKeys.java   |  1 -
 .../org/apache/hadoop/test/TestHdfsHelper.java     |  4 +++-
 .../hadoop/hdfs/nfs/nfs3/TestRpcProgramNfs3.java   |  4 ++--
 .../java/org/apache/hadoop/hdfs/DFSConfigKeys.java |  2 --
 .../src/main/resources/hdfs-default.xml            |  8 --------
 .../src/site/markdown/TransparentEncryption.md     |  2 +-
 .../org/apache/hadoop/cli/TestCryptoAdminCLI.java  |  4 ++--
 .../org/apache/hadoop/hdfs/TestAclsEndToEnd.java   |  3 ++-
 .../java/org/apache/hadoop/hdfs/TestDFSUtil.java   | 12 ++++++++----
 .../apache/hadoop/hdfs/TestEncryptionZones.java    | 17 ++++++++++-------
 .../hadoop/hdfs/TestEncryptionZonesWithHA.java     |  3 ++-
 .../apache/hadoop/hdfs/TestKeyProviderCache.java   | 10 +++++-----
 .../apache/hadoop/hdfs/TestReservedRawPaths.java   |  3 ++-
 .../hdfs/TestSecureEncryptionZoneWithKMS.java      |  6 ++++--
 .../server/namenode/TestNestedEncryptionZones.java |  4 +++-
 .../namenode/metrics/TestNameNodeMetrics.java      |  3 ++-
 24 files changed, 83 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea839bd4/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderFactory.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderFactory.java
index ce99d79..b16960c 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderFactory.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderFactory.java
@@ -29,6 +29,7 @@ import java.util.ServiceLoader;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 
 /**
  * A factory to create a list of KeyProvider based on the path given in a
@@ -39,7 +40,7 @@ import org.apache.hadoop.conf.Configuration;
 @InterfaceStability.Unstable
 public abstract class KeyProviderFactory {
   public static final String KEY_PROVIDER_PATH =
-      "hadoop.security.key.provider.path";
+      CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH;
 
   public abstract KeyProvider createProvider(URI providerName,
                                              Configuration conf

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea839bd4/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
index 0a3afb7..b5b107c 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
@@ -628,6 +628,14 @@ public class CommonConfigurationKeysPublic {
   public static final String  HADOOP_SECURITY_IMPERSONATION_PROVIDER_CLASS =
     "hadoop.security.impersonation.provider.class";
 
+  /**
+   * @see
+   * <a href="{@docRoot}/../hadoop-project-dist/hadoop-common/core-default.xml">
+   * core-default.xml</a>
+   */
+  public static final String HADOOP_SECURITY_KEY_PROVIDER_PATH =
+      "hadoop.security.key.provider.path";
+
   //  <!-- KMSClientProvider configurations -->
   /**
    * @see

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea839bd4/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
index fda8c9b..5b8d49d 100644
--- a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
+++ b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
@@ -2038,6 +2038,14 @@
 </property>
 
 <property>
+  <name>hadoop.security.key.provider.path</name>
+  <description>
+    The KeyProvider to use when managing zone keys, and interacting with
+    encryption keys when reading and writing to an encryption zone.
+  </description>
+</property>
+
+<property>
   <name>fs.har.impl.disable.cache</name>
   <value>true</value>
   <description>Don't cache 'har' filesystem instances.</description>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea839bd4/hadoop-common-project/hadoop-common/src/site/markdown/DeprecatedProperties.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/DeprecatedProperties.md b/hadoop-common-project/hadoop-common/src/site/markdown/DeprecatedProperties.md
index adebe6e..94ed68e 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/DeprecatedProperties.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/DeprecatedProperties.md
@@ -28,6 +28,7 @@ The following table lists the configuration property names that are deprecated i
 | dfs.data.dir | dfs.datanode.data.dir |
 | dfs.datanode.max.xcievers | dfs.datanode.max.transfer.threads |
 | dfs.df.interval | fs.df.interval |
+| dfs.encryption.key.provider.uri | hadoop.security.key.provider.path |
 | dfs.federation.nameservice.id | dfs.nameservice.id |
 | dfs.federation.nameservices | dfs.nameservices |
 | dfs.http.address | dfs.namenode.http-address |

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea839bd4/hadoop-common-project/hadoop-kms/src/site/markdown/index.md.vm
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-kms/src/site/markdown/index.md.vm b/hadoop-common-project/hadoop-kms/src/site/markdown/index.md.vm
index 889dbaf1..0c6d0b2 100644
--- a/hadoop-common-project/hadoop-kms/src/site/markdown/index.md.vm
+++ b/hadoop-common-project/hadoop-kms/src/site/markdown/index.md.vm
@@ -37,10 +37,10 @@ KMS Client Configuration
 The KMS client `KeyProvider` uses the **kms** scheme, and the embedded URL must be the URL of the KMS. For example, for a KMS running on `http://localhost:9600/kms`, the KeyProvider URI is `kms://http@localhost:9600/kms`. And, for a KMS running on `https://localhost:9600/kms`, the KeyProvider URI is `kms://https@localhost:9600/kms`
 
 The following is an example to configure HDFS NameNode as a KMS client in
-`hdfs-site.xml`:
+`core-site.xml`:
 
     <property>
-      <name>dfs.encryption.key.provider.uri</name>
+      <name>hadoop.security.key.provider.path</name>
       <value>kms://http@localhost:9600/kms</value>
       <description>
         The KeyProvider to use when interacting with encryption keys used
@@ -664,15 +664,15 @@ is to use LoadBalancingKMSClientProvider. Using this approach, a KMS client
 (for example, a HDFS NameNode) is aware of multiple KMS instances, and it sends
 requests to them in a round-robin fashion. LoadBalancingKMSClientProvider is
 implicitly used when more than one URI is specified in
-`dfs.encryption.key.provider.uri`.
+`hadoop.security.key.provider.path`.
 
-The following example in `hdfs-site.xml` configures two KMS
+The following example in `core-site.xml` configures two KMS
 instances, `kms01.example.com` and `kms02.example.com`.
 The hostnames are separated by semi-colons, and all KMS instances must run
 on the same port.
 
     <property>
-      <name>dfs.encryption.key.provider.uri</name>
+      <name>hadoop.security.key.provider.path</name>
       <value>kms://https@kms01.example.com;kms02.example.com:9600/kms</value>
       <description>
         The KeyProvider to use when interacting with encryption keys used

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea839bd4/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
index c511596..b93632a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
@@ -526,7 +526,7 @@ public class DFSUtilClient {
   }
 
   private static String keyProviderUriKeyName =
-      HdfsClientConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI;
+      CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH;
 
   /**
    * Set the key provider uri configuration key name for creating key providers.
@@ -616,16 +616,17 @@ public class DFSUtilClient {
   }
 
   /**
-   * Probe for HDFS Encryption being enabled; this uses the value of
-   * the option {@link HdfsClientConfigKeys#DFS_ENCRYPTION_KEY_PROVIDER_URI},
-   * returning true if that property contains a non-empty, non-whitespace
+   * Probe for HDFS Encryption being enabled; this uses the value of the option
+   * {@link CommonConfigurationKeysPublic#HADOOP_SECURITY_KEY_PROVIDER_PATH}
+   * , returning true if that property contains a non-empty, non-whitespace
    * string.
    * @param conf configuration to probe
    * @return true if encryption is considered enabled.
    */
   public static boolean isHDFSEncryptionEnabled(Configuration conf) {
-    return !conf.getTrimmed(
-        HdfsClientConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI, "").isEmpty();
+    return !(conf.getTrimmed(
+        CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH, "")
+        .isEmpty());
   }
 
   public static InetSocketAddress getNNAddress(String address) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea839bd4/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/HdfsConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/HdfsConfiguration.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/HdfsConfiguration.java
index 580e7f9..3f6eae0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/HdfsConfiguration.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/HdfsConfiguration.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DeprecatedKeys;
@@ -141,6 +142,8 @@ public class HdfsConfiguration extends Configuration {
             HdfsClientConfigKeys.DFS_NAMESERVICES),
         new DeprecationDelta("dfs.federation.nameservice.id",
             DeprecatedKeys.DFS_NAMESERVICE_ID),
+        new DeprecationDelta("dfs.encryption.key.provider.uri",
+            CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH),
     });
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea839bd4/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/KeyProviderCache.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/KeyProviderCache.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/KeyProviderCache.java
index 0b0e006..3491010 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/KeyProviderCache.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/KeyProviderCache.java
@@ -25,7 +25,7 @@ import java.util.concurrent.TimeUnit;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.crypto.key.KeyProvider;
-import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.cache.Cache;
@@ -86,11 +86,11 @@ public class KeyProviderCache {
 
   private URI createKeyProviderURI(Configuration conf) {
     final String providerUriStr = conf.getTrimmed(
-        HdfsClientConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI, "");
+        CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH, "");
     // No provider set in conf
     if (providerUriStr.isEmpty()) {
       LOG.error("Could not find uri with key ["
-          + HdfsClientConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI
+          + CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH
           + "] to create a keyProvider !!");
       return null;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea839bd4/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
index 3ea42c3..642d4c8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
@@ -139,7 +139,6 @@ public interface HdfsClientConfigKeys {
       "dfs.datanode.kerberos.principal";
   String  DFS_DATANODE_READAHEAD_BYTES_KEY = "dfs.datanode.readahead.bytes";
   long    DFS_DATANODE_READAHEAD_BYTES_DEFAULT = 4 * 1024 * 1024; // 4MB
-  String  DFS_ENCRYPTION_KEY_PROVIDER_URI = "dfs.encryption.key.provider.uri";
 
   String DFS_ENCRYPT_DATA_TRANSFER_CIPHER_SUITES_KEY =
       "dfs.encrypt.data.transfer.cipher.suites";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea839bd4/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestHdfsHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestHdfsHelper.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestHdfsHelper.java
index 5695285..0e701f7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestHdfsHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestHdfsHelper.java
@@ -22,6 +22,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.crypto.key.JavaKeyStoreProvider;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystemTestHelper;
 import org.apache.hadoop.fs.Path;
@@ -158,7 +159,8 @@ public class TestHdfsHelper extends TestDirHelper {
       FileSystemTestHelper helper = new FileSystemTestHelper();
       final String jceksPath = JavaKeyStoreProvider.SCHEME_NAME + "://file" +
           new Path(helper.getTestRootDir(), "test.jks").toUri();
-      conf.set(DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI, jceksPath);
+      conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH,
+          jceksPath);
       MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf);
       builder.numDataNodes(2);
       MiniDFSCluster miniHdfs = builder.build();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea839bd4/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestRpcProgramNfs3.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestRpcProgramNfs3.java b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestRpcProgramNfs3.java
index 811aeda..0fabb30 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestRpcProgramNfs3.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestRpcProgramNfs3.java
@@ -30,10 +30,10 @@ import java.util.EnumSet;
 
 import org.apache.hadoop.crypto.key.JavaKeyStoreProvider;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystemTestHelper;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
@@ -135,7 +135,7 @@ public class TestRpcProgramNfs3 {
     String testRoot = fsHelper.getTestRootDir();
     testRootDir = new File(testRoot).getAbsoluteFile();
     final Path jksPath = new Path(testRootDir.toString(), "test.jks");
-    config.set(DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI,
+    config.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH,
         JavaKeyStoreProvider.SCHEME_NAME + "://file" + jksPath.toUri());
     ProxyUsers.refreshSuperUserGroupsConfiguration(config);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea839bd4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index caf6b60..3532d25 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
@@ -804,8 +804,6 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
       HdfsClientConfigKeys.DFS_DATA_TRANSFER_SASL_PROPS_RESOLVER_CLASS_KEY;
   public static final int    DFS_NAMENODE_LIST_ENCRYPTION_ZONES_NUM_RESPONSES_DEFAULT = 100;
   public static final String DFS_NAMENODE_LIST_ENCRYPTION_ZONES_NUM_RESPONSES = "dfs.namenode.list.encryption.zones.num.responses";
-  public static final String DFS_ENCRYPTION_KEY_PROVIDER_URI =
-      HdfsClientConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI;
   public static final String DFS_NAMENODE_EDEKCACHELOADER_INTERVAL_MS_KEY = "dfs.namenode.edekcacheloader.interval.ms";
   public static final int DFS_NAMENODE_EDEKCACHELOADER_INTERVAL_MS_DEFAULT = 1000;
   public static final String DFS_NAMENODE_EDEKCACHELOADER_INITIAL_DELAY_MS_KEY = "dfs.namenode.edekcacheloader.initial.delay.ms";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea839bd4/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
index 29c9ef2..ebaefde 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
@@ -2757,14 +2757,6 @@
 </property>
 
 <property>
-  <name>dfs.encryption.key.provider.uri</name>
-  <description>
-    The KeyProvider to use when interacting with encryption keys used
-    when reading and writing to an encryption zone.
-  </description>
-</property>
-
-<property>
   <name>dfs.storage.policy.enabled</name>
   <value>true</value>
   <description>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea839bd4/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/TransparentEncryption.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/TransparentEncryption.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/TransparentEncryption.md
index 6d0e52c..ee98df8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/TransparentEncryption.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/TransparentEncryption.md
@@ -117,7 +117,7 @@ Once a KMS has been set up and the NameNode and HDFS clients have been correctly
 
 ### <a name="Configuring_the_cluster_KeyProvider"></a>Configuring the cluster KeyProvider
 
-#### dfs.encryption.key.provider.uri
+#### hadoop.security.key.provider.path
 
 The KeyProvider to use when interacting with encryption keys used when reading and writing to an encryption zone.
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea839bd4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/TestCryptoAdminCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/TestCryptoAdminCLI.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/TestCryptoAdminCLI.java
index 7c6aaeb..99a7c2a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/TestCryptoAdminCLI.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/TestCryptoAdminCLI.java
@@ -35,7 +35,7 @@ import org.apache.hadoop.cli.util.CommandExecutor.Result;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.crypto.key.JavaKeyStoreProvider;
 import org.apache.hadoop.crypto.key.KeyProvider;
-import org.apache.hadoop.crypto.key.KeyProviderFactory;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
@@ -66,7 +66,7 @@ public class TestCryptoAdminCLI extends CLITestHelperDFS {
 
     tmpDir = GenericTestUtils.getTestDir(UUID.randomUUID().toString());
     final Path jksPath = new Path(tmpDir.toString(), "test.jks");
-    conf.set(DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI,
+    conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH,
         JavaKeyStoreProvider.SCHEME_NAME + "://file" + jksPath.toUri());
 
     dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea839bd4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAclsEndToEnd.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAclsEndToEnd.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAclsEndToEnd.java
index 2b515d0..3ff7055 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAclsEndToEnd.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAclsEndToEnd.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.crypto.key.kms.KMSClientProvider;
 import org.apache.hadoop.crypto.key.kms.server.KMSConfiguration;
 import org.apache.hadoop.crypto.key.kms.server.KeyAuthorizationKeyProvider;
 import org.apache.hadoop.crypto.key.kms.server.MiniKMS;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystemTestHelper;
@@ -190,7 +191,7 @@ public class TestAclsEndToEnd {
         "keyadmin,hdfs,user");
     conf.set(ProxyUsers.CONF_HADOOP_PROXYUSER + "." + realUser + ".hosts",
         "*");
-    conf.set(DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI,
+    conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH,
         getKeyProviderURI());
     conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY,
         true);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea839bd4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
index b7db601..7257bbd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
@@ -59,6 +59,7 @@ import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -1031,16 +1032,19 @@ public class TestDFSUtil {
   @Test
   public void testEncryptionProbe() throws Throwable {
     Configuration conf = new Configuration(false);
-    conf.unset(DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI);
+    conf.unset(CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH);
     assertFalse("encryption enabled on no provider key",
         DFSUtilClient.isHDFSEncryptionEnabled(conf));
-    conf.set(DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI, "");
+    conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH,
+        "");
     assertFalse("encryption enabled on empty provider key",
         DFSUtilClient.isHDFSEncryptionEnabled(conf));
-    conf.set(DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI, "\n\t\n");
+    conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH,
+        "\n\t\n");
     assertFalse("encryption enabled on whitespace provider key",
         DFSUtilClient.isHDFSEncryptionEnabled(conf));
-    conf.set(DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI, "http://hadoop.apache.org");
+    conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH,
+        "http://hadoop.apache.org");
     assertTrue("encryption disabled on valid provider key",
         DFSUtilClient.isHDFSEncryptionEnabled(conf));
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea839bd4/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 53c12ec..b634dd2 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
@@ -150,7 +150,8 @@ public class TestEncryptionZones {
     // Set up java key store
     String testRoot = fsHelper.getTestRootDir();
     testRootDir = new File(testRoot).getAbsoluteFile();
-    conf.set(DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI, getKeyProviderURI());
+    conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH,
+        getKeyProviderURI());
     conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY, true);
     // Lower the batch size for testing
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_LIST_ENCRYPTION_ZONES_NUM_RESPONSES,
@@ -845,9 +846,9 @@ public class TestEncryptionZones {
     // Check KeyProvider state
     // Flushing the KP on the NN, since it caches, and init a test one
     cluster.getNamesystem().getProvider().flush();
-    KeyProvider provider = KeyProviderFactory
-        .get(new URI(conf.getTrimmed(DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI)),
-            conf);
+    KeyProvider provider = KeyProviderFactory.get(new URI(conf.getTrimmed(
+        CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH)),
+        conf);
     List<String> keys = provider.getKeys();
     assertEquals("Expected NN to have created one key per zone", 1,
         keys.size());
@@ -931,7 +932,8 @@ public class TestEncryptionZones {
   public void testCreateEZWithNoProvider() throws Exception {
     // Unset the key provider and make sure EZ ops don't work
     final Configuration clusterConf = cluster.getConfiguration(0);
-    clusterConf.unset(DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI);
+    clusterConf
+        .unset(CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH);
     cluster.restartNameNode(true);
     cluster.waitActive();
     final Path zone1 = new Path("/zone1");
@@ -943,8 +945,9 @@ public class TestEncryptionZones {
       assertExceptionContains("since no key provider is available", e);
     }
     final Path jksPath = new Path(testRootDir.toString(), "test.jks");
-    clusterConf.set(DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI,
-        JavaKeyStoreProvider.SCHEME_NAME + "://file" + jksPath.toUri()
+    clusterConf
+        .set(CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH,
+            JavaKeyStoreProvider.SCHEME_NAME + "://file" + jksPath.toUri()
     );
     // Try listing EZs as well
     assertNumZones(0);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea839bd4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZonesWithHA.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZonesWithHA.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZonesWithHA.java
index f3a2640..bb8b79b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZonesWithHA.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZonesWithHA.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.crypto.key.JavaKeyStoreProvider;
 import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.FileSystemTestHelper;
 import org.apache.hadoop.fs.Path;
@@ -62,7 +63,7 @@ public class TestEncryptionZonesWithHA {
     fsHelper = new FileSystemTestHelper();
     String testRoot = fsHelper.getTestRootDir();
     testRootDir = new File(testRoot).getAbsoluteFile();
-    conf.set(DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI,
+    conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH,
         JavaKeyStoreProvider.SCHEME_NAME + "://file" +
         new Path(testRootDir.toString(), "test.jks").toUri()
     );

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea839bd4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestKeyProviderCache.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestKeyProviderCache.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestKeyProviderCache.java
index 4cbe871..0868b54 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestKeyProviderCache.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestKeyProviderCache.java
@@ -24,7 +24,7 @@ import java.util.List;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.crypto.key.KeyProvider;
 import org.apache.hadoop.crypto.key.KeyProviderFactory;
-import org.apache.hadoop.crypto.key.kms.KMSClientProvider;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -94,26 +94,26 @@ public class TestKeyProviderCache {
   public void testCache() throws Exception {
     KeyProviderCache kpCache = new KeyProviderCache(10000);
     Configuration conf = new Configuration();
-    conf.set(DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI,
+    conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH,
         "dummy://foo:bar@test_provider1");
     KeyProvider keyProvider1 = kpCache.get(conf);
     Assert.assertNotNull("Returned Key Provider is null !!", keyProvider1);
 
-    conf.set(DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI,
+    conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH,
         "dummy://foo:bar@test_provider1");
     KeyProvider keyProvider2 = kpCache.get(conf);
 
     Assert.assertTrue("Different KeyProviders returned !!",
         keyProvider1 == keyProvider2);
 
-    conf.set(DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI,
+    conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH,
         "dummy://test_provider3");
     KeyProvider keyProvider3 = kpCache.get(conf);
 
     Assert.assertFalse("Same KeyProviders returned !!",
         keyProvider1 == keyProvider3);
 
-    conf.set(DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI,
+    conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH,
         "dummy://hello:there@test_provider1");
     KeyProvider keyProvider4 = kpCache.get(conf);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea839bd4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReservedRawPaths.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReservedRawPaths.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReservedRawPaths.java
index c09d346..5416739 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReservedRawPaths.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReservedRawPaths.java
@@ -25,6 +25,7 @@ import java.util.EnumSet;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.crypto.key.JavaKeyStoreProvider;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.FileContextTestWrapper;
 import org.apache.hadoop.fs.FileStatus;
@@ -77,7 +78,7 @@ public class TestReservedRawPaths {
     String testRoot = fsHelper.getTestRootDir();
     File testRootDir = new File(testRoot).getAbsoluteFile();
     final Path jksPath = new Path(testRootDir.toString(), "test.jks");
-    conf.set(DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI,
+    conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH,
         JavaKeyStoreProvider.SCHEME_NAME + "://file" + jksPath.toUri()
     );
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea839bd4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSecureEncryptionZoneWithKMS.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSecureEncryptionZoneWithKMS.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSecureEncryptionZoneWithKMS.java
index b85c069..96e39ba 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSecureEncryptionZoneWithKMS.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSecureEncryptionZoneWithKMS.java
@@ -49,6 +49,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.crypto.key.kms.KMSClientProvider;
 import org.apache.hadoop.crypto.key.kms.server.KMSConfiguration;
 import org.apache.hadoop.crypto.key.kms.server.MiniKMS;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystemTestWrapper;
 import org.apache.hadoop.fs.FileUtil;
@@ -237,8 +238,9 @@ public class TestSecureEncryptionZoneWithKMS {
   @Before
   public void setup() throws Exception {
     // Start MiniDFS Cluster
-    baseConf.set(DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI,
-        getKeyProviderURI());
+    baseConf
+        .set(CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH,
+            getKeyProviderURI());
     baseConf.setBoolean(DFSConfigKeys
         .DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY, true);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea839bd4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNestedEncryptionZones.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNestedEncryptionZones.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNestedEncryptionZones.java
index 8847c91..13fc985 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNestedEncryptionZones.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNestedEncryptionZones.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.crypto.key.JavaKeyStoreProvider;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileSystemTestHelper;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
@@ -85,7 +86,8 @@ public class TestNestedEncryptionZones {
     // Set up java key store
     String testRoot = fsHelper.getTestRootDir();
     testRootDir = new File(testRoot).getAbsoluteFile();
-    conf.set(DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI, getKeyProviderURI());
+    conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH,
+        getKeyProviderURI());
     conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY, true);
     // Lower the batch size for testing
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_LIST_ENCRYPTION_ZONES_NUM_RESPONSES,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea839bd4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java
index 9e95650..90d61ee 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.server.namenode.metrics;
 
 import org.apache.hadoop.crypto.key.JavaKeyStoreProvider;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileSystemTestHelper;
 import org.apache.hadoop.fs.FileSystemTestWrapper;
 import org.apache.hadoop.fs.permission.FsPermission;
@@ -642,7 +643,7 @@ public class TestNameNodeMetrics {
     // Set up java key store
     String testRoot = fsHelper.getTestRootDir();
     File testRootDir = new File(testRoot).getAbsoluteFile();
-    conf.set(DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI,
+    conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH,
         JavaKeyStoreProvider.SCHEME_NAME + "://file" +
         new Path(testRootDir.toString(), "test.jks").toUri());
     conf.setBoolean(DFSConfigKeys


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


[17/44] hadoop git commit: YARN-5163. Migrate TestClientToAMTokens and TestClientRMTokens tests from the old RPC engine. Contributed by Wei Zhou and Kai Zheng

Posted by su...@apache.org.
YARN-5163. Migrate TestClientToAMTokens and TestClientRMTokens tests from the old RPC engine. Contributed by Wei Zhou and Kai Zheng


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

Branch: refs/heads/YARN-2915
Commit: 58bae3544747ad47373925add35b39bda8be8c5a
Parents: 8a40953
Author: Kai Zheng <ka...@intel.com>
Authored: Sun Sep 18 08:43:36 2016 +0800
Committer: Kai Zheng <ka...@intel.com>
Committed: Sun Sep 18 08:43:36 2016 +0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/ipc/TestRpcBase.java |   4 +-
 .../src/test/proto/test_rpc_service.proto       |   4 +
 .../security/TestClientToAMTokens.java          | 108 ++++++++++---------
 3 files changed, 65 insertions(+), 51 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/58bae354/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRpcBase.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRpcBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRpcBase.java
index 3c5885f..0d2f975 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRpcBase.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRpcBase.java
@@ -31,8 +31,6 @@ import org.junit.Assert;
 
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.retry.RetryPolicy;
-import org.apache.hadoop.ipc.protobuf.TestProtos;
-import org.apache.hadoop.ipc.protobuf.TestRpcServiceProtos;
 import org.apache.hadoop.security.KerberosInfo;
 import org.apache.hadoop.security.SaslRpcServer.AuthMethod;
 import org.apache.hadoop.security.token.Token;
@@ -481,7 +479,7 @@ public class TestRpcBase {
     }
   }
 
-  protected static TestProtos.EmptyRequestProto newEmptyRequest() {
+  public static TestProtos.EmptyRequestProto newEmptyRequest() {
     return TestProtos.EmptyRequestProto.newBuilder().build();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/58bae354/hadoop-common-project/hadoop-common/src/test/proto/test_rpc_service.proto
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/proto/test_rpc_service.proto b/hadoop-common-project/hadoop-common/src/test/proto/test_rpc_service.proto
index 06f6c4f..32eafeb 100644
--- a/hadoop-common-project/hadoop-common/src/test/proto/test_rpc_service.proto
+++ b/hadoop-common-project/hadoop-common/src/test/proto/test_rpc_service.proto
@@ -67,3 +67,7 @@ service NewerProtobufRpcProto {
   rpc ping(EmptyRequestProto) returns (EmptyResponseProto);
   rpc echo(EmptyRequestProto) returns (EmptyResponseProto);
 }
+
+service CustomProto {
+  rpc ping(EmptyRequestProto) returns (EmptyResponseProto);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/58bae354/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestClientToAMTokens.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/security/TestClientToAMTokens.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestClientToAMTokens.java
index d36fb9f..a451356 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestClientToAMTokens.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestClientToAMTokens.java
@@ -18,33 +18,19 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.security;
 
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.server.resourcemanager
-    .ParameterizedSchedulerTestBase;
-import static org.junit.Assert.fail;
-import org.junit.Before;
-import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-import java.io.IOException;
-import java.lang.annotation.Annotation;
-import java.net.InetSocketAddress;
-import java.nio.ByteBuffer;
-import java.security.PrivilegedAction;
-import java.security.PrivilegedExceptionAction;
-import java.util.Timer;
-import java.util.TimerTask;
-
-import javax.security.sasl.SaslException;
-
-import org.junit.Assert;
-
+import com.google.protobuf.BlockingService;
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.ProtocolInfo;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.Server;
+import org.apache.hadoop.ipc.TestRpcBase;
+import org.apache.hadoop.ipc.protobuf.TestProtos;
+import org.apache.hadoop.ipc.protobuf.TestRpcServiceProtos;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.KerberosInfo;
 import org.apache.hadoop.security.SecurityInfo;
@@ -63,9 +49,9 @@ import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.event.DrainDispatcher;
-import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.security.client.ClientToAMTokenIdentifier;
 import org.apache.hadoop.yarn.security.client.ClientToAMTokenSecretManager;
@@ -75,12 +61,30 @@ 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.MockRMWithCustomAMLauncher;
+import org.apache.hadoop.yarn.server.resourcemanager.ParameterizedSchedulerTestBase;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.util.Records;
+import org.junit.Assert;
+import org.junit.Before;
 import org.junit.Test;
 
+import javax.security.sasl.SaslException;
+import java.io.IOException;
+import java.lang.annotation.Annotation;
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import java.security.PrivilegedAction;
+import java.security.PrivilegedExceptionAction;
+import java.util.Timer;
+import java.util.TimerTask;
+
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
 public class TestClientToAMTokens extends ParameterizedSchedulerTestBase {
   private YarnConfiguration conf;
 
@@ -89,11 +93,12 @@ public class TestClientToAMTokens extends ParameterizedSchedulerTestBase {
     conf = getConf();
   }
 
-  private interface CustomProtocol {
-    @SuppressWarnings("unused")
-    public static final long versionID = 1L;
-
-    public void ping() throws YarnException, IOException;
+  @TokenInfo(ClientToAMTokenSelector.class)
+  @ProtocolInfo(protocolName =
+      "org.apache.hadoop.yarn.server.resourcemanager.security$CustomProtocol",
+      protocolVersion = 1)
+  public interface CustomProtocol
+      extends TestRpcServiceProtos.CustomProto.BlockingInterface {
   }
 
   private static class CustomSecurityInfo extends SecurityInfo {
@@ -137,8 +142,10 @@ public class TestClientToAMTokens extends ParameterizedSchedulerTestBase {
     }
 
     @Override
-    public void ping() throws YarnException, IOException {
+    public TestProtos.EmptyResponseProto ping(RpcController unused,
+                TestProtos.EmptyRequestProto request) throws ServiceException {
       this.pinged = true;
+      return TestProtos.EmptyResponseProto.newBuilder().build();
     }
     
     public ClientToAMTokenSecretManager getClientToAMTokenSecretManager() {
@@ -148,6 +155,13 @@ public class TestClientToAMTokens extends ParameterizedSchedulerTestBase {
     @Override
     protected void serviceStart() throws Exception {
       Configuration conf = getConfig();
+      // Set RPC engine to protobuf RPC engine
+      RPC.setProtocolEngine(conf, CustomProtocol.class,
+          ProtobufRpcEngine.class);
+      UserGroupInformation.setConfiguration(conf);
+
+      BlockingService service = TestRpcServiceProtos.CustomProto
+          .newReflectiveBlockingService(this);
 
       Server server;
       try {
@@ -158,7 +172,7 @@ public class TestClientToAMTokens extends ParameterizedSchedulerTestBase {
               .setProtocol(CustomProtocol.class)
               .setNumHandlers(1)
               .setSecretManager(secretMgr)
-              .setInstance(this).build();
+              .setInstance(service).build();
       } catch (Exception e) {
         throw new YarnRuntimeException(e);
       }
@@ -176,6 +190,8 @@ public class TestClientToAMTokens extends ParameterizedSchedulerTestBase {
   public void testClientToAMTokens() throws Exception {
     conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION,
       "kerberos");
+    // Set RPC engine to protobuf RPC engine
+    RPC.setProtocolEngine(conf, CustomProtocol.class, ProtobufRpcEngine.class);
     UserGroupInformation.setConfiguration(conf);
 
     ContainerManagementProtocol containerManager =
@@ -268,10 +284,9 @@ public class TestClientToAMTokens extends ParameterizedSchedulerTestBase {
 
     // Verify denial for unauthenticated user
     try {
-      CustomProtocol client =
-          (CustomProtocol) RPC.getProxy(CustomProtocol.class, 1L, am.address,
-            conf);
-      client.ping();
+      CustomProtocol client = RPC.getProxy(CustomProtocol.class,
+          1L, am.address, conf);
+      client.ping(null, TestRpcBase.newEmptyRequest());
       fail("Access by unauthenticated user should fail!!");
     } catch (Exception e) {
       Assert.assertFalse(am.pinged);
@@ -335,17 +350,16 @@ public class TestClientToAMTokens extends ParameterizedSchedulerTestBase {
         @Override
         public Void run() throws Exception {
           try {
-            CustomProtocol client =
-                (CustomProtocol) RPC.getProxy(CustomProtocol.class, 1L,
+            CustomProtocol client = RPC.getProxy(CustomProtocol.class, 1L,
                   am.address, conf);
-            client.ping();
+            client.ping(null, TestRpcBase.newEmptyRequest());
             fail("Connection initiation with illegally modified "
                 + "tokens is expected to fail.");
             return null;
-          } catch (YarnException ex) {
-            fail("Cannot get a YARN remote exception as "
-                + "it will indicate RPC success");
-            throw ex;
+          } catch (ServiceException ex) {
+            //fail("Cannot get a YARN remote exception as "
+            //    + "it indicates RPC success");
+            throw (Exception) ex.getCause();
           }
         }
       });
@@ -383,9 +397,8 @@ public class TestClientToAMTokens extends ParameterizedSchedulerTestBase {
       @Override
       public Void run() throws Exception {
         CustomProtocol client =
-            (CustomProtocol) RPC.getProxy(CustomProtocol.class, 1L, am.address,
-              conf);
-        client.ping();
+            RPC.getProxy(CustomProtocol.class, 1L, am.address, conf);
+        client.ping(null, TestRpcBase.newEmptyRequest());
         Assert.assertTrue(am.pinged);
         return null;
       }
@@ -402,10 +415,9 @@ public class TestClientToAMTokens extends ParameterizedSchedulerTestBase {
     ugi.doAs(new PrivilegedExceptionAction<Void>() {
       @Override
       public Void run() throws Exception {
-        CustomProtocol client =
-            (CustomProtocol) RPC.getProxy(CustomProtocol.class, 1L, am.address,
-              conf);
-        client.ping();
+        CustomProtocol client = RPC.getProxy(CustomProtocol.class,
+            1L, am.address, conf);
+        client.ping(null, TestRpcBase.newEmptyRequest());
         Assert.assertTrue(am.pinged);
         return null;
       }


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


[29/44] 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/eb4c9673/hadoop-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/eb4c9673/hadoop-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/eb4c9673/hadoop-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/eb4c9673/hadoop-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/eb4c9673/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/eb4c9673/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


[43/44] 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/9313ae14
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/9313ae14
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/9313ae14

Branch: refs/heads/YARN-2915
Commit: 9313ae14a5436c84851198fa1f82ac5b9caec5d9
Parents: eb4c967
Author: Subru Krishnan <su...@apache.org>
Authored: Fri Aug 5 12:34:58 2016 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon Sep 19 12:58:34 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/9313ae14/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/9313ae14/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/9313ae14/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/9313ae14/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/9313ae14/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/9313ae14/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/9313ae14/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/9313ae14/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/9313ae14/hadoop-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/9313ae14/hadoop-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/9313ae14/hadoop-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/9313ae14/hadoop-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/9313ae14/hadoop-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/9313ae14/hadoop-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/9313ae14/hadoop-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/9313ae14/hadoop-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/9313ae14/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/9313ae14/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


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

Branch: refs/heads/YARN-2915
Commit: b627de259d729dddec5eb4863c6ac4022ebb1c66
Parents: 5ae955b
Author: Subru Krishnan <su...@apache.org>
Authored: Thu Sep 1 13:55:54 2016 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon Sep 19 12:58:34 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/b627de25/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/b627de25/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


[10/44] hadoop git commit: HADOOP-13412. Move dev-support/checkstyle/suppressions.xml to hadoop-build-tools. Contributed by John Zhuge.

Posted by su...@apache.org.
HADOOP-13412. Move dev-support/checkstyle/suppressions.xml to hadoop-build-tools. Contributed by John Zhuge.


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

Branch: refs/heads/YARN-2915
Commit: cd099a63d58c6981c45db2b169430d4935e43bd1
Parents: 0e68e14
Author: Wei-Chiu Chuang <we...@apache.org>
Authored: Fri Sep 16 14:26:42 2016 -0700
Committer: Wei-Chiu Chuang <we...@apache.org>
Committed: Fri Sep 16 14:26:42 2016 -0700

----------------------------------------------------------------------
 dev-support/checkstyle/suppressions.xml         | 21 --------------------
 .../main/resources/checkstyle/suppressions.xml  | 21 ++++++++++++++++++++
 pom.xml                                         |  2 +-
 3 files changed, 22 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/cd099a63/dev-support/checkstyle/suppressions.xml
----------------------------------------------------------------------
diff --git a/dev-support/checkstyle/suppressions.xml b/dev-support/checkstyle/suppressions.xml
deleted file mode 100644
index ccc89c8..0000000
--- a/dev-support/checkstyle/suppressions.xml
+++ /dev/null
@@ -1,21 +0,0 @@
-<?xml version="1.0"?>
-<!--
-  Licensed under the Apache License, Version 2.0 (the "License");
-  you may not use this file except in compliance with the License.
-  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-  Unless required by applicable law or agreed to in writing, software
-  distributed under the License is distributed on an "AS IS" BASIS,
-  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  See the License for the specific language governing permissions and
-  limitations under the License. See accompanying LICENSE file.
--->
-<!DOCTYPE suppressions PUBLIC
-"-//Puppy Crawl//DTD Suppressions 1.1//EN"
-"http://www.puppycrawl.com/dtds/suppressions_1_1.dtd">
-
-<suppressions>
-  <suppress checks="JavadocPackage" files="[\\/]src[\\/]test[\\/].*"/>
-</suppressions>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cd099a63/hadoop-build-tools/src/main/resources/checkstyle/suppressions.xml
----------------------------------------------------------------------
diff --git a/hadoop-build-tools/src/main/resources/checkstyle/suppressions.xml b/hadoop-build-tools/src/main/resources/checkstyle/suppressions.xml
new file mode 100644
index 0000000..ccc89c8
--- /dev/null
+++ b/hadoop-build-tools/src/main/resources/checkstyle/suppressions.xml
@@ -0,0 +1,21 @@
+<?xml version="1.0"?>
+<!--
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License. See accompanying LICENSE file.
+-->
+<!DOCTYPE suppressions PUBLIC
+"-//Puppy Crawl//DTD Suppressions 1.1//EN"
+"http://www.puppycrawl.com/dtds/suppressions_1_1.dtd">
+
+<suppressions>
+  <suppress checks="JavadocPackage" files="[\\/]src[\\/]test[\\/].*"/>
+</suppressions>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cd099a63/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index fb50419..250f5a1 100644
--- a/pom.xml
+++ b/pom.xml
@@ -318,7 +318,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs
           </dependencies>
           <configuration>
             <configLocation>checkstyle/checkstyle.xml</configLocation>
-            <suppressionsLocation>dev-support/checkstyle/suppressions.xml</suppressionsLocation>
+            <suppressionsLocation>checkstyle/suppressions.xml</suppressionsLocation>
             <includeTestSourceDirectory>true</includeTestSourceDirectory>
             <failOnViolation>false</failOnViolation>
             <format>xml</format>


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


[40/44] 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/1f264546
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/1f264546
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/1f264546

Branch: refs/heads/YARN-2915
Commit: 1f264546d1213bbd3a68ade40df304e78e4e5eb6
Parents: ab3ceda
Author: Subru Krishnan <su...@apache.org>
Authored: Thu Aug 4 15:58:31 2016 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon Sep 19 12:58:34 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/1f264546/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 86e8a95..473ee0f 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
@@ -2482,6 +2482,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/1f264546/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 423b78b..218ac79 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
@@ -2631,6 +2631,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/1f264546/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/1f264546/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/1f264546/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/1f264546/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/1f264546/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/1f264546/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/1f264546/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/1f264546/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


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

Branch: refs/heads/YARN-2915
Commit: faf61ee8b801170c06bc712af40b9560b48bbafa
Parents: 0c0de59
Author: Subru Krishnan <su...@apache.org>
Authored: Tue Aug 9 16:07:55 2016 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon Sep 19 12:58:34 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/faf61ee8/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/faf61ee8/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/faf61ee8/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/faf61ee8/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/faf61ee8/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/faf61ee8/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/faf61ee8/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/faf61ee8/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/faf61ee8/hadoop-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/faf61ee8/hadoop-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/faf61ee8/hadoop-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/faf61ee8/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/faf61ee8/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/faf61ee8/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


[07/44] hadoop git commit: HDFS-9895. Remove unnecessary conf cache from DataNode. Contributed by Xiaobing Zhou.

Posted by su...@apache.org.
HDFS-9895. Remove unnecessary conf cache from DataNode. Contributed by Xiaobing Zhou.


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

Branch: refs/heads/YARN-2915
Commit: ec3ea188753e2c138a64e10303c8751116dc4e80
Parents: fcbac00
Author: Arpit Agarwal <ar...@apache.org>
Authored: Thu Sep 15 15:25:47 2016 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Sep 15 15:25:47 2016 -0700

----------------------------------------------------------------------
 .../hdfs/server/datanode/BlockScanner.java      |   4 +
 .../hadoop/hdfs/server/datanode/DNConf.java     |  95 +++++++------
 .../hadoop/hdfs/server/datanode/DataNode.java   | 133 ++++++++++---------
 .../server/datanode/TestBPOfferService.java     |   4 +-
 .../TestDataXceiverLazyPersistHint.java         |   5 +-
 .../fsdataset/impl/TestFsDatasetImpl.java       |   6 +-
 6 files changed, 130 insertions(+), 117 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ec3ea188/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockScanner.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockScanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockScanner.java
index be6aa83..456dcc1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockScanner.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockScanner.java
@@ -165,6 +165,10 @@ public class BlockScanner {
     }
   }
 
+  public BlockScanner(DataNode datanode) {
+    this(datanode, datanode.getConf());
+  }
+
   public BlockScanner(DataNode datanode, Configuration conf) {
     this.datanode = datanode;
     this.conf = new Conf(conf);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ec3ea188/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java
index 757d7f5..823d05c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java
@@ -56,6 +56,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.IGNORE_SECURE_PORTS_FOR_TESTI
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_BP_READY_TIMEOUT_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_BP_READY_TIMEOUT_DEFAULT;
 
+import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
@@ -72,7 +73,6 @@ import java.util.concurrent.TimeUnit;
  */
 @InterfaceAudience.Private
 public class DNConf {
-  final Configuration conf;
   final int socketTimeout;
   final int socketWriteTimeout;
   final int socketKeepaliveTimeout;
@@ -120,73 +120,77 @@ public class DNConf {
   private final int volFailuresTolerated;
   private final int volsConfigured;
   private final int maxDataLength;
+  private Configurable dn;
 
-  public DNConf(Configuration conf) {
-    this.conf = conf;
-    socketTimeout = conf.getInt(DFS_CLIENT_SOCKET_TIMEOUT_KEY,
+  public DNConf(final Configurable dn) {
+    this.dn = dn;
+    socketTimeout = getConf().getInt(DFS_CLIENT_SOCKET_TIMEOUT_KEY,
         HdfsConstants.READ_TIMEOUT);
-    socketWriteTimeout = conf.getInt(DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY,
+    socketWriteTimeout = getConf().getInt(DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY,
         HdfsConstants.WRITE_TIMEOUT);
-    socketKeepaliveTimeout = conf.getInt(
+    socketKeepaliveTimeout = getConf().getInt(
         DFSConfigKeys.DFS_DATANODE_SOCKET_REUSE_KEEPALIVE_KEY,
         DFSConfigKeys.DFS_DATANODE_SOCKET_REUSE_KEEPALIVE_DEFAULT);
-    this.transferSocketSendBufferSize = conf.getInt(
+    this.transferSocketSendBufferSize = getConf().getInt(
         DFSConfigKeys.DFS_DATANODE_TRANSFER_SOCKET_SEND_BUFFER_SIZE_KEY,
         DFSConfigKeys.DFS_DATANODE_TRANSFER_SOCKET_SEND_BUFFER_SIZE_DEFAULT);
-    this.transferSocketRecvBufferSize = conf.getInt(
+    this.transferSocketRecvBufferSize = getConf().getInt(
         DFSConfigKeys.DFS_DATANODE_TRANSFER_SOCKET_RECV_BUFFER_SIZE_KEY,
         DFSConfigKeys.DFS_DATANODE_TRANSFER_SOCKET_RECV_BUFFER_SIZE_DEFAULT);
-    this.tcpNoDelay = conf.getBoolean(
+    this.tcpNoDelay = getConf().getBoolean(
         DFSConfigKeys.DFS_DATA_TRANSFER_SERVER_TCPNODELAY,
         DFSConfigKeys.DFS_DATA_TRANSFER_SERVER_TCPNODELAY_DEFAULT);
 
     /* Based on results on different platforms, we might need set the default
      * to false on some of them. */
-    transferToAllowed = conf.getBoolean(
+    transferToAllowed = getConf().getBoolean(
         DFS_DATANODE_TRANSFERTO_ALLOWED_KEY,
         DFS_DATANODE_TRANSFERTO_ALLOWED_DEFAULT);
 
-    writePacketSize = conf.getInt(DFS_CLIENT_WRITE_PACKET_SIZE_KEY, 
+    writePacketSize = getConf().getInt(DFS_CLIENT_WRITE_PACKET_SIZE_KEY,
         DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT);
     
-    readaheadLength = conf.getLong(
+    readaheadLength = getConf().getLong(
         HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_KEY,
         HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT);
-    maxDataLength = conf.getInt(DFSConfigKeys.IPC_MAXIMUM_DATA_LENGTH,
+    maxDataLength = getConf().getInt(DFSConfigKeys.IPC_MAXIMUM_DATA_LENGTH,
         DFSConfigKeys.IPC_MAXIMUM_DATA_LENGTH_DEFAULT);
-    dropCacheBehindWrites = conf.getBoolean(
+    dropCacheBehindWrites = getConf().getBoolean(
         DFSConfigKeys.DFS_DATANODE_DROP_CACHE_BEHIND_WRITES_KEY,
         DFSConfigKeys.DFS_DATANODE_DROP_CACHE_BEHIND_WRITES_DEFAULT);
-    syncBehindWrites = conf.getBoolean(
+    syncBehindWrites = getConf().getBoolean(
         DFSConfigKeys.DFS_DATANODE_SYNC_BEHIND_WRITES_KEY,
         DFSConfigKeys.DFS_DATANODE_SYNC_BEHIND_WRITES_DEFAULT);
-    syncBehindWritesInBackground = conf.getBoolean(
+    syncBehindWritesInBackground = getConf().getBoolean(
         DFSConfigKeys.DFS_DATANODE_SYNC_BEHIND_WRITES_IN_BACKGROUND_KEY,
         DFSConfigKeys.DFS_DATANODE_SYNC_BEHIND_WRITES_IN_BACKGROUND_DEFAULT);
-    dropCacheBehindReads = conf.getBoolean(
+    dropCacheBehindReads = getConf().getBoolean(
         DFSConfigKeys.DFS_DATANODE_DROP_CACHE_BEHIND_READS_KEY,
         DFSConfigKeys.DFS_DATANODE_DROP_CACHE_BEHIND_READS_DEFAULT);
-    connectToDnViaHostname = conf.getBoolean(
+    connectToDnViaHostname = getConf().getBoolean(
         DFSConfigKeys.DFS_DATANODE_USE_DN_HOSTNAME,
         DFSConfigKeys.DFS_DATANODE_USE_DN_HOSTNAME_DEFAULT);
-    this.blockReportInterval = conf.getLong(DFS_BLOCKREPORT_INTERVAL_MSEC_KEY,
+    this.blockReportInterval = getConf().getLong(
+        DFS_BLOCKREPORT_INTERVAL_MSEC_KEY,
         DFS_BLOCKREPORT_INTERVAL_MSEC_DEFAULT);
-    this.ibrInterval = conf.getLong(
+    this.ibrInterval = getConf().getLong(
         DFSConfigKeys.DFS_BLOCKREPORT_INCREMENTAL_INTERVAL_MSEC_KEY,
         DFSConfigKeys.DFS_BLOCKREPORT_INCREMENTAL_INTERVAL_MSEC_DEFAULT);
-    this.blockReportSplitThreshold = conf.getLong(DFS_BLOCKREPORT_SPLIT_THRESHOLD_KEY,
-                                            DFS_BLOCKREPORT_SPLIT_THRESHOLD_DEFAULT);
-    this.cacheReportInterval = conf.getLong(DFS_CACHEREPORT_INTERVAL_MSEC_KEY,
+    this.blockReportSplitThreshold = getConf().getLong(
+        DFS_BLOCKREPORT_SPLIT_THRESHOLD_KEY,
+        DFS_BLOCKREPORT_SPLIT_THRESHOLD_DEFAULT);
+    this.cacheReportInterval = getConf().getLong(
+        DFS_CACHEREPORT_INTERVAL_MSEC_KEY,
         DFS_CACHEREPORT_INTERVAL_MSEC_DEFAULT);
 
-    this.dfsclientSlowIoWarningThresholdMs = conf.getLong(
+    this.dfsclientSlowIoWarningThresholdMs = getConf().getLong(
         HdfsClientConfigKeys.DFS_CLIENT_SLOW_IO_WARNING_THRESHOLD_KEY,
         HdfsClientConfigKeys.DFS_CLIENT_SLOW_IO_WARNING_THRESHOLD_DEFAULT);
-    this.datanodeSlowIoWarningThresholdMs = conf.getLong(
+    this.datanodeSlowIoWarningThresholdMs = getConf().getLong(
         DFSConfigKeys.DFS_DATANODE_SLOW_IO_WARNING_THRESHOLD_KEY,
         DFSConfigKeys.DFS_DATANODE_SLOW_IO_WARNING_THRESHOLD_DEFAULT);
 
-    long initBRDelay = conf.getTimeDuration(
+    long initBRDelay = getConf().getTimeDuration(
         DFS_BLOCKREPORT_INITIAL_DELAY_KEY,
         DFS_BLOCKREPORT_INITIAL_DELAY_DEFAULT, TimeUnit.SECONDS) * 1000L;
     if (initBRDelay >= blockReportInterval) {
@@ -197,11 +201,11 @@ public class DNConf {
     }
     initialBlockReportDelayMs = initBRDelay;
     
-    heartBeatInterval = conf.getTimeDuration(DFS_HEARTBEAT_INTERVAL_KEY,
+    heartBeatInterval = getConf().getTimeDuration(DFS_HEARTBEAT_INTERVAL_KEY,
         DFS_HEARTBEAT_INTERVAL_DEFAULT, TimeUnit.SECONDS) * 1000L;
     long confLifelineIntervalMs =
-        conf.getLong(DFS_DATANODE_LIFELINE_INTERVAL_SECONDS_KEY,
-        3 * conf.getTimeDuration(DFS_HEARTBEAT_INTERVAL_KEY,
+        getConf().getLong(DFS_DATANODE_LIFELINE_INTERVAL_SECONDS_KEY,
+        3 * getConf().getTimeDuration(DFS_HEARTBEAT_INTERVAL_KEY,
         DFS_HEARTBEAT_INTERVAL_DEFAULT, TimeUnit.SECONDS)) * 1000L;
     if (confLifelineIntervalMs <= heartBeatInterval) {
       confLifelineIntervalMs = 3 * heartBeatInterval;
@@ -215,47 +219,50 @@ public class DNConf {
     lifelineIntervalMs = confLifelineIntervalMs;
     
     // do we need to sync block file contents to disk when blockfile is closed?
-    this.syncOnClose = conf.getBoolean(DFS_DATANODE_SYNCONCLOSE_KEY, 
+    this.syncOnClose = getConf().getBoolean(DFS_DATANODE_SYNCONCLOSE_KEY,
         DFS_DATANODE_SYNCONCLOSE_DEFAULT);
 
-    this.minimumNameNodeVersion = conf.get(DFS_DATANODE_MIN_SUPPORTED_NAMENODE_VERSION_KEY,
+    this.minimumNameNodeVersion = getConf().get(
+        DFS_DATANODE_MIN_SUPPORTED_NAMENODE_VERSION_KEY,
         DFS_DATANODE_MIN_SUPPORTED_NAMENODE_VERSION_DEFAULT);
     
-    this.encryptDataTransfer = conf.getBoolean(DFS_ENCRYPT_DATA_TRANSFER_KEY,
+    this.encryptDataTransfer = getConf().getBoolean(
+        DFS_ENCRYPT_DATA_TRANSFER_KEY,
         DFS_ENCRYPT_DATA_TRANSFER_DEFAULT);
-    this.encryptionAlgorithm = conf.get(DFS_DATA_ENCRYPTION_ALGORITHM_KEY);
-    this.trustedChannelResolver = TrustedChannelResolver.getInstance(conf);
+    this.encryptionAlgorithm = getConf().get(DFS_DATA_ENCRYPTION_ALGORITHM_KEY);
+    this.trustedChannelResolver = TrustedChannelResolver.getInstance(getConf());
     this.saslPropsResolver = DataTransferSaslUtil.getSaslPropertiesResolver(
-      conf);
-    this.ignoreSecurePortsForTesting = conf.getBoolean(
+      getConf());
+    this.ignoreSecurePortsForTesting = getConf().getBoolean(
         IGNORE_SECURE_PORTS_FOR_TESTING_KEY,
         IGNORE_SECURE_PORTS_FOR_TESTING_DEFAULT);
     
-    this.xceiverStopTimeout = conf.getLong(
+    this.xceiverStopTimeout = getConf().getLong(
         DFS_DATANODE_XCEIVER_STOP_TIMEOUT_MILLIS_KEY,
         DFS_DATANODE_XCEIVER_STOP_TIMEOUT_MILLIS_DEFAULT);
 
-    this.maxLockedMemory = conf.getLong(
+    this.maxLockedMemory = getConf().getLong(
         DFS_DATANODE_MAX_LOCKED_MEMORY_KEY,
         DFS_DATANODE_MAX_LOCKED_MEMORY_DEFAULT);
 
-    this.restartReplicaExpiry = conf.getLong(
+    this.restartReplicaExpiry = getConf().getLong(
         DFS_DATANODE_RESTART_REPLICA_EXPIRY_KEY,
         DFS_DATANODE_RESTART_REPLICA_EXPIRY_DEFAULT) * 1000L;
 
-    this.allowNonLocalLazyPersist = conf.getBoolean(
+    this.allowNonLocalLazyPersist = getConf().getBoolean(
         DFS_DATANODE_NON_LOCAL_LAZY_PERSIST,
         DFS_DATANODE_NON_LOCAL_LAZY_PERSIST_DEFAULT);
 
-    this.bpReadyTimeout = conf.getTimeDuration(
+    this.bpReadyTimeout = getConf().getTimeDuration(
         DFS_DATANODE_BP_READY_TIMEOUT_KEY,
         DFS_DATANODE_BP_READY_TIMEOUT_DEFAULT, TimeUnit.SECONDS);
 
     this.volFailuresTolerated =
-        conf.getInt(DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY,
+        getConf().getInt(
+            DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY,
             DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_DEFAULT);
     String[] dataDirs =
-        conf.getTrimmedStrings(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY);
+        getConf().getTrimmedStrings(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY);
     this.volsConfigured = (dataDirs == null) ? 0 : dataDirs.length;
   }
 
@@ -270,7 +277,7 @@ public class DNConf {
    * @return Configuration the configuration
    */
   public Configuration getConf() {
-    return conf;
+    return this.dn.getConf();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ec3ea188/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index 09ecac1..abff796 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@ -363,7 +363,6 @@ public class DataNode extends ReconfigurableBase
   private SecureResources secureResources = null;
   // dataDirs must be accessed while holding the DataNode lock.
   private List<StorageLocation> dataDirs;
-  private Configuration conf;
   private final String confVersion;
   private final long maxNumberOfBlocksToLog;
   private final boolean pipelineSupportECN;
@@ -419,7 +418,7 @@ public class DataNode extends ReconfigurableBase
     this.confVersion = null;
     this.usersWithLocalPathAccess = null;
     this.connectToDnViaHostname = false;
-    this.blockScanner = new BlockScanner(this, conf);
+    this.blockScanner = new BlockScanner(this, this.getConf());
     this.pipelineSupportECN = false;
     this.checkDiskErrorInterval =
         ThreadLocalRandom.current().nextInt(5000, (int) (5000 * 1.25));
@@ -438,7 +437,7 @@ public class DataNode extends ReconfigurableBase
     this.tracer = createTracer(conf);
     this.tracerConfigurationManager =
         new TracerConfigurationManager(DATANODE_HTRACE_PREFIX, conf);
-    this.blockScanner = new BlockScanner(this, conf);
+    this.blockScanner = new BlockScanner(this);
     this.lastDiskErrorCheck = 0;
     this.maxNumberOfBlocksToLog = conf.getLong(DFS_MAX_NUM_BLOCKS_TO_LOG_KEY,
         DFS_MAX_NUM_BLOCKS_TO_LOG_DEFAULT);
@@ -487,7 +486,7 @@ public class DataNode extends ReconfigurableBase
     try {
       hostName = getHostName(conf);
       LOG.info("Configured hostname is " + hostName);
-      startDataNode(conf, dataDirs, resources);
+      startDataNode(dataDirs, resources);
     } catch (IOException ie) {
       shutdown();
       throw ie;
@@ -527,7 +526,7 @@ public class DataNode extends ReconfigurableBase
         try {
           LOG.info("Reconfiguring " + property + " to " + newVal);
           this.refreshVolumes(newVal);
-          return conf.get(DFS_DATANODE_DATA_DIR_KEY);
+          return getConf().get(DFS_DATANODE_DATA_DIR_KEY);
         } catch (IOException e) {
           rootException = e;
         } finally {
@@ -650,7 +649,7 @@ public class DataNode extends ReconfigurableBase
 
     // Use the existing StorageLocation to detect storage type changes.
     Map<String, StorageLocation> existingLocations = new HashMap<>();
-    for (StorageLocation loc : getStorageLocations(this.conf)) {
+    for (StorageLocation loc : getStorageLocations(getConf())) {
       existingLocations.put(loc.getFile().getCanonicalPath(), loc);
     }
 
@@ -846,7 +845,7 @@ public class DataNode extends ReconfigurableBase
         it.remove();
       }
     }
-    conf.set(DFS_DATANODE_DATA_DIR_KEY, Joiner.on(",").join(dataDirs));
+    getConf().set(DFS_DATANODE_DATA_DIR_KEY, Joiner.on(",").join(dataDirs));
 
     if (ioe != null) {
       throw ioe;
@@ -904,14 +903,14 @@ public class DataNode extends ReconfigurableBase
    * for information related to the different configuration options and
    * Http Policy is decided.
    */
-  private void startInfoServer(Configuration conf)
+  private void startInfoServer()
     throws IOException {
     // SecureDataNodeStarter will bind the privileged port to the channel if
     // the DN is started by JSVC, pass it along.
     ServerSocketChannel httpServerChannel = secureResources != null ?
         secureResources.getHttpServerChannel() : null;
 
-    this.httpServer = new DatanodeHttpServer(conf, this, httpServerChannel);
+    httpServer = new DatanodeHttpServer(getConf(), this, httpServerChannel);
     httpServer.start();
     if (httpServer.getHttpAddress() != null) {
       infoPort = httpServer.getHttpAddress().getPort();
@@ -933,24 +932,24 @@ public class DataNode extends ReconfigurableBase
     }
   }
 
-  private void initIpcServer(Configuration conf) throws IOException {
+  private void initIpcServer() throws IOException {
     InetSocketAddress ipcAddr = NetUtils.createSocketAddr(
-        conf.getTrimmed(DFS_DATANODE_IPC_ADDRESS_KEY));
+        getConf().getTrimmed(DFS_DATANODE_IPC_ADDRESS_KEY));
     
     // Add all the RPC protocols that the Datanode implements    
-    RPC.setProtocolEngine(conf, ClientDatanodeProtocolPB.class,
+    RPC.setProtocolEngine(getConf(), ClientDatanodeProtocolPB.class,
         ProtobufRpcEngine.class);
     ClientDatanodeProtocolServerSideTranslatorPB clientDatanodeProtocolXlator = 
           new ClientDatanodeProtocolServerSideTranslatorPB(this);
     BlockingService service = ClientDatanodeProtocolService
         .newReflectiveBlockingService(clientDatanodeProtocolXlator);
-    ipcServer = new RPC.Builder(conf)
+    ipcServer = new RPC.Builder(getConf())
         .setProtocol(ClientDatanodeProtocolPB.class)
         .setInstance(service)
         .setBindAddress(ipcAddr.getHostName())
         .setPort(ipcAddr.getPort())
         .setNumHandlers(
-            conf.getInt(DFS_DATANODE_HANDLER_COUNT_KEY,
+            getConf().getInt(DFS_DATANODE_HANDLER_COUNT_KEY,
                 DFS_DATANODE_HANDLER_COUNT_DEFAULT)).setVerbose(false)
         .setSecretManager(blockPoolTokenSecretManager).build();
 
@@ -958,29 +957,32 @@ public class DataNode extends ReconfigurableBase
         = new ReconfigurationProtocolServerSideTranslatorPB(this);
     service = ReconfigurationProtocolService
         .newReflectiveBlockingService(reconfigurationProtocolXlator);
-    DFSUtil.addPBProtocol(conf, ReconfigurationProtocolPB.class, service,
+    DFSUtil.addPBProtocol(getConf(), ReconfigurationProtocolPB.class, service,
         ipcServer);
 
     InterDatanodeProtocolServerSideTranslatorPB interDatanodeProtocolXlator = 
         new InterDatanodeProtocolServerSideTranslatorPB(this);
     service = InterDatanodeProtocolService
         .newReflectiveBlockingService(interDatanodeProtocolXlator);
-    DFSUtil.addPBProtocol(conf, InterDatanodeProtocolPB.class, service,
+    DFSUtil.addPBProtocol(getConf(), InterDatanodeProtocolPB.class, service,
         ipcServer);
 
     TraceAdminProtocolServerSideTranslatorPB traceAdminXlator =
         new TraceAdminProtocolServerSideTranslatorPB(this);
     BlockingService traceAdminService = TraceAdminService
         .newReflectiveBlockingService(traceAdminXlator);
-    DFSUtil.addPBProtocol(conf, TraceAdminProtocolPB.class, traceAdminService,
+    DFSUtil.addPBProtocol(
+        getConf(),
+        TraceAdminProtocolPB.class,
+        traceAdminService,
         ipcServer);
 
     LOG.info("Opened IPC server at " + ipcServer.getListenerAddress());
 
     // set service-level authorization security policy
-    if (conf.getBoolean(
+    if (getConf().getBoolean(
         CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION, false)) {
-      ipcServer.refreshServiceAcl(conf, new HDFSPolicyProvider());
+      ipcServer.refreshServiceAcl(getConf(), new HDFSPolicyProvider());
     }
   }
 
@@ -1071,17 +1073,17 @@ public class DataNode extends ReconfigurableBase
     }
   }
 
-  private void initDataXceiver(Configuration conf) throws IOException {
+  private void initDataXceiver() throws IOException {
     // find free port or use privileged port provided
     TcpPeerServer tcpPeerServer;
     if (secureResources != null) {
       tcpPeerServer = new TcpPeerServer(secureResources);
     } else {
-      int backlogLength = conf.getInt(
+      int backlogLength = getConf().getInt(
           CommonConfigurationKeysPublic.IPC_SERVER_LISTEN_QUEUE_SIZE_KEY,
           CommonConfigurationKeysPublic.IPC_SERVER_LISTEN_QUEUE_SIZE_DEFAULT);
       tcpPeerServer = new TcpPeerServer(dnConf.socketWriteTimeout,
-          DataNode.getStreamingAddr(conf), backlogLength);
+          DataNode.getStreamingAddr(getConf()), backlogLength);
     }
     if (dnConf.getTransferSocketRecvBufferSize() > 0) {
       tcpPeerServer.setReceiveBufferSize(
@@ -1090,24 +1092,27 @@ public class DataNode extends ReconfigurableBase
     streamingAddr = tcpPeerServer.getStreamingAddr();
     LOG.info("Opened streaming server at " + streamingAddr);
     this.threadGroup = new ThreadGroup("dataXceiverServer");
-    xserver = new DataXceiverServer(tcpPeerServer, conf, this);
+    xserver = new DataXceiverServer(tcpPeerServer, getConf(), this);
     this.dataXceiverServer = new Daemon(threadGroup, xserver);
     this.threadGroup.setDaemon(true); // auto destroy when empty
 
-    if (conf.getBoolean(HdfsClientConfigKeys.Read.ShortCircuit.KEY,
-              HdfsClientConfigKeys.Read.ShortCircuit.DEFAULT) ||
-        conf.getBoolean(HdfsClientConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC,
-              HdfsClientConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC_DEFAULT)) {
+    if (getConf().getBoolean(
+        HdfsClientConfigKeys.Read.ShortCircuit.KEY,
+        HdfsClientConfigKeys.Read.ShortCircuit.DEFAULT) ||
+        getConf().getBoolean(
+            HdfsClientConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC,
+            HdfsClientConfigKeys
+              .DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC_DEFAULT)) {
       DomainPeerServer domainPeerServer =
-                getDomainPeerServer(conf, streamingAddr.getPort());
+                getDomainPeerServer(getConf(), streamingAddr.getPort());
       if (domainPeerServer != null) {
         this.localDataXceiverServer = new Daemon(threadGroup,
-            new DataXceiverServer(domainPeerServer, conf, this));
+            new DataXceiverServer(domainPeerServer, getConf(), this));
         LOG.info("Listening on UNIX domain socket: " +
             domainPeerServer.getBindPath());
       }
     }
-    this.shortCircuitRegistry = new ShortCircuitRegistry(conf);
+    this.shortCircuitRegistry = new ShortCircuitRegistry(getConf());
   }
 
   private static DomainPeerServer getDomainPeerServer(Configuration conf,
@@ -1288,26 +1293,23 @@ public class DataNode extends ReconfigurableBase
   /**
    * This method starts the data node with the specified conf.
    * 
-   * @param conf - the configuration
-   *  if conf's CONFIG_PROPERTY_SIMULATED property is set
-   *  then a simulated storage based data node is created.
+   * If conf's CONFIG_PROPERTY_SIMULATED property is set
+   * then a simulated storage based data node is created.
    * 
    * @param dataDirs - only for a non-simulated storage data node
    * @throws IOException
    */
-  void startDataNode(Configuration conf, 
-                     List<StorageLocation> dataDirs,
+  void startDataNode(List<StorageLocation> dataDirectories,
                      SecureResources resources
                      ) throws IOException {
 
     // settings global for all BPs in the Data Node
     this.secureResources = resources;
     synchronized (this) {
-      this.dataDirs = dataDirs;
+      this.dataDirs = dataDirectories;
     }
-    this.conf = conf;
-    this.dnConf = new DNConf(conf);
-    checkSecureConfig(dnConf, conf, resources);
+    this.dnConf = new DNConf(this);
+    checkSecureConfig(dnConf, getConf(), resources);
 
     if (dnConf.maxLockedMemory > 0) {
       if (!NativeIO.POSIX.getCacheManipulator().verifyCanMlock()) {
@@ -1347,10 +1349,10 @@ public class DataNode extends ReconfigurableBase
     
     // global DN settings
     registerMXBean();
-    initDataXceiver(conf);
-    startInfoServer(conf);
+    initDataXceiver();
+    startInfoServer();
     pauseMonitor = new JvmPauseMonitor();
-    pauseMonitor.init(conf);
+    pauseMonitor.init(getConf());
     pauseMonitor.start();
   
     // BlockPoolTokenSecretManager is required to create ipc server.
@@ -1360,24 +1362,24 @@ public class DataNode extends ReconfigurableBase
     dnUserName = UserGroupInformation.getCurrentUser().getShortUserName();
     LOG.info("dnUserName = " + dnUserName);
     LOG.info("supergroup = " + supergroup);
-    initIpcServer(conf);
+    initIpcServer();
 
-    metrics = DataNodeMetrics.create(conf, getDisplayName());
+    metrics = DataNodeMetrics.create(getConf(), getDisplayName());
     metrics.getJvmMetrics().setPauseMonitor(pauseMonitor);
 
-    ecWorker = new ErasureCodingWorker(conf, this);
+    ecWorker = new ErasureCodingWorker(getConf(), this);
     blockRecoveryWorker = new BlockRecoveryWorker(this);
 
     blockPoolManager = new BlockPoolManager(this);
-    blockPoolManager.refreshNamenodes(conf);
+    blockPoolManager.refreshNamenodes(getConf());
 
     // Create the ReadaheadPool from the DataNode context so we can
     // exit without having to explicitly shutdown its thread pool.
     readaheadPool = ReadaheadPool.getInstance();
-    saslClient = new SaslDataTransferClient(dnConf.conf, 
+    saslClient = new SaslDataTransferClient(dnConf.getConf(),
         dnConf.saslPropsResolver, dnConf.trustedChannelResolver);
     saslServer = new SaslDataTransferServer(dnConf, blockPoolTokenSecretManager);
-    startMetricsLogger(conf);
+    startMetricsLogger();
   }
 
   /**
@@ -1592,10 +1594,10 @@ public class DataNode extends ReconfigurableBase
     // failures.
     checkDiskError();
 
-    data.addBlockPool(nsInfo.getBlockPoolID(), conf);
+    data.addBlockPool(nsInfo.getBlockPoolID(), getConf());
     blockScanner.enableBlockPoolId(bpos.getBlockPoolId());
-    initDirectoryScanner(conf);
-    initDiskBalancer(data, conf);
+    initDirectoryScanner(getConf());
+    initDiskBalancer(data, getConf());
   }
 
   List<BPOfferService> getAllBpOs() {
@@ -1616,10 +1618,10 @@ public class DataNode extends ReconfigurableBase
    */
   private void initStorage(final NamespaceInfo nsInfo) throws IOException {
     final FsDatasetSpi.Factory<? extends FsDatasetSpi<?>> factory
-        = FsDatasetSpi.Factory.getFactory(conf);
+        = FsDatasetSpi.Factory.getFactory(getConf());
     
     if (!factory.isSimulated()) {
-      final StartupOption startOpt = getStartupOption(conf);
+      final StartupOption startOpt = getStartupOption(getConf());
       if (startOpt == null) {
         throw new IOException("Startup option not set.");
       }
@@ -1639,7 +1641,7 @@ public class DataNode extends ReconfigurableBase
 
     synchronized(this)  {
       if (data == null) {
-        data = factory.newInstance(this, storage, conf);
+        data = factory.newInstance(this, storage, getConf());
       }
     }
   }
@@ -1720,7 +1722,7 @@ public class DataNode extends ReconfigurableBase
    */
   DatanodeProtocolClientSideTranslatorPB connectToNN(
       InetSocketAddress nnAddr) throws IOException {
-    return new DatanodeProtocolClientSideTranslatorPB(nnAddr, conf);
+    return new DatanodeProtocolClientSideTranslatorPB(nnAddr, getConf());
   }
 
   /**
@@ -1733,7 +1735,7 @@ public class DataNode extends ReconfigurableBase
   DatanodeLifelineProtocolClientSideTranslatorPB connectToLifelineNN(
       InetSocketAddress lifelineNnAddr) throws IOException {
     return new DatanodeLifelineProtocolClientSideTranslatorPB(lifelineNnAddr,
-        conf);
+        getConf());
   }
 
   public static InterDatanodeProtocol createInterDataNodeProtocolProxy(
@@ -2388,7 +2390,7 @@ public class DataNode extends ReconfigurableBase
         unbufIn = saslStreams.in;
         
         out = new DataOutputStream(new BufferedOutputStream(unbufOut,
-            DFSUtilClient.getSmallBufferSize(conf)));
+            DFSUtilClient.getSmallBufferSize(getConf())));
         in = new DataInputStream(unbufIn);
         blockSender = new BlockSender(b, 0, b.getNumBytes(), 
             false, false, true, DataNode.this, null, cachingStrategy);
@@ -2508,7 +2510,7 @@ public class DataNode extends ReconfigurableBase
     }
     ipcServer.setTracer(tracer);
     ipcServer.start();
-    startPlugins(conf);
+    startPlugins(getConf());
   }
 
   /**
@@ -3051,8 +3053,8 @@ public class DataNode extends ReconfigurableBase
   @Override // ClientDatanodeProtocol
   public void refreshNamenodes() throws IOException {
     checkSuperuserPrivilege();
-    conf = new Configuration();
-    refreshNamenodes(conf);
+    setConf(new Configuration());
+    refreshNamenodes(getConf());
   }
   
   @Override // ClientDatanodeProtocol
@@ -3327,8 +3329,8 @@ public class DataNode extends ReconfigurableBase
                            Token<BlockTokenIdentifier> blockToken)
       throws IOException {
 
-    return DFSUtilClient.connectToDN(datanodeID, timeout, conf, saslClient,
-        NetUtils.getDefaultSocketFactory(getConf()), false,
+    return DFSUtilClient.connectToDN(datanodeID, timeout, getConf(),
+        saslClient, NetUtils.getDefaultSocketFactory(getConf()), false,
         getDataEncryptionKeyFactoryForBlock(block), blockToken);
   }
 
@@ -3341,7 +3343,7 @@ public class DataNode extends ReconfigurableBase
     final int numOobTypes = oobEnd - oobStart + 1;
     oobTimeouts = new long[numOobTypes];
 
-    final String[] ele = conf.get(DFS_DATANODE_OOB_TIMEOUT_KEY,
+    final String[] ele = getConf().get(DFS_DATANODE_OOB_TIMEOUT_KEY,
         DFS_DATANODE_OOB_TIMEOUT_DEFAULT).split(",");
     for (int i = 0; i < numOobTypes; i++) {
       oobTimeouts[i] = (i < ele.length) ? Long.parseLong(ele[i]) : 0;
@@ -3367,10 +3369,9 @@ public class DataNode extends ReconfigurableBase
    * Start a timer to periodically write DataNode metrics to the log file. This
    * behavior can be disabled by configuration.
    *
-   * @param metricConf
    */
-  protected void startMetricsLogger(Configuration metricConf) {
-    long metricsLoggerPeriodSec = metricConf.getInt(
+  protected void startMetricsLogger() {
+    long metricsLoggerPeriodSec = getConf().getInt(
         DFS_DATANODE_METRICS_LOGGER_PERIOD_SECONDS_KEY,
         DFS_DATANODE_METRICS_LOGGER_PERIOD_SECONDS_DEFAULT);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ec3ea188/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java
index 29db702..2d50c75 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java
@@ -116,7 +116,7 @@ public class TestBPOfferService {
     File dnDataDir = new File(new File(TEST_BUILD_DATA, "dfs"), "data");
     conf.set(DFS_DATANODE_DATA_DIR_KEY, dnDataDir.toURI().toString());
     Mockito.doReturn(conf).when(mockDn).getConf();
-    Mockito.doReturn(new DNConf(conf)).when(mockDn).getDnConf();
+    Mockito.doReturn(new DNConf(mockDn)).when(mockDn).getDnConf();
     Mockito.doReturn(DataNodeMetrics.create(conf, "fake dn"))
     .when(mockDn).getMetrics();
 
@@ -338,7 +338,7 @@ public class TestBPOfferService {
       new File(TEST_BUILD_DATA, "testBPInitErrorHandling"), "data");
     conf.set(DFS_DATANODE_DATA_DIR_KEY, dnDataDir.toURI().toString());
     Mockito.doReturn(conf).when(mockDn).getConf();
-    Mockito.doReturn(new DNConf(conf)).when(mockDn).getDnConf();
+    Mockito.doReturn(new DNConf(mockDn)).when(mockDn).getDnConf();
     Mockito.doReturn(DataNodeMetrics.create(conf, "fake dn")).
       when(mockDn).getMetrics();
     final AtomicInteger count = new AtomicInteger();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ec3ea188/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataXceiverLazyPersistHint.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataXceiverLazyPersistHint.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataXceiverLazyPersistHint.java
index c21cc86..b2bfe49 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataXceiverLazyPersistHint.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataXceiverLazyPersistHint.java
@@ -171,12 +171,13 @@ public class TestDataXceiverLazyPersistHint {
     conf.setBoolean(
         DFS_DATANODE_NON_LOCAL_LAZY_PERSIST,
         nonLocalLazyPersist == NonLocalLazyPersist.ALLOWED);
-    DNConf dnConf = new DNConf(conf);
+
     DatanodeRegistration mockDnReg = mock(DatanodeRegistration.class);
     DataNodeMetrics mockMetrics = mock(DataNodeMetrics.class);
     DataNode mockDn = mock(DataNode.class);
-    when(mockDn.getDnConf()).thenReturn(dnConf);
     when(mockDn.getConf()).thenReturn(conf);
+    DNConf dnConf = new DNConf(mockDn);
+    when(mockDn.getDnConf()).thenReturn(dnConf);
     when(mockDn.getMetrics()).thenReturn(mockMetrics);
     when(mockDn.getDNRegistrationForBP("Dummy-pool")).thenReturn(mockDnReg);
     return mockDn;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ec3ea188/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java
index a330fbf..179b617 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java
@@ -163,11 +163,11 @@ public class TestFsDatasetImpl {
     storage = mock(DataStorage.class);
     this.conf = new Configuration();
     this.conf.setLong(DFS_DATANODE_SCAN_PERIOD_HOURS_KEY, 0);
-    final DNConf dnConf = new DNConf(conf);
 
     when(datanode.getConf()).thenReturn(conf);
+    final DNConf dnConf = new DNConf(datanode);
     when(datanode.getDnConf()).thenReturn(dnConf);
-    final BlockScanner disabledBlockScanner = new BlockScanner(datanode, conf);
+    final BlockScanner disabledBlockScanner = new BlockScanner(datanode);
     when(datanode.getBlockScanner()).thenReturn(disabledBlockScanner);
     final ShortCircuitRegistry shortCircuitRegistry =
         new ShortCircuitRegistry(conf);
@@ -326,7 +326,7 @@ public class TestFsDatasetImpl {
     RoundRobinVolumeChoosingPolicy<FsVolumeImpl> blockChooser =
         new RoundRobinVolumeChoosingPolicy<>();
     conf.setLong(DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_KEY, -1);
-    final BlockScanner blockScanner = new BlockScanner(datanode, conf);
+    final BlockScanner blockScanner = new BlockScanner(datanode);
     final FsVolumeList volumeList = new FsVolumeList(
         Collections.<VolumeFailureInfo>emptyList(), blockScanner, blockChooser);
     final List<FsVolumeImpl> oldVolumes = new ArrayList<>();


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


[26/44] 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/7d8b1146/hadoop-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/7d8b1146/hadoop-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/7d8b1146/hadoop-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/7d8b1146/hadoop-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/7d8b1146/hadoop-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/7d8b1146/hadoop-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/7d8b1146/hadoop-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/7d8b1146/hadoop-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/7d8b1146/hadoop-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/7d8b1146/hadoop-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/7d8b1146/hadoop-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/7d8b1146/hadoop-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/7d8b1146/hadoop-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/7d8b1146/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/7d8b1146/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/7d8b1146/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


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

Branch: refs/heads/YARN-2915
Commit: eb4c9673728ea7cca9691c5cf1ccf25c5fe4cc01
Parents: 1f26454
Author: Subru Krishnan <su...@apache.org>
Authored: Fri Aug 5 11:52:44 2016 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon Sep 19 12:58:34 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/eb4c9673/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/eb4c9673/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/eb4c9673/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/eb4c9673/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/eb4c9673/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/eb4c9673/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/eb4c9673/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/eb4c9673/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/eb4c9673/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/eb4c9673/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/eb4c9673/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/eb4c9673/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/eb4c9673/hadoop-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/eb4c9673/hadoop-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/eb4c9673/hadoop-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/eb4c9673/hadoop-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/eb4c9673/hadoop-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/eb4c9673/hadoop-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/eb4c9673/hadoop-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/eb4c9673/hadoop-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