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

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

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

Branch: refs/heads/YARN-2915
Commit: 5b109eb3ac7390431750fd39062f9486452c1220
Parents: 7e46cf0
Author: Subru Krishnan <su...@apache.org>
Authored: Fri Aug 5 12:34:58 2016 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Wed Jun 7 14:46:40 2017 -0700

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


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