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 jh...@apache.org on 2017/08/17 20:57:43 UTC

[01/50] [abbrv] hadoop git commit: HADOOP-14741. Refactor curator based ZooKeeper communication into common library. (Íñigo Goiri via Subru). [Forced Update!]

Repository: hadoop
Updated Branches:
  refs/heads/YARN-5734 96063d5a6 -> 0fd690498 (forced update)


HADOOP-14741. Refactor curator based ZooKeeper communication into common library. (Íñigo Goiri via Subru).


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

Branch: refs/heads/YARN-5734
Commit: bbbf0e2a4136b30cad9dfd36ef138650a1adea60
Parents: 8c4b6d1
Author: Subru Krishnan <su...@apache.org>
Authored: Fri Aug 11 13:58:45 2017 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Fri Aug 11 13:58:45 2017 -0700

----------------------------------------------------------------------
 .../hadoop/fs/CommonConfigurationKeys.java      |  21 ++
 .../hadoop/util/curator/ZKCuratorManager.java   | 294 +++++++++++++++++++
 .../hadoop/util/curator/package-info.java       |  27 ++
 .../src/main/resources/core-default.xml         |  46 +++
 .../util/curator/TestZKCuratorManager.java      |  95 ++++++
 .../hadoop/yarn/conf/YarnConfiguration.java     |  13 +-
 .../yarn/conf/TestYarnConfigurationFields.java  |   9 +
 .../src/main/resources/yarn-default.xml         |  53 ----
 ...ActiveStandbyElectorBasedElectorService.java |   5 +-
 .../yarn/server/resourcemanager/RMZKUtils.java  |  81 -----
 .../server/resourcemanager/ResourceManager.java |  83 +++---
 .../recovery/ZKRMStateStore.java                |  38 ++-
 .../server/resourcemanager/RMHATestBase.java    |   5 +-
 13 files changed, 567 insertions(+), 203 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/bbbf0e2a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
index e53f71e..0da4bbd 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
@@ -377,4 +377,25 @@ public class CommonConfigurationKeys extends CommonConfigurationKeysPublic {
 
   // HDFS client HTrace configuration.
   public static final String  FS_CLIENT_HTRACE_PREFIX = "fs.client.htrace.";
+
+  // Global ZooKeeper configuration keys
+  public static final String ZK_PREFIX = "hadoop.zk.";
+  /** ACL for the ZooKeeper ensemble. */
+  public static final String ZK_ACL = ZK_PREFIX + "acl";
+  public static final String ZK_ACL_DEFAULT = "world:anyone:rwcda";
+  /** Authentication for the ZooKeeper ensemble. */
+  public static final String ZK_AUTH = ZK_PREFIX + "auth";
+
+  /** Address of the ZooKeeper ensemble. */
+  public static final String ZK_ADDRESS = ZK_PREFIX + "address";
+  /** Maximum number of retries for a ZooKeeper operation. */
+  public static final String ZK_NUM_RETRIES = ZK_PREFIX + "num-retries";
+  public static final int    ZK_NUM_RETRIES_DEFAULT = 1000;
+  /** Timeout for a ZooKeeper operation in ZooKeeper in milliseconds. */
+  public static final String ZK_TIMEOUT_MS = ZK_PREFIX + "timeout-ms";
+  public static final int    ZK_TIMEOUT_MS_DEFAULT = 10000;
+  /** How often to retry a ZooKeeper operation  in milliseconds. */
+  public static final String ZK_RETRY_INTERVAL_MS =
+      ZK_PREFIX + "retry-interval-ms";
+  public static final int    ZK_RETRY_INTERVAL_MS_DEFAULT = 1000;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bbbf0e2a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/curator/ZKCuratorManager.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/curator/ZKCuratorManager.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/curator/ZKCuratorManager.java
new file mode 100644
index 0000000..3adf028
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/curator/ZKCuratorManager.java
@@ -0,0 +1,294 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.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.util.curator;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.curator.framework.AuthInfo;
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.retry.RetryNTimes;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.util.ZKUtil;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.data.ACL;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Helper class that provides utility methods specific to ZK operations.
+ */
+@InterfaceAudience.Private
+public final class ZKCuratorManager {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ZKCuratorManager.class);
+
+  /** Configuration for the ZooKeeper connection. */
+  private final Configuration conf;
+
+  /** Curator for ZooKeeper. */
+  private CuratorFramework curator;
+
+
+  public ZKCuratorManager(Configuration config) throws IOException {
+    this.conf = config;
+  }
+
+  /**
+   * Get the curator framework managing the ZooKeeper connection.
+   * @return Curator framework.
+   */
+  public CuratorFramework getCurator() {
+    return curator;
+  }
+
+  /**
+   * Close the connection with ZooKeeper.
+   */
+  public void close() {
+    if (curator != null) {
+      curator.close();
+    }
+  }
+
+  /**
+   * Utility method to fetch the ZK ACLs from the configuration.
+   * @throws java.io.IOException if the Zookeeper ACLs configuration file
+   * cannot be read
+   */
+  public static List<ACL> getZKAcls(Configuration conf) throws IOException {
+    // Parse authentication from configuration.
+    String zkAclConf = conf.get(CommonConfigurationKeys.ZK_ACL,
+        CommonConfigurationKeys.ZK_ACL_DEFAULT);
+    try {
+      zkAclConf = ZKUtil.resolveConfIndirection(zkAclConf);
+      return ZKUtil.parseACLs(zkAclConf);
+    } catch (IOException | ZKUtil.BadAclFormatException e) {
+      LOG.error("Couldn't read ACLs based on {}",
+          CommonConfigurationKeys.ZK_ACL);
+      throw e;
+    }
+  }
+
+  /**
+   * Utility method to fetch ZK auth info from the configuration.
+   * @throws java.io.IOException if the Zookeeper ACLs configuration file
+   * cannot be read
+   */
+  public static List<ZKUtil.ZKAuthInfo> getZKAuths(Configuration conf)
+      throws IOException {
+    String zkAuthConf = conf.get(CommonConfigurationKeys.ZK_AUTH);
+    try {
+      zkAuthConf = ZKUtil.resolveConfIndirection(zkAuthConf);
+      if (zkAuthConf != null) {
+        return ZKUtil.parseAuth(zkAuthConf);
+      } else {
+        return Collections.emptyList();
+      }
+    } catch (IOException | ZKUtil.BadAuthFormatException e) {
+      LOG.error("Couldn't read Auth based on {}",
+          CommonConfigurationKeys.ZK_AUTH);
+      throw e;
+    }
+  }
+
+  /**
+   * Start the connection to the ZooKeeper ensemble.
+   * @param conf Configuration for the connection.
+   * @throws IOException If the connection cannot be started.
+   */
+  public void start() throws IOException {
+    this.start(new ArrayList<>());
+  }
+
+  /**
+   * Start the connection to the ZooKeeper ensemble.
+   * @param conf Configuration for the connection.
+   * @param authInfos List of authentication keys.
+   * @throws IOException If the connection cannot be started.
+   */
+  public void start(List<AuthInfo> authInfos) throws IOException {
+
+    // Connect to the ZooKeeper ensemble
+    String zkHostPort = conf.get(CommonConfigurationKeys.ZK_ADDRESS);
+    if (zkHostPort == null) {
+      throw new IOException(
+          CommonConfigurationKeys.ZK_ADDRESS + " is not configured.");
+    }
+    int numRetries = conf.getInt(CommonConfigurationKeys.ZK_NUM_RETRIES,
+        CommonConfigurationKeys.ZK_NUM_RETRIES_DEFAULT);
+    int zkSessionTimeout = conf.getInt(CommonConfigurationKeys.ZK_TIMEOUT_MS,
+        CommonConfigurationKeys.ZK_TIMEOUT_MS_DEFAULT);
+    int zkRetryInterval = conf.getInt(
+        CommonConfigurationKeys.ZK_RETRY_INTERVAL_MS,
+        CommonConfigurationKeys.ZK_RETRY_INTERVAL_MS_DEFAULT);
+    RetryNTimes retryPolicy = new RetryNTimes(numRetries, zkRetryInterval);
+
+    // Set up ZK auths
+    List<ZKUtil.ZKAuthInfo> zkAuths = getZKAuths(conf);
+    if (authInfos == null) {
+      authInfos = new ArrayList<>();
+    }
+    for (ZKUtil.ZKAuthInfo zkAuth : zkAuths) {
+      authInfos.add(new AuthInfo(zkAuth.getScheme(), zkAuth.getAuth()));
+    }
+
+    CuratorFramework client = CuratorFrameworkFactory.builder()
+        .connectString(zkHostPort)
+        .sessionTimeoutMs(zkSessionTimeout)
+        .retryPolicy(retryPolicy)
+        .authorization(authInfos)
+        .build();
+    client.start();
+
+    this.curator = client;
+  }
+
+  /**
+   * Get ACLs for a ZNode.
+   * @param path Path of the ZNode.
+   * @return The list of ACLs.
+   * @throws Exception
+   */
+  public List<ACL> getACL(final String path) throws Exception {
+    return curator.getACL().forPath(path);
+  }
+
+  /**
+   * Get the data in a ZNode.
+   * @param path Path of the ZNode.
+   * @param stat Output statistics of the ZNode.
+   * @return The data in the ZNode.
+   * @throws Exception If it cannot contact Zookeeper.
+   */
+  public byte[] getData(final String path) throws Exception {
+    return curator.getData().forPath(path);
+  }
+
+  /**
+   * Get the data in a ZNode.
+   * @param path Path of the ZNode.
+   * @param stat Output statistics of the ZNode.
+   * @return The data in the ZNode.
+   * @throws Exception If it cannot contact Zookeeper.
+   */
+  public String getSringData(final String path) throws Exception {
+    byte[] bytes = getData(path);
+    return new String(bytes, Charset.forName("UTF-8"));
+  }
+
+  /**
+   * Set data into a ZNode.
+   * @param path Path of the ZNode.
+   * @param data Data to set.
+   * @param version Version of the data to store.
+   * @throws Exception If it cannot contact Zookeeper.
+   */
+  public void setData(String path, byte[] data, int version) throws Exception {
+    curator.setData().withVersion(version).forPath(path, data);
+  }
+
+  /**
+   * Set data into a ZNode.
+   * @param path Path of the ZNode.
+   * @param data Data to set as String.
+   * @param version Version of the data to store.
+   * @throws Exception If it cannot contact Zookeeper.
+   */
+  public void setData(String path, String data, int version) throws Exception {
+    byte[] bytes = data.getBytes(Charset.forName("UTF-8"));
+    setData(path, bytes, version);
+  }
+
+  /**
+   * Get children of a ZNode.
+   * @param path Path of the ZNode.
+   * @return The list of children.
+   * @throws Exception If it cannot contact Zookeeper.
+   */
+  public List<String> getChildren(final String path) throws Exception {
+    return curator.getChildren().forPath(path);
+  }
+
+  /**
+   * Check if a ZNode exists.
+   * @param path Path of the ZNode.
+   * @return If the ZNode exists.
+   * @throws Exception If it cannot contact Zookeeper.
+   */
+  public boolean exists(final String path) throws Exception {
+    return curator.checkExists().forPath(path) != null;
+  }
+
+  /**
+   * Create a ZNode.
+   * @param path Path of the ZNode.
+   * @return If the ZNode was created.
+   * @throws Exception If it cannot contact Zookeeper.
+   */
+  public boolean create(final String path) throws Exception {
+    return create(path, null);
+  }
+
+  /**
+   * Create a ZNode.
+   * @param path Path of the ZNode.
+   * @param zkAcl ACL for the node.
+   * @return If the ZNode was created.
+   * @throws Exception If it cannot contact Zookeeper.
+   */
+  public boolean create(final String path, List<ACL> zkAcl) throws Exception {
+    boolean created = false;
+    if (!exists(path)) {
+      curator.create()
+          .withMode(CreateMode.PERSISTENT)
+          .withACL(zkAcl)
+          .forPath(path, null);
+      created = true;
+    }
+    return created;
+  }
+
+  /**
+   * Delete a ZNode.
+   * @param path Path of the ZNode.
+   * @throws Exception If it cannot contact ZooKeeper.
+   */
+  public void delete(final String path) throws Exception {
+    if (exists(path)) {
+      curator.delete().deletingChildrenIfNeeded().forPath(path);
+    }
+  }
+
+  /**
+   * Get the path for a ZNode.
+   * @param root Root of the ZNode.
+   * @param nodeName Name of the ZNode.
+   * @return Path for the ZNode.
+   */
+  public static String getNodePath(String root, String nodeName) {
+    return root + "/" + nodeName;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bbbf0e2a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/curator/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/curator/package-info.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/curator/package-info.java
new file mode 100644
index 0000000..7b93f5a
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/curator/package-info.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.
+ */
+
+/**
+ * This package provides utilities to interact with Curator ZooKeeper.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+package org.apache.hadoop.util.curator;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bbbf0e2a/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 e6b6919..ffcab2c 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
@@ -2675,4 +2675,50 @@
       This determines the number of open file handles.
     </description>
   </property>
+
+  <property>
+    <description>Host:Port of the ZooKeeper server to be used.
+    </description>
+    <name>hadoop.zk.address</name>
+    <!--value>127.0.0.1:2181</value-->
+  </property>
+
+  <property>
+    <description>Number of tries to connect to ZooKeeper.</description>
+    <name>hadoop.zk.num-retries</name>
+    <value>1000</value>
+  </property>
+
+  <property>
+    <description>Retry interval in milliseconds when connecting to ZooKeeper.
+    </description>
+    <name>hadoop.zk.retry-interval-ms</name>
+    <value>1000</value>
+  </property>
+
+  <property>
+    <description>ZooKeeper session timeout in milliseconds. Session expiration
+    is managed by the ZooKeeper cluster itself, not by the client. This value is
+    used by the cluster to determine when the client's session expires.
+    Expirations happens when the cluster does not hear from the client within
+    the specified session timeout period (i.e. no heartbeat).</description>
+    <name>hadoop.zk.timeout-ms</name>
+    <value>10000</value>
+  </property>
+
+  <property>
+    <description>ACL's to be used for ZooKeeper znodes.</description>
+    <name>hadoop.zk.acl</name>
+    <value>world:anyone:rwcda</value>
+  </property>
+
+  <property>
+    <description>
+        Specify the auths to be used for the ACL's specified in hadoop.zk.acl.
+        This takes a comma-separated list of authentication mechanisms, each of the
+        form 'scheme:auth' (the same syntax used for the 'addAuth' command in
+        the ZK CLI).
+    </description>
+    <name>hadoop.zk.auth</name>
+  </property>
 </configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bbbf0e2a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/curator/TestZKCuratorManager.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/curator/TestZKCuratorManager.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/curator/TestZKCuratorManager.java
new file mode 100644
index 0000000..2bcf508
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/curator/TestZKCuratorManager.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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.util.curator;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.List;
+
+import org.apache.curator.test.TestingServer;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test the manager for ZooKeeper Curator.
+ */
+public class TestZKCuratorManager {
+
+  private TestingServer server;
+  private ZKCuratorManager curator;
+
+  @Before
+  public void setup() throws Exception {
+    this.server = new TestingServer();
+
+    Configuration conf = new Configuration();
+    conf.set(
+        CommonConfigurationKeys.ZK_ADDRESS, this.server.getConnectString());
+
+    this.curator = new ZKCuratorManager(conf);
+    this.curator.start();
+  }
+
+  @After
+  public void teardown() throws Exception {
+    this.curator.close();
+    if (this.server != null) {
+      this.server.close();
+      this.server = null;
+    }
+  }
+
+  @Test
+  public void testReadWriteData() throws Exception {
+    String testZNode = "/test";
+    String expectedString = "testString";
+    assertFalse(curator.exists(testZNode));
+    curator.create(testZNode);
+    assertTrue(curator.exists(testZNode));
+    curator.setData(testZNode, expectedString, -1);
+    String testString = curator.getSringData("/test");
+    assertEquals(expectedString, testString);
+  }
+
+  @Test
+  public void testChildren() throws Exception {
+    List<String> children = curator.getChildren("/");
+    assertEquals(1, children.size());
+
+    assertFalse(curator.exists("/node1"));
+    curator.create("/node1");
+    assertTrue(curator.exists("/node1"));
+
+    assertFalse(curator.exists("/node2"));
+    curator.create("/node2");
+    assertTrue(curator.exists("/node2"));
+
+    children = curator.getChildren("/");
+    assertEquals(3, children.size());
+
+    curator.delete("/node2");
+    assertFalse(curator.exists("/node2"));
+    children = curator.getChildren("/");
+    assertEquals(2, children.size());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bbbf0e2a/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 71a7134..cd4d569 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
@@ -29,6 +29,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.ha.ActiveStandbyElector;
 import org.apache.hadoop.http.HttpConfig;
 import org.apache.hadoop.net.NetUtils;
@@ -87,7 +88,17 @@ public class YarnConfiguration extends Configuration {
     });
     Configuration.addDeprecations(new DeprecationDelta[] {
         new DeprecationDelta(RM_SYSTEM_METRICS_PUBLISHER_ENABLED,
-            SYSTEM_METRICS_PUBLISHER_ENABLED)
+            SYSTEM_METRICS_PUBLISHER_ENABLED),
+        new DeprecationDelta(RM_ZK_ACL, CommonConfigurationKeys.ZK_ACL),
+        new DeprecationDelta(RM_ZK_AUTH, CommonConfigurationKeys.ZK_AUTH),
+        new DeprecationDelta(RM_ZK_ADDRESS,
+            CommonConfigurationKeys.ZK_ADDRESS),
+        new DeprecationDelta(RM_ZK_NUM_RETRIES,
+            CommonConfigurationKeys.ZK_NUM_RETRIES),
+        new DeprecationDelta(RM_ZK_TIMEOUT_MS,
+            CommonConfigurationKeys.ZK_TIMEOUT_MS),
+        new DeprecationDelta(RM_ZK_RETRY_INTERVAL_MS,
+            CommonConfigurationKeys.ZK_RETRY_INTERVAL_MS),
     });
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bbbf0e2a/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 a035dc7..b9ad31a 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
@@ -121,6 +121,15 @@ public class TestYarnConfigurationFields extends TestConfigurationFieldsBase {
     configurationPropsToSkipCompare
         .add(YarnConfiguration.RM_SYSTEM_METRICS_PUBLISHER_ENABLED);
 
+    // skip deprecated ZooKeeper settings
+    configurationPropsToSkipCompare.add(YarnConfiguration.RM_ZK_ADDRESS);
+    configurationPropsToSkipCompare.add(YarnConfiguration.RM_ZK_NUM_RETRIES);
+    configurationPropsToSkipCompare.add(YarnConfiguration.RM_ZK_TIMEOUT_MS);
+    configurationPropsToSkipCompare.add(
+        YarnConfiguration.RM_ZK_RETRY_INTERVAL_MS);
+    configurationPropsToSkipCompare.add(YarnConfiguration.RM_ZK_AUTH);
+    configurationPropsToSkipCompare.add(YarnConfiguration.RM_ZK_ACL);
+
     // Used as Java command line properties, not XML
     configurationPrefixToSkipCompare.add("yarn.app.container");
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bbbf0e2a/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 000e892..dbf115b 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
@@ -452,31 +452,6 @@
   </property>
 
   <property>
-    <description>Host:Port of the ZooKeeper server to be used by the RM. This
-      must be supplied when using the ZooKeeper based implementation of the
-      RM state store and/or embedded automatic failover in an HA setting.
-    </description>
-    <name>yarn.resourcemanager.zk-address</name>
-    <!--value>127.0.0.1:2181</value-->
-  </property>
-
-  <property>
-    <description>Number of times RM tries to connect to ZooKeeper.</description>
-    <name>yarn.resourcemanager.zk-num-retries</name>
-    <value>1000</value>
-  </property>
-
-  <property>
-    <description>Retry interval in milliseconds when connecting to ZooKeeper.
-      When HA is enabled, the value here is NOT used. It is generated
-      automatically from yarn.resourcemanager.zk-timeout-ms and
-      yarn.resourcemanager.zk-num-retries.
-    </description>
-    <name>yarn.resourcemanager.zk-retry-interval-ms</name>
-    <value>1000</value>
-  </property>
-
-  <property>
     <description>Full path of the ZooKeeper znode where RM state will be
     stored. This must be supplied when using
     org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore
@@ -486,22 +461,6 @@
   </property>
 
   <property>
-    <description>ZooKeeper session timeout in milliseconds. Session expiration
-    is managed by the ZooKeeper cluster itself, not by the client. This value is
-    used by the cluster to determine when the client's session expires.
-    Expirations happens when the cluster does not hear from the client within
-    the specified session timeout period (i.e. no heartbeat).</description>
-    <name>yarn.resourcemanager.zk-timeout-ms</name>
-    <value>10000</value>
-  </property>
-
-  <property>
-    <description>ACL's to be used for ZooKeeper znodes.</description>
-    <name>yarn.resourcemanager.zk-acl</name>
-    <value>world:anyone:rwcda</value>
-  </property>
-
-  <property>
     <description>
       ACLs to be used for the root znode when using ZKRMStateStore in an HA
       scenario for fencing.
@@ -527,18 +486,6 @@
   </property>
 
   <property>
-    <description>
-        Specify the auths to be used for the ACL's specified in both the
-        yarn.resourcemanager.zk-acl and
-        yarn.resourcemanager.zk-state-store.root-node.acl properties.  This
-        takes a comma-separated list of authentication mechanisms, each of the
-        form 'scheme:auth' (the same syntax used for the 'addAuth' command in
-        the ZK CLI).
-    </description>
-    <name>yarn.resourcemanager.zk-auth</name>
-  </property>
-
-  <property>
     <description>URI pointing to the location of the FileSystem path where
     RM state will be stored. This must be supplied when using
     org.apache.hadoop.yarn.server.resourcemanager.recovery.FileSystemRMStateStore

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bbbf0e2a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ActiveStandbyElectorBasedElectorService.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/ActiveStandbyElectorBasedElectorService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ActiveStandbyElectorBasedElectorService.java
index a8dcda4..c5c9211 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ActiveStandbyElectorBasedElectorService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ActiveStandbyElectorBasedElectorService.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.ha.ServiceFailedException;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.ZKUtil;
+import org.apache.hadoop.util.curator.ZKCuratorManager;
 import org.apache.hadoop.yarn.conf.HAUtil;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
@@ -96,8 +97,8 @@ public class ActiveStandbyElectorBasedElectorService extends AbstractService
     zkSessionTimeout = conf.getLong(YarnConfiguration.RM_ZK_TIMEOUT_MS,
         YarnConfiguration.DEFAULT_RM_ZK_TIMEOUT_MS);
 
-    List<ACL> zkAcls = RMZKUtils.getZKAcls(conf);
-    List<ZKUtil.ZKAuthInfo> zkAuths = RMZKUtils.getZKAuths(conf);
+    List<ACL> zkAcls = ZKCuratorManager.getZKAcls(conf);
+    List<ZKUtil.ZKAuthInfo> zkAuths = ZKCuratorManager.getZKAuths(conf);
 
     int maxRetryNum =
         conf.getInt(YarnConfiguration.RM_HA_FC_ELECTOR_ZK_RETRIES_KEY, conf

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bbbf0e2a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMZKUtils.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/RMZKUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMZKUtils.java
deleted file mode 100644
index 4b8561d..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMZKUtils.java
+++ /dev/null
@@ -1,81 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.server.resourcemanager;
-
-import java.io.IOException;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.util.ZKUtil;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.zookeeper.data.ACL;
-
-import java.util.Collections;
-import java.util.List;
-
-/**
- * Helper class that provides utility methods specific to ZK operations
- */
-@InterfaceAudience.Private
-public class RMZKUtils {
-  private static final Log LOG = LogFactory.getLog(RMZKUtils.class);
-
-  /**
-   * Utility method to fetch the ZK ACLs from the configuration.
-   *
-   * @throws java.io.IOException if the Zookeeper ACLs configuration file
-   * cannot be read
-   */
-  public static List<ACL> getZKAcls(Configuration conf) throws IOException {
-    // Parse authentication from configuration.
-    String zkAclConf =
-        conf.get(YarnConfiguration.RM_ZK_ACL,
-            YarnConfiguration.DEFAULT_RM_ZK_ACL);
-    try {
-      zkAclConf = ZKUtil.resolveConfIndirection(zkAclConf);
-      return ZKUtil.parseACLs(zkAclConf);
-    } catch (IOException | ZKUtil.BadAclFormatException e) {
-      LOG.error("Couldn't read ACLs based on " + YarnConfiguration.RM_ZK_ACL);
-      throw e;
-    }
-  }
-
-  /**
-   * Utility method to fetch ZK auth info from the configuration.
-   *
-   * @throws java.io.IOException if the Zookeeper ACLs configuration file
-   * cannot be read
-   */
-  public static List<ZKUtil.ZKAuthInfo> getZKAuths(Configuration conf)
-      throws IOException {
-    String zkAuthConf = conf.get(YarnConfiguration.RM_ZK_AUTH);
-    try {
-      zkAuthConf = ZKUtil.resolveConfIndirection(zkAuthConf);
-      if (zkAuthConf != null) {
-        return ZKUtil.parseAuth(zkAuthConf);
-      } else {
-        return Collections.emptyList();
-      }
-    } catch (IOException | ZKUtil.BadAuthFormatException e) {
-      LOG.error("Couldn't read Auth based on " + YarnConfiguration.RM_ZK_AUTH);
-      throw e;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bbbf0e2a/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 9691885..cb7daf9 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
@@ -22,8 +22,6 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.curator.framework.AuthInfo;
 import org.apache.curator.framework.CuratorFramework;
-import org.apache.curator.framework.CuratorFrameworkFactory;
-import org.apache.curator.retry.RetryNTimes;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.ha.HAServiceProtocol;
@@ -46,7 +44,7 @@ import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.ShutdownHookManager;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.VersionInfo;
-import org.apache.hadoop.util.ZKUtil;
+import org.apache.hadoop.util.curator.ZKCuratorManager;
 import org.apache.hadoop.yarn.YarnUncaughtExceptionHandler;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -192,7 +190,7 @@ public class ResourceManager extends CompositeService implements Recoverable {
   protected ResourceTrackerService resourceTracker;
   private JvmMetrics jvmMetrics;
   private boolean curatorEnabled = false;
-  private CuratorFramework curator;
+  private ZKCuratorManager zkManager;
   private final String zkRootNodePassword =
       Long.toString(new SecureRandom().nextLong());
   private boolean recoveryEnabled;
@@ -345,7 +343,7 @@ public class ResourceManager extends CompositeService implements Recoverable {
         conf.getBoolean(YarnConfiguration.CURATOR_LEADER_ELECTOR,
             YarnConfiguration.DEFAULT_CURATOR_LEADER_ELECTOR_ENABLED);
     if (curatorEnabled) {
-      this.curator = createAndStartCurator(conf);
+      this.zkManager = createAndStartZKManager(conf);
       elector = new CuratorBasedElectorService(this);
     } else {
       elector = new ActiveStandbyElectorBasedElectorService(this);
@@ -353,50 +351,49 @@ public class ResourceManager extends CompositeService implements Recoverable {
     return elector;
   }
 
-  public CuratorFramework createAndStartCurator(Configuration conf)
+  /**
+   * Create and ZooKeeper Curator manager.
+   * @param config Configuration for the ZooKeeper curator.
+   * @return New ZooKeeper Curator manager.
+   * @throws IOException If it cannot create the manager.
+   */
+  public ZKCuratorManager createAndStartZKManager(Configuration config)
       throws IOException {
-    String zkHostPort = conf.get(YarnConfiguration.RM_ZK_ADDRESS);
-    if (zkHostPort == null) {
-      throw new YarnRuntimeException(
-          YarnConfiguration.RM_ZK_ADDRESS + " is not configured.");
-    }
-    int numRetries = conf.getInt(YarnConfiguration.RM_ZK_NUM_RETRIES,
-        YarnConfiguration.DEFAULT_ZK_RM_NUM_RETRIES);
-    int zkSessionTimeout = conf.getInt(YarnConfiguration.RM_ZK_TIMEOUT_MS,
-        YarnConfiguration.DEFAULT_RM_ZK_TIMEOUT_MS);
-    int zkRetryInterval = conf.getInt(YarnConfiguration.RM_ZK_RETRY_INTERVAL_MS,
-        YarnConfiguration.DEFAULT_RM_ZK_RETRY_INTERVAL_MS);
-
-    // set up zk auths
-    List<ZKUtil.ZKAuthInfo> zkAuths = RMZKUtils.getZKAuths(conf);
+    ZKCuratorManager manager = new ZKCuratorManager(config);
+
+    // Get authentication
     List<AuthInfo> authInfos = new ArrayList<>();
-    for (ZKUtil.ZKAuthInfo zkAuth : zkAuths) {
-      authInfos.add(new AuthInfo(zkAuth.getScheme(), zkAuth.getAuth()));
+    if (HAUtil.isHAEnabled(config) && HAUtil.getConfValueForRMInstance(
+        YarnConfiguration.ZK_RM_STATE_STORE_ROOT_NODE_ACL, config) == null) {
+      String zkRootNodeUsername = HAUtil.getConfValueForRMInstance(
+          YarnConfiguration.RM_ADDRESS,
+          YarnConfiguration.DEFAULT_RM_ADDRESS, config);
+      String defaultFencingAuth =
+          zkRootNodeUsername + ":" + zkRootNodePassword;
+      byte[] defaultFencingAuthData =
+          defaultFencingAuth.getBytes(Charset.forName("UTF-8"));
+      String scheme = new DigestAuthenticationProvider().getScheme();
+      AuthInfo authInfo = new AuthInfo(scheme, defaultFencingAuthData);
+      authInfos.add(authInfo);
     }
 
-    if (HAUtil.isHAEnabled(conf) && HAUtil.getConfValueForRMInstance(
-        YarnConfiguration.ZK_RM_STATE_STORE_ROOT_NODE_ACL, conf) == null) {
-      String zkRootNodeUsername = HAUtil
-          .getConfValueForRMInstance(YarnConfiguration.RM_ADDRESS,
-              YarnConfiguration.DEFAULT_RM_ADDRESS, conf);
-      byte[] defaultFencingAuth =
-          (zkRootNodeUsername + ":" + zkRootNodePassword)
-              .getBytes(Charset.forName("UTF-8"));
-      authInfos.add(new AuthInfo(new DigestAuthenticationProvider().getScheme(),
-          defaultFencingAuth));
-    }
+    manager.start(authInfos);
+    return manager;
+  }
 
-    CuratorFramework client =  CuratorFrameworkFactory.builder()
-        .connectString(zkHostPort)
-        .sessionTimeoutMs(zkSessionTimeout)
-        .retryPolicy(new RetryNTimes(numRetries, zkRetryInterval))
-        .authorization(authInfos).build();
-    client.start();
-    return client;
+  /**
+   * Get the ZooKeeper Curator manager.
+   * @return ZooKeeper Curator manager.
+   */
+  public ZKCuratorManager getZKManager() {
+    return this.zkManager;
   }
 
   public CuratorFramework getCurator() {
-    return this.curator;
+    if (this.zkManager == null) {
+      return null;
+    }
+    return this.zkManager.getCurator();
   }
 
   public String getZkRootNodePassword() {
@@ -1264,8 +1261,8 @@ public class ResourceManager extends CompositeService implements Recoverable {
       configurationProvider.close();
     }
     super.serviceStop();
-    if (curator != null) {
-      curator.close();
+    if (zkManager != null) {
+      zkManager.close();
     }
     transitionToStandby(false);
     rmContext.setHAServiceState(HAServiceState.STOPPING);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bbbf0e2a/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 1b3b367..4b6e82c 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
@@ -31,6 +31,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.security.token.delegation.DelegationKey;
 import org.apache.hadoop.util.ZKUtil;
+import org.apache.hadoop.util.curator.ZKCuratorManager;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ReservationId;
@@ -46,7 +47,6 @@ import org.apache.hadoop.yarn.proto.YarnProtos.ReservationAllocationStateProto;
 import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
 import org.apache.hadoop.yarn.server.records.Version;
 import org.apache.hadoop.yarn.server.records.impl.pb.VersionPBImpl;
-import org.apache.hadoop.yarn.server.resourcemanager.RMZKUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.AMRMTokenSecretManagerState;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.ApplicationAttemptStateData;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.ApplicationStateData;
@@ -201,8 +201,8 @@ public class ZKRMStateStore extends RMStateStore {
   private final String zkRootNodeAuthScheme =
       new DigestAuthenticationProvider().getScheme();
 
-  @VisibleForTesting
-  protected CuratorFramework curatorFramework;
+  /** Manager for the ZooKeeper connection. */
+  private ZKCuratorManager zkManager;
 
   /*
    * Indicates different app attempt state store operations.
@@ -298,12 +298,11 @@ public class ZKRMStateStore extends RMStateStore {
       appIdNodeSplitIndex = YarnConfiguration.DEFAULT_ZK_APPID_NODE_SPLIT_INDEX;
     }
 
-    zkAcl = RMZKUtils.getZKAcls(conf);
+    zkAcl = ZKCuratorManager.getZKAcls(conf);
 
     if (HAUtil.isHAEnabled(conf)) {
       String zkRootNodeAclConf = HAUtil.getConfValueForRMInstance
           (YarnConfiguration.ZK_RM_STATE_STORE_ROOT_NODE_ACL, conf);
-
       if (zkRootNodeAclConf != null) {
         zkRootNodeAclConf = ZKUtil.resolveConfIndirection(zkRootNodeAclConf);
 
@@ -330,10 +329,9 @@ public class ZKRMStateStore extends RMStateStore {
     amrmTokenSecretManagerRoot =
         getNodePath(zkRootNodePath, AMRMTOKEN_SECRET_MANAGER_ROOT);
     reservationRoot = getNodePath(zkRootNodePath, RESERVATION_SYSTEM_ROOT);
-    curatorFramework = resourceManager.getCurator();
-
-    if (curatorFramework == null) {
-      curatorFramework = resourceManager.createAndStartCurator(conf);
+    zkManager = resourceManager.getZKManager();
+    if (zkManager == null) {
+      zkManager = resourceManager.createAndStartZKManager(conf);
     }
   }
 
@@ -382,6 +380,7 @@ public class ZKRMStateStore extends RMStateStore {
       logRootNodeAcls("Before setting ACLs'\n");
     }
 
+    CuratorFramework curatorFramework = zkManager.getCurator();
     if (HAUtil.isHAEnabled(getConfig())) {
       curatorFramework.setACL().withACL(zkRootNodeAcl).forPath(zkRootNodePath);
     } else {
@@ -401,6 +400,7 @@ public class ZKRMStateStore extends RMStateStore {
     }
 
     if (!HAUtil.isHAEnabled(getConfig())) {
+      CuratorFramework curatorFramework = zkManager.getCurator();
       IOUtils.closeStream(curatorFramework);
     }
   }
@@ -936,6 +936,7 @@ public class ZKRMStateStore extends RMStateStore {
       }
       safeDelete(appIdRemovePath);
     } else {
+      CuratorFramework curatorFramework = zkManager.getCurator();
       curatorFramework.delete().deletingChildrenIfNeeded().
           forPath(appIdRemovePath);
     }
@@ -1236,38 +1237,32 @@ public class ZKRMStateStore extends RMStateStore {
 
   @VisibleForTesting
   byte[] getData(final String path) throws Exception {
-    return curatorFramework.getData().forPath(path);
+    return zkManager.getData(path);
   }
 
   @VisibleForTesting
   List<ACL> getACL(final String path) throws Exception {
-    return curatorFramework.getACL().forPath(path);
+    return zkManager.getACL(path);
   }
 
   @VisibleForTesting
   List<String> getChildren(final String path) throws Exception {
-    return curatorFramework.getChildren().forPath(path);
+    return zkManager.getChildren(path);
   }
 
   @VisibleForTesting
   boolean exists(final String path) throws Exception {
-    return curatorFramework.checkExists().forPath(path) != null;
+    return zkManager.exists(path);
   }
 
   @VisibleForTesting
   void create(final String path) throws Exception {
-    if (!exists(path)) {
-      curatorFramework.create()
-          .withMode(CreateMode.PERSISTENT).withACL(zkAcl)
-          .forPath(path, null);
-    }
+    zkManager.create(path, zkAcl);
   }
 
   @VisibleForTesting
   void delete(final String path) throws Exception {
-    if (exists(path)) {
-      curatorFramework.delete().deletingChildrenIfNeeded().forPath(path);
-    }
+    zkManager.delete(path);
   }
 
   private void safeCreate(String path, byte[] data, List<ACL> acl,
@@ -1310,6 +1305,7 @@ public class ZKRMStateStore extends RMStateStore {
     private CuratorTransactionFinal transactionFinal;
 
     SafeTransaction() throws Exception {
+      CuratorFramework curatorFramework = zkManager.getCurator();
       CuratorTransaction transaction = curatorFramework.inTransaction();
       transactionFinal = transaction.create()
           .withMode(CreateMode.PERSISTENT).withACL(zkAcl)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bbbf0e2a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/RMHATestBase.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/RMHATestBase.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/RMHATestBase.java
index c95bcdf..4d8b20d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/RMHATestBase.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/RMHATestBase.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.yarn.server.resourcemanager;
 import java.io.IOException;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.ha.ClientBaseWithFixes;
 import org.apache.hadoop.ha.HAServiceProtocol;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
@@ -61,8 +62,8 @@ public abstract class RMHATestBase extends ClientBaseWithFixes{
     configuration.setBoolean(YarnConfiguration.RECOVERY_ENABLED, true);
     configuration.set(YarnConfiguration.RM_STORE,
         ZKRMStateStore.class.getName());
-    configuration.set(YarnConfiguration.RM_ZK_ADDRESS, hostPort);
-    configuration.setInt(YarnConfiguration.RM_ZK_TIMEOUT_MS, ZK_TIMEOUT_MS);
+    configuration.set(CommonConfigurationKeys.ZK_ADDRESS, hostPort);
+    configuration.setInt(CommonConfigurationKeys.ZK_TIMEOUT_MS, ZK_TIMEOUT_MS);
     configuration.setBoolean(YarnConfiguration.AUTO_FAILOVER_ENABLED, false);
     configuration.set(YarnConfiguration.RM_CLUSTER_ID, "test-yarn-cluster");
     int base = 100;


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


[35/50] [abbrv] hadoop git commit: MAPREDUCE-6936. Remove unnecessary dependency of hadoop-yarn-server-common from hadoop-mapreduce-client-common (haibochen via rkanter)

Posted by jh...@apache.org.
MAPREDUCE-6936. Remove unnecessary dependency of hadoop-yarn-server-common from hadoop-mapreduce-client-common (haibochen via rkanter)


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

Branch: refs/heads/YARN-5734
Commit: ab051bd42ee1d7c4d3b7cc71e6b2734a0955e767
Parents: 0acc5e0
Author: Robert Kanter <rk...@apache.org>
Authored: Wed Aug 16 16:14:04 2017 -0700
Committer: Robert Kanter <rk...@apache.org>
Committed: Wed Aug 16 16:14:04 2017 -0700

----------------------------------------------------------------------
 .../hadoop-mapreduce-client-common/pom.xml                       | 4 ----
 1 file changed, 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab051bd4/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/pom.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/pom.xml
index db8ae49..b88b012 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/pom.xml
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/pom.xml
@@ -46,10 +46,6 @@
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-mapreduce-client-core</artifactId>
     </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-yarn-server-common</artifactId>
-    </dependency>
   </dependencies>
 
   <build>


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


[18/50] [abbrv] hadoop git commit: YARN-6917. Queue path is recomputed from scratch on every allocation. Contributed by Eric Payne

Posted by jh...@apache.org.
YARN-6917. Queue path is recomputed from scratch on every allocation. Contributed by Eric Payne


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

Branch: refs/heads/YARN-5734
Commit: 5558792894169425bff054364a1ab4c48b347fb9
Parents: 3325ef6
Author: Jason Lowe <jl...@yahoo-inc.com>
Authored: Mon Aug 14 15:31:34 2017 -0500
Committer: Jason Lowe <jl...@yahoo-inc.com>
Committed: Mon Aug 14 15:31:34 2017 -0500

----------------------------------------------------------------------
 .../resourcemanager/scheduler/capacity/AbstractCSQueue.java  | 8 ++++++++
 .../server/resourcemanager/scheduler/capacity/LeafQueue.java | 5 -----
 .../resourcemanager/scheduler/capacity/ParentQueue.java      | 6 ------
 3 files changed, 8 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/55587928/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.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/AbstractCSQueue.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/AbstractCSQueue.java
index 5fbdead..d7c452a 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/AbstractCSQueue.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/AbstractCSQueue.java
@@ -76,6 +76,7 @@ public abstract class AbstractCSQueue implements CSQueue {
   private static final Log LOG = LogFactory.getLog(AbstractCSQueue.class);  
   volatile CSQueue parent;
   final String queueName;
+  private final String queuePath;
   volatile int numContainers;
   
   final Resource minimumAllocation;
@@ -119,6 +120,8 @@ public abstract class AbstractCSQueue implements CSQueue {
     this.labelManager = cs.getRMContext().getNodeLabelManager();
     this.parent = parent;
     this.queueName = queueName;
+    this.queuePath =
+      ((parent == null) ? "" : (parent.getQueuePath() + ".")) + this.queueName;
     this.resourceCalculator = cs.getResourceCalculator();
     this.activitiesManager = cs.getActivitiesManager();
     
@@ -150,6 +153,11 @@ public abstract class AbstractCSQueue implements CSQueue {
         queueCapacities,
         parent == null ? null : parent.getQueueCapacities());
   }
+
+  @Override
+  public String getQueuePath() {
+    return queuePath;
+  }
   
   @Override
   public float getCapacity() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/55587928/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 2e502b7..d15431e 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
@@ -299,11 +299,6 @@ public class LeafQueue extends AbstractCSQueue {
     }
   }
 
-  @Override
-  public String getQueuePath() {
-    return getParent().getQueuePath() + "." + getQueueName();
-  }
-
   /**
    * Used only by tests.
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/55587928/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 e0baa07..2e48000 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
@@ -199,12 +199,6 @@ public class ParentQueue extends AbstractCSQueue {
   }
 
   @Override
-  public String getQueuePath() {
-    String parentPath = ((parent == null) ? "" : (parent.getQueuePath() + "."));
-    return parentPath + getQueueName();
-  }
-
-  @Override
   public QueueInfo getQueueInfo(
       boolean includeChildQueues, boolean recursive) {
     try {


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


[24/50] [abbrv] hadoop git commit: HDFS-11696. Fix warnings from Spotbugs in hadoop-hdfs. Contributed by Yiqun Lin.

Posted by jh...@apache.org.
HDFS-11696. Fix warnings from Spotbugs in hadoop-hdfs. Contributed by Yiqun Lin.


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

Branch: refs/heads/YARN-5734
Commit: 2e43c28e01fe006210e71aab179527669f6412ed
Parents: 645a8f2
Author: Yiqun Lin <yq...@apache.org>
Authored: Tue Aug 15 16:48:49 2017 +0800
Committer: Yiqun Lin <yq...@apache.org>
Committed: Tue Aug 15 16:48:49 2017 +0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |  7 +++--
 .../hdfs/server/protocol/SlowDiskReports.java   |  5 ++--
 .../dev-support/findbugsExcludeFile.xml         | 26 +++++++++++++++++++
 .../hdfs/qjournal/server/JournalNode.java       | 16 +++++++-----
 .../hdfs/server/datanode/DataStorage.java       | 12 ++++++---
 .../namenode/NNStorageRetentionManager.java     | 27 +++++++++++---------
 .../org/apache/hadoop/hdfs/tools/DFSAdmin.java  |  6 ++---
 .../offlineImageViewer/ImageLoaderCurrent.java  | 10 +++++---
 .../namenode/TestNameNodeOptionParsing.java     | 27 +++++++++++++++++++-
 9 files changed, 103 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e43c28e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index 677ea35..88b273a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -2901,9 +2901,12 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     }
     synchronized (DFSClient.class) {
       if (STRIPED_READ_THREAD_POOL == null) {
-        STRIPED_READ_THREAD_POOL = DFSUtilClient.getThreadPoolExecutor(1,
+        // Only after thread pool is fully constructed then save it to
+        // volatile field.
+        ThreadPoolExecutor threadPool = DFSUtilClient.getThreadPoolExecutor(1,
             numThreads, 60, "StripedRead-", true);
-        STRIPED_READ_THREAD_POOL.allowCoreThreadTimeOut(true);
+        threadPool.allowCoreThreadTimeOut(true);
+        STRIPED_READ_THREAD_POOL = threadPool;
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e43c28e/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/SlowDiskReports.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/SlowDiskReports.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/SlowDiskReports.java
index 8095c2a..496389a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/SlowDiskReports.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/SlowDiskReports.java
@@ -101,8 +101,9 @@ public final class SlowDiskReports {
     }
 
     boolean areEqual;
-    for (String disk : this.slowDisks.keySet()) {
-      if (!this.slowDisks.get(disk).equals(that.slowDisks.get(disk))) {
+    for (Map.Entry<String, Map<DiskOp, Double>> entry : this.slowDisks
+        .entrySet()) {
+      if (!entry.getValue().equals(that.slowDisks.get(entry.getKey()))) {
         return false;
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e43c28e/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml b/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
index 2a7824a..9582fcb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
@@ -264,4 +264,30 @@
         <Field name="locations" />
         <Bug pattern="SE_TRANSIENT_FIELD_NOT_RESTORED" />
     </Match>
+    <Match>
+       <Class name="org.apache.hadoop.hdfs.server.namenode.NNUpgradeUtil$1" />
+       <Method name="visitFile" />
+       <Bug pattern="NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE" />
+     </Match>
+     <!-- Ignore warnings for not changing the startup option parsing behavior. -->
+     <Match>
+       <Class name="org.apache.hadoop.hdfs.server.common.HdfsServerConstants$StartupOption" />
+       <Method name="setClusterId" />
+       <Bug pattern="ME_ENUM_FIELD_SETTER" />
+     </Match>
+     <Match>
+       <Class name="org.apache.hadoop.hdfs.server.common.HdfsServerConstants$StartupOption" />
+       <Method name="setForce" />
+       <Bug pattern="ME_ENUM_FIELD_SETTER" />
+     </Match>
+     <Match>
+       <Class name="org.apache.hadoop.hdfs.server.common.HdfsServerConstants$StartupOption" />
+       <Method name="setForceFormat" />
+       <Bug pattern="ME_ENUM_FIELD_SETTER" />
+     </Match>
+     <Match>
+      <Class name="org.apache.hadoop.hdfs.server.common.HdfsServerConstants$StartupOption" />
+      <Method name="setInteractiveFormat" />
+      <Bug pattern="ME_ENUM_FIELD_SETTER" />
+    </Match>
  </FindBugsFilter>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e43c28e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNode.java
index af7a84f..6056e34 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNode.java
@@ -299,14 +299,18 @@ public class JournalNode implements Tool, Configurable, JournalNodeMXBean {
         return file.isDirectory();
       }
     });
-    for (File journalDir : journalDirs) {
-      String jid = journalDir.getName();
-      if (!status.containsKey(jid)) {
-        Map<String, String> jMap = new HashMap<String, String>();
-        jMap.put("Formatted", "true");
-        status.put(jid, jMap);
+
+    if (journalDirs != null) {
+      for (File journalDir : journalDirs) {
+        String jid = journalDir.getName();
+        if (!status.containsKey(jid)) {
+          Map<String, String> jMap = new HashMap<String, String>();
+          jMap.put("Formatted", "true");
+          status.put(jid, jMap);
+        }
       }
     }
+
     return JSON.toString(status);
   }
   

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e43c28e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
index 9a71081..6d6e96a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
@@ -1336,10 +1336,14 @@ public class DataStorage extends Storage {
           return name.startsWith(BLOCK_SUBDIR_PREFIX);
         }
       });
-    for(int i = 0; i < otherNames.length; i++)
-      linkBlocksHelper(new File(from, otherNames[i]),
-          new File(to, otherNames[i]), oldLV, hl, upgradeToIdBasedLayout,
-          blockRoot, idBasedLayoutSingleLinks);
+
+    if (otherNames != null) {
+      for (int i = 0; i < otherNames.length; i++) {
+        linkBlocksHelper(new File(from, otherNames[i]),
+            new File(to, otherNames[i]), oldLV, hl, upgradeToIdBasedLayout,
+            blockRoot, idBasedLayoutSingleLinks);
+      }
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e43c28e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorageRetentionManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorageRetentionManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorageRetentionManager.java
index 98b7e9a..2a83541 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorageRetentionManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorageRetentionManager.java
@@ -255,24 +255,27 @@ public class NNStorageRetentionManager {
     });
 
     // Check whether there is any work to do.
-    if (filesInStorage.length <= numCheckpointsToRetain) {
+    if (filesInStorage != null
+        && filesInStorage.length <= numCheckpointsToRetain) {
       return;
     }
 
     // Create a sorted list of txids from the file names.
     TreeSet<Long> sortedTxIds = new TreeSet<Long>();
-    for (String fName : filesInStorage) {
-      // Extract the transaction id from the file name.
-      long fTxId;
-      try {
-        fTxId = Long.parseLong(fName.substring(oivImagePrefix.length() + 1));
-      } catch (NumberFormatException nfe) {
-        // This should not happen since we have already filtered it.
-        // Log and continue.
-        LOG.warn("Invalid file name. Skipping " + fName);
-        continue;
+    if (filesInStorage != null) {
+      for (String fName : filesInStorage) {
+        // Extract the transaction id from the file name.
+        long fTxId;
+        try {
+          fTxId = Long.parseLong(fName.substring(oivImagePrefix.length() + 1));
+        } catch (NumberFormatException nfe) {
+          // This should not happen since we have already filtered it.
+          // Log and continue.
+          LOG.warn("Invalid file name. Skipping " + fName);
+          continue;
+        }
+        sortedTxIds.add(Long.valueOf(fTxId));
       }
-      sortedTxIds.add(Long.valueOf(fTxId));
     }
 
     int numFilesToDelete = sortedTxIds.size() - numCheckpointsToRetain;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e43c28e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
index f2233eb..1fb1d5f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
@@ -1992,7 +1992,7 @@ public class DFSAdmin extends FsShell {
         return exitCode;
       }
     } else if ("-report".equals(cmd)) {
-      if (argv.length < 1) {
+      if (argv.length > 6) {
         printUsage(cmd);
         return exitCode;
       }
@@ -2022,7 +2022,7 @@ public class DFSAdmin extends FsShell {
         return exitCode;
       }
     } else if (RollingUpgradeCommand.matches(cmd)) {
-      if (argv.length < 1 || argv.length > 2) {
+      if (argv.length > 2) {
         printUsage(cmd);
         return exitCode;
       }
@@ -2097,7 +2097,7 @@ public class DFSAdmin extends FsShell {
         return exitCode;
       }
     } else if ("-triggerBlockReport".equals(cmd)) {
-      if (argv.length < 1) {
+      if ((argv.length != 2) && (argv.length != 3)) {
         printUsage(cmd);
         return exitCode;
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e43c28e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java
index f2c7427..2e2eaf4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java
@@ -722,9 +722,13 @@ class ImageLoaderCurrent implements ImageLoader {
       if (supportSnapshot && supportInodeId) {
         dirNodeMap.put(inodeId, pathName);
       }
-      v.visit(ImageElement.NS_QUOTA, numBlocks == -1 ? in.readLong() : -1);
-      if (NameNodeLayoutVersion.supports(Feature.DISKSPACE_QUOTA, imageVersion))
-        v.visit(ImageElement.DS_QUOTA, numBlocks == -1 ? in.readLong() : -1);
+
+      v.visit(ImageElement.NS_QUOTA, in.readLong());
+      if (NameNodeLayoutVersion.supports(Feature.DISKSPACE_QUOTA,
+          imageVersion)) {
+        v.visit(ImageElement.DS_QUOTA, in.readLong());
+      }
+
       if (supportSnapshot) {
         boolean snapshottable = in.readBoolean();
         if (!snapshottable) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e43c28e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeOptionParsing.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeOptionParsing.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeOptionParsing.java
index 7ee49a9..92b96a5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeOptionParsing.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeOptionParsing.java
@@ -138,5 +138,30 @@ public class TestNameNodeOptionParsing {
       }
     }
   }
-    
+
+  @Test
+  public void testFormat() {
+    String[] args = new String[] {"-format"};
+    StartupOption opt = NameNode.parseArguments(args);
+    assertEquals(StartupOption.FORMAT, opt);
+    assertEquals(true, opt.getInteractiveFormat());
+    assertEquals(false, opt.getForceFormat());
+
+    args = new String[] {"-format", "-nonInteractive"};
+    opt = NameNode.parseArguments(args);
+    assertEquals(StartupOption.FORMAT, opt);
+    assertEquals(false, opt.getInteractiveFormat());
+    assertEquals(false, opt.getForceFormat());
+
+    args = new String[] {"-format", "-nonInteractive", "-force"};
+    opt = NameNode.parseArguments(args);
+    assertEquals(StartupOption.FORMAT, opt);
+    assertEquals(false, opt.getInteractiveFormat());
+    assertEquals(true, opt.getForceFormat());
+
+    // test error condition
+    args = new String[] {"-nonInteractive"};
+    opt = NameNode.parseArguments(args);
+    assertNull(opt);
+  }
 }


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


[08/50] [abbrv] hadoop git commit: HDFS-11303. Hedged read might hang infinitely if read data from all DN failed . Contributed by Chen Zhang, Wei-chiu Chuang, and John Zhuge.

Posted by jh...@apache.org.
HDFS-11303. Hedged read might hang infinitely if read data from all DN failed . Contributed by Chen Zhang, Wei-chiu Chuang, and John Zhuge.


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

Branch: refs/heads/YARN-5734
Commit: 8b242f09a61a7536d2422546bfa6c2aaf1d57ed6
Parents: 28d97b7
Author: John Zhuge <jz...@cloudera.com>
Authored: Thu Aug 10 14:04:36 2017 -0700
Committer: John Zhuge <jz...@apache.org>
Committed: Fri Aug 11 19:42:07 2017 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/DFSInputStream.java  | 11 ++--
 .../java/org/apache/hadoop/hdfs/TestPread.java  | 63 ++++++++++++++++++++
 2 files changed, 70 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8b242f09/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
index dcc997c..6bff172 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
@@ -1131,8 +1131,9 @@ public class DFSInputStream extends FSInputStream
         Future<ByteBuffer> firstRequest = hedgedService
             .submit(getFromDataNodeCallable);
         futures.add(firstRequest);
+        Future<ByteBuffer> future = null;
         try {
-          Future<ByteBuffer> future = hedgedService.poll(
+          future = hedgedService.poll(
               conf.getHedgedReadThresholdMillis(), TimeUnit.MILLISECONDS);
           if (future != null) {
             ByteBuffer result = future.get();
@@ -1142,16 +1143,18 @@ public class DFSInputStream extends FSInputStream
           }
           DFSClient.LOG.debug("Waited {}ms to read from {}; spawning hedged "
               + "read", conf.getHedgedReadThresholdMillis(), chosenNode.info);
-          // Ignore this node on next go around.
-          ignored.add(chosenNode.info);
           dfsClient.getHedgedReadMetrics().incHedgedReadOps();
           // continue; no need to refresh block locations
         } catch (ExecutionException e) {
-          // Ignore
+          futures.remove(future);
         } catch (InterruptedException e) {
           throw new InterruptedIOException(
               "Interrupted while waiting for reading task");
         }
+        // Ignore this node on next go around.
+        // If poll timeout and the request still ongoing, don't consider it
+        // again. If read data failed, don't consider it either.
+        ignored.add(chosenNode.info);
       } else {
         // We are starting up a 'hedged' read. We have a read already
         // ongoing. Call getBestNodeDNAddrPair instead of chooseDataNode.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8b242f09/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java
index 85fc97b..bcb02b3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java
@@ -59,6 +59,8 @@ import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
 import com.google.common.base.Supplier;
+import org.slf4j.LoggerFactory;
+import org.slf4j.Logger;
 
 /**
  * This class tests the DFS positional read functionality in a single node
@@ -72,6 +74,9 @@ public class TestPread {
   boolean simulatedStorage;
   boolean isHedgedRead;
 
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestPread.class.getName());
+
   @Before
   public void setup() {
     simulatedStorage = false;
@@ -551,6 +556,64 @@ public class TestPread {
     }
   }
 
+  @Test(timeout=30000)
+  public void testHedgedReadFromAllDNFailed() throws IOException {
+    Configuration conf = new Configuration();
+    int numHedgedReadPoolThreads = 5;
+    final int hedgedReadTimeoutMillis = 50;
+
+    conf.setInt(HdfsClientConfigKeys.HedgedRead.THREADPOOL_SIZE_KEY,
+        numHedgedReadPoolThreads);
+    conf.setLong(HdfsClientConfigKeys.HedgedRead.THRESHOLD_MILLIS_KEY,
+        hedgedReadTimeoutMillis);
+    conf.setInt(HdfsClientConfigKeys.Retry.WINDOW_BASE_KEY, 0);
+    // Set up the InjectionHandler
+    DFSClientFaultInjector.set(Mockito.mock(DFSClientFaultInjector.class));
+    DFSClientFaultInjector injector = DFSClientFaultInjector.get();
+    Mockito.doAnswer(new Answer<Void>() {
+      @Override
+      public Void answer(InvocationOnMock invocation) throws Throwable {
+        if (true) {
+          LOG.info("-------------- throw Checksum Exception");
+          throw new ChecksumException("ChecksumException test", 100);
+        }
+        return null;
+      }
+    }).when(injector).fetchFromDatanodeException();
+
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3)
+        .format(true).build();
+    DistributedFileSystem fileSys = cluster.getFileSystem();
+    DFSClient dfsClient = fileSys.getClient();
+    FSDataOutputStream output = null;
+    DFSInputStream input = null;
+    String filename = "/hedgedReadMaxOut.dat";
+    DFSHedgedReadMetrics metrics = dfsClient.getHedgedReadMetrics();
+    // Metrics instance is static, so we need to reset counts from prior tests.
+    metrics.hedgedReadOps.set(0);
+    try {
+      Path file = new Path(filename);
+      output = fileSys.create(file, (short) 2);
+      byte[] data = new byte[64 * 1024];
+      output.write(data);
+      output.flush();
+      output.close();
+      byte[] buffer = new byte[64 * 1024];
+      input = dfsClient.open(filename);
+      input.read(0, buffer, 0, 1024);
+      Assert.fail("Reading the block should have thrown BlockMissingException");
+    } catch (BlockMissingException e) {
+      assertEquals(3, input.getHedgedReadOpsLoopNumForTesting());
+      assertTrue(metrics.getHedgedReadOps() == 0);
+    } finally {
+      Mockito.reset(injector);
+      IOUtils.cleanupWithLogger(LOG, input);
+      IOUtils.cleanupWithLogger(LOG, output);
+      fileSys.close();
+      cluster.shutdown();
+    }
+  }
+
   /**
    * Scenario: 1. Write a file with RF=2, DN1 and DN2<br>
    * 2. Open the stream, Consider Locations are [DN1, DN2] in LocatedBlock.<br>


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


[45/50] [abbrv] hadoop git commit: YARN-6575. Support global configuration mutation in MutableConfProvider. (Jonathan Hung via Xuan Gong)

Posted by jh...@apache.org.
YARN-6575. Support global configuration mutation in MutableConfProvider. (Jonathan Hung via Xuan Gong)


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

Branch: refs/heads/YARN-5734
Commit: 7cf82f3661a19ce899bbac7f9d5fae1e0e8ccefd
Parents: 771d951
Author: Xuan <xg...@apache.org>
Authored: Mon Jun 5 16:30:38 2017 -0700
Committer: Jonathan Hung <jh...@linkedin.com>
Committed: Thu Aug 17 13:48:55 2017 -0700

----------------------------------------------------------------------
 .../ConfigurationMutationACLPolicy.java         |   4 +-
 .../DefaultConfigurationMutationACLPolicy.java  |   4 +-
 .../scheduler/MutableConfScheduler.java         |   4 +-
 .../scheduler/MutableConfigurationProvider.java |   4 +-
 .../scheduler/capacity/CapacityScheduler.java   |   4 +-
 .../conf/MutableCSConfigurationProvider.java    |  10 +-
 ...ueueAdminConfigurationMutationACLPolicy.java |  22 +++-
 .../resourcemanager/webapp/RMWebServices.java   |   4 +-
 .../webapp/dao/QueueConfigsUpdateInfo.java      |  60 -----------
 .../webapp/dao/SchedConfUpdateInfo.java         |  69 +++++++++++++
 .../TestConfigurationMutationACLPolicies.java   |  28 ++++-
 .../TestMutableCSConfigurationProvider.java     |  10 +-
 .../TestRMWebServicesConfigurationMutation.java | 101 +++++++++++++------
 13 files changed, 205 insertions(+), 119 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7cf82f36/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ConfigurationMutationACLPolicy.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/ConfigurationMutationACLPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ConfigurationMutationACLPolicy.java
index 724487b..3a388fe 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ConfigurationMutationACLPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ConfigurationMutationACLPolicy.java
@@ -21,7 +21,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.QueueConfigsUpdateInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.SchedConfUpdateInfo;
 
 /**
  * Interface for determining whether configuration mutations are allowed.
@@ -41,7 +41,7 @@ public interface ConfigurationMutationACLPolicy {
    * @param confUpdate configurations to be updated
    * @return whether provided mutation is allowed or not
    */
-  boolean isMutationAllowed(UserGroupInformation user, QueueConfigsUpdateInfo
+  boolean isMutationAllowed(UserGroupInformation user, SchedConfUpdateInfo
       confUpdate);
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7cf82f36/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/DefaultConfigurationMutationACLPolicy.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/DefaultConfigurationMutationACLPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/DefaultConfigurationMutationACLPolicy.java
index 680c3b8..6648668 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/DefaultConfigurationMutationACLPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/DefaultConfigurationMutationACLPolicy.java
@@ -22,7 +22,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.security.YarnAuthorizationProvider;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.QueueConfigsUpdateInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.SchedConfUpdateInfo;
 
 /**
  * Default configuration mutation ACL policy. Checks if user is YARN admin.
@@ -39,7 +39,7 @@ public class DefaultConfigurationMutationACLPolicy implements
 
   @Override
   public boolean isMutationAllowed(UserGroupInformation user,
-      QueueConfigsUpdateInfo confUpdate) {
+      SchedConfUpdateInfo confUpdate) {
     return authorizer.isAdmin(user);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7cf82f36/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/MutableConfScheduler.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/MutableConfScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/MutableConfScheduler.java
index 93a935e..027d944 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/MutableConfScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/MutableConfScheduler.java
@@ -19,7 +19,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.QueueConfigsUpdateInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.SchedConfUpdateInfo;
 
 import java.io.IOException;
 
@@ -36,7 +36,7 @@ public interface MutableConfScheduler extends ResourceScheduler {
    * @throws IOException if update is invalid
    */
   void updateConfiguration(UserGroupInformation user,
-      QueueConfigsUpdateInfo confUpdate) throws IOException;
+      SchedConfUpdateInfo confUpdate) throws IOException;
 
   /**
    * Get the scheduler configuration.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7cf82f36/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/MutableConfigurationProvider.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/MutableConfigurationProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/MutableConfigurationProvider.java
index f04c128..6b8306c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/MutableConfigurationProvider.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/MutableConfigurationProvider.java
@@ -19,7 +19,7 @@
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
 
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.QueueConfigsUpdateInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.SchedConfUpdateInfo;
 
 import java.io.IOException;
 
@@ -34,7 +34,7 @@ public interface MutableConfigurationProvider {
    * @param confUpdate Key-value pairs for configurations to be updated.
    * @throws IOException if scheduler could not be reinitialized
    */
-  void mutateConfiguration(UserGroupInformation user, QueueConfigsUpdateInfo
+  void mutateConfiguration(UserGroupInformation user, SchedConfUpdateInfo
       confUpdate) throws IOException;
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7cf82f36/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.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/CapacityScheduler.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/CapacityScheduler.java
index 4fbccbd..ebd3a49 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/CapacityScheduler.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/CapacityScheduler.java
@@ -135,7 +135,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.Placeme
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.SimplePlacementSet;
 import org.apache.hadoop.yarn.server.resourcemanager.security.AppPriorityACLsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.QueueConfigsUpdateInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.SchedConfUpdateInfo;
 import org.apache.hadoop.yarn.server.utils.Lock;
 import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
@@ -2562,7 +2562,7 @@ public class CapacityScheduler extends
 
   @Override
   public void updateConfiguration(UserGroupInformation user,
-      QueueConfigsUpdateInfo confUpdate) throws IOException {
+      SchedConfUpdateInfo confUpdate) throws IOException {
     if (csConfProvider instanceof MutableConfigurationProvider) {
       ((MutableConfigurationProvider) csConfProvider).mutateConfiguration(
           user, confUpdate);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7cf82f36/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/MutableCSConfigurationProvider.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/conf/MutableCSConfigurationProvider.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/conf/MutableCSConfigurationProvider.java
index 8b879b0..eb97260 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/conf/MutableCSConfigurationProvider.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/conf/MutableCSConfigurationProvider.java
@@ -32,7 +32,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Capacity
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.conf.YarnConfigurationStore.LogMutation;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.QueueConfigInfo;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.QueueConfigsUpdateInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.SchedConfUpdateInfo;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -98,7 +98,7 @@ public class MutableCSConfigurationProvider implements CSConfigurationProvider,
 
   @Override
   public void mutateConfiguration(UserGroupInformation user,
-      QueueConfigsUpdateInfo confUpdate) throws IOException {
+      SchedConfUpdateInfo confUpdate) throws IOException {
     if (!aclMutationPolicy.isMutationAllowed(user, confUpdate)) {
       throw new AccessControlException("User is not admin of all modified" +
           " queues.");
@@ -126,7 +126,7 @@ public class MutableCSConfigurationProvider implements CSConfigurationProvider,
 
 
   private Map<String, String> constructKeyValueConfUpdate(
-      QueueConfigsUpdateInfo mutationInfo) throws IOException {
+      SchedConfUpdateInfo mutationInfo) throws IOException {
     CapacityScheduler cs = (CapacityScheduler) rmContext.getScheduler();
     CapacitySchedulerConfiguration proposedConf =
         new CapacitySchedulerConfiguration(cs.getConfiguration(), false);
@@ -140,6 +140,10 @@ public class MutableCSConfigurationProvider implements CSConfigurationProvider,
     for (QueueConfigInfo updateQueueInfo : mutationInfo.getUpdateQueueInfo()) {
       updateQueue(updateQueueInfo, proposedConf, confUpdate);
     }
+    for (Map.Entry<String, String> global : mutationInfo.getGlobalParams()
+        .entrySet()) {
+      confUpdate.put(global.getKey(), global.getValue());
+    }
     return confUpdate;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7cf82f36/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/QueueAdminConfigurationMutationACLPolicy.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/conf/QueueAdminConfigurationMutationACLPolicy.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/conf/QueueAdminConfigurationMutationACLPolicy.java
index 1f94c1c..0a82d50 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/conf/QueueAdminConfigurationMutationACLPolicy.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/conf/QueueAdminConfigurationMutationACLPolicy.java
@@ -22,15 +22,17 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueInfo;
+import org.apache.hadoop.yarn.security.YarnAuthorizationProvider;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ConfigurationMutationACLPolicy;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.MutableConfScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.QueueConfigInfo;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.QueueConfigsUpdateInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.SchedConfUpdateInfo;
 
 import java.io.IOException;
 import java.util.HashSet;
+import java.util.Map;
 import java.util.Set;
 
 /**
@@ -40,16 +42,29 @@ import java.util.Set;
 public class QueueAdminConfigurationMutationACLPolicy implements
     ConfigurationMutationACLPolicy {
 
+  private Configuration conf;
   private RMContext rmContext;
+  private YarnAuthorizationProvider authorizer;
 
   @Override
-  public void init(Configuration conf, RMContext context) {
+  public void init(Configuration config, RMContext context) {
+    this.conf = config;
     this.rmContext = context;
+    this.authorizer = YarnAuthorizationProvider.getInstance(conf);
   }
 
   @Override
   public boolean isMutationAllowed(UserGroupInformation user,
-      QueueConfigsUpdateInfo confUpdate) {
+      SchedConfUpdateInfo confUpdate) {
+    // If there are global config changes, check if user is admin.
+    Map<String, String> globalParams = confUpdate.getGlobalParams();
+    if (globalParams != null && globalParams.size() != 0) {
+      if (!authorizer.isAdmin(user)) {
+        return false;
+      }
+    }
+
+    // Check if user is admin of all modified queues.
     Set<String> queues = new HashSet<>();
     for (QueueConfigInfo addQueueInfo : confUpdate.getAddQueueInfo()) {
       queues.add(addQueueInfo.getQueue());
@@ -71,7 +86,6 @@ public class QueueAdminConfigurationMutationACLPolicy implements
         // Queue is not found, do nothing.
       }
       String parentPath = queuePath;
-      // TODO: handle global config change.
       while (queueInfo == null) {
         // We are adding a queue (whose parent we are possibly also adding).
         // Check ACL of lowest parent queue which already exists.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7cf82f36/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
index dd476aa..358b57b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
@@ -2409,11 +2409,11 @@ public class RMWebServices extends WebServices implements RMWebServiceProtocol {
   }
 
   @PUT
-  @Path("/queues")
+  @Path("/sched-conf")
   @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
       MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   @Consumes({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
-  public Response updateSchedulerConfiguration(QueueConfigsUpdateInfo
+  public Response updateSchedulerConfiguration(SchedConfUpdateInfo
       mutationInfo, @Context HttpServletRequest hsr)
       throws AuthorizationException, InterruptedException {
     init();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7cf82f36/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/QueueConfigsUpdateInfo.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/QueueConfigsUpdateInfo.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/QueueConfigsUpdateInfo.java
deleted file mode 100644
index 644ec90..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/QueueConfigsUpdateInfo.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.server.resourcemanager.webapp.dao;
-
-import java.util.ArrayList;
-
-import javax.xml.bind.annotation.XmlAccessType;
-import javax.xml.bind.annotation.XmlAccessorType;
-import javax.xml.bind.annotation.XmlElement;
-import javax.xml.bind.annotation.XmlRootElement;
-
-/**
- * Information for making scheduler configuration changes (supports adding,
- * removing, or updating a queue).
- */
-@XmlRootElement(name = "schedConf")
-@XmlAccessorType(XmlAccessType.FIELD)
-public class QueueConfigsUpdateInfo {
-
-  @XmlElement(name = "add")
-  private ArrayList<QueueConfigInfo> addQueueInfo = new ArrayList<>();
-
-  @XmlElement(name = "remove")
-  private ArrayList<String> removeQueueInfo = new ArrayList<>();
-
-  @XmlElement(name = "update")
-  private ArrayList<QueueConfigInfo> updateQueueInfo = new ArrayList<>();
-
-  public QueueConfigsUpdateInfo() {
-    // JAXB needs this
-  }
-
-  public ArrayList<QueueConfigInfo> getAddQueueInfo() {
-    return addQueueInfo;
-  }
-
-  public ArrayList<String> getRemoveQueueInfo() {
-    return removeQueueInfo;
-  }
-
-  public ArrayList<QueueConfigInfo> getUpdateQueueInfo() {
-    return updateQueueInfo;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7cf82f36/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/SchedConfUpdateInfo.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/SchedConfUpdateInfo.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/SchedConfUpdateInfo.java
new file mode 100644
index 0000000..b7c585e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/SchedConfUpdateInfo.java
@@ -0,0 +1,69 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, 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.webapp.dao;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlElementWrapper;
+import javax.xml.bind.annotation.XmlRootElement;
+
+/**
+ * Information for making scheduler configuration changes (supports adding,
+ * removing, or updating a queue, as well as global scheduler conf changes).
+ */
+@XmlRootElement(name = "schedConf")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class SchedConfUpdateInfo {
+
+  @XmlElement(name = "add-queue")
+  private ArrayList<QueueConfigInfo> addQueueInfo = new ArrayList<>();
+
+  @XmlElement(name = "remove-queue")
+  private ArrayList<String> removeQueueInfo = new ArrayList<>();
+
+  @XmlElement(name = "update-queue")
+  private ArrayList<QueueConfigInfo> updateQueueInfo = new ArrayList<>();
+
+  private HashMap<String, String> global = new HashMap<>();
+
+  public SchedConfUpdateInfo() {
+    // JAXB needs this
+  }
+
+  public ArrayList<QueueConfigInfo> getAddQueueInfo() {
+    return addQueueInfo;
+  }
+
+  public ArrayList<String> getRemoveQueueInfo() {
+    return removeQueueInfo;
+  }
+
+  public ArrayList<QueueConfigInfo> getUpdateQueueInfo() {
+    return updateQueueInfo;
+  }
+
+  @XmlElementWrapper(name = "global-updates")
+  public HashMap<String, String> getGlobalParams() {
+    return global;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7cf82f36/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestConfigurationMutationACLPolicies.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/scheduler/TestConfigurationMutationACLPolicies.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestConfigurationMutationACLPolicies.java
index 4016dcf..0f5a3d8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestConfigurationMutationACLPolicies.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestConfigurationMutationACLPolicies.java
@@ -26,7 +26,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.conf.QueueAdminConfigurationMutationACLPolicy;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.QueueConfigInfo;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.QueueConfigsUpdateInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.SchedConfUpdateInfo;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -77,6 +77,7 @@ public class TestConfigurationMutationACLPolicies {
         .thenReturn(false);
     when(scheduler.getQueue(eq(queueName))).thenReturn(queue);
   }
+
   @Test
   public void testDefaultPolicy() {
     Configuration conf = new Configuration();
@@ -98,7 +99,7 @@ public class TestConfigurationMutationACLPolicies {
         ConfigurationMutationACLPolicy.class);
     policy = ConfigurationMutationACLPolicyFactory.getPolicy(conf);
     policy.init(conf, rmContext);
-    QueueConfigsUpdateInfo updateInfo = new QueueConfigsUpdateInfo();
+    SchedConfUpdateInfo updateInfo = new SchedConfUpdateInfo();
     QueueConfigInfo configInfo = new QueueConfigInfo("root.a", EMPTY_MAP);
     updateInfo.getUpdateQueueInfo().add(configInfo);
     assertTrue(policy.isMutationAllowed(GOOD_USER, updateInfo));
@@ -114,7 +115,7 @@ public class TestConfigurationMutationACLPolicies {
     policy = ConfigurationMutationACLPolicyFactory.getPolicy(conf);
     policy.init(conf, rmContext);
     // Add root.b.b1. Should check ACL of root.b queue.
-    QueueConfigsUpdateInfo updateInfo = new QueueConfigsUpdateInfo();
+    SchedConfUpdateInfo updateInfo = new SchedConfUpdateInfo();
     QueueConfigInfo configInfo = new QueueConfigInfo("root.b.b2", EMPTY_MAP);
     updateInfo.getAddQueueInfo().add(configInfo);
     assertTrue(policy.isMutationAllowed(GOOD_USER, updateInfo));
@@ -130,7 +131,7 @@ public class TestConfigurationMutationACLPolicies {
     policy = ConfigurationMutationACLPolicyFactory.getPolicy(conf);
     policy.init(conf, rmContext);
     // Add root.b.b1.b11. Should check ACL of root.b queue.
-    QueueConfigsUpdateInfo updateInfo = new QueueConfigsUpdateInfo();
+    SchedConfUpdateInfo updateInfo = new SchedConfUpdateInfo();
     QueueConfigInfo configInfo = new QueueConfigInfo("root.b.b2.b21", EMPTY_MAP);
     updateInfo.getAddQueueInfo().add(configInfo);
     assertTrue(policy.isMutationAllowed(GOOD_USER, updateInfo));
@@ -146,9 +147,26 @@ public class TestConfigurationMutationACLPolicies {
     policy = ConfigurationMutationACLPolicyFactory.getPolicy(conf);
     policy.init(conf, rmContext);
     // Remove root.b.b1.
-    QueueConfigsUpdateInfo updateInfo = new QueueConfigsUpdateInfo();
+    SchedConfUpdateInfo updateInfo = new SchedConfUpdateInfo();
     updateInfo.getRemoveQueueInfo().add("root.b.b1");
     assertTrue(policy.isMutationAllowed(GOOD_USER, updateInfo));
     assertFalse(policy.isMutationAllowed(BAD_USER, updateInfo));
   }
+
+  @Test
+  public void testQueueAdminPolicyGlobal() {
+    Configuration conf = new Configuration();
+    conf.set(YarnConfiguration.YARN_ADMIN_ACL, GOOD_USER.getShortUserName());
+    conf.setClass(YarnConfiguration.RM_SCHEDULER_MUTATION_ACL_POLICY_CLASS,
+        QueueAdminConfigurationMutationACLPolicy.class,
+        ConfigurationMutationACLPolicy.class);
+    policy = ConfigurationMutationACLPolicyFactory.getPolicy(conf);
+    policy.init(conf, rmContext);
+    SchedConfUpdateInfo updateInfo = new SchedConfUpdateInfo();
+    assertTrue(policy.isMutationAllowed(GOOD_USER, updateInfo));
+    assertTrue(policy.isMutationAllowed(BAD_USER, updateInfo));
+    updateInfo.getGlobalParams().put("globalKey", "globalValue");
+    assertTrue(policy.isMutationAllowed(GOOD_USER, updateInfo));
+    assertFalse(policy.isMutationAllowed(BAD_USER, updateInfo));
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7cf82f36/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/TestMutableCSConfigurationProvider.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/scheduler/capacity/conf/TestMutableCSConfigurationProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/TestMutableCSConfigurationProvider.java
index 13229b1..3216781 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/TestMutableCSConfigurationProvider.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/TestMutableCSConfigurationProvider.java
@@ -24,7 +24,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.QueueConfigInfo;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.QueueConfigsUpdateInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.SchedConfUpdateInfo;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -47,8 +47,8 @@ public class TestMutableCSConfigurationProvider {
 
   private MutableCSConfigurationProvider confProvider;
   private RMContext rmContext;
-  private QueueConfigsUpdateInfo goodUpdate;
-  private QueueConfigsUpdateInfo badUpdate;
+  private SchedConfUpdateInfo goodUpdate;
+  private SchedConfUpdateInfo badUpdate;
   private CapacityScheduler cs;
 
   private static final UserGroupInformation TEST_USER = UserGroupInformation
@@ -62,14 +62,14 @@ public class TestMutableCSConfigurationProvider {
     when(cs.getConfiguration()).thenReturn(
         new CapacitySchedulerConfiguration());
     confProvider = new MutableCSConfigurationProvider(rmContext);
-    goodUpdate = new QueueConfigsUpdateInfo();
+    goodUpdate = new SchedConfUpdateInfo();
     Map<String, String> goodUpdateMap = new HashMap<>();
     goodUpdateMap.put("goodKey", "goodVal");
     QueueConfigInfo goodUpdateInfo = new
         QueueConfigInfo("root.a", goodUpdateMap);
     goodUpdate.getUpdateQueueInfo().add(goodUpdateInfo);
 
-    badUpdate = new QueueConfigsUpdateInfo();
+    badUpdate = new SchedConfUpdateInfo();
     Map<String, String> badUpdateMap = new HashMap<>();
     badUpdateMap.put("badKey", "badVal");
     QueueConfigInfo badUpdateInfo = new

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7cf82f36/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesConfigurationMutation.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/webapp/TestRMWebServicesConfigurationMutation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesConfigurationMutation.java
index d149055..5fbe36f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesConfigurationMutation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesConfigurationMutation.java
@@ -36,7 +36,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.QueueConfigInfo;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.QueueConfigsUpdateInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.SchedConfUpdateInfo;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
 import org.apache.hadoop.yarn.webapp.GuiceServletConfig;
 import org.apache.hadoop.yarn.webapp.JerseyTestBase;
@@ -162,7 +162,7 @@ public class TestRMWebServicesConfigurationMutation extends JerseyTestBase {
     ClientResponse response;
 
     // Add parent queue root.d with two children d1 and d2.
-    QueueConfigsUpdateInfo updateInfo = new QueueConfigsUpdateInfo();
+    SchedConfUpdateInfo updateInfo = new SchedConfUpdateInfo();
     Map<String, String> d1Capacity = new HashMap<>();
     d1Capacity.put(CapacitySchedulerConfiguration.CAPACITY, "25");
     d1Capacity.put(CapacitySchedulerConfiguration.MAXIMUM_CAPACITY, "25");
@@ -181,9 +181,9 @@ public class TestRMWebServicesConfigurationMutation extends JerseyTestBase {
     updateInfo.getAddQueueInfo().add(d);
     response =
         r.path("ws").path("v1").path("cluster")
-            .path("queues").queryParam("user.name", userName)
+            .path("sched-conf").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON)
-            .entity(toJson(updateInfo, QueueConfigsUpdateInfo.class),
+            .entity(toJson(updateInfo, SchedConfUpdateInfo.class),
                 MediaType.APPLICATION_JSON)
             .put(ClientResponse.class);
 
@@ -205,7 +205,7 @@ public class TestRMWebServicesConfigurationMutation extends JerseyTestBase {
     ClientResponse response;
 
     // Add root.d with capacity 25, reducing root.b capacity from 75 to 50.
-    QueueConfigsUpdateInfo updateInfo = new QueueConfigsUpdateInfo();
+    SchedConfUpdateInfo updateInfo = new SchedConfUpdateInfo();
     Map<String, String> dCapacity = new HashMap<>();
     dCapacity.put(CapacitySchedulerConfiguration.CAPACITY, "25");
     Map<String, String> bCapacity = new HashMap<>();
@@ -216,9 +216,9 @@ public class TestRMWebServicesConfigurationMutation extends JerseyTestBase {
     updateInfo.getUpdateQueueInfo().add(b);
     response =
         r.path("ws").path("v1").path("cluster")
-            .path("queues").queryParam("user.name", userName)
+            .path("sched-conf").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON)
-            .entity(toJson(updateInfo, QueueConfigsUpdateInfo.class),
+            .entity(toJson(updateInfo, SchedConfUpdateInfo.class),
                 MediaType.APPLICATION_JSON)
             .put(ClientResponse.class);
 
@@ -238,13 +238,13 @@ public class TestRMWebServicesConfigurationMutation extends JerseyTestBase {
 
     stopQueue("root.a.a2");
     // Remove root.a.a2
-    QueueConfigsUpdateInfo updateInfo = new QueueConfigsUpdateInfo();
+    SchedConfUpdateInfo updateInfo = new SchedConfUpdateInfo();
     updateInfo.getRemoveQueueInfo().add("root.a.a2");
     response =
         r.path("ws").path("v1").path("cluster")
-            .path("queues").queryParam("user.name", userName)
+            .path("sched-conf").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON)
-            .entity(toJson(updateInfo, QueueConfigsUpdateInfo.class),
+            .entity(toJson(updateInfo, SchedConfUpdateInfo.class),
                 MediaType.APPLICATION_JSON)
             .put(ClientResponse.class);
 
@@ -263,13 +263,13 @@ public class TestRMWebServicesConfigurationMutation extends JerseyTestBase {
 
     stopQueue("root.c", "root.c.c1");
     // Remove root.c (parent queue)
-    QueueConfigsUpdateInfo updateInfo = new QueueConfigsUpdateInfo();
+    SchedConfUpdateInfo updateInfo = new SchedConfUpdateInfo();
     updateInfo.getRemoveQueueInfo().add("root.c");
     response =
         r.path("ws").path("v1").path("cluster")
-            .path("queues").queryParam("user.name", userName)
+            .path("sched-conf").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON)
-            .entity(toJson(updateInfo, QueueConfigsUpdateInfo.class),
+            .entity(toJson(updateInfo, SchedConfUpdateInfo.class),
                 MediaType.APPLICATION_JSON)
             .put(ClientResponse.class);
 
@@ -288,7 +288,7 @@ public class TestRMWebServicesConfigurationMutation extends JerseyTestBase {
 
     stopQueue("root.a", "root.a.a1", "root.a.a2");
     // Remove root.a (parent queue) with capacity 25
-    QueueConfigsUpdateInfo updateInfo = new QueueConfigsUpdateInfo();
+    SchedConfUpdateInfo updateInfo = new SchedConfUpdateInfo();
     updateInfo.getRemoveQueueInfo().add("root.a");
 
     // Set root.b capacity to 100
@@ -298,9 +298,9 @@ public class TestRMWebServicesConfigurationMutation extends JerseyTestBase {
     updateInfo.getUpdateQueueInfo().add(b);
     response =
         r.path("ws").path("v1").path("cluster")
-            .path("queues").queryParam("user.name", userName)
+            .path("sched-conf").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON)
-            .entity(toJson(updateInfo, QueueConfigsUpdateInfo.class),
+            .entity(toJson(updateInfo, SchedConfUpdateInfo.class),
                 MediaType.APPLICATION_JSON)
             .put(ClientResponse.class);
 
@@ -320,7 +320,7 @@ public class TestRMWebServicesConfigurationMutation extends JerseyTestBase {
 
     stopQueue("root.b", "root.c", "root.c.c1");
     // Remove root.b and root.c
-    QueueConfigsUpdateInfo updateInfo = new QueueConfigsUpdateInfo();
+    SchedConfUpdateInfo updateInfo = new SchedConfUpdateInfo();
     updateInfo.getRemoveQueueInfo().add("root.b");
     updateInfo.getRemoveQueueInfo().add("root.c");
     Map<String, String> aCapacity = new HashMap<>();
@@ -330,9 +330,9 @@ public class TestRMWebServicesConfigurationMutation extends JerseyTestBase {
     updateInfo.getUpdateQueueInfo().add(configInfo);
     response =
         r.path("ws").path("v1").path("cluster")
-            .path("queues").queryParam("user.name", userName)
+            .path("sched-conf").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON)
-            .entity(toJson(updateInfo, QueueConfigsUpdateInfo.class),
+            .entity(toJson(updateInfo, SchedConfUpdateInfo.class),
                 MediaType.APPLICATION_JSON)
             .put(ClientResponse.class);
 
@@ -348,7 +348,7 @@ public class TestRMWebServicesConfigurationMutation extends JerseyTestBase {
     ClientResponse response;
 
     // Set state of queues to STOPPED.
-    QueueConfigsUpdateInfo updateInfo = new QueueConfigsUpdateInfo();
+    SchedConfUpdateInfo updateInfo = new SchedConfUpdateInfo();
     Map<String, String> stoppedParam = new HashMap<>();
     stoppedParam.put(CapacitySchedulerConfiguration.STATE,
         QueueState.STOPPED.toString());
@@ -358,9 +358,9 @@ public class TestRMWebServicesConfigurationMutation extends JerseyTestBase {
     }
     response =
         r.path("ws").path("v1").path("cluster")
-            .path("queues").queryParam("user.name", userName)
+            .path("sched-conf").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON)
-            .entity(toJson(updateInfo, QueueConfigsUpdateInfo.class),
+            .entity(toJson(updateInfo, SchedConfUpdateInfo.class),
                 MediaType.APPLICATION_JSON)
             .put(ClientResponse.class);
     assertEquals(Status.OK.getStatusCode(), response.getStatus());
@@ -378,7 +378,7 @@ public class TestRMWebServicesConfigurationMutation extends JerseyTestBase {
     ClientResponse response;
 
     // Update config value.
-    QueueConfigsUpdateInfo updateInfo = new QueueConfigsUpdateInfo();
+    SchedConfUpdateInfo updateInfo = new SchedConfUpdateInfo();
     Map<String, String> updateParam = new HashMap<>();
     updateParam.put(CapacitySchedulerConfiguration.MAXIMUM_AM_RESOURCE_SUFFIX,
         "0.2");
@@ -393,9 +393,9 @@ public class TestRMWebServicesConfigurationMutation extends JerseyTestBase {
         0.001f);
     response =
         r.path("ws").path("v1").path("cluster")
-            .path("queues").queryParam("user.name", userName)
+            .path("sched-conf").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON)
-            .entity(toJson(updateInfo, QueueConfigsUpdateInfo.class),
+            .entity(toJson(updateInfo, SchedConfUpdateInfo.class),
                 MediaType.APPLICATION_JSON)
             .put(ClientResponse.class);
     assertEquals(Status.OK.getStatusCode(), response.getStatus());
@@ -411,9 +411,9 @@ public class TestRMWebServicesConfigurationMutation extends JerseyTestBase {
     updateInfo.getUpdateQueueInfo().add(aUpdateInfo);
     response =
         r.path("ws").path("v1").path("cluster")
-            .path("queues").queryParam("user.name", userName)
+            .path("sched-conf").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON)
-            .entity(toJson(updateInfo, QueueConfigsUpdateInfo.class),
+            .entity(toJson(updateInfo, SchedConfUpdateInfo.class),
                 MediaType.APPLICATION_JSON)
             .put(ClientResponse.class);
     assertEquals(Status.OK.getStatusCode(), response.getStatus());
@@ -431,7 +431,7 @@ public class TestRMWebServicesConfigurationMutation extends JerseyTestBase {
     ClientResponse response;
 
     // Update root.a and root.b capacity to 50.
-    QueueConfigsUpdateInfo updateInfo = new QueueConfigsUpdateInfo();
+    SchedConfUpdateInfo updateInfo = new SchedConfUpdateInfo();
     Map<String, String> updateParam = new HashMap<>();
     updateParam.put(CapacitySchedulerConfiguration.CAPACITY, "50");
     QueueConfigInfo aUpdateInfo = new QueueConfigInfo("root.a", updateParam);
@@ -441,9 +441,9 @@ public class TestRMWebServicesConfigurationMutation extends JerseyTestBase {
 
     response =
         r.path("ws").path("v1").path("cluster")
-            .path("queues").queryParam("user.name", userName)
+            .path("sched-conf").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON)
-            .entity(toJson(updateInfo, QueueConfigsUpdateInfo.class),
+            .entity(toJson(updateInfo, SchedConfUpdateInfo.class),
                 MediaType.APPLICATION_JSON)
             .put(ClientResponse.class);
     assertEquals(Status.OK.getStatusCode(), response.getStatus());
@@ -453,6 +453,47 @@ public class TestRMWebServicesConfigurationMutation extends JerseyTestBase {
     assertEquals(50.0f, newCSConf.getNonLabeledQueueCapacity("root.b"), 0.01f);
   }
 
+  @Test
+  public void testGlobalConfChange() throws Exception {
+    WebResource r = resource();
+
+    ClientResponse response;
+
+    // Set maximum-applications to 30000.
+    SchedConfUpdateInfo updateInfo = new SchedConfUpdateInfo();
+    updateInfo.getGlobalParams().put(CapacitySchedulerConfiguration.PREFIX +
+        "maximum-applications", "30000");
+
+    response =
+        r.path("ws").path("v1").path("cluster")
+            .path("sched-conf").queryParam("user.name", userName)
+            .accept(MediaType.APPLICATION_JSON)
+            .entity(toJson(updateInfo, SchedConfUpdateInfo.class),
+                MediaType.APPLICATION_JSON)
+            .put(ClientResponse.class);
+    assertEquals(Status.OK.getStatusCode(), response.getStatus());
+    CapacitySchedulerConfiguration newCSConf =
+        ((CapacityScheduler) rm.getResourceScheduler()).getConfiguration();
+    assertEquals(30000, newCSConf.getMaximumSystemApplications());
+
+    updateInfo.getGlobalParams().put(CapacitySchedulerConfiguration.PREFIX +
+        "maximum-applications", null);
+    // Unset maximum-applications. Should be set to default.
+    response =
+        r.path("ws").path("v1").path("cluster")
+            .path("sched-conf").queryParam("user.name", userName)
+            .accept(MediaType.APPLICATION_JSON)
+            .entity(toJson(updateInfo, SchedConfUpdateInfo.class),
+                MediaType.APPLICATION_JSON)
+            .put(ClientResponse.class);
+    assertEquals(Status.OK.getStatusCode(), response.getStatus());
+    newCSConf =
+        ((CapacityScheduler) rm.getResourceScheduler()).getConfiguration();
+    assertEquals(CapacitySchedulerConfiguration
+        .DEFAULT_MAXIMUM_SYSTEM_APPLICATIIONS,
+        newCSConf.getMaximumSystemApplications());
+  }
+
   @Override
   @After
   public void tearDown() throws Exception {


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


[31/50] [abbrv] hadoop git commit: YARN-6965. Duplicate instantiation in FairSchedulerQueueInfo. Contributed by Masahiro Tanaka.

Posted by jh...@apache.org.
YARN-6965. Duplicate instantiation in FairSchedulerQueueInfo. Contributed by Masahiro Tanaka.


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

Branch: refs/heads/YARN-5734
Commit: 588c190afd49bdbd5708f7805bf6c68f09fee142
Parents: 75dd866
Author: Akira Ajisaka <aa...@apache.org>
Authored: Wed Aug 16 14:06:22 2017 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Wed Aug 16 14:06:22 2017 +0900

----------------------------------------------------------------------
 .../server/resourcemanager/webapp/dao/FairSchedulerQueueInfo.java   | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/588c190a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/FairSchedulerQueueInfo.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/FairSchedulerQueueInfo.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/FairSchedulerQueueInfo.java
index a4607c2..79339c7 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/FairSchedulerQueueInfo.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/FairSchedulerQueueInfo.java
@@ -99,7 +99,6 @@ public class FairSchedulerQueueInfo {
     steadyFairResources = new ResourceInfo(queue.getSteadyFairShare());
     fairResources = new ResourceInfo(queue.getFairShare());
     minResources = new ResourceInfo(queue.getMinShare());
-    maxResources = new ResourceInfo(queue.getMaxShare());
     maxResources = new ResourceInfo(
         Resources.componentwiseMin(queue.getMaxShare(),
             scheduler.getClusterResource()));


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


[44/50] [abbrv] hadoop git commit: YARN-5949. Add pluggable configuration ACL policy interface and implementation. (Jonathan Hung via wangda)

Posted by jh...@apache.org.
YARN-5949. Add pluggable configuration ACL policy interface and implementation. (Jonathan Hung via wangda)

Change-Id: Ib98e82ff753bede21fcab2e6ca9ec1e7a5a2008f


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

Branch: refs/heads/YARN-5734
Commit: 771d951e3f54c69370ed16ebdb48137911167eb0
Parents: eadb07d
Author: Wangda Tan <wa...@apache.org>
Authored: Mon May 22 13:38:31 2017 -0700
Committer: Jonathan Hung <jh...@linkedin.com>
Committed: Thu Aug 17 13:48:55 2017 -0700

----------------------------------------------------------------------
 .../hadoop/yarn/conf/YarnConfiguration.java     |   3 +
 .../src/main/resources/yarn-default.xml         |  11 ++
 .../ConfigurationMutationACLPolicy.java         |  47 ++++++
 .../ConfigurationMutationACLPolicyFactory.java  |  49 ++++++
 .../DefaultConfigurationMutationACLPolicy.java  |  45 ++++++
 .../scheduler/MutableConfScheduler.java         |  19 ++-
 .../scheduler/MutableConfigurationProvider.java |   8 +-
 .../scheduler/capacity/CapacityScheduler.java   |   6 +-
 .../conf/MutableCSConfigurationProvider.java    | 151 +++++++++++++++++-
 ...ueueAdminConfigurationMutationACLPolicy.java |  96 ++++++++++++
 .../resourcemanager/webapp/RMWebServices.java   | 131 +---------------
 .../TestConfigurationMutationACLPolicies.java   | 154 +++++++++++++++++++
 .../TestMutableCSConfigurationProvider.java     |  40 +++--
 13 files changed, 610 insertions(+), 150 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/771d951e/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 2accb31..e7cbcfa 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
@@ -641,6 +641,9 @@ public class YarnConfiguration extends Configuration {
   public static final String DEFAULT_CONFIGURATION_STORE =
       MEMORY_CONFIGURATION_STORE;
 
+  public static final String RM_SCHEDULER_MUTATION_ACL_POLICY_CLASS =
+      YARN_PREFIX + "scheduler.configuration.mutation.acl-policy.class";
+
   public static final String YARN_AUTHORIZATION_PROVIDER = YARN_PREFIX
       + "authorization-provider";
   private static final List<String> RM_SERVICES_ADDRESS_CONF_KEYS_HTTP =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/771d951e/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 49634a5..dae3a86 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
@@ -3230,4 +3230,15 @@
     <value>memory</value>
   </property>
 
+  <property>
+    <description>
+      The class to use for configuration mutation ACL policy if using a mutable
+      configuration provider. Controls whether a mutation request is allowed.
+      The DefaultConfigurationMutationACLPolicy checks if the requestor is a
+      YARN admin.
+    </description>
+    <name>yarn.scheduler.configuration.mutation.acl-policy.class</name>
+    <value>org.apache.hadoop.yarn.server.resourcemanager.scheduler.DefaultConfigurationMutationACLPolicy</value>
+  </property>
+
 </configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/771d951e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ConfigurationMutationACLPolicy.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/ConfigurationMutationACLPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ConfigurationMutationACLPolicy.java
new file mode 100644
index 0000000..724487b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ConfigurationMutationACLPolicy.java
@@ -0,0 +1,47 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.QueueConfigsUpdateInfo;
+
+/**
+ * Interface for determining whether configuration mutations are allowed.
+ */
+public interface ConfigurationMutationACLPolicy {
+
+  /**
+   * Initialize ACL policy with configuration and RMContext.
+   * @param conf Configuration to initialize with.
+   * @param rmContext rmContext
+   */
+  void init(Configuration conf, RMContext rmContext);
+
+  /**
+   * Check if mutation is allowed.
+   * @param user User issuing the request
+   * @param confUpdate configurations to be updated
+   * @return whether provided mutation is allowed or not
+   */
+  boolean isMutationAllowed(UserGroupInformation user, QueueConfigsUpdateInfo
+      confUpdate);
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/771d951e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ConfigurationMutationACLPolicyFactory.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/ConfigurationMutationACLPolicyFactory.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ConfigurationMutationACLPolicyFactory.java
new file mode 100644
index 0000000..2898785
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ConfigurationMutationACLPolicyFactory.java
@@ -0,0 +1,49 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+
+/**
+ * Factory class for creating instances of
+ * {@link ConfigurationMutationACLPolicy}.
+ */
+public final class ConfigurationMutationACLPolicyFactory {
+
+  private static final Log LOG = LogFactory.getLog(
+      ConfigurationMutationACLPolicyFactory.class);
+
+  private ConfigurationMutationACLPolicyFactory() {
+    // Unused.
+  }
+
+  public static ConfigurationMutationACLPolicy getPolicy(Configuration conf) {
+    Class<? extends ConfigurationMutationACLPolicy> policyClass =
+        conf.getClass(YarnConfiguration.RM_SCHEDULER_MUTATION_ACL_POLICY_CLASS,
+            DefaultConfigurationMutationACLPolicy.class,
+            ConfigurationMutationACLPolicy.class);
+    LOG.info("Using ConfigurationMutationACLPolicy implementation - " +
+        policyClass);
+    return ReflectionUtils.newInstance(policyClass, conf);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/771d951e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/DefaultConfigurationMutationACLPolicy.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/DefaultConfigurationMutationACLPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/DefaultConfigurationMutationACLPolicy.java
new file mode 100644
index 0000000..680c3b8
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/DefaultConfigurationMutationACLPolicy.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.resourcemanager.scheduler;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.security.YarnAuthorizationProvider;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.QueueConfigsUpdateInfo;
+
+/**
+ * Default configuration mutation ACL policy. Checks if user is YARN admin.
+ */
+public class DefaultConfigurationMutationACLPolicy implements
+    ConfigurationMutationACLPolicy {
+
+  private YarnAuthorizationProvider authorizer;
+
+  @Override
+  public void init(Configuration conf, RMContext rmContext) {
+    authorizer = YarnAuthorizationProvider.getInstance(conf);
+  }
+
+  @Override
+  public boolean isMutationAllowed(UserGroupInformation user,
+      QueueConfigsUpdateInfo confUpdate) {
+    return authorizer.isAdmin(user);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/771d951e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/MutableConfScheduler.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/MutableConfScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/MutableConfScheduler.java
index 35e36e1..93a935e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/MutableConfScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/MutableConfScheduler.java
@@ -17,10 +17,11 @@
  */
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.QueueConfigsUpdateInfo;
 
 import java.io.IOException;
-import java.util.Map;
 
 /**
  * Interface for a scheduler that supports changing configuration at runtime.
@@ -31,10 +32,22 @@ public interface MutableConfScheduler extends ResourceScheduler {
   /**
    * Update the scheduler's configuration.
    * @param user Caller of this update
-   * @param confUpdate key-value map of the configuration update
+   * @param confUpdate configuration update
    * @throws IOException if update is invalid
    */
   void updateConfiguration(UserGroupInformation user,
-      Map<String, String> confUpdate) throws IOException;
+      QueueConfigsUpdateInfo confUpdate) throws IOException;
 
+  /**
+   * Get the scheduler configuration.
+   * @return the scheduler configuration
+   */
+  Configuration getConfiguration();
+
+  /**
+   * Get queue object based on queue name.
+   * @param queueName the queue name
+   * @return the queue object
+   */
+  Queue getQueue(String queueName);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/771d951e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/MutableConfigurationProvider.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/MutableConfigurationProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/MutableConfigurationProvider.java
index 889c3bc..f04c128 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/MutableConfigurationProvider.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/MutableConfigurationProvider.java
@@ -18,8 +18,10 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
 
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.QueueConfigsUpdateInfo;
+
 import java.io.IOException;
-import java.util.Map;
 
 /**
  * Interface for allowing changing scheduler configurations.
@@ -32,7 +34,7 @@ public interface MutableConfigurationProvider {
    * @param confUpdate Key-value pairs for configurations to be updated.
    * @throws IOException if scheduler could not be reinitialized
    */
-  void mutateConfiguration(String user, Map<String, String> confUpdate)
-      throws IOException;
+  void mutateConfiguration(UserGroupInformation user, QueueConfigsUpdateInfo
+      confUpdate) throws IOException;
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/771d951e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.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/CapacityScheduler.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/CapacityScheduler.java
index f49286c..4fbccbd 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/CapacityScheduler.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/CapacityScheduler.java
@@ -135,6 +135,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.Placeme
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.SimplePlacementSet;
 import org.apache.hadoop.yarn.server.resourcemanager.security.AppPriorityACLsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.QueueConfigsUpdateInfo;
 import org.apache.hadoop.yarn.server.utils.Lock;
 import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
@@ -647,6 +648,7 @@ public class CapacityScheduler extends
     preemptionManager.refreshQueues(null, this.getRootQueue());
   }
 
+  @Override
   public CSQueue getQueue(String queueName) {
     if (queueName == null) {
       return null;
@@ -2560,10 +2562,10 @@ public class CapacityScheduler extends
 
   @Override
   public void updateConfiguration(UserGroupInformation user,
-      Map<String, String> confUpdate) throws IOException {
+      QueueConfigsUpdateInfo confUpdate) throws IOException {
     if (csConfProvider instanceof MutableConfigurationProvider) {
       ((MutableConfigurationProvider) csConfProvider).mutateConfiguration(
-          user.getShortUserName(), confUpdate);
+          user, confUpdate);
     } else {
       throw new UnsupportedOperationException("Configured CS configuration " +
           "provider does not support updating configuration.");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/771d951e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/MutableCSConfigurationProvider.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/conf/MutableCSConfigurationProvider.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/conf/MutableCSConfigurationProvider.java
index ea1b3c0..8b879b0 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/conf/MutableCSConfigurationProvider.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/conf/MutableCSConfigurationProvider.java
@@ -18,14 +18,27 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.conf;
 
+import com.google.common.base.Joiner;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ConfigurationMutationACLPolicy;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ConfigurationMutationACLPolicyFactory;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.MutableConfigurationProvider;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.conf.YarnConfigurationStore.LogMutation;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.QueueConfigInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.QueueConfigsUpdateInfo;
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 
 /**
@@ -38,6 +51,7 @@ public class MutableCSConfigurationProvider implements CSConfigurationProvider,
 
   private Configuration schedConf;
   private YarnConfigurationStore confStore;
+  private ConfigurationMutationACLPolicy aclMutationPolicy;
   private RMContext rmContext;
   private Configuration conf;
 
@@ -68,6 +82,9 @@ public class MutableCSConfigurationProvider implements CSConfigurationProvider,
       schedConf.set(kv.getKey(), kv.getValue());
     }
     confStore.initialize(config, schedConf);
+    this.aclMutationPolicy = ConfigurationMutationACLPolicyFactory
+        .getPolicy(config);
+    aclMutationPolicy.init(config, rmContext);
     this.conf = config;
   }
 
@@ -80,12 +97,17 @@ public class MutableCSConfigurationProvider implements CSConfigurationProvider,
   }
 
   @Override
-  public void mutateConfiguration(String user,
-      Map<String, String> confUpdate) throws IOException {
+  public void mutateConfiguration(UserGroupInformation user,
+      QueueConfigsUpdateInfo confUpdate) throws IOException {
+    if (!aclMutationPolicy.isMutationAllowed(user, confUpdate)) {
+      throw new AccessControlException("User is not admin of all modified" +
+          " queues.");
+    }
     Configuration oldConf = new Configuration(schedConf);
-    LogMutation log = new LogMutation(confUpdate, user);
+    Map<String, String> kvUpdate = constructKeyValueConfUpdate(confUpdate);
+    LogMutation log = new LogMutation(kvUpdate, user.getShortUserName());
     long id = confStore.logMutation(log);
-    for (Map.Entry<String, String> kv : confUpdate.entrySet()) {
+    for (Map.Entry<String, String> kv : kvUpdate.entrySet()) {
       if (kv.getValue() == null) {
         schedConf.unset(kv.getKey());
       } else {
@@ -101,4 +123,125 @@ public class MutableCSConfigurationProvider implements CSConfigurationProvider,
     }
     confStore.confirmMutation(id, true);
   }
+
+
+  private Map<String, String> constructKeyValueConfUpdate(
+      QueueConfigsUpdateInfo mutationInfo) throws IOException {
+    CapacityScheduler cs = (CapacityScheduler) rmContext.getScheduler();
+    CapacitySchedulerConfiguration proposedConf =
+        new CapacitySchedulerConfiguration(cs.getConfiguration(), false);
+    Map<String, String> confUpdate = new HashMap<>();
+    for (String queueToRemove : mutationInfo.getRemoveQueueInfo()) {
+      removeQueue(queueToRemove, proposedConf, confUpdate);
+    }
+    for (QueueConfigInfo addQueueInfo : mutationInfo.getAddQueueInfo()) {
+      addQueue(addQueueInfo, proposedConf, confUpdate);
+    }
+    for (QueueConfigInfo updateQueueInfo : mutationInfo.getUpdateQueueInfo()) {
+      updateQueue(updateQueueInfo, proposedConf, confUpdate);
+    }
+    return confUpdate;
+  }
+
+  private void removeQueue(
+      String queueToRemove, CapacitySchedulerConfiguration proposedConf,
+      Map<String, String> confUpdate) throws IOException {
+    if (queueToRemove == null) {
+      return;
+    } else {
+      CapacityScheduler cs = (CapacityScheduler) rmContext.getScheduler();
+      String queueName = queueToRemove.substring(
+          queueToRemove.lastIndexOf('.') + 1);
+      CSQueue queue = cs.getQueue(queueName);
+      if (queue == null ||
+          !queue.getQueuePath().equals(queueToRemove)) {
+        throw new IOException("Queue " + queueToRemove + " not found");
+      } else if (queueToRemove.lastIndexOf('.') == -1) {
+        throw new IOException("Can't remove queue " + queueToRemove);
+      }
+      String parentQueuePath = queueToRemove.substring(0, queueToRemove
+          .lastIndexOf('.'));
+      String[] siblingQueues = proposedConf.getQueues(parentQueuePath);
+      List<String> newSiblingQueues = new ArrayList<>();
+      for (String siblingQueue : siblingQueues) {
+        if (!siblingQueue.equals(queueName)) {
+          newSiblingQueues.add(siblingQueue);
+        }
+      }
+      proposedConf.setQueues(parentQueuePath, newSiblingQueues
+          .toArray(new String[0]));
+      String queuesConfig = CapacitySchedulerConfiguration.PREFIX
+          + parentQueuePath + CapacitySchedulerConfiguration.DOT
+          + CapacitySchedulerConfiguration.QUEUES;
+      if (newSiblingQueues.size() == 0) {
+        confUpdate.put(queuesConfig, null);
+      } else {
+        confUpdate.put(queuesConfig, Joiner.on(',').join(newSiblingQueues));
+      }
+      for (Map.Entry<String, String> confRemove : proposedConf.getValByRegex(
+          ".*" + queueToRemove.replaceAll("\\.", "\\.") + "\\..*")
+          .entrySet()) {
+        proposedConf.unset(confRemove.getKey());
+        confUpdate.put(confRemove.getKey(), null);
+      }
+    }
+  }
+
+  private void addQueue(
+      QueueConfigInfo addInfo, CapacitySchedulerConfiguration proposedConf,
+      Map<String, String> confUpdate) throws IOException {
+    if (addInfo == null) {
+      return;
+    } else {
+      CapacityScheduler cs = (CapacityScheduler) rmContext.getScheduler();
+      String queuePath = addInfo.getQueue();
+      String queueName = queuePath.substring(queuePath.lastIndexOf('.') + 1);
+      if (cs.getQueue(queueName) != null) {
+        throw new IOException("Can't add existing queue " + queuePath);
+      } else if (queuePath.lastIndexOf('.') == -1) {
+        throw new IOException("Can't add invalid queue " + queuePath);
+      }
+      String parentQueue = queuePath.substring(0, queuePath.lastIndexOf('.'));
+      String[] siblings = proposedConf.getQueues(parentQueue);
+      List<String> siblingQueues = siblings == null ? new ArrayList<>() :
+          new ArrayList<>(Arrays.<String>asList(siblings));
+      siblingQueues.add(queuePath.substring(queuePath.lastIndexOf('.') + 1));
+      proposedConf.setQueues(parentQueue,
+          siblingQueues.toArray(new String[0]));
+      confUpdate.put(CapacitySchedulerConfiguration.PREFIX
+              + parentQueue + CapacitySchedulerConfiguration.DOT
+              + CapacitySchedulerConfiguration.QUEUES,
+          Joiner.on(',').join(siblingQueues));
+      String keyPrefix = CapacitySchedulerConfiguration.PREFIX
+          + queuePath + CapacitySchedulerConfiguration.DOT;
+      for (Map.Entry<String, String> kv : addInfo.getParams().entrySet()) {
+        if (kv.getValue() == null) {
+          proposedConf.unset(keyPrefix + kv.getKey());
+        } else {
+          proposedConf.set(keyPrefix + kv.getKey(), kv.getValue());
+        }
+        confUpdate.put(keyPrefix + kv.getKey(), kv.getValue());
+      }
+    }
+  }
+
+  private void updateQueue(QueueConfigInfo updateInfo,
+      CapacitySchedulerConfiguration proposedConf,
+      Map<String, String> confUpdate) {
+    if (updateInfo == null) {
+      return;
+    } else {
+      String queuePath = updateInfo.getQueue();
+      String keyPrefix = CapacitySchedulerConfiguration.PREFIX
+          + queuePath + CapacitySchedulerConfiguration.DOT;
+      for (Map.Entry<String, String> kv : updateInfo.getParams().entrySet()) {
+        if (kv.getValue() == null) {
+          proposedConf.unset(keyPrefix + kv.getKey());
+        } else {
+          proposedConf.set(keyPrefix + kv.getKey(), kv.getValue());
+        }
+        confUpdate.put(keyPrefix + kv.getKey(), kv.getValue());
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/771d951e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/QueueAdminConfigurationMutationACLPolicy.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/conf/QueueAdminConfigurationMutationACLPolicy.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/conf/QueueAdminConfigurationMutationACLPolicy.java
new file mode 100644
index 0000000..1f94c1c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/QueueAdminConfigurationMutationACLPolicy.java
@@ -0,0 +1,96 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, 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.scheduler.capacity.conf;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.records.QueueACL;
+import org.apache.hadoop.yarn.api.records.QueueInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ConfigurationMutationACLPolicy;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.MutableConfScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.QueueConfigInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.QueueConfigsUpdateInfo;
+
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Set;
+
+/**
+ * A configuration mutation ACL policy which checks that user has admin
+ * privileges on all queues they are changing.
+ */
+public class QueueAdminConfigurationMutationACLPolicy implements
+    ConfigurationMutationACLPolicy {
+
+  private RMContext rmContext;
+
+  @Override
+  public void init(Configuration conf, RMContext context) {
+    this.rmContext = context;
+  }
+
+  @Override
+  public boolean isMutationAllowed(UserGroupInformation user,
+      QueueConfigsUpdateInfo confUpdate) {
+    Set<String> queues = new HashSet<>();
+    for (QueueConfigInfo addQueueInfo : confUpdate.getAddQueueInfo()) {
+      queues.add(addQueueInfo.getQueue());
+    }
+    for (String removeQueue : confUpdate.getRemoveQueueInfo()) {
+      queues.add(removeQueue);
+    }
+    for (QueueConfigInfo updateQueueInfo : confUpdate.getUpdateQueueInfo()) {
+      queues.add(updateQueueInfo.getQueue());
+    }
+    for (String queuePath : queues) {
+      String queueName = queuePath.lastIndexOf('.') != -1 ?
+          queuePath.substring(queuePath.lastIndexOf('.') + 1) : queuePath;
+      QueueInfo queueInfo = null;
+      try {
+        queueInfo = rmContext.getScheduler()
+            .getQueueInfo(queueName, false, false);
+      } catch (IOException e) {
+        // Queue is not found, do nothing.
+      }
+      String parentPath = queuePath;
+      // TODO: handle global config change.
+      while (queueInfo == null) {
+        // We are adding a queue (whose parent we are possibly also adding).
+        // Check ACL of lowest parent queue which already exists.
+        parentPath = parentPath.substring(0, parentPath.lastIndexOf('.'));
+        String parentName = parentPath.lastIndexOf('.') != -1 ?
+            parentPath.substring(parentPath.lastIndexOf('.') + 1) : parentPath;
+        try {
+          queueInfo = rmContext.getScheduler()
+              .getQueueInfo(parentName, false, false);
+        } catch (IOException e) {
+          // Queue is not found, do nothing.
+        }
+      }
+      Queue queue = ((MutableConfScheduler) rmContext.getScheduler())
+          .getQueue(queueInfo.getQueueName());
+      if (queue != null && !queue.hasAccess(QueueACL.ADMINISTER_QUEUE, user)) {
+        return false;
+      }
+    }
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/771d951e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
index 5ed1263..dd476aa 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
@@ -135,7 +135,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivitiesManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler;
@@ -2434,10 +2433,8 @@ public class RMWebServices extends WebServices implements RMWebServiceProtocol {
         callerUGI.doAs(new PrivilegedExceptionAction<Void>() {
           @Override
           public Void run() throws IOException, YarnException {
-            Map<String, String> confUpdate =
-                constructKeyValueConfUpdate(mutationInfo);
-            ((CapacityScheduler) scheduler).updateConfiguration(callerUGI,
-                confUpdate);
+            ((MutableConfScheduler) scheduler).updateConfiguration(callerUGI,
+                mutationInfo);
             return null;
           }
         });
@@ -2449,129 +2446,9 @@ public class RMWebServices extends WebServices implements RMWebServiceProtocol {
           "successfully applied.").build();
     } else {
       return Response.status(Status.BAD_REQUEST)
-          .entity("Configuration change only supported by CapacityScheduler.")
+          .entity("Configuration change only supported by " +
+              "MutableConfScheduler.")
           .build();
     }
   }
-
-  private Map<String, String> constructKeyValueConfUpdate(
-      QueueConfigsUpdateInfo mutationInfo) throws IOException {
-    CapacitySchedulerConfiguration currentConf =
-        ((CapacityScheduler) rm.getResourceScheduler()).getConfiguration();
-    CapacitySchedulerConfiguration proposedConf =
-        new CapacitySchedulerConfiguration(currentConf, false);
-    Map<String, String> confUpdate = new HashMap<>();
-    for (String queueToRemove : mutationInfo.getRemoveQueueInfo()) {
-      removeQueue(queueToRemove, proposedConf, confUpdate);
-    }
-    for (QueueConfigInfo addQueueInfo : mutationInfo.getAddQueueInfo()) {
-      addQueue(addQueueInfo, proposedConf, confUpdate);
-    }
-    for (QueueConfigInfo updateQueueInfo : mutationInfo.getUpdateQueueInfo()) {
-      updateQueue(updateQueueInfo, proposedConf, confUpdate);
-    }
-    return confUpdate;
-  }
-
-  private void removeQueue(
-      String queueToRemove, CapacitySchedulerConfiguration proposedConf,
-      Map<String, String> confUpdate) throws IOException {
-    if (queueToRemove == null) {
-      return;
-    } else {
-      CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
-      String queueName = queueToRemove.substring(
-          queueToRemove.lastIndexOf('.') + 1);
-      CSQueue queue = cs.getQueue(queueName);
-      if (queue == null ||
-          !queue.getQueuePath().equals(queueToRemove)) {
-        throw new IOException("Queue " + queueToRemove + " not found");
-      } else if (queueToRemove.lastIndexOf('.') == -1) {
-        throw new IOException("Can't remove queue " + queueToRemove);
-      }
-      String parentQueuePath = queueToRemove.substring(0, queueToRemove
-          .lastIndexOf('.'));
-      String[] siblingQueues = proposedConf.getQueues(parentQueuePath);
-      List<String> newSiblingQueues = new ArrayList<>();
-      for (String siblingQueue : siblingQueues) {
-        if (!siblingQueue.equals(queueName)) {
-          newSiblingQueues.add(siblingQueue);
-        }
-      }
-      proposedConf.setQueues(parentQueuePath, newSiblingQueues
-          .toArray(new String[0]));
-      String queuesConfig = CapacitySchedulerConfiguration.PREFIX +
-          parentQueuePath + CapacitySchedulerConfiguration.DOT +
-          CapacitySchedulerConfiguration.QUEUES;
-      if (newSiblingQueues.size() == 0) {
-        confUpdate.put(queuesConfig, null);
-      } else {
-        confUpdate.put(queuesConfig, Joiner.on(',').join(newSiblingQueues));
-      }
-      for (Map.Entry<String, String> confRemove : proposedConf.getValByRegex(
-          ".*" + queueToRemove.replaceAll("\\.", "\\.") + "\\..*")
-          .entrySet()) {
-        proposedConf.unset(confRemove.getKey());
-        confUpdate.put(confRemove.getKey(), null);
-      }
-    }
-  }
-
-  private void addQueue(
-      QueueConfigInfo addInfo, CapacitySchedulerConfiguration proposedConf,
-      Map<String, String> confUpdate) throws IOException {
-    if (addInfo == null) {
-      return;
-    } else {
-      CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
-      String queuePath = addInfo.getQueue();
-      String queueName = queuePath.substring(queuePath.lastIndexOf('.') + 1);
-      if (cs.getQueue(queueName) != null) {
-        throw new IOException("Can't add existing queue " + queuePath);
-      } else if (queuePath.lastIndexOf('.') == -1) {
-        throw new IOException("Can't add invalid queue " + queuePath);
-      }
-      String parentQueue = queuePath.substring(0, queuePath.lastIndexOf('.'));
-      String[] siblings = proposedConf.getQueues(parentQueue);
-      List<String> siblingQueues = siblings == null ? new ArrayList<>() :
-          new ArrayList<>(Arrays.<String>asList(siblings));
-      siblingQueues.add(queuePath.substring(queuePath.lastIndexOf('.') + 1));
-      proposedConf.setQueues(parentQueue,
-          siblingQueues.toArray(new String[0]));
-      confUpdate.put(CapacitySchedulerConfiguration.PREFIX +
-          parentQueue + CapacitySchedulerConfiguration.DOT +
-          CapacitySchedulerConfiguration.QUEUES,
-          Joiner.on(',').join(siblingQueues));
-      String keyPrefix = CapacitySchedulerConfiguration.PREFIX +
-          queuePath + CapacitySchedulerConfiguration.DOT;
-      for (Map.Entry<String, String> kv : addInfo.getParams().entrySet()) {
-        if (kv.getValue() == null) {
-          proposedConf.unset(keyPrefix + kv.getKey());
-        } else {
-          proposedConf.set(keyPrefix + kv.getKey(), kv.getValue());
-        }
-        confUpdate.put(keyPrefix + kv.getKey(), kv.getValue());
-      }
-    }
-  }
-
-  private void updateQueue(QueueConfigInfo updateInfo,
-      CapacitySchedulerConfiguration proposedConf,
-      Map<String, String> confUpdate) {
-    if (updateInfo == null) {
-      return;
-    } else {
-      String queuePath = updateInfo.getQueue();
-      String keyPrefix = CapacitySchedulerConfiguration.PREFIX +
-          queuePath + CapacitySchedulerConfiguration.DOT;
-      for (Map.Entry<String, String> kv : updateInfo.getParams().entrySet()) {
-        if (kv.getValue() == null) {
-          proposedConf.unset(keyPrefix + kv.getKey());
-        } else {
-          proposedConf.set(keyPrefix + kv.getKey(), kv.getValue());
-        }
-        confUpdate.put(keyPrefix + kv.getKey(), kv.getValue());
-      }
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/771d951e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestConfigurationMutationACLPolicies.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/scheduler/TestConfigurationMutationACLPolicies.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestConfigurationMutationACLPolicies.java
new file mode 100644
index 0000000..4016dcf
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestConfigurationMutationACLPolicies.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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, 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.scheduler;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.records.QueueACL;
+import org.apache.hadoop.yarn.api.records.QueueInfo;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.conf.QueueAdminConfigurationMutationACLPolicy;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.QueueConfigInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.QueueConfigsUpdateInfo;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Map;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.anyBoolean;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class TestConfigurationMutationACLPolicies {
+
+  private ConfigurationMutationACLPolicy policy;
+  private RMContext rmContext;
+  private MutableConfScheduler scheduler;
+
+  private static final UserGroupInformation GOOD_USER = UserGroupInformation
+      .createUserForTesting("goodUser", new String[] {});
+  private static final UserGroupInformation BAD_USER = UserGroupInformation
+      .createUserForTesting("badUser", new String[] {});
+  private static final Map<String, String> EMPTY_MAP =
+      Collections.<String, String>emptyMap();
+
+  @Before
+  public void setUp() throws IOException {
+    rmContext = mock(RMContext.class);
+    scheduler = mock(MutableConfScheduler.class);
+    when(rmContext.getScheduler()).thenReturn(scheduler);
+    mockQueue("a", scheduler);
+    mockQueue("b", scheduler);
+    mockQueue("b1", scheduler);
+  }
+
+  private void mockQueue(String queueName, MutableConfScheduler scheduler)
+      throws IOException {
+    QueueInfo queueInfo = QueueInfo.newInstance(queueName, 0, 0, 0, null, null,
+        null, null, null, null, false);
+    when(scheduler.getQueueInfo(eq(queueName), anyBoolean(), anyBoolean()))
+        .thenReturn(queueInfo);
+    Queue queue = mock(Queue.class);
+    when(queue.hasAccess(eq(QueueACL.ADMINISTER_QUEUE), eq(GOOD_USER)))
+        .thenReturn(true);
+    when(queue.hasAccess(eq(QueueACL.ADMINISTER_QUEUE), eq(BAD_USER)))
+        .thenReturn(false);
+    when(scheduler.getQueue(eq(queueName))).thenReturn(queue);
+  }
+  @Test
+  public void testDefaultPolicy() {
+    Configuration conf = new Configuration();
+    conf.set(YarnConfiguration.YARN_ADMIN_ACL, GOOD_USER.getShortUserName());
+    conf.setClass(YarnConfiguration.RM_SCHEDULER_MUTATION_ACL_POLICY_CLASS,
+        DefaultConfigurationMutationACLPolicy.class,
+        ConfigurationMutationACLPolicy.class);
+    policy = ConfigurationMutationACLPolicyFactory.getPolicy(conf);
+    policy.init(conf, rmContext);
+    assertTrue(policy.isMutationAllowed(GOOD_USER, null));
+    assertFalse(policy.isMutationAllowed(BAD_USER, null));
+  }
+  
+  @Test
+  public void testQueueAdminBasedPolicy() {
+    Configuration conf = new Configuration();
+    conf.setClass(YarnConfiguration.RM_SCHEDULER_MUTATION_ACL_POLICY_CLASS,
+        QueueAdminConfigurationMutationACLPolicy.class,
+        ConfigurationMutationACLPolicy.class);
+    policy = ConfigurationMutationACLPolicyFactory.getPolicy(conf);
+    policy.init(conf, rmContext);
+    QueueConfigsUpdateInfo updateInfo = new QueueConfigsUpdateInfo();
+    QueueConfigInfo configInfo = new QueueConfigInfo("root.a", EMPTY_MAP);
+    updateInfo.getUpdateQueueInfo().add(configInfo);
+    assertTrue(policy.isMutationAllowed(GOOD_USER, updateInfo));
+    assertFalse(policy.isMutationAllowed(BAD_USER, updateInfo));
+  }
+
+  @Test
+  public void testQueueAdminPolicyAddQueue() {
+    Configuration conf = new Configuration();
+    conf.setClass(YarnConfiguration.RM_SCHEDULER_MUTATION_ACL_POLICY_CLASS,
+        QueueAdminConfigurationMutationACLPolicy.class,
+        ConfigurationMutationACLPolicy.class);
+    policy = ConfigurationMutationACLPolicyFactory.getPolicy(conf);
+    policy.init(conf, rmContext);
+    // Add root.b.b1. Should check ACL of root.b queue.
+    QueueConfigsUpdateInfo updateInfo = new QueueConfigsUpdateInfo();
+    QueueConfigInfo configInfo = new QueueConfigInfo("root.b.b2", EMPTY_MAP);
+    updateInfo.getAddQueueInfo().add(configInfo);
+    assertTrue(policy.isMutationAllowed(GOOD_USER, updateInfo));
+    assertFalse(policy.isMutationAllowed(BAD_USER, updateInfo));
+  }
+
+  @Test
+  public void testQueueAdminPolicyAddNestedQueue() {
+    Configuration conf = new Configuration();
+    conf.setClass(YarnConfiguration.RM_SCHEDULER_MUTATION_ACL_POLICY_CLASS,
+        QueueAdminConfigurationMutationACLPolicy.class,
+        ConfigurationMutationACLPolicy.class);
+    policy = ConfigurationMutationACLPolicyFactory.getPolicy(conf);
+    policy.init(conf, rmContext);
+    // Add root.b.b1.b11. Should check ACL of root.b queue.
+    QueueConfigsUpdateInfo updateInfo = new QueueConfigsUpdateInfo();
+    QueueConfigInfo configInfo = new QueueConfigInfo("root.b.b2.b21", EMPTY_MAP);
+    updateInfo.getAddQueueInfo().add(configInfo);
+    assertTrue(policy.isMutationAllowed(GOOD_USER, updateInfo));
+    assertFalse(policy.isMutationAllowed(BAD_USER, updateInfo));
+  }
+
+  @Test
+  public void testQueueAdminPolicyRemoveQueue() {
+    Configuration conf = new Configuration();
+    conf.setClass(YarnConfiguration.RM_SCHEDULER_MUTATION_ACL_POLICY_CLASS,
+        QueueAdminConfigurationMutationACLPolicy.class,
+        ConfigurationMutationACLPolicy.class);
+    policy = ConfigurationMutationACLPolicyFactory.getPolicy(conf);
+    policy.init(conf, rmContext);
+    // Remove root.b.b1.
+    QueueConfigsUpdateInfo updateInfo = new QueueConfigsUpdateInfo();
+    updateInfo.getRemoveQueueInfo().add("root.b.b1");
+    assertTrue(policy.isMutationAllowed(GOOD_USER, updateInfo));
+    assertFalse(policy.isMutationAllowed(BAD_USER, updateInfo));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/771d951e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/TestMutableCSConfigurationProvider.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/scheduler/capacity/conf/TestMutableCSConfigurationProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/TestMutableCSConfigurationProvider.java
index 254da31..13229b1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/TestMutableCSConfigurationProvider.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/TestMutableCSConfigurationProvider.java
@@ -19,8 +19,12 @@
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.conf;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.QueueConfigInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.QueueConfigsUpdateInfo;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -43,22 +47,34 @@ public class TestMutableCSConfigurationProvider {
 
   private MutableCSConfigurationProvider confProvider;
   private RMContext rmContext;
-  private Map<String, String> goodUpdate;
-  private Map<String, String> badUpdate;
+  private QueueConfigsUpdateInfo goodUpdate;
+  private QueueConfigsUpdateInfo badUpdate;
   private CapacityScheduler cs;
 
-  private static final String TEST_USER = "testUser";
+  private static final UserGroupInformation TEST_USER = UserGroupInformation
+      .createUserForTesting("testUser", new String[] {});
 
   @Before
   public void setUp() {
     cs = mock(CapacityScheduler.class);
     rmContext = mock(RMContext.class);
     when(rmContext.getScheduler()).thenReturn(cs);
+    when(cs.getConfiguration()).thenReturn(
+        new CapacitySchedulerConfiguration());
     confProvider = new MutableCSConfigurationProvider(rmContext);
-    goodUpdate = new HashMap<>();
-    goodUpdate.put("goodKey", "goodVal");
-    badUpdate = new HashMap<>();
-    badUpdate.put("badKey", "badVal");
+    goodUpdate = new QueueConfigsUpdateInfo();
+    Map<String, String> goodUpdateMap = new HashMap<>();
+    goodUpdateMap.put("goodKey", "goodVal");
+    QueueConfigInfo goodUpdateInfo = new
+        QueueConfigInfo("root.a", goodUpdateMap);
+    goodUpdate.getUpdateQueueInfo().add(goodUpdateInfo);
+
+    badUpdate = new QueueConfigsUpdateInfo();
+    Map<String, String> badUpdateMap = new HashMap<>();
+    badUpdateMap.put("badKey", "badVal");
+    QueueConfigInfo badUpdateInfo = new
+        QueueConfigInfo("root.a", badUpdateMap);
+    badUpdate.getUpdateQueueInfo().add(badUpdateInfo);
   }
 
   @Test
@@ -66,15 +82,16 @@ public class TestMutableCSConfigurationProvider {
     Configuration conf = new Configuration();
     confProvider.init(conf);
     assertNull(confProvider.loadConfiguration(conf)
-        .get("goodKey"));
+        .get("yarn.scheduler.capacity.root.a.goodKey"));
 
     doNothing().when(cs).reinitialize(any(Configuration.class),
         any(RMContext.class));
     confProvider.mutateConfiguration(TEST_USER, goodUpdate);
     assertEquals("goodVal", confProvider.loadConfiguration(conf)
-        .get("goodKey"));
+        .get("yarn.scheduler.capacity.root.a.goodKey"));
 
-    assertNull(confProvider.loadConfiguration(conf).get("badKey"));
+    assertNull(confProvider.loadConfiguration(conf).get(
+        "yarn.scheduler.capacity.root.a.badKey"));
     doThrow(new IOException()).when(cs).reinitialize(any(Configuration.class),
         any(RMContext.class));
     try {
@@ -82,6 +99,7 @@ public class TestMutableCSConfigurationProvider {
     } catch (IOException e) {
       // Expected exception.
     }
-    assertNull(confProvider.loadConfiguration(conf).get("badKey"));
+    assertNull(confProvider.loadConfiguration(conf).get(
+        "yarn.scheduler.capacity.root.a.badKey"));
   }
 }


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


[14/50] [abbrv] hadoop git commit: HDFS-12162. Update listStatus document to describe the behavior when the argument is a file. Contributed by Ajay Kumar.

Posted by jh...@apache.org.
HDFS-12162. Update listStatus document to describe the behavior when the argument is a file. Contributed by Ajay Kumar.


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

Branch: refs/heads/YARN-5734
Commit: d72124a44268e21ada036242bfbccafc23c52ed0
Parents: 18f3603
Author: Anu Engineer <ae...@apache.org>
Authored: Mon Aug 14 11:32:49 2017 -0700
Committer: Anu Engineer <ae...@apache.org>
Committed: Mon Aug 14 11:32:49 2017 -0700

----------------------------------------------------------------------
 .../hadoop/fs/http/server/FSOperations.java     |  2 +-
 .../hadoop-hdfs/src/site/markdown/WebHDFS.md    | 39 ++++++++++++++++++++
 2 files changed, 40 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d72124a4/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 c008802..4b5918a 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
@@ -669,7 +669,7 @@ public class FSOperations {
     /**
      * Creates a list-status executor.
      *
-     * @param path the directory to retrieve the status of its contents.
+     * @param path the directory/file to retrieve the status of its contents.
      * @param filter glob filter to use.
      *
      * @throws IOException thrown if the filter expression is incorrect.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d72124a4/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md
index 7544c80..03834eb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md
@@ -495,6 +495,45 @@ See also: [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).getFileSt
 
 See also: [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).listStatus
 
+### List a File
+
+* Submit a HTTP GET request.
+
+        curl -i  "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?op=LISTSTATUS"
+
+    The client receives a response with a [`FileStatuses` JSON object](#FileStatuses_JSON_Schema):
+
+        HTTP/1.1 200 OK
+        Content-Type: application/json
+        Content-Length: 427
+
+        {
+          "FileStatuses":
+          {
+            "FileStatus":
+            [
+              {
+                "accessTime"      : 1320171722771,
+                "blockSize"       : 33554432,
+                "childrenNum"     : 0,
+                "fileId"          : 16390,
+                "group"           : "supergroup",
+                "length"          : 1366,
+                "modificationTime": 1501770633062,
+                "owner"           : "webuser",
+                "pathSuffix"      : "",
+                "permission"      : "644",
+                "replication"     : 1,
+                "storagePolicy"   : 0,
+                "type"            : "FILE"
+              }
+            ]
+          }
+        }
+
+See also: [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).listStatus
+
+
 ### Iteratively List a Directory
 
 * Submit a HTTP GET request.


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


[06/50] [abbrv] hadoop git commit: YARN-6896. Federation: routing REST invocations transparently to multiple RMs (part 1 - basic execution). (Contributed by Giovanni Matteo Fumarola via curino)

Posted by jh...@apache.org.
YARN-6896. Federation: routing REST invocations transparently to multiple RMs (part 1 - basic execution). (Contributed by Giovanni Matteo Fumarola via curino)


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

Branch: refs/heads/YARN-5734
Commit: cc59b5fb26ccf58dffcd8850fa12ec65250f127d
Parents: 0996acd
Author: Carlo Curino <cu...@apache.org>
Authored: Fri Aug 11 15:58:01 2017 -0700
Committer: Carlo Curino <cu...@apache.org>
Committed: Fri Aug 11 15:58:01 2017 -0700

----------------------------------------------------------------------
 .../hadoop/yarn/conf/YarnConfiguration.java     |  10 +
 .../yarn/conf/TestYarnConfigurationFields.java  |   2 +
 .../webapp/DefaultRequestInterceptorREST.java   |  16 +-
 .../webapp/FederationInterceptorREST.java       | 750 +++++++++++++++++++
 .../webapp/BaseRouterWebServicesTest.java       |  37 +-
 .../MockDefaultRequestInterceptorREST.java      | 136 ++++
 .../webapp/TestFederationInterceptorREST.java   | 379 ++++++++++
 .../TestFederationInterceptorRESTRetry.java     | 274 +++++++
 .../TestableFederationInterceptorREST.java      |  54 ++
 .../src/site/markdown/Federation.md             |   2 +-
 10 files changed, 1646 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc59b5fb/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 cd4d569..8acaef8 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
@@ -2721,6 +2721,16 @@ public class YarnConfiguration extends Configuration {
       "org.apache.hadoop.yarn.server.router.webapp."
           + "DefaultRequestInterceptorREST";
 
+  /**
+   * The interceptor class used in FederationInterceptorREST to communicate with
+   * each SubCluster.
+   */
+  public static final String ROUTER_WEBAPP_DEFAULT_INTERCEPTOR_CLASS =
+      ROUTER_WEBAPP_PREFIX + "default-interceptor-class";
+  public static final String DEFAULT_ROUTER_WEBAPP_DEFAULT_INTERCEPTOR_CLASS =
+      "org.apache.hadoop.yarn.server.router.webapp."
+          + "DefaultRequestInterceptorREST";
+
   ////////////////////////////////
   // Other Configs
   ////////////////////////////////

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc59b5fb/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 b9ad31a..91a8b0a 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
@@ -81,6 +81,8 @@ public class TestYarnConfigurationFields extends TestConfigurationFieldsBase {
         .add(YarnConfiguration.ROUTER_CLIENTRM_ADDRESS);
     configurationPropsToSkipCompare
         .add(YarnConfiguration.ROUTER_RMADMIN_ADDRESS);
+    configurationPropsToSkipCompare
+        .add(YarnConfiguration.ROUTER_WEBAPP_DEFAULT_INTERCEPTOR_CLASS);
 
     // Federation policies configs to be ignored
     configurationPropsToSkipCompare

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc59b5fb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/DefaultRequestInterceptorREST.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/DefaultRequestInterceptorREST.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/DefaultRequestInterceptorREST.java
index aa8e3eb..abd8ca6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/DefaultRequestInterceptorREST.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/DefaultRequestInterceptorREST.java
@@ -30,6 +30,7 @@ import javax.ws.rs.core.Response;
 import org.apache.hadoop.security.authorize.AuthorizationException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ActivitiesInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppActivitiesInfo;
@@ -66,10 +67,23 @@ import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
  * implementation that simply forwards the client requests to the resource
  * manager.
  */
-public final class DefaultRequestInterceptorREST
+public class DefaultRequestInterceptorREST
     extends AbstractRESTRequestInterceptor {
 
   private String webAppAddress;
+  private SubClusterId subClusterId = null;
+
+  public void setWebAppAddress(String webAppAddress) {
+    this.webAppAddress = webAppAddress;
+  }
+
+  protected void setSubClusterId(SubClusterId scId) {
+    this.subClusterId = scId;
+  }
+
+  protected SubClusterId getSubClusterId() {
+    return this.subClusterId;
+  }
 
   @Override
   public void init(String user) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc59b5fb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/FederationInterceptorREST.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/FederationInterceptorREST.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/FederationInterceptorREST.java
new file mode 100644
index 0000000..8ecc19d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/FederationInterceptorREST.java
@@ -0,0 +1,750 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.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.router.webapp;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.commons.lang.NotImplementedException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.authorize.AuthorizationException;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.hadoop.yarn.server.federation.policies.FederationPolicyUtils;
+import org.apache.hadoop.yarn.server.federation.policies.RouterPolicyFacade;
+import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
+import org.apache.hadoop.yarn.server.federation.store.records.ApplicationHomeSubCluster;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
+import org.apache.hadoop.yarn.server.federation.utils.FederationStateStoreFacade;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWebAppUtil;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.*;
+import org.apache.hadoop.yarn.server.router.RouterServerUtil;
+import org.apache.hadoop.yarn.server.webapp.dao.AppAttemptInfo;
+import org.apache.hadoop.yarn.server.webapp.dao.ContainerInfo;
+import org.apache.hadoop.yarn.server.webapp.dao.ContainersInfo;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.Response.Status;
+import java.io.IOException;
+import java.util.*;
+
+/**
+ * Extends the {@code AbstractRESTRequestInterceptor} class and provides an
+ * implementation for federation of YARN RM and scaling an application across
+ * multiple YARN SubClusters. All the federation specific implementation is
+ * encapsulated in this class. This is always the last intercepter in the chain.
+ */
+public class FederationInterceptorREST extends AbstractRESTRequestInterceptor {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(FederationInterceptorREST.class);
+
+  private int numSubmitRetries;
+  private FederationStateStoreFacade federationFacade;
+  private Random rand;
+  private RouterPolicyFacade policyFacade;
+
+  private Map<SubClusterId, DefaultRequestInterceptorREST> interceptors;
+
+  @Override
+  public void init(String user) {
+    federationFacade = FederationStateStoreFacade.getInstance();
+    rand = new Random(System.currentTimeMillis());
+
+    final Configuration conf = this.getConf();
+
+    try {
+      policyFacade = new RouterPolicyFacade(conf, federationFacade,
+          this.federationFacade.getSubClusterResolver(), null);
+    } catch (FederationPolicyInitializationException e) {
+      LOG.error(e.getMessage());
+    }
+
+    numSubmitRetries =
+        conf.getInt(YarnConfiguration.ROUTER_CLIENTRM_SUBMIT_RETRY,
+            YarnConfiguration.DEFAULT_ROUTER_CLIENTRM_SUBMIT_RETRY);
+
+    interceptors = new HashMap<SubClusterId, DefaultRequestInterceptorREST>();
+  }
+
+  private SubClusterId getRandomActiveSubCluster(
+      Map<SubClusterId, SubClusterInfo> activeSubclusters,
+      List<SubClusterId> blackListSubClusters) throws YarnException {
+
+    if (activeSubclusters == null || activeSubclusters.size() < 1) {
+      RouterServerUtil.logAndThrowException(
+          FederationPolicyUtils.NO_ACTIVE_SUBCLUSTER_AVAILABLE, null);
+    }
+    List<SubClusterId> list = new ArrayList<>(activeSubclusters.keySet());
+
+    FederationPolicyUtils.validateSubClusterAvailability(list,
+        blackListSubClusters);
+
+    if (blackListSubClusters != null) {
+
+      // Remove from the active SubClusters from StateStore the blacklisted ones
+      for (SubClusterId scId : blackListSubClusters) {
+        list.remove(scId);
+      }
+    }
+
+    return list.get(rand.nextInt(list.size()));
+  }
+
+  @VisibleForTesting
+  protected DefaultRequestInterceptorREST getInterceptorForSubCluster(
+      SubClusterId subClusterId) {
+    if (interceptors.containsKey(subClusterId)) {
+      return interceptors.get(subClusterId);
+    } else {
+      LOG.error("The interceptor for SubCluster " + subClusterId
+          + " does not exist in the cache.");
+      return null;
+    }
+  }
+
+  private DefaultRequestInterceptorREST createInterceptorForSubCluster(
+      SubClusterId subClusterId, String webAppAddress) {
+
+    final Configuration conf = this.getConf();
+
+    String interceptorClassName =
+        conf.get(YarnConfiguration.ROUTER_WEBAPP_DEFAULT_INTERCEPTOR_CLASS,
+            YarnConfiguration.DEFAULT_ROUTER_WEBAPP_DEFAULT_INTERCEPTOR_CLASS);
+    DefaultRequestInterceptorREST interceptorInstance = null;
+    try {
+      Class<?> interceptorClass = conf.getClassByName(interceptorClassName);
+      if (DefaultRequestInterceptorREST.class
+          .isAssignableFrom(interceptorClass)) {
+        interceptorInstance = (DefaultRequestInterceptorREST) ReflectionUtils
+            .newInstance(interceptorClass, conf);
+
+      } else {
+        throw new YarnRuntimeException(
+            "Class: " + interceptorClassName + " not instance of "
+                + DefaultRequestInterceptorREST.class.getCanonicalName());
+      }
+    } catch (ClassNotFoundException e) {
+      throw new YarnRuntimeException(
+          "Could not instantiate ApplicationMasterRequestInterceptor: "
+              + interceptorClassName,
+          e);
+    }
+
+    interceptorInstance.setWebAppAddress(webAppAddress);
+    interceptorInstance.setSubClusterId(subClusterId);
+    interceptors.put(subClusterId, interceptorInstance);
+    return interceptorInstance;
+  }
+
+  @VisibleForTesting
+  protected DefaultRequestInterceptorREST getOrCreateInterceptorForSubCluster(
+      SubClusterId subClusterId, String webAppAddress) {
+    DefaultRequestInterceptorREST interceptor =
+        getInterceptorForSubCluster(subClusterId);
+    if (interceptor == null) {
+      interceptor = createInterceptorForSubCluster(subClusterId, webAppAddress);
+    }
+    return interceptor;
+  }
+
+  /**
+   * Yarn Router forwards every getNewApplication requests to any RM. During
+   * this operation there will be no communication with the State Store. The
+   * Router will forward the requests to any SubCluster. The Router will retry
+   * to submit the request on #numSubmitRetries different SubClusters. The
+   * SubClusters are randomly chosen from the active ones.
+   * <p>
+   * Possible failures and behaviors:
+   * <p>
+   * Client: identical behavior as {@code RMWebServices}.
+   * <p>
+   * Router: the Client will timeout and resubmit.
+   * <p>
+   * ResourceManager: the Router will timeout and contacts another RM.
+   * <p>
+   * StateStore: not in the execution.
+   */
+  @Override
+  public Response createNewApplication(HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException {
+    Map<SubClusterId, SubClusterInfo> subClustersActive;
+    try {
+      subClustersActive = federationFacade.getSubClusters(true);
+    } catch (YarnException e) {
+      return Response.status(Status.INTERNAL_SERVER_ERROR)
+          .entity(e.getLocalizedMessage()).build();
+    }
+
+    List<SubClusterId> blacklist = new ArrayList<SubClusterId>();
+
+    for (int i = 0; i < numSubmitRetries; ++i) {
+
+      SubClusterId subClusterId;
+      try {
+        subClusterId = getRandomActiveSubCluster(subClustersActive, blacklist);
+      } catch (YarnException e) {
+        return Response.status(Status.SERVICE_UNAVAILABLE)
+            .entity(e.getLocalizedMessage()).build();
+      }
+
+      LOG.debug(
+          "getNewApplication try #" + i + " on SubCluster " + subClusterId);
+
+      DefaultRequestInterceptorREST interceptor =
+          getOrCreateInterceptorForSubCluster(subClusterId,
+              subClustersActive.get(subClusterId).getRMWebServiceAddress());
+      Response response = null;
+      try {
+        response = interceptor.createNewApplication(hsr);
+      } catch (Exception e) {
+        LOG.warn("Unable to create a new ApplicationId in SubCluster "
+            + subClusterId.getId(), e);
+      }
+
+      if (response != null && response.getStatus() == 200) {
+        return response;
+      } else {
+        // Empty response from the ResourceManager.
+        // Blacklist this subcluster for this request.
+        blacklist.add(subClusterId);
+      }
+    }
+
+    String errMsg = "Fail to create a new application.";
+    LOG.error(errMsg);
+    return Response.status(Status.INTERNAL_SERVER_ERROR).entity(errMsg).build();
+  }
+
+  /**
+   * Today, in YARN there are no checks of any applicationId submitted.
+   * <p>
+   * Base scenarios:
+   * <p>
+   * The Client submits an application to the Router. • The Router selects one
+   * SubCluster to forward the request. • The Router inserts a tuple into
+   * StateStore with the selected SubCluster (e.g. SC1) and the appId. • The
+   * State Store replies with the selected SubCluster (e.g. SC1). • The Router
+   * submits the request to the selected SubCluster.
+   * <p>
+   * In case of State Store failure:
+   * <p>
+   * The client submits an application to the Router. • The Router selects one
+   * SubCluster to forward the request. • The Router inserts a tuple into State
+   * Store with the selected SubCluster (e.g. SC1) and the appId. • Due to the
+   * State Store down the Router times out and it will retry depending on the
+   * FederationFacade settings. • The Router replies to the client with an error
+   * message.
+   * <p>
+   * If State Store fails after inserting the tuple: identical behavior as
+   * {@code RMWebServices}.
+   * <p>
+   * In case of Router failure:
+   * <p>
+   * Scenario 1 – Crash before submission to the ResourceManager
+   * <p>
+   * The Client submits an application to the Router. • The Router selects one
+   * SubCluster to forward the request. • The Router inserts a tuple into State
+   * Store with the selected SubCluster (e.g. SC1) and the appId. • The Router
+   * crashes. • The Client timeouts and resubmits the application. • The Router
+   * selects one SubCluster to forward the request. • The Router inserts a tuple
+   * into State Store with the selected SubCluster (e.g. SC2) and the appId. •
+   * Because the tuple is already inserted in the State Store, it returns the
+   * previous selected SubCluster (e.g. SC1). • The Router submits the request
+   * to the selected SubCluster (e.g. SC1).
+   * <p>
+   * Scenario 2 – Crash after submission to the ResourceManager
+   * <p>
+   * • The Client submits an application to the Router. • The Router selects one
+   * SubCluster to forward the request. • The Router inserts a tuple into State
+   * Store with the selected SubCluster (e.g. SC1) and the appId. • The Router
+   * submits the request to the selected SubCluster. • The Router crashes. • The
+   * Client timeouts and resubmit the application. • The Router selects one
+   * SubCluster to forward the request. • The Router inserts a tuple into State
+   * Store with the selected SubCluster (e.g. SC2) and the appId. • The State
+   * Store replies with the selected SubCluster (e.g. SC1). • The Router submits
+   * the request to the selected SubCluster (e.g. SC1). When a client re-submits
+   * the same application to the same RM, it does not raise an exception and
+   * replies with operation successful message.
+   * <p>
+   * In case of Client failure: identical behavior as {@code RMWebServices}.
+   * <p>
+   * In case of ResourceManager failure:
+   * <p>
+   * The Client submits an application to the Router. • The Router selects one
+   * SubCluster to forward the request. • The Router inserts a tuple into State
+   * Store with the selected SubCluster (e.g. SC1) and the appId. • The Router
+   * submits the request to the selected SubCluster. • The entire SubCluster is
+   * down – all the RMs in HA or the master RM is not reachable. • The Router
+   * times out. • The Router selects a new SubCluster to forward the request. •
+   * The Router update a tuple into State Store with the selected SubCluster
+   * (e.g. SC2) and the appId. • The State Store replies with OK answer. • The
+   * Router submits the request to the selected SubCluster (e.g. SC2).
+   */
+  @Override
+  public Response submitApplication(ApplicationSubmissionContextInfo newApp,
+      HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException {
+    if (newApp == null || newApp.getApplicationId() == null) {
+      String errMsg = "Missing ApplicationSubmissionContextInfo or "
+          + "applicationSubmissionContex information.";
+      return Response.status(Status.BAD_REQUEST).entity(errMsg).build();
+    }
+
+    ApplicationId applicationId = null;
+    try {
+      applicationId = ApplicationId.fromString(newApp.getApplicationId());
+    } catch (IllegalArgumentException e) {
+      return Response.status(Status.BAD_REQUEST).entity(e.getLocalizedMessage())
+          .build();
+    }
+
+    List<SubClusterId> blacklist = new ArrayList<SubClusterId>();
+
+    for (int i = 0; i < numSubmitRetries; ++i) {
+
+      ApplicationSubmissionContext context =
+          RMWebAppUtil.createAppSubmissionContext(newApp, this.getConf());
+
+      SubClusterId subClusterId = null;
+      try {
+        subClusterId = policyFacade.getHomeSubcluster(context, blacklist);
+      } catch (YarnException e) {
+        return Response.status(Status.SERVICE_UNAVAILABLE)
+            .entity(e.getLocalizedMessage()).build();
+      }
+      LOG.info("submitApplication appId" + applicationId + " try #" + i
+          + " on SubCluster " + subClusterId);
+
+      ApplicationHomeSubCluster appHomeSubCluster =
+          ApplicationHomeSubCluster.newInstance(applicationId, subClusterId);
+
+      if (i == 0) {
+        try {
+          // persist the mapping of applicationId and the subClusterId which has
+          // been selected as its home
+          subClusterId =
+              federationFacade.addApplicationHomeSubCluster(appHomeSubCluster);
+        } catch (YarnException e) {
+          String errMsg = "Unable to insert the ApplicationId " + applicationId
+              + " into the FederationStateStore";
+          return Response.status(Status.SERVICE_UNAVAILABLE)
+              .entity(errMsg + " " + e.getLocalizedMessage()).build();
+        }
+      } else {
+        try {
+          // update the mapping of applicationId and the home subClusterId to
+          // the new subClusterId we have selected
+          federationFacade.updateApplicationHomeSubCluster(appHomeSubCluster);
+        } catch (YarnException e) {
+          String errMsg = "Unable to update the ApplicationId " + applicationId
+              + " into the FederationStateStore";
+          SubClusterId subClusterIdInStateStore;
+          try {
+            subClusterIdInStateStore =
+                federationFacade.getApplicationHomeSubCluster(applicationId);
+          } catch (YarnException e1) {
+            return Response.status(Status.SERVICE_UNAVAILABLE)
+                .entity(e1.getLocalizedMessage()).build();
+          }
+          if (subClusterId == subClusterIdInStateStore) {
+            LOG.info("Application " + applicationId
+                + " already submitted on SubCluster " + subClusterId);
+          } else {
+            return Response.status(Status.SERVICE_UNAVAILABLE).entity(errMsg)
+                .build();
+          }
+        }
+      }
+
+      SubClusterInfo subClusterInfo;
+      try {
+        subClusterInfo = federationFacade.getSubCluster(subClusterId);
+      } catch (YarnException e) {
+        return Response.status(Status.SERVICE_UNAVAILABLE)
+            .entity(e.getLocalizedMessage()).build();
+      }
+
+      Response response = null;
+      try {
+        response = getOrCreateInterceptorForSubCluster(subClusterId,
+            subClusterInfo.getRMWebServiceAddress()).submitApplication(newApp,
+                hsr);
+      } catch (Exception e) {
+        LOG.warn("Unable to submit the application " + applicationId
+            + "to SubCluster " + subClusterId.getId(), e);
+      }
+
+      if (response != null && response.getStatus() == 202) {
+        LOG.info("Application " + context.getApplicationName() + " with appId "
+            + applicationId + " submitted on " + subClusterId);
+        return response;
+      } else {
+        // Empty response from the ResourceManager.
+        // Blacklist this subcluster for this request.
+        blacklist.add(subClusterId);
+      }
+    }
+
+    String errMsg = "Application " + newApp.getApplicationName()
+        + " with appId " + applicationId + " failed to be submitted.";
+    LOG.error(errMsg);
+    return Response.status(Status.SERVICE_UNAVAILABLE).entity(errMsg).build();
+  }
+
+  /**
+   * The Yarn Router will forward to the respective Yarn RM in which the AM is
+   * running.
+   * <p>
+   * Possible failure:
+   * <p>
+   * Client: identical behavior as {@code RMWebServices}.
+   * <p>
+   * Router: the Client will timeout and resubmit the request.
+   * <p>
+   * ResourceManager: the Router will timeout and the call will fail.
+   * <p>
+   * State Store: the Router will timeout and it will retry depending on the
+   * FederationFacade settings - if the failure happened before the select
+   * operation.
+   */
+  @Override
+  public AppInfo getApp(HttpServletRequest hsr, String appId,
+      Set<String> unselectedFields) {
+
+    ApplicationId applicationId = null;
+    try {
+      applicationId = ApplicationId.fromString(appId);
+    } catch (IllegalArgumentException e) {
+      return null;
+    }
+
+    SubClusterInfo subClusterInfo = null;
+    SubClusterId subClusterId = null;
+    try {
+      subClusterId =
+          federationFacade.getApplicationHomeSubCluster(applicationId);
+      if (subClusterId == null) {
+        return null;
+      }
+      subClusterInfo = federationFacade.getSubCluster(subClusterId);
+    } catch (YarnException e) {
+      return null;
+    }
+
+    return getOrCreateInterceptorForSubCluster(subClusterId,
+        subClusterInfo.getRMWebServiceAddress()).getApp(hsr, appId,
+            unselectedFields);
+  }
+
+  /**
+   * The Yarn Router will forward to the respective Yarn RM in which the AM is
+   * running.
+   * <p>
+   * Possible failures and behaviors:
+   * <p>
+   * Client: identical behavior as {@code RMWebServices}.
+   * <p>
+   * Router: the Client will timeout and resubmit the request.
+   * <p>
+   * ResourceManager: the Router will timeout and the call will fail.
+   * <p>
+   * State Store: the Router will timeout and it will retry depending on the
+   * FederationFacade settings - if the failure happened before the select
+   * operation.
+   */
+  @Override
+  public Response updateAppState(AppState targetState, HttpServletRequest hsr,
+      String appId) throws AuthorizationException, YarnException,
+      InterruptedException, IOException {
+
+    ApplicationId applicationId = null;
+    try {
+      applicationId = ApplicationId.fromString(appId);
+    } catch (IllegalArgumentException e) {
+      return Response.status(Status.BAD_REQUEST).entity(e.getLocalizedMessage())
+          .build();
+    }
+
+    SubClusterId subClusterId =
+        federationFacade.getApplicationHomeSubCluster(applicationId);
+
+    SubClusterInfo subClusterInfo =
+        federationFacade.getSubCluster(subClusterId);
+
+    return getOrCreateInterceptorForSubCluster(subClusterId,
+        subClusterInfo.getRMWebServiceAddress()).updateAppState(targetState,
+            hsr, appId);
+  }
+
+  @Override
+  public ClusterInfo get() {
+    return getClusterInfo();
+  }
+
+  @Override
+  public ClusterInfo getClusterInfo() {
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public ClusterMetricsInfo getClusterMetricsInfo() {
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public SchedulerTypeInfo getSchedulerInfo() {
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public String dumpSchedulerLogs(String time, HttpServletRequest hsr)
+      throws IOException {
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public NodesInfo getNodes(String states) {
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public NodeInfo getNode(String nodeId) {
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public ActivitiesInfo getActivities(HttpServletRequest hsr, String nodeId) {
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public AppActivitiesInfo getAppActivities(HttpServletRequest hsr,
+      String appId, String time) {
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public ApplicationStatisticsInfo getAppStatistics(HttpServletRequest hsr,
+      Set<String> stateQueries, Set<String> typeQueries) {
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public AppsInfo getApps(HttpServletRequest hsr, String stateQuery,
+      Set<String> statesQuery, String finalStatusQuery, String userQuery,
+      String queueQuery, String count, String startedBegin, String startedEnd,
+      String finishBegin, String finishEnd, Set<String> applicationTypes,
+      Set<String> applicationTags, Set<String> unselectedFields) {
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public AppState getAppState(HttpServletRequest hsr, String appId)
+      throws AuthorizationException {
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public NodeToLabelsInfo getNodeToLabels(HttpServletRequest hsr)
+      throws IOException {
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public LabelsToNodesInfo getLabelsToNodes(Set<String> labels)
+      throws IOException {
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public Response replaceLabelsOnNodes(NodeToLabelsEntryList newNodeToLabels,
+      HttpServletRequest hsr) throws IOException {
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public Response replaceLabelsOnNode(Set<String> newNodeLabelsName,
+      HttpServletRequest hsr, String nodeId) throws Exception {
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public NodeLabelsInfo getClusterNodeLabels(HttpServletRequest hsr)
+      throws IOException {
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public Response addToClusterNodeLabels(NodeLabelsInfo newNodeLabels,
+      HttpServletRequest hsr) throws Exception {
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public Response removeFromCluserNodeLabels(Set<String> oldNodeLabels,
+      HttpServletRequest hsr) throws Exception {
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public NodeLabelsInfo getLabelsOnNode(HttpServletRequest hsr, String nodeId)
+      throws IOException {
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public AppPriority getAppPriority(HttpServletRequest hsr, String appId)
+      throws AuthorizationException {
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public Response updateApplicationPriority(AppPriority targetPriority,
+      HttpServletRequest hsr, String appId) throws AuthorizationException,
+      YarnException, InterruptedException, IOException {
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public AppQueue getAppQueue(HttpServletRequest hsr, String appId)
+      throws AuthorizationException {
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public Response updateAppQueue(AppQueue targetQueue, HttpServletRequest hsr,
+      String appId) throws AuthorizationException, YarnException,
+      InterruptedException, IOException {
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public Response postDelegationToken(DelegationToken tokenData,
+      HttpServletRequest hsr) throws AuthorizationException, IOException,
+      InterruptedException, Exception {
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public Response postDelegationTokenExpiration(HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException,
+      Exception {
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public Response cancelDelegationToken(HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException,
+      Exception {
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public Response createNewReservation(HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException {
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public Response submitReservation(ReservationSubmissionRequestInfo resContext,
+      HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException {
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public Response updateReservation(ReservationUpdateRequestInfo resContext,
+      HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException {
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public Response deleteReservation(ReservationDeleteRequestInfo resContext,
+      HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException {
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public Response listReservation(String queue, String reservationId,
+      long startTime, long endTime, boolean includeResourceAllocations,
+      HttpServletRequest hsr) throws Exception {
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public AppTimeoutInfo getAppTimeout(HttpServletRequest hsr, String appId,
+      String type) throws AuthorizationException {
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public AppTimeoutsInfo getAppTimeouts(HttpServletRequest hsr, String appId)
+      throws AuthorizationException {
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public Response updateApplicationTimeout(AppTimeoutInfo appTimeout,
+      HttpServletRequest hsr, String appId) throws AuthorizationException,
+      YarnException, InterruptedException, IOException {
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public AppAttemptsInfo getAppAttempts(HttpServletRequest hsr, String appId) {
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public AppAttemptInfo getAppAttempt(HttpServletRequest req,
+      HttpServletResponse res, String appId, String appAttemptId) {
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public ContainersInfo getContainers(HttpServletRequest req,
+      HttpServletResponse res, String appId, String appAttemptId) {
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public ContainerInfo getContainer(HttpServletRequest req,
+      HttpServletResponse res, String appId, String appAttemptId,
+      String containerId) {
+    throw new NotImplementedException();
+  }
+
+  @Override
+  public void setNextInterceptor(RESTRequestInterceptor next) {
+    throw new YarnRuntimeException("setNextInterceptor is being called on "
+        + "FederationInterceptorREST, which should be the last one "
+        + "in the chain. Check if the interceptor pipeline configuration "
+        + "is correct");
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc59b5fb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/BaseRouterWebServicesTest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/BaseRouterWebServicesTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/BaseRouterWebServicesTest.java
index 223690f..7d42084 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/BaseRouterWebServicesTest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/BaseRouterWebServicesTest.java
@@ -27,6 +27,7 @@ import java.security.PrivilegedExceptionAction;
 import javax.servlet.http.HttpServletResponse;
 import javax.ws.rs.core.Response;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AuthorizationException;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
@@ -75,30 +76,34 @@ public abstract class BaseRouterWebServicesTest {
   private RouterWebServices routerWebService;
 
   @Before
-  public void setup() {
-    conf = new YarnConfiguration();
+  public void setUp() {
+    this.conf = createConfiguration();
 
+    router = spy(new Router());
+    Mockito.doNothing().when(router).startWepApp();
+    routerWebService = new RouterWebServices(router, conf);
+    routerWebService.setResponse(mock(HttpServletResponse.class));
+
+    router.init(conf);
+    router.start();
+  }
+
+  protected YarnConfiguration createConfiguration() {
+    YarnConfiguration config = new YarnConfiguration();
     String mockPassThroughInterceptorClass =
         PassThroughRESTRequestInterceptor.class.getName();
 
     // Create a request intercepter pipeline for testing. The last one in the
     // chain will call the mock resource manager. The others in the chain will
     // simply forward it to the next one in the chain
-    conf.set(YarnConfiguration.ROUTER_WEBAPP_INTERCEPTOR_CLASS_PIPELINE,
+    config.set(YarnConfiguration.ROUTER_WEBAPP_INTERCEPTOR_CLASS_PIPELINE,
         mockPassThroughInterceptorClass + "," + mockPassThroughInterceptorClass
             + "," + mockPassThroughInterceptorClass + ","
             + MockRESTRequestInterceptor.class.getName());
 
-    conf.setInt(YarnConfiguration.ROUTER_PIPELINE_CACHE_MAX_SIZE,
+    config.setInt(YarnConfiguration.ROUTER_PIPELINE_CACHE_MAX_SIZE,
         TEST_MAX_CACHE_SIZE);
-
-    router = spy(new Router());
-    Mockito.doNothing().when(router).startWepApp();
-    routerWebService = new RouterWebServices(router, conf);
-    routerWebService.setResponse(mock(HttpServletResponse.class));
-
-    router.init(conf);
-    router.start();
+    return config;
   }
 
   @After
@@ -108,6 +113,14 @@ public abstract class BaseRouterWebServicesTest {
     }
   }
 
+  public void setUpConfig() {
+    this.conf = createConfiguration();
+  }
+
+  protected Configuration getConf() {
+    return this.conf;
+  }
+
   protected RouterWebServices getRouterWebServices() {
     Assert.assertNotNull(this.routerWebService);
     return this.routerWebService;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc59b5fb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/MockDefaultRequestInterceptorREST.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/MockDefaultRequestInterceptorREST.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/MockDefaultRequestInterceptorREST.java
new file mode 100644
index 0000000..91e601e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/MockDefaultRequestInterceptorREST.java
@@ -0,0 +1,136 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.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.router.webapp;
+
+import org.apache.hadoop.security.authorize.AuthorizationException;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.*;
+import org.apache.hadoop.yarn.webapp.NotFoundException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.servlet.http.HttpServletRequest;
+import javax.ws.rs.core.HttpHeaders;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.Response.Status;
+import java.io.IOException;
+import java.net.ConnectException;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * This class mocks the RESTRequestInterceptor.
+ */
+public class MockDefaultRequestInterceptorREST
+    extends DefaultRequestInterceptorREST {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(MockDefaultRequestInterceptorREST.class);
+  final private AtomicInteger applicationCounter = new AtomicInteger(0);
+  // True if the Mock RM is running, false otherwise.
+  // This property allows us to write tests for specific scenario as Yarn RM
+  // down e.g. network issue, failover.
+  private boolean isRunning = true;
+  private HashSet<ApplicationId> applicationMap = new HashSet<>();
+
+  private void validateRunning() throws ConnectException {
+    if (!isRunning) {
+      throw new ConnectException("RM is stopped");
+    }
+  }
+
+  @Override
+  public Response createNewApplication(HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException {
+    validateRunning();
+
+    ApplicationId applicationId =
+        ApplicationId.newInstance(Integer.valueOf(getSubClusterId().getId()),
+            applicationCounter.incrementAndGet());
+    NewApplication appId =
+        new NewApplication(applicationId.toString(), new ResourceInfo());
+    return Response.status(Status.OK).entity(appId).build();
+  }
+
+  @Override
+  public Response submitApplication(ApplicationSubmissionContextInfo newApp,
+      HttpServletRequest hsr)
+      throws AuthorizationException, IOException, InterruptedException {
+    validateRunning();
+
+    ApplicationId appId = ApplicationId.fromString(newApp.getApplicationId());
+    LOG.info("Application submitted: " + appId);
+    applicationMap.add(appId);
+    return Response.status(Status.ACCEPTED).header(HttpHeaders.LOCATION, "")
+        .entity(getSubClusterId()).build();
+  }
+
+  @Override
+  public AppInfo getApp(HttpServletRequest hsr, String appId,
+      Set<String> unselectedFields) {
+    if (!isRunning) {
+      throw new RuntimeException("RM is stopped");
+    }
+
+    ApplicationId applicationId = ApplicationId.fromString(appId);
+    if (!applicationMap.contains(applicationId)) {
+      throw new NotFoundException("app with id: " + appId + " not found");
+    }
+
+    return new AppInfo();
+  }
+
+  @Override
+  public Response updateAppState(AppState targetState, HttpServletRequest hsr,
+      String appId) throws AuthorizationException, YarnException,
+      InterruptedException, IOException {
+    validateRunning();
+
+    ApplicationId applicationId = ApplicationId.fromString(appId);
+    if (!applicationMap.remove(applicationId)) {
+      throw new ApplicationNotFoundException(
+          "Trying to kill an absent application: " + appId);
+    }
+
+    if (targetState == null) {
+      return Response.status(Status.BAD_REQUEST).build();
+    }
+
+    LOG.info("Force killing application: " + appId);
+    AppState ret = new AppState();
+    ret.setState(targetState.toString());
+    return Response.status(Status.OK).entity(ret).build();
+  }
+
+  public void setSubClusterId(int subClusterId) {
+    setSubClusterId(SubClusterId.newInstance(Integer.toString(subClusterId)));
+  }
+
+  public boolean isRunning() {
+    return isRunning;
+  }
+
+  public void setRunning(boolean runningMode) {
+    this.isRunning = runningMode;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc59b5fb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestFederationInterceptorREST.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestFederationInterceptorREST.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestFederationInterceptorREST.java
new file mode 100644
index 0000000..d918149
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestFederationInterceptorREST.java
@@ -0,0 +1,379 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.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.router.webapp;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import javax.ws.rs.core.Response;
+
+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.policies.manager.UniformBroadcastPolicyManager;
+import org.apache.hadoop.yarn.server.federation.store.impl.MemoryFederationStateStore;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.federation.utils.FederationStateStoreFacade;
+import org.apache.hadoop.yarn.server.federation.utils.FederationStateStoreTestUtil;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppState;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ApplicationSubmissionContextInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NewApplication;
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Extends the {@code BaseRouterClientRMTest} and overrides methods in order to
+ * use the {@code RouterClientRMService} pipeline test cases for testing the
+ * {@code FederationInterceptor} class. The tests for
+ * {@code RouterClientRMService} has been written cleverly so that it can be
+ * reused to validate different request intercepter chains.
+ */
+public class TestFederationInterceptorREST extends BaseRouterWebServicesTest {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestFederationInterceptorREST.class);
+  private final static int NUM_SUBCLUSTER = 4;
+  private static final int BAD_REQUEST = 400;
+  private static final int ACCEPTED = 202;
+  private static String user = "test-user";
+  private TestableFederationInterceptorREST interceptor;
+  private MemoryFederationStateStore stateStore;
+  private FederationStateStoreTestUtil stateStoreUtil;
+  private List<SubClusterId> subClusters;
+
+  @Override
+  public void setUp() {
+    super.setUpConfig();
+    interceptor = new TestableFederationInterceptorREST();
+
+    stateStore = new MemoryFederationStateStore();
+    stateStore.init(this.getConf());
+    FederationStateStoreFacade.getInstance().reinitialize(stateStore,
+        this.getConf());
+    stateStoreUtil = new FederationStateStoreTestUtil(stateStore);
+
+    interceptor.setConf(this.getConf());
+    interceptor.init(user);
+
+    subClusters = new ArrayList<>();
+
+    try {
+      for (int i = 0; i < NUM_SUBCLUSTER; i++) {
+        SubClusterId sc = SubClusterId.newInstance(Integer.toString(i));
+        stateStoreUtil.registerSubCluster(sc);
+        subClusters.add(sc);
+      }
+    } catch (YarnException e) {
+      LOG.error(e.getMessage());
+      Assert.fail();
+    }
+
+  }
+
+  @Override
+  public void tearDown() {
+    interceptor.shutdown();
+    super.tearDown();
+  }
+
+  @Override
+  protected YarnConfiguration createConfiguration() {
+    YarnConfiguration conf = new YarnConfiguration();
+    conf.setBoolean(YarnConfiguration.FEDERATION_ENABLED, true);
+    conf.set(YarnConfiguration.ROUTER_WEBAPP_DEFAULT_INTERCEPTOR_CLASS,
+        MockDefaultRequestInterceptorREST.class.getName());
+    String mockPassThroughInterceptorClass =
+        PassThroughRESTRequestInterceptor.class.getName();
+
+    // Create a request intercepter pipeline for testing. The last one in the
+    // chain is the federation intercepter that calls the mock resource manager.
+    // The others in the chain will simply forward it to the next one in the
+    // chain
+    conf.set(YarnConfiguration.ROUTER_CLIENTRM_INTERCEPTOR_CLASS_PIPELINE,
+        mockPassThroughInterceptorClass + ","
+            + TestableFederationInterceptorREST.class.getName());
+
+    conf.set(YarnConfiguration.FEDERATION_POLICY_MANAGER,
+        UniformBroadcastPolicyManager.class.getName());
+
+    // Disable StateStoreFacade cache
+    conf.setInt(YarnConfiguration.FEDERATION_CACHE_TIME_TO_LIVE_SECS, 0);
+
+    return conf;
+  }
+
+  /**
+   * This test validates the correctness of GetNewApplication. The return
+   * ApplicationId has to belong to one of the SubCluster in the cluster.
+   */
+  @Test
+  public void testGetNewApplication()
+      throws YarnException, IOException, InterruptedException {
+
+    Response response = interceptor.createNewApplication(null);
+
+    Assert.assertNotNull(response);
+    NewApplication ci = (NewApplication) response.getEntity();
+    Assert.assertNotNull(ci);
+    ApplicationId appId = ApplicationId.fromString(ci.getApplicationId());
+    Assert.assertTrue(appId.getClusterTimestamp() < NUM_SUBCLUSTER);
+    Assert.assertTrue(appId.getClusterTimestamp() >= 0);
+  }
+
+  /**
+   * This test validates the correctness of SubmitApplication. The application
+   * has to be submitted to one of the SubCluster in the cluster.
+   */
+  @Test
+  public void testSubmitApplication()
+      throws YarnException, IOException, InterruptedException {
+
+    ApplicationId appId =
+        ApplicationId.newInstance(System.currentTimeMillis(), 1);
+
+    ApplicationSubmissionContextInfo context =
+        new ApplicationSubmissionContextInfo();
+    context.setApplicationId(appId.toString());
+
+    Response response = interceptor.submitApplication(context, null);
+    Assert.assertEquals(ACCEPTED, response.getStatus());
+    SubClusterId ci = (SubClusterId) response.getEntity();
+
+    Assert.assertNotNull(response);
+    SubClusterId scIdResult = stateStoreUtil.queryApplicationHomeSC(appId);
+    Assert.assertNotNull(scIdResult);
+    Assert.assertTrue(subClusters.contains(scIdResult));
+    Assert.assertEquals(ci, scIdResult);
+  }
+
+  /**
+   * This test validates the correctness of SubmitApplication in case of
+   * multiple submission. The first retry has to be submitted to the same
+   * SubCluster of the first attempt.
+   */
+  @Test
+  public void testSubmitApplicationMultipleSubmission()
+      throws YarnException, IOException, InterruptedException {
+
+    ApplicationId appId =
+        ApplicationId.newInstance(System.currentTimeMillis(), 1);
+    ApplicationSubmissionContextInfo context =
+        new ApplicationSubmissionContextInfo();
+    context.setApplicationId(appId.toString());
+
+    // First attempt
+    Response response = interceptor.submitApplication(context, null);
+    Assert.assertNotNull(response);
+    Assert.assertEquals(ACCEPTED, response.getStatus());
+
+    SubClusterId scIdResult = stateStoreUtil.queryApplicationHomeSC(appId);
+    Assert.assertNotNull(scIdResult);
+
+    // First retry
+    response = interceptor.submitApplication(context, null);
+
+    Assert.assertNotNull(response);
+    Assert.assertEquals(ACCEPTED, response.getStatus());
+    SubClusterId scIdResult2 = stateStoreUtil.queryApplicationHomeSC(appId);
+    Assert.assertNotNull(scIdResult2);
+    Assert.assertEquals(scIdResult, scIdResult2);
+  }
+
+  /**
+   * This test validates the correctness of SubmitApplication in case of empty
+   * request.
+   */
+  @Test
+  public void testSubmitApplicationEmptyRequest()
+      throws YarnException, IOException, InterruptedException {
+
+    // ApplicationSubmissionContextInfo null
+    Response response = interceptor.submitApplication(null, null);
+
+    Assert.assertEquals(BAD_REQUEST, response.getStatus());
+
+    // ApplicationSubmissionContextInfo empty
+    response = interceptor
+        .submitApplication(new ApplicationSubmissionContextInfo(), null);
+
+    Assert.assertEquals(BAD_REQUEST, response.getStatus());
+
+    ApplicationSubmissionContextInfo context =
+        new ApplicationSubmissionContextInfo();
+    response = interceptor.submitApplication(context, null);
+    Assert.assertEquals(BAD_REQUEST, response.getStatus());
+  }
+
+  /**
+   * This test validates the correctness of SubmitApplication in case of of
+   * application in wrong format.
+   */
+  @Test
+  public void testSubmitApplicationWrongFormat()
+      throws YarnException, IOException, InterruptedException {
+
+    ApplicationSubmissionContextInfo context =
+        new ApplicationSubmissionContextInfo();
+    context.setApplicationId("Application_wrong_id");
+    Response response = interceptor.submitApplication(context, null);
+    Assert.assertEquals(BAD_REQUEST, response.getStatus());
+  }
+
+  /**
+   * This test validates the correctness of ForceKillApplication in case the
+   * application exists in the cluster.
+   */
+  @Test
+  public void testForceKillApplication()
+      throws YarnException, IOException, InterruptedException {
+
+    ApplicationId appId =
+        ApplicationId.newInstance(System.currentTimeMillis(), 1);
+    ApplicationSubmissionContextInfo context =
+        new ApplicationSubmissionContextInfo();
+    context.setApplicationId(appId.toString());
+
+    // Submit the application we are going to kill later
+    Response response = interceptor.submitApplication(context, null);
+
+    Assert.assertNotNull(response);
+    Assert.assertNotNull(stateStoreUtil.queryApplicationHomeSC(appId));
+
+    AppState appState = new AppState("KILLED");
+
+    Response responseKill =
+        interceptor.updateAppState(appState, null, appId.toString());
+    Assert.assertNotNull(responseKill);
+  }
+
+  /**
+   * This test validates the correctness of ForceKillApplication in case of
+   * application does not exist in StateStore.
+   */
+  @Test
+  public void testForceKillApplicationNotExists()
+      throws YarnException, IOException, InterruptedException {
+
+    ApplicationId appId =
+        ApplicationId.newInstance(System.currentTimeMillis(), 1);
+    AppState appState = new AppState("KILLED");
+    try {
+      interceptor.updateAppState(appState, null, appId.toString());
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertTrue(
+          e.getMessage().equals("Application " + appId + " does not exist"));
+    }
+  }
+
+  /**
+   * This test validates the correctness of ForceKillApplication in case of
+   * application in wrong format.
+   */
+  @Test
+  public void testForceKillApplicationWrongFormat()
+      throws YarnException, IOException, InterruptedException {
+
+    AppState appState = new AppState("KILLED");
+    Response response =
+        interceptor.updateAppState(appState, null, "Application_wrong_id");
+    Assert.assertEquals(BAD_REQUEST, response.getStatus());
+  }
+
+  /**
+   * This test validates the correctness of ForceKillApplication in case of
+   * empty request.
+   */
+  @Test
+  public void testForceKillApplicationEmptyRequest()
+      throws YarnException, IOException, InterruptedException {
+    ApplicationId appId =
+        ApplicationId.newInstance(System.currentTimeMillis(), 1);
+
+    ApplicationSubmissionContextInfo context =
+        new ApplicationSubmissionContextInfo();
+    context.setApplicationId(appId.toString());
+
+    // Submit the application we are going to kill later
+    interceptor.submitApplication(context, null);
+
+    Response response =
+        interceptor.updateAppState(null, null, appId.toString());
+    Assert.assertEquals(BAD_REQUEST, response.getStatus());
+
+  }
+
+  /**
+   * This test validates the correctness of GetApplicationReport in case the
+   * application exists in the cluster.
+   */
+  @Test
+  public void testGetApplicationReport()
+      throws YarnException, IOException, InterruptedException {
+
+    ApplicationId appId =
+        ApplicationId.newInstance(System.currentTimeMillis(), 1);
+    ApplicationSubmissionContextInfo context =
+        new ApplicationSubmissionContextInfo();
+    context.setApplicationId(appId.toString());
+
+    // Submit the application we want the report later
+    Response response = interceptor.submitApplication(context, null);
+
+    Assert.assertNotNull(response);
+    Assert.assertNotNull(stateStoreUtil.queryApplicationHomeSC(appId));
+
+    AppInfo responseGet = interceptor.getApp(null, appId.toString(), null);
+
+    Assert.assertNotNull(responseGet);
+  }
+
+  /**
+   * This test validates the correctness of GetApplicationReport in case the
+   * application does not exist in StateStore.
+   */
+  @Test
+  public void testGetApplicationNotExists()
+      throws YarnException, IOException, InterruptedException {
+
+    ApplicationId appId =
+        ApplicationId.newInstance(System.currentTimeMillis(), 1);
+
+    AppInfo response = interceptor.getApp(null, appId.toString(), null);
+
+    Assert.assertNull(response);
+  }
+
+  /**
+   * This test validates the correctness of GetApplicationReport in case of
+   * application in wrong format.
+   */
+  @Test
+  public void testGetApplicationWrongFormat()
+      throws YarnException, IOException, InterruptedException {
+
+    AppInfo response = interceptor.getApp(null, "Application_wrong_id", null);
+
+    Assert.assertNull(response);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc59b5fb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestFederationInterceptorRESTRetry.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestFederationInterceptorRESTRetry.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestFederationInterceptorRESTRetry.java
new file mode 100644
index 0000000..48bc1a8
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestFederationInterceptorRESTRetry.java
@@ -0,0 +1,274 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.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.router.webapp;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import javax.ws.rs.core.Response;
+
+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.policies.FederationPolicyUtils;
+import org.apache.hadoop.yarn.server.federation.policies.manager.UniformBroadcastPolicyManager;
+import org.apache.hadoop.yarn.server.federation.store.impl.MemoryFederationStateStore;
+import org.apache.hadoop.yarn.server.federation.store.records.GetApplicationHomeSubClusterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.federation.utils.FederationStateStoreFacade;
+import org.apache.hadoop.yarn.server.federation.utils.FederationStateStoreTestUtil;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ApplicationSubmissionContextInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NewApplication;
+import org.apache.hadoop.yarn.server.router.clientrm.PassThroughClientRequestInterceptor;
+import org.apache.hadoop.yarn.server.router.clientrm.TestableFederationClientInterceptor;
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Extends the {@code BaseRouterWebServicesTest} and overrides methods in order
+ * to use the {@code RouterWebServices} pipeline test cases for testing the
+ * {@code FederationInterceptorREST} class. The tests for
+ * {@code RouterWebServices} has been written cleverly so that it can be reused
+ * to validate different request interceptor chains.
+ * <p>
+ * It tests the case with SubClusters down and the Router logic of retries. We
+ * have 1 good SubCluster and 2 bad ones for all the tests.
+ */
+public class TestFederationInterceptorRESTRetry
+    extends BaseRouterWebServicesTest {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestFederationInterceptorRESTRetry.class);
+  private static final int SERVICE_UNAVAILABLE = 503;
+  private static final int ACCEPTED = 202;
+  private static final int OK = 200;
+  // running and registered
+  private static SubClusterId good;
+  // registered but not running
+  private static SubClusterId bad1;
+  private static SubClusterId bad2;
+  private static List<SubClusterId> scs = new ArrayList<SubClusterId>();
+  private TestableFederationInterceptorREST interceptor;
+  private MemoryFederationStateStore stateStore;
+  private FederationStateStoreTestUtil stateStoreUtil;
+  private String user = "test-user";
+
+  @Override
+  public void setUp() {
+    super.setUpConfig();
+    interceptor = new TestableFederationInterceptorREST();
+
+    stateStore = new MemoryFederationStateStore();
+    stateStore.init(this.getConf());
+    FederationStateStoreFacade.getInstance().reinitialize(stateStore,
+        getConf());
+    stateStoreUtil = new FederationStateStoreTestUtil(stateStore);
+
+    interceptor.setConf(this.getConf());
+    interceptor.init(user);
+
+    // Create SubClusters
+    good = SubClusterId.newInstance("0");
+    bad1 = SubClusterId.newInstance("1");
+    bad2 = SubClusterId.newInstance("2");
+    scs.add(good);
+    scs.add(bad1);
+    scs.add(bad2);
+
+    // The mock RM will not start in these SubClusters, this is done to simulate
+    // a SubCluster down
+
+    interceptor.registerBadSubCluster(bad1);
+    interceptor.registerBadSubCluster(bad2);
+  }
+
+  @Override
+  public void tearDown() {
+    interceptor.shutdown();
+    super.tearDown();
+  }
+
+  private void setupCluster(List<SubClusterId> scsToRegister)
+      throws YarnException {
+
+    try {
+      // Clean up the StateStore before every test
+      stateStoreUtil.deregisterAllSubClusters();
+
+      for (SubClusterId sc : scsToRegister) {
+        stateStoreUtil.registerSubCluster(sc);
+      }
+    } catch (YarnException e) {
+      LOG.error(e.getMessage());
+      Assert.fail();
+    }
+  }
+
+  @Override
+  protected YarnConfiguration createConfiguration() {
+    YarnConfiguration conf = new YarnConfiguration();
+    conf.setBoolean(YarnConfiguration.FEDERATION_ENABLED, true);
+
+    conf.set(YarnConfiguration.ROUTER_WEBAPP_DEFAULT_INTERCEPTOR_CLASS,
+        MockDefaultRequestInterceptorREST.class.getName());
+
+    String mockPassThroughInterceptorClass =
+        PassThroughClientRequestInterceptor.class.getName();
+
+    // Create a request intercepter pipeline for testing. The last one in the
+    // chain is the federation intercepter that calls the mock resource manager.
+    // The others in the chain will simply forward it to the next one in the
+    // chain
+    conf.set(YarnConfiguration.ROUTER_CLIENTRM_INTERCEPTOR_CLASS_PIPELINE,
+        mockPassThroughInterceptorClass + ","
+            + TestableFederationClientInterceptor.class.getName());
+
+    conf.set(YarnConfiguration.FEDERATION_POLICY_MANAGER,
+        UniformBroadcastPolicyManager.class.getName());
+
+    // Disable StateStoreFacade cache
+    conf.setInt(YarnConfiguration.FEDERATION_CACHE_TIME_TO_LIVE_SECS, 0);
+
+    return conf;
+  }
+
+  /**
+   * This test validates the correctness of GetNewApplication in case the
+   * cluster is composed of only 1 bad SubCluster.
+   */
+  @Test
+  public void testGetNewApplicationOneBadSC()
+      throws YarnException, IOException, InterruptedException {
+
+    setupCluster(Arrays.asList(bad2));
+
+    Response response = interceptor.createNewApplication(null);
+    Assert.assertEquals(SERVICE_UNAVAILABLE, response.getStatus());
+    Assert.assertEquals(FederationPolicyUtils.NO_ACTIVE_SUBCLUSTER_AVAILABLE,
+        response.getEntity());
+  }
+
+  /**
+   * This test validates the correctness of GetNewApplication in case the
+   * cluster is composed of only 2 bad SubClusters.
+   */
+  @Test
+  public void testGetNewApplicationTwoBadSCs()
+      throws YarnException, IOException, InterruptedException {
+    setupCluster(Arrays.asList(bad1, bad2));
+
+    Response response = interceptor.createNewApplication(null);
+    Assert.assertEquals(SERVICE_UNAVAILABLE, response.getStatus());
+    Assert.assertEquals(FederationPolicyUtils.NO_ACTIVE_SUBCLUSTER_AVAILABLE,
+        response.getEntity());
+  }
+
+  /**
+   * This test validates the correctness of GetNewApplication in case the
+   * cluster is composed of only 1 bad SubCluster and 1 good one.
+   */
+  @Test
+  public void testGetNewApplicationOneBadOneGood()
+      throws YarnException, IOException, InterruptedException {
+    System.out.println("Test getNewApplication with one bad, one good SC");
+    setupCluster(Arrays.asList(good, bad2));
+    Response response = interceptor.createNewApplication(null);
+
+    Assert.assertEquals(OK, response.getStatus());
+
+    NewApplication newApp = (NewApplication) response.getEntity();
+    ApplicationId appId = ApplicationId.fromString(newApp.getApplicationId());
+
+    Assert.assertEquals(Integer.parseInt(good.getId()),
+        appId.getClusterTimestamp());
+  }
+
+  /**
+   * This test validates the correctness of SubmitApplication in case the
+   * cluster is composed of only 1 bad SubCluster.
+   */
+  @Test
+  public void testSubmitApplicationOneBadSC()
+      throws YarnException, IOException, InterruptedException {
+
+    setupCluster(Arrays.asList(bad2));
+
+    ApplicationId appId =
+        ApplicationId.newInstance(System.currentTimeMillis(), 1);
+    ApplicationSubmissionContextInfo context =
+        new ApplicationSubmissionContextInfo();
+    context.setApplicationId(appId.toString());
+
+    Response response = interceptor.submitApplication(context, null);
+    Assert.assertEquals(SERVICE_UNAVAILABLE, response.getStatus());
+    Assert.assertEquals(FederationPolicyUtils.NO_ACTIVE_SUBCLUSTER_AVAILABLE,
+        response.getEntity());
+  }
+
+  /**
+   * This test validates the correctness of SubmitApplication in case the
+   * cluster is composed of only 2 bad SubClusters.
+   */
+  @Test
+  public void testSubmitApplicationTwoBadSCs()
+      throws YarnException, IOException, InterruptedException {
+    setupCluster(Arrays.asList(bad1, bad2));
+
+    ApplicationId appId =
+        ApplicationId.newInstance(System.currentTimeMillis(), 1);
+    ApplicationSubmissionContextInfo context =
+        new ApplicationSubmissionContextInfo();
+    context.setApplicationId(appId.toString());
+
+    Response response = interceptor.submitApplication(context, null);
+    Assert.assertEquals(SERVICE_UNAVAILABLE, response.getStatus());
+    Assert.assertEquals(FederationPolicyUtils.NO_ACTIVE_SUBCLUSTER_AVAILABLE,
+        response.getEntity());
+  }
+
+  /**
+   * This test validates the correctness of SubmitApplication in case the
+   * cluster is composed of only 1 bad SubCluster and a good one.
+   */
+  @Test
+  public void testSubmitApplicationOneBadOneGood()
+      throws YarnException, IOException, InterruptedException {
+    System.out.println("Test submitApplication with one bad, one good SC");
+    setupCluster(Arrays.asList(good, bad2));
+
+    ApplicationId appId =
+        ApplicationId.newInstance(System.currentTimeMillis(), 1);
+    ApplicationSubmissionContextInfo context =
+        new ApplicationSubmissionContextInfo();
+    context.setApplicationId(appId.toString());
+    Response response = interceptor.submitApplication(context, null);
+
+    Assert.assertEquals(ACCEPTED, response.getStatus());
+
+    Assert.assertEquals(good,
+        stateStore
+            .getApplicationHomeSubCluster(
+                GetApplicationHomeSubClusterRequest.newInstance(appId))
+            .getApplicationHomeSubCluster().getHomeSubCluster());
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc59b5fb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestableFederationInterceptorREST.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestableFederationInterceptorREST.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestableFederationInterceptorREST.java
new file mode 100644
index 0000000..ce5bb23
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestableFederationInterceptorREST.java
@@ -0,0 +1,54 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.router.webapp;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+
+/**
+ * Extends the FederationInterceptorREST and overrides methods to provide a
+ * testable implementation of FederationInterceptorREST.
+ */
+public class TestableFederationInterceptorREST
+    extends FederationInterceptorREST {
+
+  private List<SubClusterId> badSubCluster = new ArrayList<SubClusterId>();
+
+  /**
+   * For testing purpose, some subclusters has to be down to simulate particular
+   * scenarios as RM Failover, network issues. For this reason we keep track of
+   * these bad subclusters. This method make the subcluster unusable.
+   *
+   * @param badSC the subcluster to make unusable
+   */
+  protected void registerBadSubCluster(SubClusterId badSC) {
+
+    // Adding in the cache the bad SubCluster, in this way we can stop them
+    getOrCreateInterceptorForSubCluster(badSC, "test");
+
+    badSubCluster.add(badSC);
+    MockDefaultRequestInterceptorREST interceptor =
+        (MockDefaultRequestInterceptorREST) super.getInterceptorForSubCluster(
+            badSC);
+    interceptor.setRunning(false);
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc59b5fb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/Federation.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/Federation.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/Federation.md
index ecf61c5..3e3580c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/Federation.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/Federation.md
@@ -216,7 +216,7 @@ Optional:
 |`yarn.router.submit.retry` | `3` | The number of retries in the router before we give up. |
 |`yarn.federation.statestore.max-connections` | `10` | This is the maximum number of parallel connections each Router makes to the state-store. |
 |`yarn.federation.cache-ttl.secs` | `60` | The Router caches informations, and this is the time to leave before the cache is invalidated. |
-
+|`yarn.router.webapp.interceptor-class.pipeline` | `org.apache.hadoop.yarn.server.router.webapp.FederationInterceptorREST` | A comma-seperated list of interceptor classes to be run at the router when interfacing with the client via REST interface. The last step of this pipeline must be the Federation Interceptor REST. |
 
 ###ON NMs:
 


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


[20/50] [abbrv] hadoop git commit: HADOOP-14673. Remove leftover hadoop_xml_escape from functions. Contributed by Ajay Kumar.

Posted by jh...@apache.org.
HADOOP-14673. Remove leftover hadoop_xml_escape from functions. Contributed by Ajay Kumar.


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

Branch: refs/heads/YARN-5734
Commit: 044651139800b9e2e5b8f224772e6dbd6ded58c6
Parents: 8bef4ec
Author: Arpit Agarwal <ar...@apache.org>
Authored: Mon Aug 14 16:22:10 2017 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Mon Aug 14 16:22:10 2017 -0700

----------------------------------------------------------------------
 .../src/main/bin/hadoop-functions.sh            | 23 --------------
 .../src/test/scripts/hadoop_escape_chars.bats   | 32 --------------------
 2 files changed, 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/04465113/hadoop-common-project/hadoop-common/src/main/bin/hadoop-functions.sh
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/bin/hadoop-functions.sh b/hadoop-common-project/hadoop-common/src/main/bin/hadoop-functions.sh
index 3cf21cf..9ea4587 100755
--- a/hadoop-common-project/hadoop-common/src/main/bin/hadoop-functions.sh
+++ b/hadoop-common-project/hadoop-common/src/main/bin/hadoop-functions.sh
@@ -2578,29 +2578,6 @@ function hadoop_parse_args
   hadoop_debug "hadoop_parse: asking caller to skip ${HADOOP_PARSE_COUNTER}"
 }
 
-## @description  XML-escapes the characters (&'"<>) in the given parameter.
-## @audience     private
-## @stability    evolving
-## @replaceable  yes
-## @param        string
-## @return       XML-escaped string
-function hadoop_xml_escape
-{
-  sed -e 's/&/\&amp;/g' -e 's/"/\\\&quot;/g' \
-    -e "s/'/\\\\\&apos;/g" -e 's/</\\\&lt;/g' -e 's/>/\\\&gt;/g' <<< "$1"
-}
-
-## @description  sed-escapes the characters (\/&) in the given parameter.
-## @audience     private
-## @stability    evolving
-## @replaceable  yes
-## @param        string
-## @return       sed-escaped string
-function hadoop_sed_escape
-{
-  sed -e 's/[\/&]/\\&/g' <<< "$1"
-}
-
 ## @description Handle subcommands from main program entries
 ## @audience private
 ## @stability evolving

http://git-wip-us.apache.org/repos/asf/hadoop/blob/04465113/hadoop-common-project/hadoop-common/src/test/scripts/hadoop_escape_chars.bats
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/scripts/hadoop_escape_chars.bats b/hadoop-common-project/hadoop-common/src/test/scripts/hadoop_escape_chars.bats
deleted file mode 100755
index 9b031f2..0000000
--- a/hadoop-common-project/hadoop-common/src/test/scripts/hadoop_escape_chars.bats
+++ /dev/null
@@ -1,32 +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.
-
-load hadoop-functions_test_helper
-
-@test "hadoop_escape_sed (positive 1)" {
-  ret="$(hadoop_sed_escape "\pass&&word\0#\$asdf/g  ><'\"~\`!@#$%^&*()_+-=")"
-  expected="\\\\pass\&\&word\\\0#\$asdf\/g  ><'\"~\`!@#$%^\&*()_+-="
-  echo "actual >${ret}<"
-  echo "expected >${expected}<"
-  [ "${ret}" = "${expected}" ]
-}
-
-@test "hadoop_escape_xml (positive 1)" {
-  ret="$(hadoop_xml_escape "\pass&&word\0#\$asdf/g  ><'\"~\`!@#$%^&*()_+-=")"
-  expected="\\pass&amp;&amp;word\0#\$asdf/g  \&gt;\&lt;\&apos;\&quot;~\`!@#\$%^&amp;*()_+-="
-  echo "actual >${ret}<"
-  echo "expected >${expected}<"
-  [ "${ret}" = "${expected}" ]
-}
\ 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/50] [abbrv] hadoop git commit: HADOOP-14773. Extend ZKCuratorManager API for more reusability. (Íñigo Goiri via Subru).

Posted by jh...@apache.org.
HADOOP-14773. Extend ZKCuratorManager API for more reusability. (Íñigo Goiri via Subru).


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

Branch: refs/heads/YARN-5734
Commit: 75dd866bfb8b63cb9f13179d4365b05c48e0907d
Parents: f34646d
Author: Subru Krishnan <su...@apache.org>
Authored: Tue Aug 15 16:53:59 2017 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Tue Aug 15 16:53:59 2017 -0700

----------------------------------------------------------------------
 .../hadoop/util/curator/ZKCuratorManager.java   | 54 ++++++++++++++++++--
 .../util/curator/TestZKCuratorManager.java      |  2 +-
 .../recovery/ZKRMStateStore.java                | 19 +------
 3 files changed, 52 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/75dd866b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/curator/ZKCuratorManager.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/curator/ZKCuratorManager.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/curator/ZKCuratorManager.java
index 3adf028..9a031af 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/curator/ZKCuratorManager.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/curator/ZKCuratorManager.java
@@ -33,9 +33,12 @@ import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.util.ZKUtil;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Stat;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.base.Preconditions;
+
 /**
  * Helper class that provides utility methods specific to ZK operations.
  */
@@ -179,7 +182,6 @@ public final class ZKCuratorManager {
   /**
    * Get the data in a ZNode.
    * @param path Path of the ZNode.
-   * @param stat Output statistics of the ZNode.
    * @return The data in the ZNode.
    * @throws Exception If it cannot contact Zookeeper.
    */
@@ -190,16 +192,38 @@ public final class ZKCuratorManager {
   /**
    * Get the data in a ZNode.
    * @param path Path of the ZNode.
-   * @param stat Output statistics of the ZNode.
+   * @param stat
+   * @return The data in the ZNode.
+   * @throws Exception If it cannot contact Zookeeper.
+   */
+  public byte[] getData(final String path, Stat stat) throws Exception {
+    return curator.getData().storingStatIn(stat).forPath(path);
+  }
+
+  /**
+   * Get the data in a ZNode.
+   * @param path Path of the ZNode.
    * @return The data in the ZNode.
    * @throws Exception If it cannot contact Zookeeper.
    */
-  public String getSringData(final String path) throws Exception {
+  public String getStringData(final String path) throws Exception {
     byte[] bytes = getData(path);
     return new String(bytes, Charset.forName("UTF-8"));
   }
 
   /**
+   * Get the data in a ZNode.
+   * @param path Path of the ZNode.
+   * @param stat Output statistics of the ZNode.
+   * @return The data in the ZNode.
+   * @throws Exception If it cannot contact Zookeeper.
+   */
+  public String getStringData(final String path, Stat stat) throws Exception {
+    byte[] bytes = getData(path, stat);
+    return new String(bytes, Charset.forName("UTF-8"));
+  }
+
+  /**
    * Set data into a ZNode.
    * @param path Path of the ZNode.
    * @param data Data to set.
@@ -272,14 +296,36 @@ public final class ZKCuratorManager {
   }
 
   /**
+   * Utility function to ensure that the configured base znode exists.
+   * This recursively creates the znode as well as all of its parents.
+   * @param path Path of the znode to create.
+   * @throws Exception If it cannot create the file.
+   */
+  public void createRootDirRecursively(String path) throws Exception {
+    String[] pathParts = path.split("/");
+    Preconditions.checkArgument(
+        pathParts.length >= 1 && pathParts[0].isEmpty(),
+        "Invalid path: %s", path);
+    StringBuilder sb = new StringBuilder();
+
+    for (int i = 1; i < pathParts.length; i++) {
+      sb.append("/").append(pathParts[i]);
+      create(sb.toString());
+    }
+  }
+
+  /**
    * Delete a ZNode.
    * @param path Path of the ZNode.
+   * @return If the znode was deleted.
    * @throws Exception If it cannot contact ZooKeeper.
    */
-  public void delete(final String path) throws Exception {
+  public boolean delete(final String path) throws Exception {
     if (exists(path)) {
       curator.delete().deletingChildrenIfNeeded().forPath(path);
+      return true;
     }
+    return false;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75dd866b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/curator/TestZKCuratorManager.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/curator/TestZKCuratorManager.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/curator/TestZKCuratorManager.java
index 2bcf508..3e78a44 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/curator/TestZKCuratorManager.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/curator/TestZKCuratorManager.java
@@ -67,7 +67,7 @@ public class TestZKCuratorManager {
     curator.create(testZNode);
     assertTrue(curator.exists(testZNode));
     curator.setData(testZNode, expectedString, -1);
-    String testString = curator.getSringData("/test");
+    String testString = curator.getStringData("/test");
     assertEquals(expectedString, testString);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75dd866b/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 4b6e82c..a445e75 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
@@ -19,7 +19,6 @@
 package org.apache.hadoop.yarn.server.resourcemanager.recovery;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.curator.framework.CuratorFramework;
@@ -338,7 +337,7 @@ public class ZKRMStateStore extends RMStateStore {
   @Override
   public synchronized void startInternal() throws Exception {
     // ensure root dirs exist
-    createRootDirRecursively(znodeWorkingPath);
+    zkManager.createRootDirRecursively(znodeWorkingPath);
     create(zkRootNodePath);
     setRootNodeAcls();
     delete(fencingNodePath);
@@ -1147,22 +1146,6 @@ public class ZKRMStateStore extends RMStateStore {
   }
 
   /**
-   * Utility function to ensure that the configured base znode exists.
-   * This recursively creates the znode as well as all of its parents.
-   */
-  private void createRootDirRecursively(String path) throws Exception {
-    String pathParts[] = path.split("/");
-    Preconditions.checkArgument(pathParts.length >= 1 && pathParts[0].isEmpty(),
-        "Invalid path: %s", path);
-    StringBuilder sb = new StringBuilder();
-
-    for (int i = 1; i < pathParts.length; i++) {
-      sb.append("/").append(pathParts[i]);
-      create(sb.toString());
-    }
-  }
-
-  /**
    * Get alternate path for app id if path according to configured split index
    * does not exist. We look for path based on all possible split indices.
    * @param appId


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


[16/50] [abbrv] hadoop git commit: YARN-6881. LOG is unused in AllocationConfiguration (Contributed by weiyuan via Daniel Templeton)

Posted by jh...@apache.org.
YARN-6881. LOG is unused in AllocationConfiguration (Contributed by weiyuan via Daniel Templeton)


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

Branch: refs/heads/YARN-5734
Commit: 6b09c327057947049ef7984afbb5ed225f15fc2d
Parents: 608a06c
Author: Daniel Templeton <te...@apache.org>
Authored: Mon Aug 14 11:55:33 2017 -0700
Committer: Daniel Templeton <te...@apache.org>
Committed: Mon Aug 14 11:55:33 2017 -0700

----------------------------------------------------------------------
 .../resourcemanager/scheduler/fair/AllocationConfiguration.java   | 3 ---
 1 file changed, 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b09c327/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationConfiguration.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/AllocationConfiguration.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/AllocationConfiguration.java
index f143aa6..71e6f7f 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/AllocationConfiguration.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/AllocationConfiguration.java
@@ -23,8 +23,6 @@ import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.yarn.api.records.QueueACL;
@@ -41,7 +39,6 @@ import org.apache.hadoop.yarn.util.resource.Resources;
 import com.google.common.annotations.VisibleForTesting;
 
 public class AllocationConfiguration extends ReservationSchedulerConfiguration {
-  private static final Log LOG = LogFactory.getLog(FSQueue.class.getName());
   private static final AccessControlList EVERYBODY_ACL = new AccessControlList("*");
   private static final AccessControlList NOBODY_ACL = new AccessControlList(" ");
   private static final ResourceCalculator RESOURCE_CALCULATOR =


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


[34/50] [abbrv] hadoop git commit: MAPREDUCE-6940. Copy-paste error in the TaskAttemptUnsuccessfulCompletionEvent constructor. Contributed by Oleg Danilov

Posted by jh...@apache.org.
MAPREDUCE-6940. Copy-paste error in the TaskAttemptUnsuccessfulCompletionEvent constructor. Contributed by Oleg Danilov


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

Branch: refs/heads/YARN-5734
Commit: 0acc5e00362602f027524637a86ca1bf80982986
Parents: de462da
Author: Jason Lowe <jl...@apache.org>
Authored: Wed Aug 16 16:34:06 2017 -0500
Committer: Jason Lowe <jl...@apache.org>
Committed: Wed Aug 16 16:34:06 2017 -0500

----------------------------------------------------------------------
 .../TaskAttemptUnsuccessfulCompletionEvent.java | 28 ++++++++++----------
 1 file changed, 14 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0acc5e00/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptUnsuccessfulCompletionEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptUnsuccessfulCompletionEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptUnsuccessfulCompletionEvent.java
index 1732d91..1752967 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptUnsuccessfulCompletionEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptUnsuccessfulCompletionEvent.java
@@ -60,7 +60,7 @@ public class TaskAttemptUnsuccessfulCompletionEvent implements HistoryEvent {
   int[] physMemKbytes;
   private static final Counters EMPTY_COUNTERS = new Counters();
 
-  /** 
+  /**
    * Create an event to record the unsuccessful completion of attempts
    * @param id Attempt ID
    * @param taskType Type of the task
@@ -74,7 +74,7 @@ public class TaskAttemptUnsuccessfulCompletionEvent implements HistoryEvent {
    * @param allSplits the "splits", or a pixelated graph of various
    *        measurable worker node state variables against progress.
    *        Currently there are four; wallclock time, CPU time,
-   *        virtual memory and physical memory.  
+   *        virtual memory and physical memory.
    */
   public TaskAttemptUnsuccessfulCompletionEvent
        (TaskAttemptID id, TaskType taskType,
@@ -101,7 +101,7 @@ public class TaskAttemptUnsuccessfulCompletionEvent implements HistoryEvent {
         ProgressSplitsBlock.arrayGetPhysMemKbytes(allSplits);
   }
 
-  /** 
+  /**
    * @deprecated please use the constructor with an additional
    *              argument, an array of splits arrays instead.  See
    *              {@link org.apache.hadoop.mapred.ProgressSplitsBlock}
@@ -117,19 +117,19 @@ public class TaskAttemptUnsuccessfulCompletionEvent implements HistoryEvent {
    */
   public TaskAttemptUnsuccessfulCompletionEvent
        (TaskAttemptID id, TaskType taskType,
-        String status, long finishTime, 
+        String status, long finishTime,
         String hostname, String error) {
     this(id, taskType, status, finishTime, hostname, -1, "",
         error, EMPTY_COUNTERS, null);
   }
-  
+
   public TaskAttemptUnsuccessfulCompletionEvent
       (TaskAttemptID id, TaskType taskType,
        String status, long finishTime,
        String hostname, int port, String rackName,
        String error, int[][] allSplits) {
     this(id, taskType, status, finishTime, hostname, port,
-        rackName, error, EMPTY_COUNTERS, null);
+        rackName, error, EMPTY_COUNTERS, allSplits);
   }
 
   TaskAttemptUnsuccessfulCompletionEvent() {}
@@ -162,9 +162,9 @@ public class TaskAttemptUnsuccessfulCompletionEvent implements HistoryEvent {
     }
     return datum;
   }
-  
-  
-  
+
+
+
   public void setDatum(Object odatum) {
     this.datum =
         (TaskAttemptUnsuccessfulCompletion)odatum;
@@ -208,12 +208,12 @@ public class TaskAttemptUnsuccessfulCompletionEvent implements HistoryEvent {
   public String getHostname() { return hostname; }
   /** Get the rpc port for the host where the attempt executed */
   public int getPort() { return port; }
-  
+
   /** Get the rack name of the node where the attempt ran */
   public String getRackName() {
     return rackName == null ? null : rackName.toString();
   }
-  
+
   /** Get the error string */
   public String getError() { return error.toString(); }
   /** Get the task status */
@@ -224,12 +224,12 @@ public class TaskAttemptUnsuccessfulCompletionEvent implements HistoryEvent {
   Counters getCounters() { return counters; }
   /** Get the event type */
   public EventType getEventType() {
-    // Note that the task type can be setup/map/reduce/cleanup but the 
+    // Note that the task type can be setup/map/reduce/cleanup but the
     // attempt-type can only be map/reduce.
     // find out if the task failed or got killed
     boolean failed = TaskStatus.State.FAILED.toString().equals(getTaskStatus());
-    return getTaskId().getTaskType() == TaskType.MAP 
-           ? (failed 
+    return getTaskId().getTaskType() == TaskType.MAP
+           ? (failed
               ? EventType.MAP_ATTEMPT_FAILED
               : EventType.MAP_ATTEMPT_KILLED)
            : (failed


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


[43/50] [abbrv] hadoop git commit: YARN-5952. Create REST API for changing YARN scheduler configurations. (Jonathan Hung via wangda)

Posted by jh...@apache.org.
YARN-5952. Create REST API for changing YARN scheduler configurations. (Jonathan Hung via wangda)


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

Branch: refs/heads/YARN-5734
Commit: eadb07dbaccb5436d7771bf1c66d9412310092fb
Parents: d4c4f04
Author: Wangda Tan <wa...@apache.org>
Authored: Mon Apr 3 10:12:01 2017 -0700
Committer: Jonathan Hung <jh...@linkedin.com>
Committed: Thu Aug 17 13:48:55 2017 -0700

----------------------------------------------------------------------
 .../scheduler/MutableConfScheduler.java         |  40 ++
 .../scheduler/MutableConfigurationProvider.java |   5 +-
 .../scheduler/capacity/CapacityScheduler.java   |  16 +-
 .../conf/InMemoryConfigurationStore.java        |   6 +-
 .../conf/MutableCSConfigurationProvider.java    |  24 +-
 .../resourcemanager/webapp/RMWebServices.java   | 172 ++++++-
 .../webapp/dao/QueueConfigInfo.java             |  57 +++
 .../webapp/dao/QueueConfigsUpdateInfo.java      |  60 +++
 .../TestMutableCSConfigurationProvider.java     |   6 +-
 .../TestRMWebServicesConfigurationMutation.java | 477 +++++++++++++++++++
 10 files changed, 851 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/eadb07db/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/MutableConfScheduler.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/MutableConfScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/MutableConfScheduler.java
new file mode 100644
index 0000000..35e36e1
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/MutableConfScheduler.java
@@ -0,0 +1,40 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
+
+import org.apache.hadoop.security.UserGroupInformation;
+
+import java.io.IOException;
+import java.util.Map;
+
+/**
+ * Interface for a scheduler that supports changing configuration at runtime.
+ *
+ */
+public interface MutableConfScheduler extends ResourceScheduler {
+
+  /**
+   * Update the scheduler's configuration.
+   * @param user Caller of this update
+   * @param confUpdate key-value map of the configuration update
+   * @throws IOException if update is invalid
+   */
+  void updateConfiguration(UserGroupInformation user,
+      Map<String, String> confUpdate) throws IOException;
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eadb07db/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/MutableConfigurationProvider.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/MutableConfigurationProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/MutableConfigurationProvider.java
index da30a2b..889c3bc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/MutableConfigurationProvider.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/MutableConfigurationProvider.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
 
+import java.io.IOException;
 import java.util.Map;
 
 /**
@@ -29,7 +30,9 @@ public interface MutableConfigurationProvider {
    * Update the scheduler configuration with the provided key value pairs.
    * @param user User issuing the request
    * @param confUpdate Key-value pairs for configurations to be updated.
+   * @throws IOException if scheduler could not be reinitialized
    */
-  void mutateConfiguration(String user, Map<String, String> confUpdate);
+  void mutateConfiguration(String user, Map<String, String> confUpdate)
+      throws IOException;
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eadb07db/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.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/CapacityScheduler.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/CapacityScheduler.java
index d5abcc0..f49286c 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/CapacityScheduler.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/CapacityScheduler.java
@@ -86,6 +86,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnSched
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AppSchedulingInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerUpdates;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.MutableConfScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.MutableConfigurationProvider;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.PreemptableResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
@@ -148,7 +150,7 @@ import com.google.common.util.concurrent.SettableFuture;
 public class CapacityScheduler extends
     AbstractYarnScheduler<FiCaSchedulerApp, FiCaSchedulerNode> implements
     PreemptableResourceScheduler, CapacitySchedulerContext, Configurable,
-    ResourceAllocationCommitter {
+    ResourceAllocationCommitter, MutableConfScheduler {
 
   private static final Log LOG = LogFactory.getLog(CapacityScheduler.class);
 
@@ -2555,4 +2557,16 @@ public class CapacityScheduler extends
       writeLock.unlock();
     }
   }
+
+  @Override
+  public void updateConfiguration(UserGroupInformation user,
+      Map<String, String> confUpdate) throws IOException {
+    if (csConfProvider instanceof MutableConfigurationProvider) {
+      ((MutableConfigurationProvider) csConfProvider).mutateConfiguration(
+          user.getShortUserName(), confUpdate);
+    } else {
+      throw new UnsupportedOperationException("Configured CS configuration " +
+          "provider does not support updating configuration.");
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eadb07db/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/InMemoryConfigurationStore.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/conf/InMemoryConfigurationStore.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/conf/InMemoryConfigurationStore.java
index a208fb9..b97be1b 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/conf/InMemoryConfigurationStore.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/conf/InMemoryConfigurationStore.java
@@ -58,7 +58,11 @@ public class InMemoryConfigurationStore implements YarnConfigurationStore {
         if (isValid) {
           Map<String, String> mutations = mutation.getUpdates();
           for (Map.Entry<String, String> kv : mutations.entrySet()) {
-            schedConf.set(kv.getKey(), kv.getValue());
+            if (kv.getValue() == null) {
+              schedConf.unset(kv.getKey());
+            } else {
+              schedConf.set(kv.getKey(), kv.getValue());
+            }
           }
         }
         return true;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eadb07db/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/MutableCSConfigurationProvider.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/conf/MutableCSConfigurationProvider.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/conf/MutableCSConfigurationProvider.java
index 267ab6a..ea1b3c0 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/conf/MutableCSConfigurationProvider.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/conf/MutableCSConfigurationProvider.java
@@ -60,34 +60,44 @@ public class MutableCSConfigurationProvider implements CSConfigurationProvider,
     }
     Configuration initialSchedConf = new Configuration(false);
     initialSchedConf.addResource(YarnConfiguration.CS_CONFIGURATION_FILE);
-    this.schedConf = initialSchedConf;
-    confStore.initialize(config, initialSchedConf);
+    this.schedConf = new Configuration(false);
+    // We need to explicitly set the key-values in schedConf, otherwise
+    // these configuration keys cannot be deleted when
+    // configuration is reloaded.
+    for (Map.Entry<String, String> kv : initialSchedConf) {
+      schedConf.set(kv.getKey(), kv.getValue());
+    }
+    confStore.initialize(config, schedConf);
     this.conf = config;
   }
 
   @Override
   public CapacitySchedulerConfiguration loadConfiguration(Configuration
       configuration) throws IOException {
-    Configuration loadedConf = new Configuration(configuration);
-    loadedConf.addResource(schedConf);
+    Configuration loadedConf = new Configuration(schedConf);
+    loadedConf.addResource(configuration);
     return new CapacitySchedulerConfiguration(loadedConf, false);
   }
 
   @Override
   public void mutateConfiguration(String user,
-      Map<String, String> confUpdate) {
+      Map<String, String> confUpdate) throws IOException {
     Configuration oldConf = new Configuration(schedConf);
     LogMutation log = new LogMutation(confUpdate, user);
     long id = confStore.logMutation(log);
     for (Map.Entry<String, String> kv : confUpdate.entrySet()) {
-      schedConf.set(kv.getKey(), kv.getValue());
+      if (kv.getValue() == null) {
+        schedConf.unset(kv.getKey());
+      } else {
+        schedConf.set(kv.getKey(), kv.getValue());
+      }
     }
     try {
       rmContext.getScheduler().reinitialize(conf, rmContext);
     } catch (IOException e) {
       schedConf = oldConf;
       confStore.confirmMutation(id, false);
-      return;
+      throw e;
     }
     confStore.confirmMutation(id, true);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eadb07db/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
index 6ce47de..5ed1263 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
@@ -55,7 +55,8 @@ import javax.ws.rs.core.HttpHeaders;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
 import javax.ws.rs.core.Response.Status;
-
+import com.google.common.base.Joiner;
+import org.apache.commons.codec.binary.Base64;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -127,11 +128,14 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.MutableConfScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivitiesManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivitiesManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler;
@@ -2404,4 +2408,170 @@ public class RMWebServices extends WebServices implements RMWebServiceProtocol {
         app.getApplicationTimeouts().get(appTimeout.getTimeoutType()));
     return Response.status(Status.OK).entity(timeout).build();
   }
+
+  @PUT
+  @Path("/queues")
+  @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
+      MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
+  @Consumes({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  public Response updateSchedulerConfiguration(QueueConfigsUpdateInfo
+      mutationInfo, @Context HttpServletRequest hsr)
+      throws AuthorizationException, InterruptedException {
+    init();
+
+    UserGroupInformation callerUGI = getCallerUserGroupInformation(hsr, true);
+    ApplicationACLsManager aclsManager = rm.getApplicationACLsManager();
+    if (aclsManager.areACLsEnabled()) {
+      if (callerUGI == null || !aclsManager.isAdmin(callerUGI)) {
+        String msg = "Only admins can carry out this operation.";
+        throw new ForbiddenException(msg);
+      }
+    }
+
+    ResourceScheduler scheduler = rm.getResourceScheduler();
+    if (scheduler instanceof MutableConfScheduler) {
+      try {
+        callerUGI.doAs(new PrivilegedExceptionAction<Void>() {
+          @Override
+          public Void run() throws IOException, YarnException {
+            Map<String, String> confUpdate =
+                constructKeyValueConfUpdate(mutationInfo);
+            ((CapacityScheduler) scheduler).updateConfiguration(callerUGI,
+                confUpdate);
+            return null;
+          }
+        });
+      } catch (IOException e) {
+        return Response.status(Status.BAD_REQUEST).entity(e.getMessage())
+            .build();
+      }
+      return Response.status(Status.OK).entity("Configuration change " +
+          "successfully applied.").build();
+    } else {
+      return Response.status(Status.BAD_REQUEST)
+          .entity("Configuration change only supported by CapacityScheduler.")
+          .build();
+    }
+  }
+
+  private Map<String, String> constructKeyValueConfUpdate(
+      QueueConfigsUpdateInfo mutationInfo) throws IOException {
+    CapacitySchedulerConfiguration currentConf =
+        ((CapacityScheduler) rm.getResourceScheduler()).getConfiguration();
+    CapacitySchedulerConfiguration proposedConf =
+        new CapacitySchedulerConfiguration(currentConf, false);
+    Map<String, String> confUpdate = new HashMap<>();
+    for (String queueToRemove : mutationInfo.getRemoveQueueInfo()) {
+      removeQueue(queueToRemove, proposedConf, confUpdate);
+    }
+    for (QueueConfigInfo addQueueInfo : mutationInfo.getAddQueueInfo()) {
+      addQueue(addQueueInfo, proposedConf, confUpdate);
+    }
+    for (QueueConfigInfo updateQueueInfo : mutationInfo.getUpdateQueueInfo()) {
+      updateQueue(updateQueueInfo, proposedConf, confUpdate);
+    }
+    return confUpdate;
+  }
+
+  private void removeQueue(
+      String queueToRemove, CapacitySchedulerConfiguration proposedConf,
+      Map<String, String> confUpdate) throws IOException {
+    if (queueToRemove == null) {
+      return;
+    } else {
+      CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
+      String queueName = queueToRemove.substring(
+          queueToRemove.lastIndexOf('.') + 1);
+      CSQueue queue = cs.getQueue(queueName);
+      if (queue == null ||
+          !queue.getQueuePath().equals(queueToRemove)) {
+        throw new IOException("Queue " + queueToRemove + " not found");
+      } else if (queueToRemove.lastIndexOf('.') == -1) {
+        throw new IOException("Can't remove queue " + queueToRemove);
+      }
+      String parentQueuePath = queueToRemove.substring(0, queueToRemove
+          .lastIndexOf('.'));
+      String[] siblingQueues = proposedConf.getQueues(parentQueuePath);
+      List<String> newSiblingQueues = new ArrayList<>();
+      for (String siblingQueue : siblingQueues) {
+        if (!siblingQueue.equals(queueName)) {
+          newSiblingQueues.add(siblingQueue);
+        }
+      }
+      proposedConf.setQueues(parentQueuePath, newSiblingQueues
+          .toArray(new String[0]));
+      String queuesConfig = CapacitySchedulerConfiguration.PREFIX +
+          parentQueuePath + CapacitySchedulerConfiguration.DOT +
+          CapacitySchedulerConfiguration.QUEUES;
+      if (newSiblingQueues.size() == 0) {
+        confUpdate.put(queuesConfig, null);
+      } else {
+        confUpdate.put(queuesConfig, Joiner.on(',').join(newSiblingQueues));
+      }
+      for (Map.Entry<String, String> confRemove : proposedConf.getValByRegex(
+          ".*" + queueToRemove.replaceAll("\\.", "\\.") + "\\..*")
+          .entrySet()) {
+        proposedConf.unset(confRemove.getKey());
+        confUpdate.put(confRemove.getKey(), null);
+      }
+    }
+  }
+
+  private void addQueue(
+      QueueConfigInfo addInfo, CapacitySchedulerConfiguration proposedConf,
+      Map<String, String> confUpdate) throws IOException {
+    if (addInfo == null) {
+      return;
+    } else {
+      CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
+      String queuePath = addInfo.getQueue();
+      String queueName = queuePath.substring(queuePath.lastIndexOf('.') + 1);
+      if (cs.getQueue(queueName) != null) {
+        throw new IOException("Can't add existing queue " + queuePath);
+      } else if (queuePath.lastIndexOf('.') == -1) {
+        throw new IOException("Can't add invalid queue " + queuePath);
+      }
+      String parentQueue = queuePath.substring(0, queuePath.lastIndexOf('.'));
+      String[] siblings = proposedConf.getQueues(parentQueue);
+      List<String> siblingQueues = siblings == null ? new ArrayList<>() :
+          new ArrayList<>(Arrays.<String>asList(siblings));
+      siblingQueues.add(queuePath.substring(queuePath.lastIndexOf('.') + 1));
+      proposedConf.setQueues(parentQueue,
+          siblingQueues.toArray(new String[0]));
+      confUpdate.put(CapacitySchedulerConfiguration.PREFIX +
+          parentQueue + CapacitySchedulerConfiguration.DOT +
+          CapacitySchedulerConfiguration.QUEUES,
+          Joiner.on(',').join(siblingQueues));
+      String keyPrefix = CapacitySchedulerConfiguration.PREFIX +
+          queuePath + CapacitySchedulerConfiguration.DOT;
+      for (Map.Entry<String, String> kv : addInfo.getParams().entrySet()) {
+        if (kv.getValue() == null) {
+          proposedConf.unset(keyPrefix + kv.getKey());
+        } else {
+          proposedConf.set(keyPrefix + kv.getKey(), kv.getValue());
+        }
+        confUpdate.put(keyPrefix + kv.getKey(), kv.getValue());
+      }
+    }
+  }
+
+  private void updateQueue(QueueConfigInfo updateInfo,
+      CapacitySchedulerConfiguration proposedConf,
+      Map<String, String> confUpdate) {
+    if (updateInfo == null) {
+      return;
+    } else {
+      String queuePath = updateInfo.getQueue();
+      String keyPrefix = CapacitySchedulerConfiguration.PREFIX +
+          queuePath + CapacitySchedulerConfiguration.DOT;
+      for (Map.Entry<String, String> kv : updateInfo.getParams().entrySet()) {
+        if (kv.getValue() == null) {
+          proposedConf.unset(keyPrefix + kv.getKey());
+        } else {
+          proposedConf.set(keyPrefix + kv.getKey(), kv.getValue());
+        }
+        confUpdate.put(keyPrefix + kv.getKey(), kv.getValue());
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eadb07db/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/QueueConfigInfo.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/QueueConfigInfo.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/QueueConfigInfo.java
new file mode 100644
index 0000000..b20eda6
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/QueueConfigInfo.java
@@ -0,0 +1,57 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, 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.webapp.dao;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+
+/**
+ * Information for adding or updating a queue to scheduler configuration
+ * for this queue.
+ */
+@XmlRootElement
+@XmlAccessorType(XmlAccessType.FIELD)
+public class QueueConfigInfo {
+
+  @XmlElement(name = "queueName")
+  private String queue;
+
+  private HashMap<String, String> params = new HashMap<>();
+
+  public QueueConfigInfo() { }
+
+  public QueueConfigInfo(String queue, Map<String, String> params) {
+    this.queue = queue;
+    this.params = new HashMap<>(params);
+  }
+
+  public String getQueue() {
+    return this.queue;
+  }
+
+  public HashMap<String, String> getParams() {
+    return this.params;
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eadb07db/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/QueueConfigsUpdateInfo.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/QueueConfigsUpdateInfo.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/QueueConfigsUpdateInfo.java
new file mode 100644
index 0000000..644ec90
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/QueueConfigsUpdateInfo.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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, 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.webapp.dao;
+
+import java.util.ArrayList;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+
+/**
+ * Information for making scheduler configuration changes (supports adding,
+ * removing, or updating a queue).
+ */
+@XmlRootElement(name = "schedConf")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class QueueConfigsUpdateInfo {
+
+  @XmlElement(name = "add")
+  private ArrayList<QueueConfigInfo> addQueueInfo = new ArrayList<>();
+
+  @XmlElement(name = "remove")
+  private ArrayList<String> removeQueueInfo = new ArrayList<>();
+
+  @XmlElement(name = "update")
+  private ArrayList<QueueConfigInfo> updateQueueInfo = new ArrayList<>();
+
+  public QueueConfigsUpdateInfo() {
+    // JAXB needs this
+  }
+
+  public ArrayList<QueueConfigInfo> getAddQueueInfo() {
+    return addQueueInfo;
+  }
+
+  public ArrayList<String> getRemoveQueueInfo() {
+    return removeQueueInfo;
+  }
+
+  public ArrayList<QueueConfigInfo> getUpdateQueueInfo() {
+    return updateQueueInfo;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eadb07db/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/TestMutableCSConfigurationProvider.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/scheduler/capacity/conf/TestMutableCSConfigurationProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/TestMutableCSConfigurationProvider.java
index 3f103b1..254da31 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/TestMutableCSConfigurationProvider.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/TestMutableCSConfigurationProvider.java
@@ -77,7 +77,11 @@ public class TestMutableCSConfigurationProvider {
     assertNull(confProvider.loadConfiguration(conf).get("badKey"));
     doThrow(new IOException()).when(cs).reinitialize(any(Configuration.class),
         any(RMContext.class));
-    confProvider.mutateConfiguration(TEST_USER, badUpdate);
+    try {
+      confProvider.mutateConfiguration(TEST_USER, badUpdate);
+    } catch (IOException e) {
+      // Expected exception.
+    }
     assertNull(confProvider.loadConfiguration(conf).get("badKey"));
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eadb07db/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesConfigurationMutation.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/webapp/TestRMWebServicesConfigurationMutation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesConfigurationMutation.java
new file mode 100644
index 0000000..d149055
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesConfigurationMutation.java
@@ -0,0 +1,477 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, 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.webapp;
+
+import com.google.inject.Guice;
+import com.google.inject.servlet.ServletModule;
+import com.sun.jersey.api.client.ClientResponse;
+import com.sun.jersey.api.client.WebResource;
+import com.sun.jersey.api.json.JSONJAXBContext;
+import com.sun.jersey.api.json.JSONMarshaller;
+import com.sun.jersey.guice.spi.container.servlet.GuiceContainer;
+import com.sun.jersey.test.framework.WebAppDescriptor;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.records.QueueState;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
+import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.QueueConfigInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.QueueConfigsUpdateInfo;
+import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
+import org.apache.hadoop.yarn.webapp.GuiceServletConfig;
+import org.apache.hadoop.yarn.webapp.JerseyTestBase;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response.Status;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.StringWriter;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+/**
+ * Test scheduler configuration mutation via REST API.
+ */
+public class TestRMWebServicesConfigurationMutation extends JerseyTestBase {
+
+  private static final File CONF_FILE = new File(new File("target",
+      "test-classes"), YarnConfiguration.CS_CONFIGURATION_FILE);
+  private static final File OLD_CONF_FILE = new File(new File("target",
+      "test-classes"), YarnConfiguration.CS_CONFIGURATION_FILE + ".tmp");
+
+  private static MockRM rm;
+  private static String userName;
+  private static CapacitySchedulerConfiguration csConf;
+  private static YarnConfiguration conf;
+
+  private static class WebServletModule extends ServletModule {
+    @Override
+    protected void configureServlets() {
+      bind(JAXBContextResolver.class);
+      bind(RMWebServices.class);
+      bind(GenericExceptionHandler.class);
+      try {
+        userName = UserGroupInformation.getCurrentUser().getShortUserName();
+      } catch (IOException ioe) {
+        throw new RuntimeException("Unable to get current user name "
+            + ioe.getMessage(), ioe);
+      }
+      csConf = new CapacitySchedulerConfiguration(new Configuration(false),
+          false);
+      setupQueueConfiguration(csConf);
+      conf = new YarnConfiguration();
+      conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
+          ResourceScheduler.class);
+      conf.set(CapacitySchedulerConfiguration.CS_CONF_PROVIDER,
+          CapacitySchedulerConfiguration.STORE_CS_CONF_PROVIDER);
+      conf.set(YarnConfiguration.YARN_ADMIN_ACL, userName);
+      try {
+        if (CONF_FILE.exists()) {
+          if (!CONF_FILE.renameTo(OLD_CONF_FILE)) {
+            throw new RuntimeException("Failed to rename conf file");
+          }
+        }
+        FileOutputStream out = new FileOutputStream(CONF_FILE);
+        csConf.writeXml(out);
+        out.close();
+      } catch (IOException e) {
+        throw new RuntimeException("Failed to write XML file", e);
+      }
+      rm = new MockRM(conf);
+      bind(ResourceManager.class).toInstance(rm);
+      serve("/*").with(GuiceContainer.class);
+      filter("/*").through(TestRMWebServicesAppsModification
+          .TestRMCustomAuthFilter.class);
+    }
+  }
+
+  @Override
+  @Before
+  public void setUp() throws Exception {
+    super.setUp();
+    GuiceServletConfig.setInjector(
+        Guice.createInjector(new WebServletModule()));
+  }
+
+  private static void setupQueueConfiguration(
+      CapacitySchedulerConfiguration config) {
+    config.setQueues(CapacitySchedulerConfiguration.ROOT,
+        new String[]{"a", "b", "c"});
+
+    final String a = CapacitySchedulerConfiguration.ROOT + ".a";
+    config.setCapacity(a, 25f);
+    config.setMaximumCapacity(a, 50f);
+
+    final String a1 = a + ".a1";
+    final String a2 = a + ".a2";
+    config.setQueues(a, new String[]{"a1", "a2"});
+    config.setCapacity(a1, 100f);
+    config.setCapacity(a2, 0f);
+
+    final String b = CapacitySchedulerConfiguration.ROOT + ".b";
+    config.setCapacity(b, 75f);
+
+    final String c = CapacitySchedulerConfiguration.ROOT + ".c";
+    config.setCapacity(c, 0f);
+
+    final String c1 = c + ".c1";
+    config.setQueues(c, new String[] {"c1"});
+    config.setCapacity(c1, 0f);
+  }
+
+  public TestRMWebServicesConfigurationMutation() {
+    super(new WebAppDescriptor.Builder(
+        "org.apache.hadoop.yarn.server.resourcemanager.webapp")
+        .contextListenerClass(GuiceServletConfig.class)
+        .filterClass(com.google.inject.servlet.GuiceFilter.class)
+        .contextPath("jersey-guice-filter").servletPath("/").build());
+  }
+
+  @Test
+  public void testAddNestedQueue() throws Exception {
+    WebResource r = resource();
+
+    ClientResponse response;
+
+    // Add parent queue root.d with two children d1 and d2.
+    QueueConfigsUpdateInfo updateInfo = new QueueConfigsUpdateInfo();
+    Map<String, String> d1Capacity = new HashMap<>();
+    d1Capacity.put(CapacitySchedulerConfiguration.CAPACITY, "25");
+    d1Capacity.put(CapacitySchedulerConfiguration.MAXIMUM_CAPACITY, "25");
+    Map<String, String> nearEmptyCapacity = new HashMap<>();
+    nearEmptyCapacity.put(CapacitySchedulerConfiguration.CAPACITY, "1E-4");
+    nearEmptyCapacity.put(CapacitySchedulerConfiguration.MAXIMUM_CAPACITY,
+        "1E-4");
+    Map<String, String> d2Capacity = new HashMap<>();
+    d2Capacity.put(CapacitySchedulerConfiguration.CAPACITY, "75");
+    d2Capacity.put(CapacitySchedulerConfiguration.MAXIMUM_CAPACITY, "75");
+    QueueConfigInfo d1 = new QueueConfigInfo("root.d.d1", d1Capacity);
+    QueueConfigInfo d2 = new QueueConfigInfo("root.d.d2", d2Capacity);
+    QueueConfigInfo d = new QueueConfigInfo("root.d", nearEmptyCapacity);
+    updateInfo.getAddQueueInfo().add(d1);
+    updateInfo.getAddQueueInfo().add(d2);
+    updateInfo.getAddQueueInfo().add(d);
+    response =
+        r.path("ws").path("v1").path("cluster")
+            .path("queues").queryParam("user.name", userName)
+            .accept(MediaType.APPLICATION_JSON)
+            .entity(toJson(updateInfo, QueueConfigsUpdateInfo.class),
+                MediaType.APPLICATION_JSON)
+            .put(ClientResponse.class);
+
+    assertEquals(Status.OK.getStatusCode(), response.getStatus());
+    CapacitySchedulerConfiguration newCSConf =
+        ((CapacityScheduler) rm.getResourceScheduler()).getConfiguration();
+    assertEquals(4, newCSConf.getQueues("root").length);
+    assertEquals(2, newCSConf.getQueues("root.d").length);
+    assertEquals(25.0f, newCSConf.getNonLabeledQueueCapacity("root.d.d1"),
+        0.01f);
+    assertEquals(75.0f, newCSConf.getNonLabeledQueueCapacity("root.d.d2"),
+        0.01f);
+  }
+
+  @Test
+  public void testAddWithUpdate() throws Exception {
+    WebResource r = resource();
+
+    ClientResponse response;
+
+    // Add root.d with capacity 25, reducing root.b capacity from 75 to 50.
+    QueueConfigsUpdateInfo updateInfo = new QueueConfigsUpdateInfo();
+    Map<String, String> dCapacity = new HashMap<>();
+    dCapacity.put(CapacitySchedulerConfiguration.CAPACITY, "25");
+    Map<String, String> bCapacity = new HashMap<>();
+    bCapacity.put(CapacitySchedulerConfiguration.CAPACITY, "50");
+    QueueConfigInfo d = new QueueConfigInfo("root.d", dCapacity);
+    QueueConfigInfo b = new QueueConfigInfo("root.b", bCapacity);
+    updateInfo.getAddQueueInfo().add(d);
+    updateInfo.getUpdateQueueInfo().add(b);
+    response =
+        r.path("ws").path("v1").path("cluster")
+            .path("queues").queryParam("user.name", userName)
+            .accept(MediaType.APPLICATION_JSON)
+            .entity(toJson(updateInfo, QueueConfigsUpdateInfo.class),
+                MediaType.APPLICATION_JSON)
+            .put(ClientResponse.class);
+
+    assertEquals(Status.OK.getStatusCode(), response.getStatus());
+    CapacitySchedulerConfiguration newCSConf =
+        ((CapacityScheduler) rm.getResourceScheduler()).getConfiguration();
+    assertEquals(4, newCSConf.getQueues("root").length);
+    assertEquals(25.0f, newCSConf.getNonLabeledQueueCapacity("root.d"), 0.01f);
+    assertEquals(50.0f, newCSConf.getNonLabeledQueueCapacity("root.b"), 0.01f);
+  }
+
+  @Test
+  public void testRemoveQueue() throws Exception {
+    WebResource r = resource();
+
+    ClientResponse response;
+
+    stopQueue("root.a.a2");
+    // Remove root.a.a2
+    QueueConfigsUpdateInfo updateInfo = new QueueConfigsUpdateInfo();
+    updateInfo.getRemoveQueueInfo().add("root.a.a2");
+    response =
+        r.path("ws").path("v1").path("cluster")
+            .path("queues").queryParam("user.name", userName)
+            .accept(MediaType.APPLICATION_JSON)
+            .entity(toJson(updateInfo, QueueConfigsUpdateInfo.class),
+                MediaType.APPLICATION_JSON)
+            .put(ClientResponse.class);
+
+    assertEquals(Status.OK.getStatusCode(), response.getStatus());
+    CapacitySchedulerConfiguration newCSConf =
+        ((CapacityScheduler) rm.getResourceScheduler()).getConfiguration();
+    assertEquals(1, newCSConf.getQueues("root.a").length);
+    assertEquals("a1", newCSConf.getQueues("root.a")[0]);
+  }
+
+  @Test
+  public void testRemoveParentQueue() throws Exception {
+    WebResource r = resource();
+
+    ClientResponse response;
+
+    stopQueue("root.c", "root.c.c1");
+    // Remove root.c (parent queue)
+    QueueConfigsUpdateInfo updateInfo = new QueueConfigsUpdateInfo();
+    updateInfo.getRemoveQueueInfo().add("root.c");
+    response =
+        r.path("ws").path("v1").path("cluster")
+            .path("queues").queryParam("user.name", userName)
+            .accept(MediaType.APPLICATION_JSON)
+            .entity(toJson(updateInfo, QueueConfigsUpdateInfo.class),
+                MediaType.APPLICATION_JSON)
+            .put(ClientResponse.class);
+
+    assertEquals(Status.OK.getStatusCode(), response.getStatus());
+    CapacitySchedulerConfiguration newCSConf =
+        ((CapacityScheduler) rm.getResourceScheduler()).getConfiguration();
+    assertEquals(2, newCSConf.getQueues("root").length);
+    assertNull(newCSConf.getQueues("root.c"));
+  }
+
+  @Test
+  public void testRemoveParentQueueWithCapacity() throws Exception {
+    WebResource r = resource();
+
+    ClientResponse response;
+
+    stopQueue("root.a", "root.a.a1", "root.a.a2");
+    // Remove root.a (parent queue) with capacity 25
+    QueueConfigsUpdateInfo updateInfo = new QueueConfigsUpdateInfo();
+    updateInfo.getRemoveQueueInfo().add("root.a");
+
+    // Set root.b capacity to 100
+    Map<String, String> bCapacity = new HashMap<>();
+    bCapacity.put(CapacitySchedulerConfiguration.CAPACITY, "100");
+    QueueConfigInfo b = new QueueConfigInfo("root.b", bCapacity);
+    updateInfo.getUpdateQueueInfo().add(b);
+    response =
+        r.path("ws").path("v1").path("cluster")
+            .path("queues").queryParam("user.name", userName)
+            .accept(MediaType.APPLICATION_JSON)
+            .entity(toJson(updateInfo, QueueConfigsUpdateInfo.class),
+                MediaType.APPLICATION_JSON)
+            .put(ClientResponse.class);
+
+    assertEquals(Status.OK.getStatusCode(), response.getStatus());
+    CapacitySchedulerConfiguration newCSConf =
+        ((CapacityScheduler) rm.getResourceScheduler()).getConfiguration();
+    assertEquals(2, newCSConf.getQueues("root").length);
+    assertEquals(100.0f, newCSConf.getNonLabeledQueueCapacity("root.b"),
+        0.01f);
+  }
+
+  @Test
+  public void testRemoveMultipleQueues() throws Exception {
+    WebResource r = resource();
+
+    ClientResponse response;
+
+    stopQueue("root.b", "root.c", "root.c.c1");
+    // Remove root.b and root.c
+    QueueConfigsUpdateInfo updateInfo = new QueueConfigsUpdateInfo();
+    updateInfo.getRemoveQueueInfo().add("root.b");
+    updateInfo.getRemoveQueueInfo().add("root.c");
+    Map<String, String> aCapacity = new HashMap<>();
+    aCapacity.put(CapacitySchedulerConfiguration.CAPACITY, "100");
+    aCapacity.put(CapacitySchedulerConfiguration.MAXIMUM_CAPACITY, "100");
+    QueueConfigInfo configInfo = new QueueConfigInfo("root.a", aCapacity);
+    updateInfo.getUpdateQueueInfo().add(configInfo);
+    response =
+        r.path("ws").path("v1").path("cluster")
+            .path("queues").queryParam("user.name", userName)
+            .accept(MediaType.APPLICATION_JSON)
+            .entity(toJson(updateInfo, QueueConfigsUpdateInfo.class),
+                MediaType.APPLICATION_JSON)
+            .put(ClientResponse.class);
+
+    assertEquals(Status.OK.getStatusCode(), response.getStatus());
+    CapacitySchedulerConfiguration newCSConf =
+        ((CapacityScheduler) rm.getResourceScheduler()).getConfiguration();
+    assertEquals(1, newCSConf.getQueues("root").length);
+  }
+
+  private void stopQueue(String... queuePaths) throws Exception {
+    WebResource r = resource();
+
+    ClientResponse response;
+
+    // Set state of queues to STOPPED.
+    QueueConfigsUpdateInfo updateInfo = new QueueConfigsUpdateInfo();
+    Map<String, String> stoppedParam = new HashMap<>();
+    stoppedParam.put(CapacitySchedulerConfiguration.STATE,
+        QueueState.STOPPED.toString());
+    for (String queue : queuePaths) {
+      QueueConfigInfo stoppedInfo = new QueueConfigInfo(queue, stoppedParam);
+      updateInfo.getUpdateQueueInfo().add(stoppedInfo);
+    }
+    response =
+        r.path("ws").path("v1").path("cluster")
+            .path("queues").queryParam("user.name", userName)
+            .accept(MediaType.APPLICATION_JSON)
+            .entity(toJson(updateInfo, QueueConfigsUpdateInfo.class),
+                MediaType.APPLICATION_JSON)
+            .put(ClientResponse.class);
+    assertEquals(Status.OK.getStatusCode(), response.getStatus());
+    CapacitySchedulerConfiguration newCSConf =
+        ((CapacityScheduler) rm.getResourceScheduler()).getConfiguration();
+    for (String queue : queuePaths) {
+      assertEquals(QueueState.STOPPED, newCSConf.getState(queue));
+    }
+  }
+
+  @Test
+  public void testUpdateQueue() throws Exception {
+    WebResource r = resource();
+
+    ClientResponse response;
+
+    // Update config value.
+    QueueConfigsUpdateInfo updateInfo = new QueueConfigsUpdateInfo();
+    Map<String, String> updateParam = new HashMap<>();
+    updateParam.put(CapacitySchedulerConfiguration.MAXIMUM_AM_RESOURCE_SUFFIX,
+        "0.2");
+    QueueConfigInfo aUpdateInfo = new QueueConfigInfo("root.a", updateParam);
+    updateInfo.getUpdateQueueInfo().add(aUpdateInfo);
+    CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
+
+    assertEquals(CapacitySchedulerConfiguration
+            .DEFAULT_MAXIMUM_APPLICATIONMASTERS_RESOURCE_PERCENT,
+        cs.getConfiguration()
+            .getMaximumApplicationMasterResourcePerQueuePercent("root.a"),
+        0.001f);
+    response =
+        r.path("ws").path("v1").path("cluster")
+            .path("queues").queryParam("user.name", userName)
+            .accept(MediaType.APPLICATION_JSON)
+            .entity(toJson(updateInfo, QueueConfigsUpdateInfo.class),
+                MediaType.APPLICATION_JSON)
+            .put(ClientResponse.class);
+    assertEquals(Status.OK.getStatusCode(), response.getStatus());
+    CapacitySchedulerConfiguration newCSConf = cs.getConfiguration();
+    assertEquals(0.2f, newCSConf
+        .getMaximumApplicationMasterResourcePerQueuePercent("root.a"), 0.001f);
+
+    // Remove config. Config value should be reverted to default.
+    updateParam.put(CapacitySchedulerConfiguration.MAXIMUM_AM_RESOURCE_SUFFIX,
+        null);
+    aUpdateInfo = new QueueConfigInfo("root.a", updateParam);
+    updateInfo.getUpdateQueueInfo().clear();
+    updateInfo.getUpdateQueueInfo().add(aUpdateInfo);
+    response =
+        r.path("ws").path("v1").path("cluster")
+            .path("queues").queryParam("user.name", userName)
+            .accept(MediaType.APPLICATION_JSON)
+            .entity(toJson(updateInfo, QueueConfigsUpdateInfo.class),
+                MediaType.APPLICATION_JSON)
+            .put(ClientResponse.class);
+    assertEquals(Status.OK.getStatusCode(), response.getStatus());
+    newCSConf = cs.getConfiguration();
+    assertEquals(CapacitySchedulerConfiguration
+        .DEFAULT_MAXIMUM_APPLICATIONMASTERS_RESOURCE_PERCENT, newCSConf
+            .getMaximumApplicationMasterResourcePerQueuePercent("root.a"),
+        0.001f);
+  }
+
+  @Test
+  public void testUpdateQueueCapacity() throws Exception {
+    WebResource r = resource();
+
+    ClientResponse response;
+
+    // Update root.a and root.b capacity to 50.
+    QueueConfigsUpdateInfo updateInfo = new QueueConfigsUpdateInfo();
+    Map<String, String> updateParam = new HashMap<>();
+    updateParam.put(CapacitySchedulerConfiguration.CAPACITY, "50");
+    QueueConfigInfo aUpdateInfo = new QueueConfigInfo("root.a", updateParam);
+    QueueConfigInfo bUpdateInfo = new QueueConfigInfo("root.b", updateParam);
+    updateInfo.getUpdateQueueInfo().add(aUpdateInfo);
+    updateInfo.getUpdateQueueInfo().add(bUpdateInfo);
+
+    response =
+        r.path("ws").path("v1").path("cluster")
+            .path("queues").queryParam("user.name", userName)
+            .accept(MediaType.APPLICATION_JSON)
+            .entity(toJson(updateInfo, QueueConfigsUpdateInfo.class),
+                MediaType.APPLICATION_JSON)
+            .put(ClientResponse.class);
+    assertEquals(Status.OK.getStatusCode(), response.getStatus());
+    CapacitySchedulerConfiguration newCSConf =
+        ((CapacityScheduler) rm.getResourceScheduler()).getConfiguration();
+    assertEquals(50.0f, newCSConf.getNonLabeledQueueCapacity("root.a"), 0.01f);
+    assertEquals(50.0f, newCSConf.getNonLabeledQueueCapacity("root.b"), 0.01f);
+  }
+
+  @Override
+  @After
+  public void tearDown() throws Exception {
+    if (rm != null) {
+      rm.stop();
+    }
+    CONF_FILE.delete();
+    if (!OLD_CONF_FILE.renameTo(CONF_FILE)) {
+      throw new RuntimeException("Failed to re-copy old configuration file");
+    }
+    super.tearDown();
+  }
+
+  @SuppressWarnings("rawtypes")
+  private String toJson(Object nsli, Class klass) throws Exception {
+    StringWriter sw = new StringWriter();
+    JSONJAXBContext ctx = new JSONJAXBContext(klass);
+    JSONMarshaller jm = ctx.createJSONMarshaller();
+    jm.marshallToJSON(nsli, sw);
+    return sw.toString();
+  }
+}


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


[23/50] [abbrv] hadoop git commit: HADOOP-14726. Mark FileStatus::isDir as final

Posted by jh...@apache.org.
HADOOP-14726. Mark FileStatus::isDir as final


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

Branch: refs/heads/YARN-5734
Commit: 645a8f2a4d09acb5a21820f52ee78784d9e4cc8a
Parents: 4d7be1d
Author: Chris Douglas <cd...@apache.org>
Authored: Mon Aug 14 21:57:20 2017 -0700
Committer: Chris Douglas <cd...@apache.org>
Committed: Mon Aug 14 21:57:20 2017 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/fs/FileStatus.java    | 19 +++++++++----------
 .../hadoop/fs/viewfs/ViewFsFileStatus.java       |  8 +-------
 .../fs/viewfs/ViewFsLocatedFileStatus.java       |  6 ------
 .../hadoop/hdfs/protocolPB/PBHelperClient.java   |  2 +-
 .../apache/hadoop/hdfs/nfs/nfs3/Nfs3Utils.java   |  6 ++++--
 .../hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java     |  8 ++++----
 .../apache/hadoop/hdfs/server/mover/Mover.java   |  2 +-
 .../hdfs/server/namenode/NamenodeFsck.java       |  4 ++--
 .../hadoop/hdfs/TestDFSUpgradeFromImage.java     |  3 +--
 .../hdfs/server/mover/TestStorageMover.java      |  2 +-
 .../hadoop/hdfs/server/namenode/TestStartup.java |  4 ++--
 .../server/namenode/ha/TestEditLogTailer.java    |  4 ++--
 .../namenode/ha/TestFailureToReadEdits.java      |  6 +++---
 .../namenode/ha/TestInitializeSharedEdits.java   |  2 +-
 .../lib/input/TestCombineFileInputFormat.java    |  2 +-
 .../azure/TestOutOfBandAzureBlobOperations.java  |  8 ++++----
 .../hadoop/fs/swift/snative/SwiftFileStatus.java | 16 ++++------------
 .../snative/SwiftNativeFileSystemStore.java      |  4 ++--
 .../fs/swift/TestSwiftFileSystemDirectories.java |  4 ++--
 .../TestSwiftFileSystemPartitionedUploads.java   |  2 +-
 20 files changed, 46 insertions(+), 66 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/645a8f2a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileStatus.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileStatus.java
index 2f22ea0..8575439 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileStatus.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileStatus.java
@@ -172,7 +172,7 @@ public class FileStatus implements Writable, Comparable<Object>,
    * @return true if this is a file
    */
   public boolean isFile() {
-    return !isdir && !isSymlink();
+    return !isDirectory() && !isSymlink();
   }
 
   /**
@@ -182,20 +182,20 @@ public class FileStatus implements Writable, Comparable<Object>,
   public boolean isDirectory() {
     return isdir;
   }
-  
+
   /**
-   * Old interface, instead use the explicit {@link FileStatus#isFile()}, 
-   * {@link FileStatus#isDirectory()}, and {@link FileStatus#isSymlink()} 
+   * Old interface, instead use the explicit {@link FileStatus#isFile()},
+   * {@link FileStatus#isDirectory()}, and {@link FileStatus#isSymlink()}
    * @return true if this is a directory.
-   * @deprecated Use {@link FileStatus#isFile()},  
-   * {@link FileStatus#isDirectory()}, and {@link FileStatus#isSymlink()} 
+   * @deprecated Use {@link FileStatus#isFile()},
+   * {@link FileStatus#isDirectory()}, and {@link FileStatus#isSymlink()}
    * instead.
    */
   @Deprecated
-  public boolean isDir() {
-    return isdir;
+  public final boolean isDir() {
+    return isDirectory();
   }
-  
+
   /**
    * Is this a symbolic link?
    * @return true if this is a symbolic link
@@ -448,7 +448,6 @@ public class FileStatus implements Writable, Comparable<Object>,
     FileStatus other = PBHelper.convert(proto);
     isdir = other.isDirectory();
     length = other.getLen();
-    isdir = other.isDirectory();
     block_replication = other.getReplication();
     blocksize = other.getBlockSize();
     modification_time = other.getModificationTime();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/645a8f2a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFsFileStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFsFileStatus.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFsFileStatus.java
index e0f62e4..ce03ced 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFsFileStatus.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFsFileStatus.java
@@ -61,13 +61,7 @@ class ViewFsFileStatus extends FileStatus {
    public boolean isDirectory() {
      return  myFs.isDirectory();
    }
-   
-   @Override
-   @SuppressWarnings("deprecation")
-   public boolean isDir() {
-     return myFs.isDirectory();
-   }
-   
+
    @Override
    public boolean isSymlink() {
      return myFs.isSymlink();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/645a8f2a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFsLocatedFileStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFsLocatedFileStatus.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFsLocatedFileStatus.java
index 4e681a7..c7c8b28 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFsLocatedFileStatus.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFsLocatedFileStatus.java
@@ -50,12 +50,6 @@ class ViewFsLocatedFileStatus extends LocatedFileStatus {
   }
 
   @Override
-  @SuppressWarnings("deprecation")
-  public boolean isDir() {
-    return myFs.isDirectory();
-  }
-
-  @Override
   public boolean isSymlink() {
     return myFs.isSymlink();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/645a8f2a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
index a0d2c5a..5b1a687 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
@@ -2095,7 +2095,7 @@ public class PBHelperClient {
     if (fs == null)
       return null;
     FileType fType = FileType.IS_FILE;
-    if (fs.isDir()) {
+    if (fs.isDirectory()) {
       fType = FileType.IS_DIR;
     } else if (fs.isSymlink()) {
       fType = FileType.IS_SYMLINK;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/645a8f2a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3Utils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3Utils.java b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3Utils.java
index cc17394..abaa5ca 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3Utils.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3Utils.java
@@ -65,7 +65,9 @@ public class Nfs3Utils {
      * client takes only the lower 32bit of the fileId and treats it as signed
      * int. When the 32th bit is 1, the client considers it invalid.
      */
-    NfsFileType fileType = fs.isDir() ? NfsFileType.NFSDIR : NfsFileType.NFSREG;
+    NfsFileType fileType = fs.isDirectory()
+        ? NfsFileType.NFSDIR
+        : NfsFileType.NFSREG;
     fileType = fs.isSymlink() ? NfsFileType.NFSLNK : fileType;
     int nlink = (fileType == NfsFileType.NFSDIR) ? fs.getChildrenNum() + 2 : 1;
     long size = (fileType == NfsFileType.NFSDIR) ? getDirSize(fs
@@ -98,7 +100,7 @@ public class Nfs3Utils {
       return null;
     }
 
-    long size = fstat.isDir() ? getDirSize(fstat.getChildrenNum()) : fstat
+    long size = fstat.isDirectory() ? getDirSize(fstat.getChildrenNum()) : fstat
         .getLen();
     return new WccAttr(size, new NfsTime(fstat.getModificationTime()),
         new NfsTime(fstat.getModificationTime()));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/645a8f2a/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 d6bb71d..7a6aa89 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
@@ -1208,7 +1208,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
       if (fstat == null) {
         return new REMOVE3Response(Nfs3Status.NFS3ERR_NOENT, errWcc);
       }
-      if (fstat.isDir()) {
+      if (fstat.isDirectory()) {
         return new REMOVE3Response(Nfs3Status.NFS3ERR_ISDIR, errWcc);
       }
 
@@ -1289,7 +1289,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
       if (fstat == null) {
         return new RMDIR3Response(Nfs3Status.NFS3ERR_NOENT, errWcc);
       }
-      if (!fstat.isDir()) {
+      if (!fstat.isDirectory()) {
         return new RMDIR3Response(Nfs3Status.NFS3ERR_NOTDIR, errWcc);
       }
 
@@ -1565,7 +1565,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
         LOG.info("Can't get path for fileId: " + handle.getFileId());
         return new READDIR3Response(Nfs3Status.NFS3ERR_STALE);
       }
-      if (!dirStatus.isDir()) {
+      if (!dirStatus.isDirectory()) {
         LOG.error("Can't readdir for regular file, fileId: "
             + handle.getFileId());
         return new READDIR3Response(Nfs3Status.NFS3ERR_NOTDIR);
@@ -1732,7 +1732,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
         LOG.info("Can't get path for fileId: " + handle.getFileId());
         return new READDIRPLUS3Response(Nfs3Status.NFS3ERR_STALE);
       }
-      if (!dirStatus.isDir()) {
+      if (!dirStatus.isDirectory()) {
         LOG.error("Can't readdirplus for regular file, fileId: "
             + handle.getFileId());
         return new READDIRPLUS3Response(Nfs3Status.NFS3ERR_NOTDIR);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/645a8f2a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
index 1a2c889..b653f4f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
@@ -348,7 +348,7 @@ public class Mover {
     private void processRecursively(String parent, HdfsFileStatus status,
         Result result) {
       String fullPath = status.getFullName(parent);
-      if (status.isDir()) {
+      if (status.isDirectory()) {
         if (!fullPath.endsWith(Path.SEPARATOR)) {
           fullPath = fullPath + Path.SEPARATOR;
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/645a8f2a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
index eddab3f..5872955 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
@@ -471,7 +471,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
   void check(String parent, HdfsFileStatus file, Result replRes, Result ecRes)
       throws IOException {
     String path = file.getFullName(parent);
-    if (file.isDir()) {
+    if (file.isDirectory()) {
       checkDir(path, replRes, ecRes);
       return;
     }
@@ -1115,7 +1115,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
       if (lfStatus == null) { // not exists
         lfInitedOk = dfs.mkdirs(lfName, null, true);
         lostFound = lfName;
-      } else if (!lfStatus.isDir()) { // exists but not a directory
+      } else if (!lfStatus.isDirectory()) { // exists but not a directory
         LOG.warn("Cannot use /lost+found : a regular file with this name exists.");
         lfInitedOk = false;
       }  else { // exists and is a directory

http://git-wip-us.apache.org/repos/asf/hadoop/blob/645a8f2a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java
index 5dee6e0..e42e08c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java
@@ -51,7 +51,6 @@ import org.apache.hadoop.util.StringUtils;
 import org.apache.log4j.Logger;
 import org.junit.Test;
 
-import static org.apache.hadoop.hdfs.inotify.Event.CreateEvent;
 import static org.junit.Assert.*;
 
 /**
@@ -572,7 +571,7 @@ public class TestDFSUpgradeFromImage {
       Path path) throws IOException {
     String pathStr = path.toString();
     HdfsFileStatus status = dfs.getFileInfo(pathStr);
-    if (!status.isDir()) {
+    if (!status.isDirectory()) {
       for (int retries = 10; retries > 0; retries--) {
         if (dfs.recoverLease(pathStr)) {
           return;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/645a8f2a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestStorageMover.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestStorageMover.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestStorageMover.java
index 077997f..764a0db 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestStorageMover.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestStorageMover.java
@@ -283,7 +283,7 @@ public class TestStorageMover {
 
     private void verifyRecursively(final Path parent,
         final HdfsFileStatus status) throws Exception {
-      if (status.isDir()) {
+      if (status.isDirectory()) {
         Path fullPath = parent == null ?
             new Path("/") : status.getFullPath(parent);
         DirectoryListing children = dfs.getClient().listPaths(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/645a8f2a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java
index 29a6064..94172bb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java
@@ -450,7 +450,7 @@ public class TestStartup {
     namenode.getNamesystem().mkdirs("/test",
         new PermissionStatus("hairong", null, FsPermission.getDefault()), true);
     NamenodeProtocols nnRpc = namenode.getRpcServer();
-    assertTrue(nnRpc.getFileInfo("/test").isDir());
+    assertTrue(nnRpc.getFileInfo("/test").isDirectory());
     nnRpc.setSafeMode(SafeModeAction.SAFEMODE_ENTER, false);
     nnRpc.saveNamespace(0, 0);
     namenode.stop();
@@ -481,7 +481,7 @@ public class TestStartup {
   private void checkNameSpace(Configuration conf) throws IOException {
     NameNode namenode = new NameNode(conf);
     NamenodeProtocols nnRpc = namenode.getRpcServer();
-    assertTrue(nnRpc.getFileInfo("/test").isDir());
+    assertTrue(nnRpc.getFileInfo("/test").isDirectory());
     nnRpc.setSafeMode(SafeModeAction.SAFEMODE_ENTER, false);
     nnRpc.saveNamespace(0, 0);
     namenode.stop();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/645a8f2a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogTailer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogTailer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogTailer.java
index 5a2aff9..b3bb3dd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogTailer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogTailer.java
@@ -124,7 +124,7 @@ public class TestEditLogTailer {
       
       for (int i = 0; i < DIRS_TO_MAKE / 2; i++) {
         assertTrue(NameNodeAdapter.getFileInfo(nn2,
-            getDirPath(i), false).isDir());
+            getDirPath(i), false).isDirectory());
       }
       
       for (int i = DIRS_TO_MAKE / 2; i < DIRS_TO_MAKE; i++) {
@@ -137,7 +137,7 @@ public class TestEditLogTailer {
       
       for (int i = DIRS_TO_MAKE / 2; i < DIRS_TO_MAKE; i++) {
         assertTrue(NameNodeAdapter.getFileInfo(nn2,
-            getDirPath(i), false).isDir());
+            getDirPath(i), false).isDirectory());
       }
     } finally {
       cluster.shutdown();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/645a8f2a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailureToReadEdits.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailureToReadEdits.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailureToReadEdits.java
index 38c2b2d..93c717c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailureToReadEdits.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailureToReadEdits.java
@@ -205,7 +205,7 @@ public class TestFailureToReadEdits {
         TEST_DIR1, false));
     // Should have been successfully created.
     assertTrue(NameNodeAdapter.getFileInfo(nn1,
-        TEST_DIR2, false).isDir());
+        TEST_DIR2, false).isDirectory());
     // Null because it hasn't been created yet.
     assertNull(NameNodeAdapter.getFileInfo(nn1,
         TEST_DIR3, false));
@@ -219,10 +219,10 @@ public class TestFailureToReadEdits {
         TEST_DIR1, false));
     // Should have been successfully created.
     assertTrue(NameNodeAdapter.getFileInfo(nn1,
-        TEST_DIR2, false).isDir());
+        TEST_DIR2, false).isDirectory());
     // Should now have been successfully created.
     assertTrue(NameNodeAdapter.getFileInfo(nn1,
-        TEST_DIR3, false).isDir());
+        TEST_DIR3, false).isDirectory());
   }
   
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/645a8f2a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestInitializeSharedEdits.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestInitializeSharedEdits.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestInitializeSharedEdits.java
index 856ed8f..8eeb853 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestInitializeSharedEdits.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestInitializeSharedEdits.java
@@ -128,7 +128,7 @@ public class TestInitializeSharedEdits {
       HATestUtil.waitForStandbyToCatchUp(cluster.getNameNode(0),
           cluster.getNameNode(1));
       assertTrue(NameNodeAdapter.getFileInfo(cluster.getNameNode(1),
-          newPath.toString(), false).isDir());
+          newPath.toString(), false).isDirectory());
     } finally {
       if (fs != null) {
         fs.close();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/645a8f2a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineFileInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineFileInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineFileInputFormat.java
index 1fca5c9..cc97a14d 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineFileInputFormat.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineFileInputFormat.java
@@ -154,7 +154,7 @@ public class TestCombineFileInputFormat {
     @Override
     public BlockLocation[] getFileBlockLocations(
         FileStatus stat, long start, long len) throws IOException {
-      if (stat.isDir()) {
+      if (stat.isDirectory()) {
         return null;
       }
       System.out.println("File " + stat.getPath());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/645a8f2a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestOutOfBandAzureBlobOperations.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestOutOfBandAzureBlobOperations.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestOutOfBandAzureBlobOperations.java
index 9d5d6a22c..544d6ab 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestOutOfBandAzureBlobOperations.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestOutOfBandAzureBlobOperations.java
@@ -73,20 +73,20 @@ public class TestOutOfBandAzureBlobOperations {
     FileStatus[] obtained = fs.listStatus(new Path("/root/b"));
     assertNotNull(obtained);
     assertEquals(1, obtained.length);
-    assertFalse(obtained[0].isDir());
+    assertFalse(obtained[0].isDirectory());
     assertEquals("/root/b", obtained[0].getPath().toUri().getPath());
 
     // List the directory
     obtained = fs.listStatus(new Path("/root"));
     assertNotNull(obtained);
     assertEquals(1, obtained.length);
-    assertFalse(obtained[0].isDir());
+    assertFalse(obtained[0].isDirectory());
     assertEquals("/root/b", obtained[0].getPath().toUri().getPath());
 
     // Get the directory's file status
     FileStatus dirStatus = fs.getFileStatus(new Path("/root"));
     assertNotNull(dirStatus);
-    assertTrue(dirStatus.isDir());
+    assertTrue(dirStatus.isDirectory());
     assertEquals("/root", dirStatus.getPath().toUri().getPath());
   }
 
@@ -114,7 +114,7 @@ public class TestOutOfBandAzureBlobOperations {
     FileStatus[] listResult = fs.listStatus(new Path("/root/b"));
     // File should win.
     assertEquals(1, listResult.length);
-    assertFalse(listResult[0].isDir());
+    assertFalse(listResult[0].isDirectory());
     try {
       // Trying to delete root/b/c would cause a dilemma for WASB, so
       // it should throw.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/645a8f2a/hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/snative/SwiftFileStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/snative/SwiftFileStatus.java b/hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/snative/SwiftFileStatus.java
index d010d08..725cae1 100644
--- a/hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/snative/SwiftFileStatus.java
+++ b/hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/snative/SwiftFileStatus.java
@@ -71,7 +71,7 @@ public class SwiftFileStatus extends FileStatus {
    * @return true if the status is considered to be a file
    */
   @Override
-  public boolean isDir() {
+  public boolean isDirectory() {
     return super.isDirectory() || getLen() == 0;
   }
 
@@ -79,19 +79,11 @@ public class SwiftFileStatus extends FileStatus {
    * A entry is a file if it is not a directory.
    * By implementing it <i>and not marking as an override</i> this
    * subclass builds and runs in both Hadoop versions.
-   * @return the opposite value to {@link #isDir()}
+   * @return the opposite value to {@link #isDirectory()}
    */
   @Override
   public boolean isFile() {
-    return !isDir();
-  }
-
-  /**
-   * Directory test
-   * @return true if the file is considered to be a directory
-   */
-  public boolean isDirectory() {
-    return isDir();
+    return !this.isDirectory();
   }
 
   @Override
@@ -100,7 +92,7 @@ public class SwiftFileStatus extends FileStatus {
     sb.append(getClass().getSimpleName());
     sb.append("{ ");
     sb.append("path=").append(getPath());
-    sb.append("; isDirectory=").append(isDir());
+    sb.append("; isDirectory=").append(isDirectory());
     sb.append("; length=").append(getLen());
     sb.append("; blocksize=").append(getBlockSize());
     sb.append("; modification_time=").append(getModificationTime());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/645a8f2a/hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/snative/SwiftNativeFileSystemStore.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/snative/SwiftNativeFileSystemStore.java b/hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/snative/SwiftNativeFileSystemStore.java
index f2ecb0f..a44051a 100644
--- a/hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/snative/SwiftNativeFileSystemStore.java
+++ b/hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/snative/SwiftNativeFileSystemStore.java
@@ -578,7 +578,7 @@ public class SwiftNativeFileSystemStore {
 
     //enum the child entries and everything underneath
     List<FileStatus> childStats = listDirectory(srcObject, true, true);
-    boolean srcIsFile = !srcMetadata.isDir();
+    boolean srcIsFile = !srcMetadata.isDirectory();
     if (srcIsFile) {
 
       //source is a simple file OR a partitioned file
@@ -945,7 +945,7 @@ public class SwiftNativeFileSystemStore {
     //>1 entry implies directory with children. Run through them,
     // but first check for the recursive flag and reject it *unless it looks
     // like a partitioned file (len > 0 && has children)
-    if (!fileStatus.isDir()) {
+    if (!fileStatus.isDirectory()) {
       LOG.debug("Multiple child entries but entry has data: assume partitioned");
     } else if (!recursive) {
       //if there are children, unless this is a recursive operation, fail immediately

http://git-wip-us.apache.org/repos/asf/hadoop/blob/645a8f2a/hadoop-tools/hadoop-openstack/src/test/java/org/apache/hadoop/fs/swift/TestSwiftFileSystemDirectories.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-openstack/src/test/java/org/apache/hadoop/fs/swift/TestSwiftFileSystemDirectories.java b/hadoop-tools/hadoop-openstack/src/test/java/org/apache/hadoop/fs/swift/TestSwiftFileSystemDirectories.java
index 21fe918..9b4ba5e 100644
--- a/hadoop-tools/hadoop-openstack/src/test/java/org/apache/hadoop/fs/swift/TestSwiftFileSystemDirectories.java
+++ b/hadoop-tools/hadoop-openstack/src/test/java/org/apache/hadoop/fs/swift/TestSwiftFileSystemDirectories.java
@@ -87,7 +87,7 @@ public class TestSwiftFileSystemDirectories extends SwiftFileSystemBaseTest {
     assertEquals("Wrong number of elements in file status " + statusString, 1,
                  statuses.length);
     SwiftFileStatus stat = (SwiftFileStatus) statuses[0];
-    assertTrue("isDir(): Not a directory: " + stat, stat.isDir());
+    assertTrue("isDir(): Not a directory: " + stat, stat.isDirectory());
     extraStatusAssertions(stat);
   }
 
@@ -135,7 +135,7 @@ public class TestSwiftFileSystemDirectories extends SwiftFileSystemBaseTest {
     SwiftTestUtils.writeTextFile(fs, src, "testMultiByteFilesAreFiles", false);
     assertIsFile(src);
     FileStatus status = fs.getFileStatus(src);
-    assertFalse(status.isDir());
+    assertFalse(status.isDirectory());
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/645a8f2a/hadoop-tools/hadoop-openstack/src/test/java/org/apache/hadoop/fs/swift/TestSwiftFileSystemPartitionedUploads.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-openstack/src/test/java/org/apache/hadoop/fs/swift/TestSwiftFileSystemPartitionedUploads.java b/hadoop-tools/hadoop-openstack/src/test/java/org/apache/hadoop/fs/swift/TestSwiftFileSystemPartitionedUploads.java
index b42abcd..419d030 100644
--- a/hadoop-tools/hadoop-openstack/src/test/java/org/apache/hadoop/fs/swift/TestSwiftFileSystemPartitionedUploads.java
+++ b/hadoop-tools/hadoop-openstack/src/test/java/org/apache/hadoop/fs/swift/TestSwiftFileSystemPartitionedUploads.java
@@ -228,7 +228,7 @@ public class TestSwiftFileSystemPartitionedUploads extends
                  status.getLen());
     String fileInfo = qualifiedPath + "  " + status;
     assertFalse("File claims to be a directory " + fileInfo,
-                status.isDir());
+                status.isDirectory());
 
     FileStatus listedFileStat = resolveChild(parentDirListing, qualifiedPath);
     assertNotNull("Did not find " + path + " in " + parentDirLS,


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


[10/50] [abbrv] hadoop git commit: YARN-6741. Deleting all children of a Parent Queue on refresh throws exception. Contributed by Naganarasimha G R.

Posted by jh...@apache.org.
YARN-6741. Deleting all children of a Parent Queue on refresh throws exception. Contributed by Naganarasimha G R.


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

Branch: refs/heads/YARN-5734
Commit: d8f74c3964fa429a4a53c3651d175792cf00ac81
Parents: 7769e96
Author: bibinchundatt <bi...@apache.org>
Authored: Mon Aug 14 09:39:00 2017 +0530
Committer: bibinchundatt <bi...@apache.org>
Committed: Mon Aug 14 09:39:00 2017 +0530

----------------------------------------------------------------------
 .../capacity/CapacitySchedulerQueueManager.java |   4 +
 .../scheduler/capacity/ParentQueue.java         |  39 +++----
 .../capacity/TestCapacityScheduler.java         | 114 ++++++++++++++++++-
 3 files changed, 137 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8f74c39/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerQueueManager.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/CapacitySchedulerQueueManager.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/CapacitySchedulerQueueManager.java
index e33fbb3..1ceb6fb 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/CapacitySchedulerQueueManager.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/CapacitySchedulerQueueManager.java
@@ -327,6 +327,10 @@ public class CapacitySchedulerQueueManager implements SchedulerQueueManager<
                 + "it is not yet in stopped state. Current State : "
                 + oldQueue.getState());
           }
+        } else if (oldQueue instanceof ParentQueue
+            && newQueue instanceof LeafQueue) {
+          LOG.info("Converting the parent queue: " + oldQueue.getQueuePath()
+              + " to leaf queue.");
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8f74c39/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 f6ada4f..e0baa07 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
@@ -18,6 +18,14 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -34,7 +42,6 @@ import org.apache.hadoop.yarn.api.records.QueueInfo;
 import org.apache.hadoop.yarn.api.records.QueueState;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.security.AccessType;
@@ -45,7 +52,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerStat
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedContainerChangeRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivitiesLogger;
@@ -62,14 +68,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.Placeme
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.PlacementSetUtils;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-
 @Private
 @Evolving
 public class ParentQueue extends AbstractCSQueue {
@@ -315,18 +313,21 @@ public class ParentQueue extends AbstractCSQueue {
 
         // Check if the child-queue already exists
         if (childQueue != null) {
-          // Check if the child-queue has been converted into parent queue.
-          // The CS has already checked to ensure that this child-queue is in
-          // STOPPED state.
-          if (childQueue instanceof LeafQueue
-              && newChildQueue instanceof ParentQueue) {
-            // We would convert this LeafQueue to ParentQueue, consider this
-            // as the combination of DELETE then ADD.
+          // Check if the child-queue has been converted into parent queue or
+          // parent Queue has been converted to child queue. The CS has already
+          // checked to ensure that this child-queue is in STOPPED state if
+          // Child queue has been converted to ParentQueue.
+          if ((childQueue instanceof LeafQueue
+              && newChildQueue instanceof ParentQueue)
+              || (childQueue instanceof ParentQueue
+                  && newChildQueue instanceof LeafQueue)) {
+            // We would convert this LeafQueue to ParentQueue, or vice versa.
+            // consider this as the combination of DELETE then ADD.
             newChildQueue.setParent(this);
             currentChildQueues.put(newChildQueueName, newChildQueue);
             // inform CapacitySchedulerQueueManager
-            CapacitySchedulerQueueManager queueManager = this.csContext
-                .getCapacitySchedulerQueueManager();
+            CapacitySchedulerQueueManager queueManager =
+                this.csContext.getCapacitySchedulerQueueManager();
             queueManager.addQueue(newChildQueueName, newChildQueue);
             continue;
           }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d8f74c39/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.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/scheduler/capacity/TestCapacityScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
index 64e0df4..a526222 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -42,7 +43,6 @@ import java.util.Set;
 import java.util.concurrent.BrokenBarrierException;
 import java.util.concurrent.CyclicBarrier;
 
-import com.google.common.base.Supplier;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -167,6 +167,7 @@ import org.junit.Before;
 import org.junit.Test;
 import org.mockito.Mockito;
 
+import com.google.common.base.Supplier;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Sets;
@@ -667,6 +668,36 @@ public class TestCapacityScheduler {
 
   /**
    * @param conf, to be modified
+   * @return, CS configuration which has deleted all childred of queue(b)
+   *           root
+   *          /     \
+   *        a        b
+   *       / \
+   *      a1  a2
+   */
+  private CapacitySchedulerConfiguration setupQueueConfWithOutChildrenOfB(
+      CapacitySchedulerConfiguration conf) {
+
+    // Define top-level queues
+    conf.setQueues(CapacitySchedulerConfiguration.ROOT,
+        new String[] {"a","b"});
+
+    conf.setCapacity(A, A_CAPACITY);
+    conf.setCapacity(B, B_CAPACITY);
+
+    // Define 2nd-level queues
+    conf.setQueues(A, new String[] {"a1","a2"});
+    conf.setCapacity(A1, A1_CAPACITY);
+    conf.setUserLimitFactor(A1, 100.0f);
+    conf.setCapacity(A2, A2_CAPACITY);
+    conf.setUserLimitFactor(A2, 100.0f);
+
+    LOG.info("Setup top-level queues a and b (without children)");
+    return conf;
+  }
+
+  /**
+   * @param conf, to be modified
    * @return, CS configuration which has deleted a queue(b1)
    *           root
    *          /     \
@@ -4643,6 +4674,10 @@ public class TestCapacityScheduler {
     try {
       cs.reinitialize(conf, mockContext);
     } catch (IOException e) {
+      LOG.error(
+          "Expected to NOT throw exception when refresh queue tries to delete"
+              + " a queue WITHOUT running apps",
+          e);
       fail("Expected to NOT throw exception when refresh queue tries to delete"
           + " a queue WITHOUT running apps");
     }
@@ -4713,6 +4748,83 @@ public class TestCapacityScheduler {
   }
 
   /**
+   * Test for all child queue deletion and thus making parent queue a child.
+   * @throws Exception
+   */
+  @Test
+  public void testRefreshQueuesWithAllChildQueuesDeleted() throws Exception {
+    CapacityScheduler cs = new CapacityScheduler();
+    CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
+    RMContextImpl rmContext = new RMContextImpl(null, null, null, null, null,
+        null, new RMContainerTokenSecretManager(conf),
+        new NMTokenSecretManagerInRM(conf),
+        new ClientToAMTokenSecretManagerInRM(), null);
+    setupQueueConfiguration(conf);
+    cs.setConf(new YarnConfiguration());
+    cs.setRMContext(resourceManager.getRMContext());
+    cs.init(conf);
+    cs.start();
+    cs.reinitialize(conf, rmContext);
+    checkQueueCapacities(cs, A_CAPACITY, B_CAPACITY);
+
+    // test delete all leaf queues when there is no application running.
+    Map<String, CSQueue> queues =
+        cs.getCapacitySchedulerQueueManager().getQueues();
+
+    CSQueue bQueue = Mockito.spy((LeafQueue) queues.get("b1"));
+    when(bQueue.getState()).thenReturn(QueueState.RUNNING)
+        .thenReturn(QueueState.STOPPED);
+    queues.put("b1", bQueue);
+
+    bQueue = Mockito.spy((LeafQueue) queues.get("b2"));
+    when(bQueue.getState()).thenReturn(QueueState.STOPPED);
+    queues.put("b2", bQueue);
+
+    bQueue = Mockito.spy((LeafQueue) queues.get("b3"));
+    when(bQueue.getState()).thenReturn(QueueState.STOPPED);
+    queues.put("b3", bQueue);
+
+    conf = new CapacitySchedulerConfiguration();
+    setupQueueConfWithOutChildrenOfB(conf);
+
+    // test convert parent queue to leaf queue(root.b) when there is no
+    // application running.
+    try {
+      cs.reinitialize(conf, mockContext);
+      fail("Expected to throw exception when refresh queue tries to make parent"
+          + " queue a child queue when one of its children is still running.");
+    } catch (IOException e) {
+      //do not do anything, expected exception
+    }
+
+    // test delete leaf queues(root.b.b1,b2,b3) when there is no application
+    // running.
+    try {
+      cs.reinitialize(conf, mockContext);
+    } catch (IOException e) {
+      e.printStackTrace();
+      fail("Expected to NOT throw exception when refresh queue tries to delete"
+          + " all children of a parent queue(without running apps).");
+    }
+    CSQueue rootQueue = cs.getRootQueue();
+    CSQueue queueB = findQueue(rootQueue, B);
+    assertNotNull("Parent Queue B should not be deleted", queueB);
+    Assert.assertTrue("As Queue'B children are not deleted",
+        queueB instanceof LeafQueue);
+
+    String message =
+        "Refresh needs to support delete of all children of Parent queue.";
+    assertNull(message,
+        cs.getCapacitySchedulerQueueManager().getQueues().get("b3"));
+    assertNull(message,
+        cs.getCapacitySchedulerQueueManager().getQueues().get("b1"));
+    assertNull(message,
+        cs.getCapacitySchedulerQueueManager().getQueues().get("b2"));
+
+    cs.stop();
+  }
+
+  /**
    * Test if we can convert a leaf queue to a parent queue
    * @throws Exception
    */


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


[49/50] [abbrv] hadoop git commit: YARN-6322: Disable queue refresh when configuration mutation is enabled. Contributed by Jonathan Hung

Posted by jh...@apache.org.
YARN-6322: Disable queue refresh when configuration mutation is enabled. Contributed by Jonathan Hung


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

Branch: refs/heads/YARN-5734
Commit: 0fd690498cb66516d1e32928a71bb2e3e40d99e1
Parents: 279456e
Author: Xuan <xg...@apache.org>
Authored: Tue Aug 1 08:48:04 2017 -0700
Committer: Jonathan Hung <jh...@linkedin.com>
Committed: Thu Aug 17 13:50:11 2017 -0700

----------------------------------------------------------------------
 .../server/resourcemanager/AdminService.java    | 12 +++++++++-
 .../scheduler/MutableConfScheduler.java         | 12 ++++++++--
 .../scheduler/MutableConfigurationProvider.java |  4 +++-
 .../scheduler/capacity/CapacityScheduler.java   |  9 ++++++--
 .../conf/MutableCSConfigurationProvider.java    | 11 +++++-----
 .../resourcemanager/TestRMAdminService.java     | 23 ++++++++++++++++++++
 .../TestMutableCSConfigurationProvider.java     | 14 +++++++-----
 7 files changed, 67 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0fd69049/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
index 3457ae3..fd9e849 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
@@ -29,6 +29,7 @@ import java.util.Set;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.ha.HAServiceProtocol;
@@ -92,6 +93,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSyst
 import org.apache.hadoop.yarn.server.resourcemanager.resource.DynamicResourceConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeResourceUpdateEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.MutableConfScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.security.authorize.RMPolicyProvider;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -384,6 +387,12 @@ public class AdminService extends CompositeService implements
     RefreshQueuesResponse response =
         recordFactory.newRecordInstance(RefreshQueuesResponse.class);
     try {
+      ResourceScheduler scheduler = rm.getRMContext().getScheduler();
+      if (scheduler instanceof MutableConfScheduler
+          && ((MutableConfScheduler) scheduler).isConfigurationMutable()) {
+        throw new IOException("Scheduler configuration is mutable. " +
+            operation + " is not allowed in this scenario.");
+      }
       refreshQueues();
       RMAuditLogger.logSuccess(user.getShortUserName(), operation,
           "AdminService");
@@ -393,7 +402,8 @@ public class AdminService extends CompositeService implements
     }
   }
 
-  private void refreshQueues() throws IOException, YarnException {
+  @Private
+  public void refreshQueues() throws IOException, YarnException {
     rm.getRMContext().getScheduler().reinitialize(getConfig(),
         this.rm.getRMContext());
     // refresh the reservation system

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0fd69049/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/MutableConfScheduler.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/MutableConfScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/MutableConfScheduler.java
index 007dc29..313bf6a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/MutableConfScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/MutableConfScheduler.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.webapp.dao.SchedConfUpdateInfo;
 
 import java.io.IOException;
@@ -33,10 +34,11 @@ public interface MutableConfScheduler extends ResourceScheduler {
    * Update the scheduler's configuration.
    * @param user Caller of this update
    * @param confUpdate configuration update
-   * @throws IOException if update is invalid
+   * @throws IOException if scheduler could not be reinitialized
+   * @throws YarnException if reservation system could not be reinitialized
    */
   void updateConfiguration(UserGroupInformation user,
-      SchedConfUpdateInfo confUpdate) throws IOException;
+      SchedConfUpdateInfo confUpdate) throws IOException, YarnException;
 
   /**
    * Get the scheduler configuration.
@@ -50,4 +52,10 @@ public interface MutableConfScheduler extends ResourceScheduler {
    * @return the queue object
    */
   Queue getQueue(String queueName);
+
+  /**
+   * Return whether the scheduler configuration is mutable.
+   * @return whether scheduler configuration is mutable or not.
+   */
+  boolean isConfigurationMutable();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0fd69049/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/MutableConfigurationProvider.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/MutableConfigurationProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/MutableConfigurationProvider.java
index 1f13467..9baf1ad 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/MutableConfigurationProvider.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/MutableConfigurationProvider.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
 
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.webapp.dao.SchedConfUpdateInfo;
 
 import java.io.IOException;
@@ -39,8 +40,9 @@ public interface MutableConfigurationProvider {
    * @param user User issuing the request
    * @param confUpdate Key-value pairs for configurations to be updated.
    * @throws IOException if scheduler could not be reinitialized
+   * @throws YarnException if reservation system could not be reinitialized
    */
   void mutateConfiguration(UserGroupInformation user, SchedConfUpdateInfo
-      confUpdate) throws IOException;
+      confUpdate) throws IOException, YarnException;
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0fd69049/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.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/CapacityScheduler.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/CapacityScheduler.java
index 0d46483..1091fcd 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/CapacityScheduler.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/CapacityScheduler.java
@@ -2565,8 +2565,8 @@ public class CapacityScheduler extends
 
   @Override
   public void updateConfiguration(UserGroupInformation user,
-      SchedConfUpdateInfo confUpdate) throws IOException {
-    if (csConfProvider instanceof MutableConfigurationProvider) {
+      SchedConfUpdateInfo confUpdate) throws IOException, YarnException {
+    if (isConfigurationMutable()) {
       ((MutableConfigurationProvider) csConfProvider).mutateConfiguration(
           user, confUpdate);
     } else {
@@ -2574,4 +2574,9 @@ public class CapacityScheduler extends
           "provider does not support updating configuration.");
     }
   }
+
+  @Override
+  public boolean isConfigurationMutable() {
+    return csConfProvider instanceof MutableConfigurationProvider;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0fd69049/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/MutableCSConfigurationProvider.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/conf/MutableCSConfigurationProvider.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/conf/MutableCSConfigurationProvider.java
index 9ccc146..d03b2e2 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/conf/MutableCSConfigurationProvider.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/conf/MutableCSConfigurationProvider.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ConfigurationMutationACLPolicy;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ConfigurationMutationACLPolicyFactory;
@@ -58,7 +59,6 @@ public class MutableCSConfigurationProvider implements CSConfigurationProvider,
   private YarnConfigurationStore confStore;
   private ConfigurationMutationACLPolicy aclMutationPolicy;
   private RMContext rmContext;
-  private Configuration conf;
 
   public MutableCSConfigurationProvider(RMContext rmContext) {
     this.rmContext = rmContext;
@@ -96,7 +96,6 @@ public class MutableCSConfigurationProvider implements CSConfigurationProvider,
     this.aclMutationPolicy = ConfigurationMutationACLPolicyFactory
         .getPolicy(config);
     aclMutationPolicy.init(config, rmContext);
-    this.conf = config;
   }
 
   @Override
@@ -109,7 +108,7 @@ public class MutableCSConfigurationProvider implements CSConfigurationProvider,
 
   @Override
   public synchronized void mutateConfiguration(UserGroupInformation user,
-      SchedConfUpdateInfo confUpdate) throws IOException {
+      SchedConfUpdateInfo confUpdate) throws IOException, YarnException {
     if (!aclMutationPolicy.isMutationAllowed(user, confUpdate)) {
       throw new AccessControlException("User is not admin of all modified" +
           " queues.");
@@ -126,8 +125,8 @@ public class MutableCSConfigurationProvider implements CSConfigurationProvider,
       }
     }
     try {
-      rmContext.getScheduler().reinitialize(conf, rmContext);
-    } catch (IOException e) {
+      rmContext.getRMAdminService().refreshQueues();
+    } catch (IOException | YarnException e) {
       schedConf = oldConf;
       confStore.confirmMutation(id, false);
       throw e;
@@ -148,7 +147,7 @@ public class MutableCSConfigurationProvider implements CSConfigurationProvider,
         }
       }
       try {
-        rmContext.getScheduler().reinitialize(conf, rmContext);
+        rmContext.getScheduler().reinitialize(schedConf, rmContext);
       } catch (IOException e) {
         schedConf = oldConf;
         confStore.confirmMutation(mutation.getId(), false);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0fd69049/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.java
index 512c14a..0d409f0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.java
@@ -193,6 +193,29 @@ public class TestRMAdminService {
   }
 
   @Test
+  public void testAdminRefreshQueuesWithMutableSchedulerConfiguration() {
+    configuration.set(CapacitySchedulerConfiguration.CS_CONF_PROVIDER,
+        CapacitySchedulerConfiguration.STORE_CS_CONF_PROVIDER);
+
+    try {
+      rm = new MockRM(configuration);
+      rm.init(configuration);
+      rm.start();
+    } catch (Exception ex) {
+      fail("Should not get any exceptions");
+    }
+
+    try {
+      rm.adminService.refreshQueues(RefreshQueuesRequest.newInstance());
+      fail("Expected exception while calling refreshQueues when scheduler" +
+          " configuration is mutable.");
+    } catch (Exception ex) {
+      assertTrue(ex.getMessage().endsWith("Scheduler configuration is " +
+          "mutable. refreshQueues is not allowed in this scenario."));
+    }
+  }
+
+  @Test
   public void testAdminRefreshNodesWithoutConfiguration()
       throws IOException, YarnException {
     configuration.set(YarnConfiguration.RM_CONFIGURATION_PROVIDER_CLASS,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0fd69049/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/TestMutableCSConfigurationProvider.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/scheduler/capacity/conf/TestMutableCSConfigurationProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/TestMutableCSConfigurationProvider.java
index 9104f16..635a184 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/TestMutableCSConfigurationProvider.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/TestMutableCSConfigurationProvider.java
@@ -20,6 +20,8 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.conf;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.resourcemanager.AdminService;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
@@ -34,7 +36,6 @@ import java.util.Map;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
-import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.doNothing;
 import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.mock;
@@ -50,6 +51,7 @@ public class TestMutableCSConfigurationProvider {
   private SchedConfUpdateInfo goodUpdate;
   private SchedConfUpdateInfo badUpdate;
   private CapacityScheduler cs;
+  private AdminService adminService;
 
   private static final UserGroupInformation TEST_USER = UserGroupInformation
       .createUserForTesting("testUser", new String[] {});
@@ -61,6 +63,8 @@ public class TestMutableCSConfigurationProvider {
     when(rmContext.getScheduler()).thenReturn(cs);
     when(cs.getConfiguration()).thenReturn(
         new CapacitySchedulerConfiguration());
+    adminService = mock(AdminService.class);
+    when(rmContext.getRMAdminService()).thenReturn(adminService);
     confProvider = new MutableCSConfigurationProvider(rmContext);
     goodUpdate = new SchedConfUpdateInfo();
     Map<String, String> goodUpdateMap = new HashMap<>();
@@ -78,22 +82,20 @@ public class TestMutableCSConfigurationProvider {
   }
 
   @Test
-  public void testInMemoryBackedProvider() throws IOException {
+  public void testInMemoryBackedProvider() throws IOException, YarnException {
     Configuration conf = new Configuration();
     confProvider.init(conf);
     assertNull(confProvider.loadConfiguration(conf)
         .get("yarn.scheduler.capacity.root.a.goodKey"));
 
-    doNothing().when(cs).reinitialize(any(Configuration.class),
-        any(RMContext.class));
+    doNothing().when(adminService).refreshQueues();
     confProvider.mutateConfiguration(TEST_USER, goodUpdate);
     assertEquals("goodVal", confProvider.loadConfiguration(conf)
         .get("yarn.scheduler.capacity.root.a.goodKey"));
 
     assertNull(confProvider.loadConfiguration(conf).get(
         "yarn.scheduler.capacity.root.a.badKey"));
-    doThrow(new IOException()).when(cs).reinitialize(any(Configuration.class),
-        any(RMContext.class));
+    doThrow(new IOException()).when(adminService).refreshQueues();
     try {
       confProvider.mutateConfiguration(TEST_USER, badUpdate);
     } catch (IOException e) {


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


[47/50] [abbrv] hadoop git commit: YARN-5953 addendum: Move QueueConfigInfo and SchedConfUpdateInfo to package org.apache.hadoop.yarn.webapp.dao

Posted by jh...@apache.org.
YARN-5953 addendum: Move QueueConfigInfo and SchedConfUpdateInfo to package org.apache.hadoop.yarn.webapp.dao


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

Branch: refs/heads/YARN-5734
Commit: b44f573ffd255811885ac0590cc94e4d1f2efb9f
Parents: cb27fc4
Author: Xuan <xg...@apache.org>
Authored: Mon Jul 31 11:49:05 2017 -0700
Committer: Jonathan Hung <jh...@linkedin.com>
Committed: Thu Aug 17 13:50:10 2017 -0700

----------------------------------------------------------------------
 .../hadoop/yarn/webapp/dao/QueueConfigInfo.java | 57 +++++++++++++
 .../yarn/webapp/dao/SchedConfUpdateInfo.java    | 85 ++++++++++++++++++++
 .../webapp/dao/QueueConfigInfo.java             | 57 -------------
 .../webapp/dao/SchedConfUpdateInfo.java         | 85 --------------------
 4 files changed, 142 insertions(+), 142 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b44f573f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/dao/QueueConfigInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/dao/QueueConfigInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/dao/QueueConfigInfo.java
new file mode 100644
index 0000000..d1d91c2
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/dao/QueueConfigInfo.java
@@ -0,0 +1,57 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.webapp.dao;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+
+/**
+ * Information for adding or updating a queue to scheduler configuration
+ * for this queue.
+ */
+@XmlRootElement
+@XmlAccessorType(XmlAccessType.FIELD)
+public class QueueConfigInfo {
+
+  @XmlElement(name = "queueName")
+  private String queue;
+
+  private HashMap<String, String> params = new HashMap<>();
+
+  public QueueConfigInfo() { }
+
+  public QueueConfigInfo(String queue, Map<String, String> params) {
+    this.queue = queue;
+    this.params = new HashMap<>(params);
+  }
+
+  public String getQueue() {
+    return this.queue;
+  }
+
+  public HashMap<String, String> getParams() {
+    return this.params;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b44f573f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/dao/SchedConfUpdateInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/dao/SchedConfUpdateInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/dao/SchedConfUpdateInfo.java
new file mode 100644
index 0000000..bb84096
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/dao/SchedConfUpdateInfo.java
@@ -0,0 +1,85 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.webapp.dao;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlElementWrapper;
+import javax.xml.bind.annotation.XmlRootElement;
+
+/**
+ * Information for making scheduler configuration changes (supports adding,
+ * removing, or updating a queue, as well as global scheduler conf changes).
+ */
+@XmlRootElement(name = "schedConf")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class SchedConfUpdateInfo {
+
+  @XmlElement(name = "add-queue")
+  private ArrayList<QueueConfigInfo> addQueueInfo = new ArrayList<>();
+
+  @XmlElement(name = "remove-queue")
+  private ArrayList<String> removeQueueInfo = new ArrayList<>();
+
+  @XmlElement(name = "update-queue")
+  private ArrayList<QueueConfigInfo> updateQueueInfo = new ArrayList<>();
+
+  private HashMap<String, String> global = new HashMap<>();
+
+  public SchedConfUpdateInfo() {
+    // JAXB needs this
+  }
+
+  public ArrayList<QueueConfigInfo> getAddQueueInfo() {
+    return addQueueInfo;
+  }
+
+  public void setAddQueueInfo(ArrayList<QueueConfigInfo> addQueueInfo) {
+    this.addQueueInfo = addQueueInfo;
+  }
+
+  public ArrayList<String> getRemoveQueueInfo() {
+    return removeQueueInfo;
+  }
+
+  public void setRemoveQueueInfo(ArrayList<String> removeQueueInfo) {
+    this.removeQueueInfo = removeQueueInfo;
+  }
+
+  public ArrayList<QueueConfigInfo> getUpdateQueueInfo() {
+    return updateQueueInfo;
+  }
+
+  public void setUpdateQueueInfo(ArrayList<QueueConfigInfo> updateQueueInfo) {
+    this.updateQueueInfo = updateQueueInfo;
+  }
+
+  @XmlElementWrapper(name = "global-updates")
+  public HashMap<String, String> getGlobalParams() {
+    return global;
+  }
+
+  public void setGlobalParams(HashMap<String, String> globalInfo) {
+    this.global = globalInfo;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b44f573f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/QueueConfigInfo.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/QueueConfigInfo.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/QueueConfigInfo.java
deleted file mode 100644
index d1d91c2..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/QueueConfigInfo.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.webapp.dao;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import javax.xml.bind.annotation.XmlAccessType;
-import javax.xml.bind.annotation.XmlAccessorType;
-import javax.xml.bind.annotation.XmlElement;
-import javax.xml.bind.annotation.XmlRootElement;
-
-/**
- * Information for adding or updating a queue to scheduler configuration
- * for this queue.
- */
-@XmlRootElement
-@XmlAccessorType(XmlAccessType.FIELD)
-public class QueueConfigInfo {
-
-  @XmlElement(name = "queueName")
-  private String queue;
-
-  private HashMap<String, String> params = new HashMap<>();
-
-  public QueueConfigInfo() { }
-
-  public QueueConfigInfo(String queue, Map<String, String> params) {
-    this.queue = queue;
-    this.params = new HashMap<>(params);
-  }
-
-  public String getQueue() {
-    return this.queue;
-  }
-
-  public HashMap<String, String> getParams() {
-    return this.params;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b44f573f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/SchedConfUpdateInfo.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/SchedConfUpdateInfo.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/SchedConfUpdateInfo.java
deleted file mode 100644
index bb84096..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/SchedConfUpdateInfo.java
+++ /dev/null
@@ -1,85 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.webapp.dao;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-
-import javax.xml.bind.annotation.XmlAccessType;
-import javax.xml.bind.annotation.XmlAccessorType;
-import javax.xml.bind.annotation.XmlElement;
-import javax.xml.bind.annotation.XmlElementWrapper;
-import javax.xml.bind.annotation.XmlRootElement;
-
-/**
- * Information for making scheduler configuration changes (supports adding,
- * removing, or updating a queue, as well as global scheduler conf changes).
- */
-@XmlRootElement(name = "schedConf")
-@XmlAccessorType(XmlAccessType.FIELD)
-public class SchedConfUpdateInfo {
-
-  @XmlElement(name = "add-queue")
-  private ArrayList<QueueConfigInfo> addQueueInfo = new ArrayList<>();
-
-  @XmlElement(name = "remove-queue")
-  private ArrayList<String> removeQueueInfo = new ArrayList<>();
-
-  @XmlElement(name = "update-queue")
-  private ArrayList<QueueConfigInfo> updateQueueInfo = new ArrayList<>();
-
-  private HashMap<String, String> global = new HashMap<>();
-
-  public SchedConfUpdateInfo() {
-    // JAXB needs this
-  }
-
-  public ArrayList<QueueConfigInfo> getAddQueueInfo() {
-    return addQueueInfo;
-  }
-
-  public void setAddQueueInfo(ArrayList<QueueConfigInfo> addQueueInfo) {
-    this.addQueueInfo = addQueueInfo;
-  }
-
-  public ArrayList<String> getRemoveQueueInfo() {
-    return removeQueueInfo;
-  }
-
-  public void setRemoveQueueInfo(ArrayList<String> removeQueueInfo) {
-    this.removeQueueInfo = removeQueueInfo;
-  }
-
-  public ArrayList<QueueConfigInfo> getUpdateQueueInfo() {
-    return updateQueueInfo;
-  }
-
-  public void setUpdateQueueInfo(ArrayList<QueueConfigInfo> updateQueueInfo) {
-    this.updateQueueInfo = updateQueueInfo;
-  }
-
-  @XmlElementWrapper(name = "global-updates")
-  public HashMap<String, String> getGlobalParams() {
-    return global;
-  }
-
-  public void setGlobalParams(HashMap<String, String> globalInfo) {
-    this.global = globalInfo;
-  }
-}


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


[48/50] [abbrv] hadoop git commit: YARN-5953:Create CLI for changing YARN configurations. (Jonathan Hung via xgong)

Posted by jh...@apache.org.
YARN-5953:Create CLI for changing YARN configurations. (Jonathan Hung via xgong)


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

Branch: refs/heads/YARN-5734
Commit: cb27fc4a18157cc9e0ffcbf6fe062e90b7bd4536
Parents: 7cf82f3
Author: Xuan <xg...@apache.org>
Authored: Fri Jul 7 14:16:46 2017 -0700
Committer: Jonathan Hung <jh...@linkedin.com>
Committed: Thu Aug 17 13:50:10 2017 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/hadoop-yarn/bin/yarn        |   4 +
 hadoop-yarn-project/hadoop-yarn/bin/yarn.cmd    |   5 +
 .../hadoop/yarn/client/cli/SchedConfCLI.java    | 238 +++++++++++++++++++
 .../yarn/client/cli/TestSchedConfCLI.java       | 160 +++++++++++++
 .../hadoop/yarn/webapp/dao/package-info.java    |  27 +++
 .../yarn/webapp/util/YarnWebServiceUtils.java   |  14 ++
 .../ConfigurationMutationACLPolicy.java         |   2 +-
 .../DefaultConfigurationMutationACLPolicy.java  |   2 +-
 .../scheduler/MutableConfScheduler.java         |   2 +-
 .../scheduler/MutableConfigurationProvider.java |   2 +-
 .../scheduler/capacity/CapacityScheduler.java   |   2 +-
 .../conf/MutableCSConfigurationProvider.java    |   4 +-
 ...ueueAdminConfigurationMutationACLPolicy.java |   4 +-
 .../resourcemanager/webapp/RMWebServices.java   |   1 +
 .../webapp/dao/QueueConfigInfo.java             |   4 +-
 .../webapp/dao/SchedConfUpdateInfo.java         |  18 +-
 .../TestConfigurationMutationACLPolicies.java   |   4 +-
 .../TestMutableCSConfigurationProvider.java     |   4 +-
 .../TestRMWebServicesConfigurationMutation.java |  65 +++--
 19 files changed, 508 insertions(+), 54 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb27fc4a/hadoop-yarn-project/hadoop-yarn/bin/yarn
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/bin/yarn b/hadoop-yarn-project/hadoop-yarn/bin/yarn
index dcde0dc..4aa0471 100755
--- a/hadoop-yarn-project/hadoop-yarn/bin/yarn
+++ b/hadoop-yarn-project/hadoop-yarn/bin/yarn
@@ -47,6 +47,7 @@ function hadoop_usage
   hadoop_add_subcommand "resourcemanager" daemon "run the ResourceManager"
   hadoop_add_subcommand "rmadmin" admin "admin tools"
   hadoop_add_subcommand "router" daemon "run the Router daemon"
+  hadoop_add_subcommand "schedconf" "modify scheduler configuration"
   hadoop_add_subcommand "scmadmin" admin "SharedCacheManager admin tools"
   hadoop_add_subcommand "sharedcachemanager" admin "run the SharedCacheManager daemon"
   hadoop_add_subcommand "timelinereader" client "run the timeline reader server"
@@ -142,6 +143,9 @@ function yarncmd_case
       HADOOP_SUBCMD_SUPPORTDAEMONIZATION="true"
       HADOOP_CLASSNAME='org.apache.hadoop.yarn.server.router.Router'
     ;;
+    schedconf)
+      HADOOP_CLASSNAME='org.apache.hadoop.yarn.client.cli.SchedConfCLI'
+    ;;
     scmadmin)
       HADOOP_CLASSNAME='org.apache.hadoop.yarn.client.SCMAdmin'
     ;;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb27fc4a/hadoop-yarn-project/hadoop-yarn/bin/yarn.cmd
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/bin/yarn.cmd b/hadoop-yarn-project/hadoop-yarn/bin/yarn.cmd
index 690badf..7ec9848 100644
--- a/hadoop-yarn-project/hadoop-yarn/bin/yarn.cmd
+++ b/hadoop-yarn-project/hadoop-yarn/bin/yarn.cmd
@@ -295,6 +295,11 @@ goto :eof
   set YARN_OPTS=%YARN_OPTS% %YARN_CLIENT_OPTS%
   goto :eof
 
+:schedconf
+  set CLASS=org.apache.hadoop.yarn.client.cli.SchedConfCLI
+  set YARN_OPTS=%YARN_OPTS% %YARN_CLIENT_OPTS%
+  goto :eof
+
 @rem This changes %1, %2 etc. Hence those cannot be used after calling this.
 :make_command_arguments
   if "%1" == "--config" (

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb27fc4a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/SchedConfCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/SchedConfCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/SchedConfCLI.java
new file mode 100644
index 0000000..e17062e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/SchedConfCLI.java
@@ -0,0 +1,238 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY 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 com.google.common.annotations.VisibleForTesting;
+import com.sun.jersey.api.client.Client;
+import com.sun.jersey.api.client.ClientResponse;
+import com.sun.jersey.api.client.WebResource;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.GnuParser;
+import org.apache.commons.cli.MissingArgumentException;
+import org.apache.commons.cli.Options;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.webapp.dao.QueueConfigInfo;
+import org.apache.hadoop.yarn.webapp.dao.SchedConfUpdateInfo;
+import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
+import org.apache.hadoop.yarn.webapp.util.YarnWebServiceUtils;
+
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response.Status;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * CLI for modifying scheduler configuration.
+ */
+@Public
+@Evolving
+public class SchedConfCLI extends Configured implements Tool {
+
+  private static final String ADD_QUEUES_OPTION = "addQueues";
+  private static final String REMOVE_QUEUES_OPTION = "removeQueues";
+  private static final String UPDATE_QUEUES_OPTION = "updateQueues";
+  private static final String GLOBAL_OPTIONS = "globalUpdates";
+  private static final String HELP_CMD = "help";
+
+  private static final String CONF_ERR_MSG = "Specify configuration key " +
+      "value as confKey=confVal.";
+
+  public SchedConfCLI() {
+    super(new YarnConfiguration());
+  }
+
+  public static void main(String[] args) throws Exception {
+    SchedConfCLI cli = new SchedConfCLI();
+    int exitCode = cli.run(args);
+    System.exit(exitCode);
+  }
+
+  @Override
+  public int run(String[] args) throws Exception {
+    Options opts = new Options();
+
+    opts.addOption("add", ADD_QUEUES_OPTION, true,
+        "Add queues with configurations");
+    opts.addOption("remove", REMOVE_QUEUES_OPTION, true,
+        "Remove queues");
+    opts.addOption("update", UPDATE_QUEUES_OPTION, true,
+        "Update queue configurations");
+    opts.addOption("global", GLOBAL_OPTIONS, true,
+        "Update global scheduler configurations");
+    opts.addOption("h", HELP_CMD, false, "Displays help for all commands.");
+
+    int exitCode = -1;
+    CommandLine parsedCli = null;
+    try {
+      parsedCli = new GnuParser().parse(opts, args);
+    } catch (MissingArgumentException ex) {
+      System.err.println("Missing argument for options");
+      printUsage();
+      return exitCode;
+    }
+
+    if (parsedCli.hasOption(HELP_CMD)) {
+      printUsage();
+      return 0;
+    }
+
+    boolean hasOption = false;
+    SchedConfUpdateInfo updateInfo = new SchedConfUpdateInfo();
+    try {
+      if (parsedCli.hasOption(ADD_QUEUES_OPTION)) {
+        hasOption = true;
+        addQueues(parsedCli.getOptionValue(ADD_QUEUES_OPTION), updateInfo);
+      }
+      if (parsedCli.hasOption(REMOVE_QUEUES_OPTION)) {
+        hasOption = true;
+        removeQueues(parsedCli.getOptionValue(REMOVE_QUEUES_OPTION),
+            updateInfo);
+      }
+      if (parsedCli.hasOption(UPDATE_QUEUES_OPTION)) {
+        hasOption = true;
+        updateQueues(parsedCli.getOptionValue(UPDATE_QUEUES_OPTION),
+            updateInfo);
+      }
+      if (parsedCli.hasOption(GLOBAL_OPTIONS)) {
+        hasOption = true;
+        globalUpdates(parsedCli.getOptionValue(GLOBAL_OPTIONS), updateInfo);
+      }
+    } catch (IllegalArgumentException e) {
+      System.err.println(e.getMessage());
+      return -1;
+    }
+
+    if (!hasOption) {
+      System.err.println("Invalid Command Usage: ");
+      printUsage();
+      return -1;
+    }
+
+    Client webServiceClient = Client.create();
+    WebResource webResource = webServiceClient.resource(WebAppUtils.
+        getRMWebAppURLWithScheme(getConf()));
+    ClientResponse response = webResource.path("ws").path("v1").path("cluster")
+        .path("sched-conf").accept(MediaType.APPLICATION_JSON)
+        .entity(YarnWebServiceUtils.toJson(updateInfo,
+            SchedConfUpdateInfo.class), MediaType.APPLICATION_JSON)
+        .put(ClientResponse.class);
+    if (response != null) {
+      if (response.getStatus() == Status.OK.getStatusCode()) {
+        System.out.println("Configuration changed successfully.");
+        return 0;
+      } else {
+        System.err.println("Configuration change unsuccessful: "
+            + response.getEntity(String.class));
+      }
+    } else {
+      System.err.println("Configuration change unsuccessful: null response");
+    }
+    return -1;
+  }
+
+  @VisibleForTesting
+  void addQueues(String args, SchedConfUpdateInfo updateInfo) {
+    if (args == null) {
+      return;
+    }
+    ArrayList<QueueConfigInfo> queueConfigInfos = new ArrayList<>();
+    for (String arg : args.split(";")) {
+      queueConfigInfos.add(getQueueConfigInfo(arg));
+    }
+    updateInfo.setAddQueueInfo(queueConfigInfos);
+  }
+
+  @VisibleForTesting
+  void removeQueues(String args, SchedConfUpdateInfo updateInfo) {
+    if (args == null) {
+      return;
+    }
+    List<String> queuesToRemove = Arrays.asList(args.split(","));
+    updateInfo.setRemoveQueueInfo(new ArrayList<>(queuesToRemove));
+  }
+
+  @VisibleForTesting
+  void updateQueues(String args, SchedConfUpdateInfo updateInfo) {
+    if (args == null) {
+      return;
+    }
+    ArrayList<QueueConfigInfo> queueConfigInfos = new ArrayList<>();
+    for (String arg : args.split(";")) {
+      queueConfigInfos.add(getQueueConfigInfo(arg));
+    }
+    updateInfo.setUpdateQueueInfo(queueConfigInfos);
+  }
+
+  @VisibleForTesting
+  void globalUpdates(String args, SchedConfUpdateInfo updateInfo) {
+    if (args == null) {
+      return;
+    }
+    HashMap<String, String> globalUpdates = new HashMap<>();
+    for (String globalUpdate : args.split(",")) {
+      putKeyValuePair(globalUpdates, globalUpdate);
+    }
+    updateInfo.setGlobalParams(globalUpdates);
+  }
+
+  private QueueConfigInfo getQueueConfigInfo(String arg) {
+    String[] queueArgs = arg.split(",");
+    String queuePath = queueArgs[0];
+    Map<String, String> queueConfigs = new HashMap<>();
+    for (int i = 1; i < queueArgs.length; ++i) {
+      putKeyValuePair(queueConfigs, queueArgs[i]);
+    }
+    return new QueueConfigInfo(queuePath, queueConfigs);
+  }
+
+  private void putKeyValuePair(Map<String, String> kv, String args) {
+    String[] argParts = args.split("=");
+    if (argParts.length == 1) {
+      if (argParts[0].isEmpty() || !args.contains("=")) {
+        throw new IllegalArgumentException(CONF_ERR_MSG);
+      } else {
+        // key specified, but no value e.g. "confKey="
+        kv.put(argParts[0], null);
+      }
+    } else if (argParts.length > 2) {
+      throw new IllegalArgumentException(CONF_ERR_MSG);
+    } else {
+      if (argParts[0].isEmpty()) {
+        throw new IllegalArgumentException(CONF_ERR_MSG);
+      }
+      kv.put(argParts[0], argParts[1]);
+    }
+  }
+
+  private void printUsage() {
+    System.out.println("yarn schedconf [-add queueAddPath1,confKey1=confVal1,"
+        + "confKey2=confVal2;queueAddPath2,confKey3=confVal3] "
+        + "[-remove queueRemovePath1,queueRemovePath2] "
+        + "[-update queueUpdatePath1,confKey1=confVal1] "
+        + "[-global globalConfKey1=globalConfVal1,"
+        + "globalConfKey2=globalConfVal2]");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb27fc4a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestSchedConfCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestSchedConfCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestSchedConfCLI.java
new file mode 100644
index 0000000..d2f0639
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestSchedConfCLI.java
@@ -0,0 +1,160 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY 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 org.junit.Before;
+import org.junit.Test;
+
+import java.io.ByteArrayOutputStream;
+import java.io.PrintStream;
+import java.util.List;
+import java.util.Map;
+import org.apache.hadoop.yarn.webapp.dao.QueueConfigInfo;
+import org.apache.hadoop.yarn.webapp.dao.SchedConfUpdateInfo;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Class for testing {@link SchedConfCLI}.
+ */
+public class TestSchedConfCLI {
+
+  private ByteArrayOutputStream sysOutStream;
+  private PrintStream sysOut;
+
+  private ByteArrayOutputStream sysErrStream;
+  private PrintStream sysErr;
+
+  private SchedConfCLI cli;
+
+  @Before
+  public void setUp() {
+    sysOutStream = new ByteArrayOutputStream();
+    sysOut =  new PrintStream(sysOutStream);
+    System.setOut(sysOut);
+
+    sysErrStream = new ByteArrayOutputStream();
+    sysErr = new PrintStream(sysErrStream);
+    System.setErr(sysErr);
+
+    cli = new SchedConfCLI();
+  }
+
+  @Test(timeout = 10000)
+  public void testInvalidConf() throws Exception {
+    // conf pair with no key should be invalid
+    int exitCode = cli.run(new String[] {"-add", "root.a,=confVal"});
+    assertTrue("Should return an error code", exitCode != 0);
+    assertTrue(sysErrStream.toString().contains("Specify configuration key " +
+        "value as confKey=confVal."));
+    exitCode = cli.run(new String[] {"-update", "root.a,=confVal"});
+    assertTrue("Should return an error code", exitCode != 0);
+    assertTrue(sysErrStream.toString().contains("Specify configuration key " +
+        "value as confKey=confVal."));
+
+    exitCode = cli.run(new String[] {"-add", "root.a,confKey=confVal=conf"});
+    assertTrue("Should return an error code", exitCode != 0);
+    assertTrue(sysErrStream.toString().contains("Specify configuration key " +
+        "value as confKey=confVal."));
+    exitCode = cli.run(new String[] {"-update", "root.a,confKey=confVal=c"});
+    assertTrue("Should return an error code", exitCode != 0);
+    assertTrue(sysErrStream.toString().contains("Specify configuration key " +
+        "value as confKey=confVal."));
+  }
+
+  @Test(timeout = 10000)
+  public void testAddQueues() {
+    SchedConfUpdateInfo schedUpdateInfo = new SchedConfUpdateInfo();
+    cli.addQueues("root.a,a1=aVal1,a2=aVal2," +
+        "a3=", schedUpdateInfo);
+    QueueConfigInfo addInfo = schedUpdateInfo.getAddQueueInfo().get(0);
+    assertEquals("root.a", addInfo.getQueue());
+    Map<String, String> params = addInfo.getParams();
+    assertEquals(3, params.size());
+    assertEquals("aVal1", params.get("a1"));
+    assertEquals("aVal2", params.get("a2"));
+    assertNull(params.get("a3"));
+
+    schedUpdateInfo = new SchedConfUpdateInfo();
+    cli.addQueues("root.b,b1=bVal1;root.c,c1=cVal1", schedUpdateInfo);
+    assertEquals(2, schedUpdateInfo.getAddQueueInfo().size());
+    QueueConfigInfo bAddInfo = schedUpdateInfo.getAddQueueInfo().get(0);
+    assertEquals("root.b", bAddInfo.getQueue());
+    Map<String, String> bParams = bAddInfo.getParams();
+    assertEquals(1, bParams.size());
+    assertEquals("bVal1", bParams.get("b1"));
+    QueueConfigInfo cAddInfo = schedUpdateInfo.getAddQueueInfo().get(1);
+    assertEquals("root.c", cAddInfo.getQueue());
+    Map<String, String> cParams = cAddInfo.getParams();
+    assertEquals(1, cParams.size());
+    assertEquals("cVal1", cParams.get("c1"));
+  }
+
+  @Test(timeout = 10000)
+  public void testRemoveQueues() {
+    SchedConfUpdateInfo schedUpdateInfo = new SchedConfUpdateInfo();
+    cli.removeQueues("root.a,root.b,root.c.c1", schedUpdateInfo);
+    List<String> removeInfo = schedUpdateInfo.getRemoveQueueInfo();
+    assertEquals(3, removeInfo.size());
+    assertEquals("root.a", removeInfo.get(0));
+    assertEquals("root.b", removeInfo.get(1));
+    assertEquals("root.c.c1", removeInfo.get(2));
+  }
+
+  @Test(timeout = 10000)
+  public void testUpdateQueues() {
+    SchedConfUpdateInfo schedUpdateInfo = new SchedConfUpdateInfo();
+    cli.updateQueues("root.a,a1=aVal1,a2=aVal2," +
+        "a3=", schedUpdateInfo);
+    QueueConfigInfo updateInfo = schedUpdateInfo.getUpdateQueueInfo().get(0);
+    assertEquals("root.a", updateInfo.getQueue());
+    Map<String, String> params = updateInfo.getParams();
+    assertEquals(3, params.size());
+    assertEquals("aVal1", params.get("a1"));
+    assertEquals("aVal2", params.get("a2"));
+    assertNull(params.get("a3"));
+
+    schedUpdateInfo = new SchedConfUpdateInfo();
+    cli.updateQueues("root.b,b1=bVal1;root.c,c1=cVal1", schedUpdateInfo);
+    assertEquals(2, schedUpdateInfo.getUpdateQueueInfo().size());
+    QueueConfigInfo bUpdateInfo = schedUpdateInfo.getUpdateQueueInfo().get(0);
+    assertEquals("root.b", bUpdateInfo.getQueue());
+    Map<String, String> bParams = bUpdateInfo.getParams();
+    assertEquals(1, bParams.size());
+    assertEquals("bVal1", bParams.get("b1"));
+    QueueConfigInfo cUpdateInfo = schedUpdateInfo.getUpdateQueueInfo().get(1);
+    assertEquals("root.c", cUpdateInfo.getQueue());
+    Map<String, String> cParams = cUpdateInfo.getParams();
+    assertEquals(1, cParams.size());
+    assertEquals("cVal1", cParams.get("c1"));
+  }
+
+  @Test(timeout = 10000)
+  public void testGlobalUpdate() {
+    SchedConfUpdateInfo schedUpdateInfo = new SchedConfUpdateInfo();
+    cli.globalUpdates("schedKey1=schedVal1,schedKey2=schedVal2",
+        schedUpdateInfo);
+    Map<String, String> globalInfo = schedUpdateInfo.getGlobalParams();
+    assertEquals(2, globalInfo.size());
+    assertEquals("schedVal1", globalInfo.get("schedKey1"));
+    assertEquals("schedVal2", globalInfo.get("schedKey2"));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb27fc4a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/dao/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/dao/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/dao/package-info.java
new file mode 100644
index 0000000..aec6762
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/dao/package-info.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.
+ */
+
+/**
+ * Data structures for scheduler configuration mutation info.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.yarn.webapp.dao;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb27fc4a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/util/YarnWebServiceUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/util/YarnWebServiceUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/util/YarnWebServiceUtils.java
index 4167e21..1cf1e97 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/util/YarnWebServiceUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/util/YarnWebServiceUtils.java
@@ -23,9 +23,14 @@ import com.sun.jersey.api.client.ClientResponse;
 import com.sun.jersey.api.client.UniformInterfaceException;
 import com.sun.jersey.api.client.WebResource;
 import javax.ws.rs.core.MediaType;
+
+import com.sun.jersey.api.json.JSONJAXBContext;
+import com.sun.jersey.api.json.JSONMarshaller;
 import org.apache.hadoop.conf.Configuration;
 import org.codehaus.jettison.json.JSONObject;
 
+import java.io.StringWriter;
+
 /**
  * This class contains several utility function which could be used to generate
  * Restful calls to RM/NM/AHS.
@@ -59,4 +64,13 @@ public final class YarnWebServiceUtils {
         .get(ClientResponse.class);
     return response.getEntity(JSONObject.class);
   }
+
+  @SuppressWarnings("rawtypes")
+  public static String toJson(Object nsli, Class klass) throws Exception {
+    StringWriter sw = new StringWriter();
+    JSONJAXBContext ctx = new JSONJAXBContext(klass);
+    JSONMarshaller jm = ctx.createJSONMarshaller();
+    jm.marshallToJSON(nsli, sw);
+    return sw.toString();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb27fc4a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ConfigurationMutationACLPolicy.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/ConfigurationMutationACLPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ConfigurationMutationACLPolicy.java
index 3a388fe..5bc5874 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ConfigurationMutationACLPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ConfigurationMutationACLPolicy.java
@@ -21,7 +21,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.SchedConfUpdateInfo;
+import org.apache.hadoop.yarn.webapp.dao.SchedConfUpdateInfo;
 
 /**
  * Interface for determining whether configuration mutations are allowed.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb27fc4a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/DefaultConfigurationMutationACLPolicy.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/DefaultConfigurationMutationACLPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/DefaultConfigurationMutationACLPolicy.java
index 6648668..1de6f6b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/DefaultConfigurationMutationACLPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/DefaultConfigurationMutationACLPolicy.java
@@ -22,7 +22,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.security.YarnAuthorizationProvider;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.SchedConfUpdateInfo;
+import org.apache.hadoop.yarn.webapp.dao.SchedConfUpdateInfo;
 
 /**
  * Default configuration mutation ACL policy. Checks if user is YARN admin.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb27fc4a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/MutableConfScheduler.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/MutableConfScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/MutableConfScheduler.java
index 027d944..007dc29 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/MutableConfScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/MutableConfScheduler.java
@@ -19,7 +19,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.SchedConfUpdateInfo;
+import org.apache.hadoop.yarn.webapp.dao.SchedConfUpdateInfo;
 
 import java.io.IOException;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb27fc4a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/MutableConfigurationProvider.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/MutableConfigurationProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/MutableConfigurationProvider.java
index 6b8306c..86be7c3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/MutableConfigurationProvider.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/MutableConfigurationProvider.java
@@ -19,7 +19,7 @@
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
 
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.SchedConfUpdateInfo;
+import org.apache.hadoop.yarn.webapp.dao.SchedConfUpdateInfo;
 
 import java.io.IOException;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb27fc4a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.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/CapacityScheduler.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/CapacityScheduler.java
index ebd3a49..fb5bb83 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/CapacityScheduler.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/CapacityScheduler.java
@@ -135,11 +135,11 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.Placeme
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.SimplePlacementSet;
 import org.apache.hadoop.yarn.server.resourcemanager.security.AppPriorityACLsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.SchedConfUpdateInfo;
 import org.apache.hadoop.yarn.server.utils.Lock;
 import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
+import org.apache.hadoop.yarn.webapp.dao.SchedConfUpdateInfo;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb27fc4a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/MutableCSConfigurationProvider.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/conf/MutableCSConfigurationProvider.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/conf/MutableCSConfigurationProvider.java
index eb97260..670c0f9 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/conf/MutableCSConfigurationProvider.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/conf/MutableCSConfigurationProvider.java
@@ -31,8 +31,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.conf.YarnConfigurationStore.LogMutation;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.QueueConfigInfo;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.SchedConfUpdateInfo;
+import org.apache.hadoop.yarn.webapp.dao.QueueConfigInfo;
+import org.apache.hadoop.yarn.webapp.dao.SchedConfUpdateInfo;
 
 import java.io.IOException;
 import java.util.ArrayList;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb27fc4a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/QueueAdminConfigurationMutationACLPolicy.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/conf/QueueAdminConfigurationMutationACLPolicy.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/conf/QueueAdminConfigurationMutationACLPolicy.java
index 0a82d50..ee53fd1 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/conf/QueueAdminConfigurationMutationACLPolicy.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/conf/QueueAdminConfigurationMutationACLPolicy.java
@@ -27,8 +27,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ConfigurationMutationACLPolicy;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.MutableConfScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.QueueConfigInfo;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.SchedConfUpdateInfo;
+import org.apache.hadoop.yarn.webapp.dao.QueueConfigInfo;
+import org.apache.hadoop.yarn.webapp.dao.SchedConfUpdateInfo;
 
 import java.io.IOException;
 import java.util.HashSet;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb27fc4a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
index 358b57b..516b6293 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java
@@ -192,6 +192,7 @@ import org.apache.hadoop.yarn.webapp.BadRequestException;
 import org.apache.hadoop.yarn.webapp.ForbiddenException;
 import org.apache.hadoop.yarn.webapp.NotFoundException;
 import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
+import org.apache.hadoop.yarn.webapp.dao.SchedConfUpdateInfo;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.inject.Inject;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb27fc4a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/QueueConfigInfo.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/QueueConfigInfo.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/QueueConfigInfo.java
index b20eda6..d1d91c2 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/QueueConfigInfo.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/QueueConfigInfo.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.yarn.server.resourcemanager.webapp.dao;
+package org.apache.hadoop.yarn.webapp.dao;
 
 import java.util.HashMap;
 import java.util.Map;
@@ -54,4 +54,4 @@ public class QueueConfigInfo {
     return this.params;
   }
 
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb27fc4a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/SchedConfUpdateInfo.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/SchedConfUpdateInfo.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/SchedConfUpdateInfo.java
index b7c585e..bb84096 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/SchedConfUpdateInfo.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/SchedConfUpdateInfo.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.yarn.server.resourcemanager.webapp.dao;
+package org.apache.hadoop.yarn.webapp.dao;
 
 import java.util.ArrayList;
 import java.util.HashMap;
@@ -54,16 +54,32 @@ public class SchedConfUpdateInfo {
     return addQueueInfo;
   }
 
+  public void setAddQueueInfo(ArrayList<QueueConfigInfo> addQueueInfo) {
+    this.addQueueInfo = addQueueInfo;
+  }
+
   public ArrayList<String> getRemoveQueueInfo() {
     return removeQueueInfo;
   }
 
+  public void setRemoveQueueInfo(ArrayList<String> removeQueueInfo) {
+    this.removeQueueInfo = removeQueueInfo;
+  }
+
   public ArrayList<QueueConfigInfo> getUpdateQueueInfo() {
     return updateQueueInfo;
   }
 
+  public void setUpdateQueueInfo(ArrayList<QueueConfigInfo> updateQueueInfo) {
+    this.updateQueueInfo = updateQueueInfo;
+  }
+
   @XmlElementWrapper(name = "global-updates")
   public HashMap<String, String> getGlobalParams() {
     return global;
   }
+
+  public void setGlobalParams(HashMap<String, String> globalInfo) {
+    this.global = globalInfo;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb27fc4a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestConfigurationMutationACLPolicies.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/scheduler/TestConfigurationMutationACLPolicies.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestConfigurationMutationACLPolicies.java
index 0f5a3d8..398e909 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestConfigurationMutationACLPolicies.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestConfigurationMutationACLPolicies.java
@@ -25,8 +25,8 @@ import org.apache.hadoop.yarn.api.records.QueueInfo;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.conf.QueueAdminConfigurationMutationACLPolicy;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.QueueConfigInfo;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.SchedConfUpdateInfo;
+import org.apache.hadoop.yarn.webapp.dao.QueueConfigInfo;
+import org.apache.hadoop.yarn.webapp.dao.SchedConfUpdateInfo;
 import org.junit.Before;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb27fc4a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/TestMutableCSConfigurationProvider.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/scheduler/capacity/conf/TestMutableCSConfigurationProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/TestMutableCSConfigurationProvider.java
index 3216781..9104f16 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/TestMutableCSConfigurationProvider.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/TestMutableCSConfigurationProvider.java
@@ -23,8 +23,8 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.QueueConfigInfo;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.SchedConfUpdateInfo;
+import org.apache.hadoop.yarn.webapp.dao.QueueConfigInfo;
+import org.apache.hadoop.yarn.webapp.dao.SchedConfUpdateInfo;
 import org.junit.Before;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb27fc4a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesConfigurationMutation.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/webapp/TestRMWebServicesConfigurationMutation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesConfigurationMutation.java
index 5fbe36f..26ef1b7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesConfigurationMutation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesConfigurationMutation.java
@@ -22,8 +22,6 @@ import com.google.inject.Guice;
 import com.google.inject.servlet.ServletModule;
 import com.sun.jersey.api.client.ClientResponse;
 import com.sun.jersey.api.client.WebResource;
-import com.sun.jersey.api.json.JSONJAXBContext;
-import com.sun.jersey.api.json.JSONMarshaller;
 import com.sun.jersey.guice.spi.container.servlet.GuiceContainer;
 import com.sun.jersey.test.framework.WebAppDescriptor;
 import org.apache.hadoop.conf.Configuration;
@@ -35,11 +33,12 @@ import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.QueueConfigInfo;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.SchedConfUpdateInfo;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
 import org.apache.hadoop.yarn.webapp.GuiceServletConfig;
 import org.apache.hadoop.yarn.webapp.JerseyTestBase;
+import org.apache.hadoop.yarn.webapp.dao.QueueConfigInfo;
+import org.apache.hadoop.yarn.webapp.dao.SchedConfUpdateInfo;
+import org.apache.hadoop.yarn.webapp.util.YarnWebServiceUtils;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -50,7 +49,6 @@ import javax.ws.rs.core.Response.Status;
 import java.io.File;
 import java.io.FileOutputStream;
 import java.io.IOException;
-import java.io.StringWriter;
 import java.util.HashMap;
 import java.util.Map;
 
@@ -183,8 +181,8 @@ public class TestRMWebServicesConfigurationMutation extends JerseyTestBase {
         r.path("ws").path("v1").path("cluster")
             .path("sched-conf").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON)
-            .entity(toJson(updateInfo, SchedConfUpdateInfo.class),
-                MediaType.APPLICATION_JSON)
+            .entity(YarnWebServiceUtils.toJson(updateInfo,
+                SchedConfUpdateInfo.class), MediaType.APPLICATION_JSON)
             .put(ClientResponse.class);
 
     assertEquals(Status.OK.getStatusCode(), response.getStatus());
@@ -218,8 +216,8 @@ public class TestRMWebServicesConfigurationMutation extends JerseyTestBase {
         r.path("ws").path("v1").path("cluster")
             .path("sched-conf").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON)
-            .entity(toJson(updateInfo, SchedConfUpdateInfo.class),
-                MediaType.APPLICATION_JSON)
+            .entity(YarnWebServiceUtils.toJson(updateInfo,
+                SchedConfUpdateInfo.class), MediaType.APPLICATION_JSON)
             .put(ClientResponse.class);
 
     assertEquals(Status.OK.getStatusCode(), response.getStatus());
@@ -244,8 +242,8 @@ public class TestRMWebServicesConfigurationMutation extends JerseyTestBase {
         r.path("ws").path("v1").path("cluster")
             .path("sched-conf").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON)
-            .entity(toJson(updateInfo, SchedConfUpdateInfo.class),
-                MediaType.APPLICATION_JSON)
+            .entity(YarnWebServiceUtils.toJson(updateInfo,
+                SchedConfUpdateInfo.class), MediaType.APPLICATION_JSON)
             .put(ClientResponse.class);
 
     assertEquals(Status.OK.getStatusCode(), response.getStatus());
@@ -269,8 +267,8 @@ public class TestRMWebServicesConfigurationMutation extends JerseyTestBase {
         r.path("ws").path("v1").path("cluster")
             .path("sched-conf").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON)
-            .entity(toJson(updateInfo, SchedConfUpdateInfo.class),
-                MediaType.APPLICATION_JSON)
+            .entity(YarnWebServiceUtils.toJson(updateInfo,
+                SchedConfUpdateInfo.class), MediaType.APPLICATION_JSON)
             .put(ClientResponse.class);
 
     assertEquals(Status.OK.getStatusCode(), response.getStatus());
@@ -300,8 +298,8 @@ public class TestRMWebServicesConfigurationMutation extends JerseyTestBase {
         r.path("ws").path("v1").path("cluster")
             .path("sched-conf").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON)
-            .entity(toJson(updateInfo, SchedConfUpdateInfo.class),
-                MediaType.APPLICATION_JSON)
+            .entity(YarnWebServiceUtils.toJson(updateInfo,
+                SchedConfUpdateInfo.class), MediaType.APPLICATION_JSON)
             .put(ClientResponse.class);
 
     assertEquals(Status.OK.getStatusCode(), response.getStatus());
@@ -332,8 +330,8 @@ public class TestRMWebServicesConfigurationMutation extends JerseyTestBase {
         r.path("ws").path("v1").path("cluster")
             .path("sched-conf").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON)
-            .entity(toJson(updateInfo, SchedConfUpdateInfo.class),
-                MediaType.APPLICATION_JSON)
+            .entity(YarnWebServiceUtils.toJson(updateInfo,
+                SchedConfUpdateInfo.class), MediaType.APPLICATION_JSON)
             .put(ClientResponse.class);
 
     assertEquals(Status.OK.getStatusCode(), response.getStatus());
@@ -360,8 +358,8 @@ public class TestRMWebServicesConfigurationMutation extends JerseyTestBase {
         r.path("ws").path("v1").path("cluster")
             .path("sched-conf").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON)
-            .entity(toJson(updateInfo, SchedConfUpdateInfo.class),
-                MediaType.APPLICATION_JSON)
+            .entity(YarnWebServiceUtils.toJson(updateInfo,
+                SchedConfUpdateInfo.class), MediaType.APPLICATION_JSON)
             .put(ClientResponse.class);
     assertEquals(Status.OK.getStatusCode(), response.getStatus());
     CapacitySchedulerConfiguration newCSConf =
@@ -395,8 +393,8 @@ public class TestRMWebServicesConfigurationMutation extends JerseyTestBase {
         r.path("ws").path("v1").path("cluster")
             .path("sched-conf").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON)
-            .entity(toJson(updateInfo, SchedConfUpdateInfo.class),
-                MediaType.APPLICATION_JSON)
+            .entity(YarnWebServiceUtils.toJson(updateInfo,
+                SchedConfUpdateInfo.class), MediaType.APPLICATION_JSON)
             .put(ClientResponse.class);
     assertEquals(Status.OK.getStatusCode(), response.getStatus());
     CapacitySchedulerConfiguration newCSConf = cs.getConfiguration();
@@ -413,8 +411,8 @@ public class TestRMWebServicesConfigurationMutation extends JerseyTestBase {
         r.path("ws").path("v1").path("cluster")
             .path("sched-conf").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON)
-            .entity(toJson(updateInfo, SchedConfUpdateInfo.class),
-                MediaType.APPLICATION_JSON)
+            .entity(YarnWebServiceUtils.toJson(updateInfo,
+                SchedConfUpdateInfo.class), MediaType.APPLICATION_JSON)
             .put(ClientResponse.class);
     assertEquals(Status.OK.getStatusCode(), response.getStatus());
     newCSConf = cs.getConfiguration();
@@ -443,8 +441,8 @@ public class TestRMWebServicesConfigurationMutation extends JerseyTestBase {
         r.path("ws").path("v1").path("cluster")
             .path("sched-conf").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON)
-            .entity(toJson(updateInfo, SchedConfUpdateInfo.class),
-                MediaType.APPLICATION_JSON)
+            .entity(YarnWebServiceUtils.toJson(updateInfo,
+                SchedConfUpdateInfo.class), MediaType.APPLICATION_JSON)
             .put(ClientResponse.class);
     assertEquals(Status.OK.getStatusCode(), response.getStatus());
     CapacitySchedulerConfiguration newCSConf =
@@ -468,8 +466,8 @@ public class TestRMWebServicesConfigurationMutation extends JerseyTestBase {
         r.path("ws").path("v1").path("cluster")
             .path("sched-conf").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON)
-            .entity(toJson(updateInfo, SchedConfUpdateInfo.class),
-                MediaType.APPLICATION_JSON)
+            .entity(YarnWebServiceUtils.toJson(updateInfo,
+                SchedConfUpdateInfo.class), MediaType.APPLICATION_JSON)
             .put(ClientResponse.class);
     assertEquals(Status.OK.getStatusCode(), response.getStatus());
     CapacitySchedulerConfiguration newCSConf =
@@ -483,8 +481,8 @@ public class TestRMWebServicesConfigurationMutation extends JerseyTestBase {
         r.path("ws").path("v1").path("cluster")
             .path("sched-conf").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON)
-            .entity(toJson(updateInfo, SchedConfUpdateInfo.class),
-                MediaType.APPLICATION_JSON)
+            .entity(YarnWebServiceUtils.toJson(updateInfo,
+                SchedConfUpdateInfo.class), MediaType.APPLICATION_JSON)
             .put(ClientResponse.class);
     assertEquals(Status.OK.getStatusCode(), response.getStatus());
     newCSConf =
@@ -506,13 +504,4 @@ public class TestRMWebServicesConfigurationMutation extends JerseyTestBase {
     }
     super.tearDown();
   }
-
-  @SuppressWarnings("rawtypes")
-  private String toJson(Object nsli, Class klass) throws Exception {
-    StringWriter sw = new StringWriter();
-    JSONJAXBContext ctx = new JSONJAXBContext(klass);
-    JSONMarshaller jm = ctx.createJSONMarshaller();
-    jm.marshallToJSON(nsli, sw);
-    return sw.toString();
-  }
 }


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


[27/50] [abbrv] hadoop git commit: YARN-5146. Support for Fair Scheduler in new YARN UI. Contributed by Abdullah Yousufi.

Posted by jh...@apache.org.
YARN-5146. Support for Fair Scheduler in new YARN UI. Contributed by Abdullah Yousufi.


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

Branch: refs/heads/YARN-5734
Commit: dadb0c2225adef5cb0126610733c285b51f4f43e
Parents: e3ae3e2
Author: Sunil G <su...@apache.org>
Authored: Tue Aug 15 21:58:44 2017 +0530
Committer: Sunil G <su...@apache.org>
Committed: Tue Aug 15 21:58:44 2017 +0530

----------------------------------------------------------------------
 .../src/main/webapp/app/adapters/yarn-queue.js  |  30 -----
 .../app/adapters/yarn-queue/capacity-queue.js   |  23 ++++
 .../app/adapters/yarn-queue/fair-queue.js       |  23 ++++
 .../app/adapters/yarn-queue/fifo-queue.js       |  23 ++++
 .../app/adapters/yarn-queue/yarn-queue.js       |  30 +++++
 .../main/webapp/app/components/tree-selector.js |  19 ++-
 .../src/main/webapp/app/models/yarn-queue.js    |  94 --------------
 .../app/models/yarn-queue/capacity-queue.js     |  95 ++++++++++++++
 .../webapp/app/models/yarn-queue/fair-queue.js  |  79 ++++++++++++
 .../webapp/app/models/yarn-queue/fifo-queue.js  |  52 ++++++++
 .../webapp/app/models/yarn-queue/yarn-queue.js  |  23 ++++
 .../main/webapp/app/routes/cluster-overview.js  |   4 +-
 .../src/main/webapp/app/routes/yarn-queue.js    |  26 ++--
 .../src/main/webapp/app/routes/yarn-queues.js   |  12 +-
 .../main/webapp/app/routes/yarn-queues/index.js |  25 ----
 .../app/routes/yarn-queues/queues-selector.js   |  25 ----
 .../main/webapp/app/serializers/yarn-queue.js   | 129 -------------------
 .../serializers/yarn-queue/capacity-queue.js    | 128 ++++++++++++++++++
 .../app/serializers/yarn-queue/fair-queue.js    |  92 +++++++++++++
 .../app/serializers/yarn-queue/fifo-queue.js    |  59 +++++++++
 .../app/serializers/yarn-queue/yarn-queue.js    |  47 +++++++
 .../components/queue-configuration-table.hbs    |  54 --------
 .../templates/components/queue-navigator.hbs    |   7 +-
 .../yarn-queue/capacity-queue-conf-table.hbs    |  54 ++++++++
 .../yarn-queue/capacity-queue-info.hbs          |  84 ++++++++++++
 .../components/yarn-queue/capacity-queue.hbs    |  63 +++++++++
 .../yarn-queue/fair-queue-conf-table.hbs        |  52 ++++++++
 .../components/yarn-queue/fair-queue-info.hbs   |  66 ++++++++++
 .../components/yarn-queue/fair-queue.hbs        |  63 +++++++++
 .../yarn-queue/fifo-queue-conf-table.hbs        |  56 ++++++++
 .../components/yarn-queue/fifo-queue-info.hbs   |  47 +++++++
 .../components/yarn-queue/fifo-queue.hbs        |  48 +++++++
 .../webapp/app/templates/yarn-queue/info.hbs    |  73 +----------
 .../main/webapp/app/templates/yarn-queues.hbs   |  54 +-------
 .../src/main/webapp/app/utils/color-utils.js    |   1 -
 35 files changed, 1266 insertions(+), 494 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-queue.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-queue.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-queue.js
deleted file mode 100644
index f2017df..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-queue.js
+++ /dev/null
@@ -1,30 +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.
- */
-
-import AbstractAdapter from './abstract';
-
-export default AbstractAdapter.extend({
-  address: "rmWebAddress",
-  restNameSpace: "cluster",
-  serverName: "RM",
-
-  pathForType(/*modelName*/) {
-    return 'scheduler'; // move to some common place, return path by modelname.
-  }
-
-});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-queue/capacity-queue.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-queue/capacity-queue.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-queue/capacity-queue.js
new file mode 100644
index 0000000..7eb9f76
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-queue/capacity-queue.js
@@ -0,0 +1,23 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import YarnQueueAdapter from './yarn-queue';
+
+export default YarnQueueAdapter.extend({
+
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-queue/fair-queue.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-queue/fair-queue.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-queue/fair-queue.js
new file mode 100644
index 0000000..7eb9f76
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-queue/fair-queue.js
@@ -0,0 +1,23 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import YarnQueueAdapter from './yarn-queue';
+
+export default YarnQueueAdapter.extend({
+
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-queue/fifo-queue.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-queue/fifo-queue.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-queue/fifo-queue.js
new file mode 100644
index 0000000..7eb9f76
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-queue/fifo-queue.js
@@ -0,0 +1,23 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import YarnQueueAdapter from './yarn-queue';
+
+export default YarnQueueAdapter.extend({
+
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-queue/yarn-queue.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-queue/yarn-queue.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-queue/yarn-queue.js
new file mode 100644
index 0000000..8184c39
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/adapters/yarn-queue/yarn-queue.js
@@ -0,0 +1,30 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import AbstractAdapter from '../abstract';
+
+export default AbstractAdapter.extend({
+  address: "rmWebAddress",
+  restNameSpace: "cluster",
+  serverName: "RM",
+
+  pathForType(/*modelName*/) {
+    return 'scheduler'; // move to some common place, return path by modelname.
+  }
+
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/tree-selector.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/tree-selector.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/tree-selector.js
index 3d72b2f..1a81a32 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/tree-selector.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/tree-selector.js
@@ -39,6 +39,9 @@ export default Ember.Component.extend({
   // mainSvg
   mainSvg: undefined,
 
+  used: undefined,
+  max: undefined,
+
   // Init data
   initData: function() {
     this.map = { };
@@ -52,7 +55,8 @@ export default Ember.Component.extend({
       }.bind(this));
 
     // var selected = this.get("selected");
-
+    this.used = this.get("used");
+    this.max = this.get("max");
     this.initQueue("root", 1, this.treeData);
   },
 
@@ -81,7 +85,6 @@ export default Ember.Component.extend({
       // Queue is not existed
       return;
     }
-
     if (depth > this.maxDepth) {
       this.maxDepth = this.maxDepth + 1;
     }
@@ -149,7 +152,9 @@ export default Ember.Component.extend({
     nodeEnter.append("circle")
       .attr("r", 1e-6)
       .style("fill", function(d) {
-        var usedCap = d.queueData.get("usedCapacity");
+        var maxCap = d.queueData.get(this.max);
+        maxCap = maxCap == undefined ? 100 : maxCap;
+        var usedCap = d.queueData.get(this.used) / maxCap * 100.0;
         if (usedCap <= 60.0) {
           return "LimeGreen";
         } else if (usedCap <= 100.0) {
@@ -157,7 +162,7 @@ export default Ember.Component.extend({
         } else {
           return "LightCoral";
         }
-      });
+      }.bind(this));
 
     // append percentage
     nodeEnter.append("text")
@@ -166,13 +171,15 @@ export default Ember.Component.extend({
       .attr("fill", "white")
       .attr("text-anchor", function() { return "middle"; })
       .text(function(d) {
-        var usedCap = d.queueData.get("usedCapacity");
+        var maxCap = d.queueData.get(this.max);
+        maxCap = maxCap == undefined ? 100 : maxCap;
+        var usedCap = d.queueData.get(this.used) / maxCap * 100.0;
         if (usedCap >= 100.0) {
           return usedCap.toFixed(0) + "%";
         } else {
           return usedCap.toFixed(1) + "%";
         }
-      })
+      }.bind(this))
       .style("fill-opacity", 1e-6);
 
     // append queue name

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-queue.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-queue.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-queue.js
deleted file mode 100644
index 27c48f7..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-queue.js
+++ /dev/null
@@ -1,94 +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.
- */
-
-import DS from 'ember-data';
-
-export default DS.Model.extend({
-  name: DS.attr('string'),
-  children: DS.attr('array'),
-  parent: DS.attr('string'),
-  capacity: DS.attr('number'),
-  maxCapacity: DS.attr('number'),
-  usedCapacity: DS.attr('number'),
-  absCapacity: DS.attr('number'),
-  absMaxCapacity: DS.attr('number'),
-  absUsedCapacity: DS.attr('number'),
-  state: DS.attr('string'),
-  userLimit: DS.attr('number'),
-  userLimitFactor: DS.attr('number'),
-  preemptionDisabled: DS.attr('number'),
-  numPendingApplications: DS.attr('number'),
-  numActiveApplications: DS.attr('number'),
-  users: DS.hasMany('YarnUser'),
-
-  isLeafQueue: function() {
-    var len = this.get("children.length");
-    if (!len) {
-      return true;
-    }
-    return len <= 0;
-  }.property("children"),
-
-  capacitiesBarChartData: function() {
-    return [
-      {
-        label: "Absolute Capacity",
-        value: this.get("name") === "root" ? 100 : this.get("absCapacity")
-      },
-      {
-        label: "Absolute Used",
-        value: this.get("name") === "root" ? this.get("usedCapacity") : this.get("absUsedCapacity")
-      },
-      {
-        label: "Absolute Max Capacity",
-        value: this.get("name") === "root" ? 100 : this.get("absMaxCapacity")
-      }
-    ];
-  }.property("absCapacity", "absUsedCapacity", "absMaxCapacity"),
-
-  userUsagesDonutChartData: function() {
-    var data = [];
-    if (this.get("users")) {
-      this.get("users").forEach(function(o) {
-        data.push({
-          label: o.get("name"),
-          value: o.get("usedMemoryMB")
-        });
-      });
-    }
-
-    return data;
-  }.property("users"),
-
-  hasUserUsages: function() {
-    return this.get("userUsagesDonutChartData").length > 0;
-  }.property(),
-
-  numOfApplicationsDonutChartData: function() {
-    return [
-      {
-        label: "Pending Apps",
-        value: this.get("numPendingApplications") || 0 // TODO, fix the REST API so root will return #applications as well.
-      },
-      {
-        label: "Active Apps",
-        value: this.get("numActiveApplications") || 0
-      }
-    ];
-  }.property()
-});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-queue/capacity-queue.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-queue/capacity-queue.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-queue/capacity-queue.js
new file mode 100644
index 0000000..1cb07bb
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-queue/capacity-queue.js
@@ -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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import DS from 'ember-data';
+
+export default DS.Model.extend({
+  name: DS.attr('string'),
+  children: DS.attr('array'),
+  parent: DS.attr('string'),
+  capacity: DS.attr('number'),
+  maxCapacity: DS.attr('number'),
+  usedCapacity: DS.attr('number'),
+  absCapacity: DS.attr('number'),
+  absMaxCapacity: DS.attr('number'),
+  absUsedCapacity: DS.attr('number'),
+  state: DS.attr('string'),
+  userLimit: DS.attr('number'),
+  userLimitFactor: DS.attr('number'),
+  preemptionDisabled: DS.attr('number'),
+  numPendingApplications: DS.attr('number'),
+  numActiveApplications: DS.attr('number'),
+  users: DS.hasMany('YarnUser'),
+  type: DS.attr('string'),
+
+  isLeafQueue: function() {
+    var len = this.get("children.length");
+    if (!len) {
+      return true;
+    }
+    return len <= 0;
+  }.property("children"),
+
+  capacitiesBarChartData: function() {
+    return [
+      {
+        label: "Absolute Capacity",
+        value: this.get("name") === "root" ? 100 : this.get("absCapacity")
+      },
+      {
+        label: "Absolute Used",
+        value: this.get("name") === "root" ? this.get("usedCapacity") : this.get("absUsedCapacity")
+      },
+      {
+        label: "Absolute Max Capacity",
+        value: this.get("name") === "root" ? 100 : this.get("absMaxCapacity")
+      }
+    ];
+  }.property("absCapacity", "usedCapacity", "absMaxCapacity"),
+
+  userUsagesDonutChartData: function() {
+    var data = [];
+    if (this.get("users")) {
+      this.get("users").forEach(function(o) {
+        data.push({
+          label: o.get("name"),
+          value: o.get("usedMemoryMB")
+        });
+      });
+    }
+
+    return data;
+  }.property("users"),
+
+  hasUserUsages: function() {
+    return this.get("userUsagesDonutChartData").length > 0;
+  }.property(),
+
+  numOfApplicationsDonutChartData: function() {
+    return [
+      {
+        label: "Pending Apps",
+        value: this.get("numPendingApplications") || 0 // TODO, fix the REST API so root will return #applications as well.
+      },
+      {
+        label: "Active Apps",
+        value: this.get("numActiveApplications") || 0
+      }
+    ];
+  }.property("numPendingApplications", "numActiveApplications")
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-queue/fair-queue.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-queue/fair-queue.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-queue/fair-queue.js
new file mode 100644
index 0000000..be71362
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-queue/fair-queue.js
@@ -0,0 +1,79 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import DS from 'ember-data';
+
+export default DS.Model.extend({
+  name: DS.attr('string'),
+  children: DS.attr('array'),
+  parent: DS.attr('string'),
+  maxApps: DS.attr('number'),
+  minResources: DS.attr(),
+  maxResources: DS.attr(),
+  usedResources: DS.attr(),
+  demandResources: DS.attr(),
+  steadyFairResources: DS.attr(),
+  fairResources: DS.attr(),
+  clusterResources: DS.attr(),
+  pendingContainers: DS.attr('number'),
+  allocatedContainers: DS.attr('number'),
+  reservedContainers: DS.attr('number'),
+  schedulingPolicy: DS.attr('string'),
+  preemptable: DS.attr('number'),
+  numPendingApplications: DS.attr('number'),
+  numActiveApplications: DS.attr('number'),
+  type: DS.attr('string'),
+
+  isLeafQueue: function() {
+    var len = this.get("children.length");
+    if (!len) {
+      return true;
+    }
+    return len <= 0;
+  }.property("children"),
+
+  capacitiesBarChartData: function() {
+    return [
+      {
+        label: "Steady Fair Memory",
+        value: this.get("steadyFairResources.memory")
+      },
+      {
+        label: "Used Memory",
+        value: this.get("usedResources.memory")
+      },
+      {
+        label: "Maximum Memory",
+        value: this.get("maxResources.memory")
+      }
+    ];
+  }.property("maxResources.memory", "usedResources.memory", "maxResources.memory"),
+
+  numOfApplicationsDonutChartData: function() {
+    return [
+      {
+        label: "Pending Apps",
+        value: this.get("numPendingApplications") || 0 // TODO, fix the REST API so root will return #applications as well.
+      },
+      {
+        label: "Active Apps",
+        value: this.get("numActiveApplications") || 0
+      }
+    ];
+  }.property()
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-queue/fifo-queue.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-queue/fifo-queue.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-queue/fifo-queue.js
new file mode 100644
index 0000000..2386dc4
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-queue/fifo-queue.js
@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import DS from 'ember-data';
+
+export default DS.Model.extend({
+  name: DS.attr('string'),
+  capacity: DS.attr('number'),
+  usedCapacity: DS.attr('number'),
+  state: DS.attr('string'),
+  minQueueMemoryCapacity: DS.attr('number'),
+  maxQueueMemoryCapacity: DS.attr('number'),
+  numNodes: DS.attr('number'),
+  usedNodeCapacity: DS.attr('number'),
+  availNodeCapacity: DS.attr('number'),
+  totalNodeCapacity: DS.attr('number'),
+  numContainers: DS.attr('number'),
+  type: DS.attr('string'),
+
+  capacitiesBarChartData: function() {
+    return [
+      {
+        label: "Available Capacity",
+        value: this.get("availNodeCapacity")
+      },
+      {
+        label: "Used Capacity",
+        value: this.get("usedNodeCapacity")
+      },
+      {
+        label: "Total Capacity",
+        value: this.get("totalNodeCapacity")
+      }
+    ];
+  }.property("availNodeCapacity", "usedNodeCapacity", "totalNodeCapacity")
+
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-queue/yarn-queue.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-queue/yarn-queue.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-queue/yarn-queue.js
new file mode 100644
index 0000000..dcf5f48
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-queue/yarn-queue.js
@@ -0,0 +1,23 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import DS from 'ember-data';
+
+export default DS.Model.extend({
+  type: DS.attr('string')
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/cluster-overview.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/cluster-overview.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/cluster-overview.js
index b5db17d..3c6abd4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/cluster-overview.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/cluster-overview.js
@@ -28,7 +28,7 @@ export default AbstractRoute.extend({
         {
           state: "RUNNING"
         }),
-      queues: this.store.query('yarn-queue', {}),
+      queues: this.store.query('yarn-queue.yarn-queue', {}),
     });
   },
 
@@ -39,6 +39,6 @@ export default AbstractRoute.extend({
   unloadAll() {
     this.store.unloadAll('ClusterMetric');
     this.store.unloadAll('yarn-app');
-    this.store.unloadAll('yarn-queue');
+    this.store.unloadAll('yarn-queue.yarn-queue');
   }
 });

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-queue.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-queue.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-queue.js
index 1c4546c..cd4ed09 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-queue.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-queue.js
@@ -22,22 +22,28 @@ import AbstractRoute from './abstract';
 
 export default AbstractRoute.extend({
   model(param) {
-    return Ember.RSVP.hash({
-      selected : param.queue_name,
-      queues: this.store.query('yarn-queue', {}),
-      selectedQueue : undefined,
-      apps: this.store.query('yarn-app', {
-        queue: param.queue_name
-      })
-    });
+      return Ember.RSVP.hash({
+        selected : param.queue_name,
+        queues: this.store.query("yarn-queue.yarn-queue", {}).then((model) => {
+          let type = model.get('firstObject').get('type');
+          return this.store.query("yarn-queue." + type + "-queue", {});
+        }),
+        selectedQueue : undefined,
+        apps: this.store.query('yarn-app', {
+          queue: param.queue_name
+        })
+      });
   },
 
   afterModel(model) {
-    model.selectedQueue = this.store.peekRecord('yarn-queue', model.selected);
+    var type = model.queues.get('firstObject').constructor.modelName;
+    model.selectedQueue = this.store.peekRecord(type, model.selected);
   },
 
   unloadAll() {
-    this.store.unloadAll('yarn-queue');
+    this.store.unloadAll('yarn-queue.capacity-queue');
+    this.store.unloadAll('yarn-queue.fair-queue');
+    this.store.unloadAll('yarn-queue.fifo-queue');
     this.store.unloadAll('yarn-app');
   }
 });

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-queues.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-queues.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-queues.js
index e4f145d..7d8a200 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-queues.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-queues.js
@@ -30,17 +30,23 @@ export default AbstractRoute.extend({
     }
     return Ember.RSVP.hash({
       selected : queueName,
-      queues: this.store.query('yarn-queue', {}),
+      queues: this.store.query("yarn-queue.yarn-queue", {}).then((model) => {
+        let type = model.get('firstObject').get('type');
+        return this.store.query("yarn-queue." + type + "-queue", {});
+      }),
       selectedQueue : undefined
     });
   },
 
   afterModel(model) {
-    model.selectedQueue = this.store.peekRecord('yarn-queue', model.selected);
+    var type = model.queues.get('firstObject').constructor.modelName;
+    model.selectedQueue = this.store.peekRecord(type, model.selected);
   },
 
   unloadAll() {
-    this.store.unloadAll('yarn-queue');
+    this.store.unloadAll('yarn-queue.capacity-queue');
+    this.store.unloadAll('yarn-queue.fair-queue');
+    this.store.unloadAll('yarn-queue.fifo-queue');
     this.store.unloadAll('yarn-app');
   },
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-queues/index.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-queues/index.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-queues/index.js
deleted file mode 100644
index 436c6d8..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-queues/index.js
+++ /dev/null
@@ -1,25 +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.
- */
-
-import Ember from 'ember';
-
-export default Ember.Route.extend({
-  beforeModel() {
-    this.transitionTo('yarn-queues.root');
-  }
-});
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-queues/queues-selector.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-queues/queues-selector.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-queues/queues-selector.js
deleted file mode 100644
index 5d14c6f..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/routes/yarn-queues/queues-selector.js
+++ /dev/null
@@ -1,25 +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.
- */
-
-import Ember from 'ember';
-
-export default Ember.Route.extend({
-  model() {
-    return this.store.findAll('yarn-queue');
-  },
-});
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-queue.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-queue.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-queue.js
deleted file mode 100644
index 4fc1a29..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-queue.js
+++ /dev/null
@@ -1,129 +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.
- */
-
-import DS from 'ember-data';
-
-export default DS.JSONAPISerializer.extend({
-
-    normalizeSingleResponse(store, primaryModelClass, payload, id,
-      requestType) {
-      var children = [];
-      if (payload.queues) {
-        payload.queues.queue.forEach(function(queue) {
-          children.push(queue.queueName);
-        });
-      }
-
-      var includedData = [];
-      var relationshipUserData = [];
-
-      // update user models
-      if (payload.users && payload.users.user) {
-        payload.users.user.forEach(function(u) {
-          includedData.push({
-            type: "YarnUser",
-            id: u.username + "_" + payload.queueName,
-            attributes: {
-              name: u.username,
-              queueName: payload.queueName,
-              usedMemoryMB: u.resourcesUsed.memory || 0,
-              usedVCore: u.resourcesUsed.vCores || 0,
-            }
-          });
-
-          relationshipUserData.push({
-            type: "YarnUser",
-            id: u.username + "_" + payload.queueName,
-          });
-        });
-      }
-
-
-      var fixedPayload = {
-        id: id,
-        type: primaryModelClass.modelName, // yarn-queue
-        attributes: {
-          name: payload.queueName,
-          parent: payload.myParent,
-          children: children,
-          capacity: payload.capacity,
-          usedCapacity: payload.usedCapacity,
-          maxCapacity: payload.maxCapacity,
-          absCapacity: payload.absoluteCapacity,
-          absMaxCapacity: payload.absoluteMaxCapacity,
-          absUsedCapacity: payload.absoluteUsedCapacity,
-          state: payload.state,
-          userLimit: payload.userLimit,
-          userLimitFactor: payload.userLimitFactor,
-          preemptionDisabled: payload.preemptionDisabled,
-          numPendingApplications: payload.numPendingApplications,
-          numActiveApplications: payload.numActiveApplications,
-        },
-        // Relationships
-        relationships: {
-          users: {
-            data: relationshipUserData
-          }
-        }
-      };
-
-      return {
-        queue: this._super(store, primaryModelClass, fixedPayload, id, requestType),
-        includedData: includedData
-      };
-    },
-
-    handleQueue(store, primaryModelClass, payload, id, requestType) {
-      var data = [];
-      var includedData = [];
-      var result = this.normalizeSingleResponse(store, primaryModelClass,
-        payload, id, requestType);
-
-      data.push(result.queue);
-      includedData = includedData.concat(result.includedData);
-
-      if (payload.queues) {
-        for (var i = 0; i < payload.queues.queue.length; i++) {
-          var queue = payload.queues.queue[i];
-          queue.myParent = payload.queueName;
-          var childResult = this.handleQueue(store, primaryModelClass, queue,
-            queue.queueName,
-            requestType);
-
-          data = data.concat(childResult.data);
-          includedData = includedData.concat(childResult.includedData);
-        }
-      }
-
-      return {
-        data: data,
-        includedData: includedData
-      };
-    },
-
-    normalizeArrayResponse(store, primaryModelClass, payload, id, requestType) {
-      var normalizedArrayResponse = {};
-      var result = this.handleQueue(store, primaryModelClass,
-        payload.scheduler.schedulerInfo, "root", requestType);
-
-      normalizedArrayResponse.data = result.data;
-      normalizedArrayResponse.included = result.includedData;
-
-      return normalizedArrayResponse;
-    }
-});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-queue/capacity-queue.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-queue/capacity-queue.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-queue/capacity-queue.js
new file mode 100644
index 0000000..c7350ef
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-queue/capacity-queue.js
@@ -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.
+ */
+
+import DS from 'ember-data';
+
+export default DS.JSONAPISerializer.extend({
+
+    normalizeSingleResponse(store, primaryModelClass, payload, id,
+      requestType) {
+      var children = [];
+      if (payload.queues) {
+        payload.queues.queue.forEach(function(queue) {
+          children.push(queue.queueName);
+        });
+      }
+
+      var includedData = [];
+      var relationshipUserData = [];
+
+      // update user models
+      if (payload.users && payload.users.user) {
+        payload.users.user.forEach(function(u) {
+          includedData.push({
+            type: "YarnUser",
+            id: u.username + "_" + payload.queueName,
+            attributes: {
+              name: u.username,
+              queueName: payload.queueName,
+              usedMemoryMB: u.resourcesUsed.memory || 0,
+              usedVCore: u.resourcesUsed.vCores || 0,
+            }
+          });
+
+          relationshipUserData.push({
+            type: "YarnUser",
+            id: u.username + "_" + payload.queueName,
+          });
+        });
+      }
+
+      var fixedPayload = {
+        id: id,
+        type: primaryModelClass.modelName, // yarn-queue
+        attributes: {
+          name: payload.queueName,
+          parent: payload.myParent,
+          children: children,
+          capacity: payload.capacity,
+          usedCapacity: payload.usedCapacity,
+          maxCapacity: payload.maxCapacity,
+          absCapacity: payload.absoluteCapacity,
+          absMaxCapacity: payload.absoluteMaxCapacity,
+          absUsedCapacity: payload.absoluteUsedCapacity,
+          state: payload.state,
+          userLimit: payload.userLimit,
+          userLimitFactor: payload.userLimitFactor,
+          preemptionDisabled: payload.preemptionDisabled,
+          numPendingApplications: payload.numPendingApplications,
+          numActiveApplications: payload.numActiveApplications,
+          type: "capacity",
+        },
+        // Relationships
+        relationships: {
+          users: {
+            data: relationshipUserData
+          }
+        }
+      };
+      return {
+        queue: this._super(store, primaryModelClass, fixedPayload, id, requestType),
+        includedData: includedData
+      };
+    },
+
+    handleQueue(store, primaryModelClass, payload, id, requestType) {
+      var data = [];
+      var includedData = [];
+      var result = this.normalizeSingleResponse(store, primaryModelClass,
+        payload, id, requestType);
+
+      data.push(result.queue);
+      includedData = includedData.concat(result.includedData);
+
+      if (payload.queues) {
+        for (var i = 0; i < payload.queues.queue.length; i++) {
+          var queue = payload.queues.queue[i];
+          queue.myParent = payload.queueName;
+          var childResult = this.handleQueue(store, primaryModelClass, queue,
+            queue.queueName,
+            requestType);
+
+          data = data.concat(childResult.data);
+          includedData = includedData.concat(childResult.includedData);
+        }
+      }
+
+      return {
+        data: data,
+        includedData: includedData
+      };
+    },
+
+    normalizeArrayResponse(store, primaryModelClass, payload, id, requestType) {
+      var normalizedArrayResponse = {};
+      var result = this.handleQueue(store, primaryModelClass,
+        payload.scheduler.schedulerInfo, "root", requestType);
+
+      normalizedArrayResponse.data = result.data;
+      normalizedArrayResponse.included = result.includedData;
+
+      return normalizedArrayResponse;
+    }
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-queue/fair-queue.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-queue/fair-queue.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-queue/fair-queue.js
new file mode 100644
index 0000000..2215d2d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-queue/fair-queue.js
@@ -0,0 +1,92 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import DS from 'ember-data';
+
+export default DS.JSONAPISerializer.extend({
+
+    normalizeSingleResponse(store, primaryModelClass, payload, id,
+      requestType) {
+      var children = [];
+      if (payload.childQueues) {
+        payload.childQueues.queue.forEach(function(queue) {
+          children.push(queue.queueName);
+        });
+      }
+
+      var fixedPayload = {
+        id: id,
+        type: primaryModelClass.modelName,
+        attributes: {
+          name: payload.queueName,
+          parent: payload.myParent,
+          children: children,
+          maxApps: payload.maxApps,
+          minResources: payload.minResources,
+          maxResources: payload.maxResources,
+          usedResources: payload.usedResources,
+          demandResources: payload.demandResources,
+          steadyFairResources: payload.steadyFairResources,
+          fairResources: payload.fairResources,
+          clusterResources: payload.clusterResources,
+          pendingContainers: payload.pendingContainers,
+          allocatedContainers: payload.allocatedContainers,
+          reservedContainers: payload.reservedContainers,
+          schedulingPolicy: payload.schedulingPolicy,
+          preemptable: payload.preemptable,
+          numPendingApplications: payload.numPendingApps,
+          numActiveApplications: payload.numActiveApps,
+          type: "fair",
+        },
+      };
+      return this._super(store, primaryModelClass, fixedPayload, id, requestType);
+    },
+
+    handleQueue(store, primaryModelClass, payload, id, requestType) {
+      var data = [];
+      var includedData = [];
+      if(!payload) return data;
+      var result = this.normalizeSingleResponse(store, primaryModelClass,
+        payload, id, requestType);
+
+      data.push(result);
+
+      if (payload.childQueues) {
+        for (var i = 0; i < payload.childQueues.queue.length; i++) {
+          var queue = payload.childQueues.queue[i];
+          queue.myParent = payload.queueName;
+          var childResult = this.handleQueue(store, primaryModelClass, queue,
+            queue.queueName,
+            requestType);
+
+          data = data.concat(childResult);
+        }
+      }
+
+      return data;
+    },
+
+    normalizeArrayResponse(store, primaryModelClass, payload, id, requestType) {
+      var normalizedArrayResponse = {};
+      var result = this.handleQueue(store, primaryModelClass,
+        payload.scheduler.schedulerInfo.rootQueue, "root", requestType);
+
+      normalizedArrayResponse.data = result;
+      return normalizedArrayResponse;
+    }
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-queue/fifo-queue.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-queue/fifo-queue.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-queue/fifo-queue.js
new file mode 100644
index 0000000..297ec18
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-queue/fifo-queue.js
@@ -0,0 +1,59 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import DS from 'ember-data';
+
+export default DS.JSONAPISerializer.extend({
+
+    normalizeSingleResponse(store, primaryModelClass, payload, id,
+      requestType) {
+
+      var fixedPayload = {
+        id: id,
+        type: primaryModelClass.modelName,
+        attributes: {
+          name: id,
+          capacity: payload.capacity * 100,
+          usedCapacity: payload.usedCapacity * 100,
+          usedNodeCapacity: payload.usedNodeCapacity,
+          availNodeCapacity: payload.availNodeCapacity,
+          totalNodeCapacity: payload.totalNodeCapacity,
+          numNodes: payload.numNodes,
+          numContainers: payload.numContainers,
+          state: payload.qstate,
+          minQueueMemoryCapacity: payload.minQueueMemoryCapacity,
+          maxQueueMemoryCapacity: payload.maxQueueMemoryCapacity,
+          type: "fifo",
+        },
+
+      };
+
+      return this._super(store, primaryModelClass, fixedPayload, id,
+        requestType);
+    },
+
+    normalizeArrayResponse(store, primaryModelClass, payload, id, requestType) {
+      var normalizedArrayResponse = {};
+      normalizedArrayResponse.data = [
+        this.normalizeSingleResponse(store, primaryModelClass,
+          payload.scheduler.schedulerInfo, "root", requestType)
+      ];
+
+      return normalizedArrayResponse;
+    }
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-queue/yarn-queue.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-queue/yarn-queue.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-queue/yarn-queue.js
new file mode 100644
index 0000000..b2e0f2f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-queue/yarn-queue.js
@@ -0,0 +1,47 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import DS from 'ember-data';
+
+export default DS.JSONAPISerializer.extend({
+
+    normalizeSingleResponse(store, primaryModelClass, payload, id,
+      requestType) {
+
+      var fixedPayload = {
+        id: id,
+        type: primaryModelClass.modelName,
+        attributes: {
+          type: payload.type.split(/(?=[A-Z])/)[0]
+        }
+      };
+      return this._super(store, primaryModelClass, fixedPayload, id,
+        requestType);
+    },
+
+    normalizeArrayResponse(store, primaryModelClass, payload, id, requestType) {
+      var normalizedArrayResponse = {};
+
+      normalizedArrayResponse.data = [
+        this.normalizeSingleResponse(store, primaryModelClass,
+          payload.scheduler.schedulerInfo, "root", requestType)
+        ];
+
+      return normalizedArrayResponse;
+    }
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/queue-configuration-table.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/queue-configuration-table.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/queue-configuration-table.hbs
deleted file mode 100644
index 17a1e1a..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/queue-configuration-table.hbs
+++ /dev/null
@@ -1,54 +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.
-}}
-
-<table id="queue-configuration-table" class="table table-striped table-bordered" cellspacing="0" width="100%" height="100%">
-  <thead>
-    <tr>
-      <td><b>Configurations</b></td>
-      <td>Value</td>
-    </tr>
-  </thead>
-  <tbody>
-    <tr>
-      <td>Configured Capacity</td>
-      <td>{{queue.capacity}}</td>
-    </tr>
-    <tr>
-      <td>Configured Max Capacity</td>
-      <td>{{queue.maxCapacity}}</td>
-    </tr>
-    <tr>
-      <td>State</td>
-      <td>{{queue.state}}</td>
-    </tr>
-  {{#if queue.isLeafQueue}}
-    <tr>
-      <td>User Limit Percent</td>
-      <td>{{queue.userLimit}}</td>
-    </tr>
-    <tr>
-      <td>User Limit Factor</td>
-      <td>{{queue.userLimitFactor}}</td>
-    </tr>
-    <tr>
-      <td>Preemption Disabled</td>
-      <td>{{queue.preemptionDisabled}}</td>
-    </tr>
-  {{/if}}
-  </tbody>
-</table>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/queue-navigator.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/queue-navigator.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/queue-navigator.hbs
index d8dd236..e3b0a90 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/queue-navigator.hbs
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/queue-navigator.hbs
@@ -20,9 +20,12 @@
 <div class="row">
   <div class="col-md-12 container-fluid">
     <div class="panel panel-default" id="tree-selector-container">
-     {{tree-selector model=model parentId="tree-selector-container" selected=selected}}
+      <div class="panel-heading">
+        Scheduler: {{model.firstObject.type}}
+      </div>
+     {{tree-selector model=model parentId="tree-selector-container" selected=selected used=used max=max}}
     </div>
   </div>
 </div>
 
-{{outlet}}
\ No newline at end of file
+{{outlet}}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/capacity-queue-conf-table.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/capacity-queue-conf-table.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/capacity-queue-conf-table.hbs
new file mode 100644
index 0000000..3f6017f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/capacity-queue-conf-table.hbs
@@ -0,0 +1,54 @@
+{{!
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+}}
+
+<table id="queue-configuration-table" class="table table-striped table-bordered" cellspacing="0" width="100%" height="100%">
+  <thead>
+    <tr>
+      <td><b>Configurations</b></td>
+      <td>Value</td>
+    </tr>
+  </thead>
+  <tbody>
+    <tr>
+      <td>Configured Capacity</td>
+      <td>{{queue.capacity}}</td>
+    </tr>
+    <tr>
+      <td>Configured Max Capacity</td>
+      <td>{{queue.maxCapacity}}</td>
+    </tr>
+    <tr>
+      <td>State</td>
+      <td>{{queue.state}}</td>
+    </tr>
+  {{#if queue.isLeafQueue}}
+    <tr>
+      <td>User Limit Percent</td>
+      <td>{{queue.userLimit}}</td>
+    </tr>
+    <tr>
+      <td>User Limit Factor</td>
+      <td>{{queue.userLimitFactor}}</td>
+    </tr>
+    <tr>
+      <td>Preemption Disabled</td>
+      <td>{{queue.preemptionDisabled}}</td>
+    </tr>
+  {{/if}}
+  </tbody>
+</table>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/capacity-queue-info.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/capacity-queue-info.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/capacity-queue-info.hbs
new file mode 100644
index 0000000..7d44e69
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/capacity-queue-info.hbs
@@ -0,0 +1,84 @@
+{{!
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+}}
+
+<div class="row">
+
+  <div class="col-lg-6 container-fluid">
+    <div class="panel panel-default">
+      <div class="panel-heading">
+        Queue Capacities: {{model.selected}}
+      </div>
+      <div class="container-fluid" id="capacity-bar-chart">
+        <br/>
+        {{bar-chart data=model.selectedQueue.capacitiesBarChartData
+        title=""
+        parentId="capacity-bar-chart"
+        textWidth=170
+        ratio=0.55
+        maxHeight=350}}
+      </div>
+    </div>
+  </div>
+
+  <div class="col-lg-6 container-fluid">
+    <div class="panel panel-default">
+      <div class="panel-heading">
+        Queue Information: {{model.selected}}
+      </div>
+        {{yarn-queue.capacity-queue-conf-table queue=model.selectedQueue}}
+    </div>
+  </div>
+
+</div>
+
+<div class="row">
+
+    <div class="col-lg-6 container-fluid">
+      <div class="panel panel-default">
+        <div class="panel-heading">
+          Running Apps: {{model.selected}}
+        </div>
+        <div class="container-fluid" id="numapplications-donut-chart">
+          {{donut-chart data=model.selectedQueue.numOfApplicationsDonutChartData
+          showLabels=true
+          parentId="numapplications-donut-chart"
+          ratio=0.6
+          maxHeight=350}}
+        </div>
+      </div>
+    </div>
+
+  {{#if model.selectedQueue.hasUserUsages}}
+    <div class="col-lg-6 container-fluid">
+      <div class="panel panel-default">
+        <div class="panel-heading">
+          User Usages: {{model.selected}}
+        </div>
+        <div class="container-fluid"  id="userusage-donut-chart">
+          {{donut-chart data=model.selectedQueue.userUsagesDonutChartData
+          showLabels=true
+          parentId="userusage-donut-chart"
+          type="memory"
+          ratio=0.6
+          maxHeight=350}}
+        </div>
+      </div>
+    </div>
+  {{/if}}
+
+</div>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/capacity-queue.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/capacity-queue.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/capacity-queue.hbs
new file mode 100644
index 0000000..8b63b66
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/capacity-queue.hbs
@@ -0,0 +1,63 @@
+{{!
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+}}
+
+{{queue-navigator model=model.queues selected=model.selected
+  used="usedCapacity" max="absMaxCapacity"}}
+
+<div class="row">
+  <div class="col-lg-4 container-fluid">
+    <div class="panel panel-default">
+      <div class="panel-heading">
+        Queue Information: {{model.selected}}
+      </div>
+        {{yarn-queue.capacity-queue-conf-table queue=model.selectedQueue}}
+    </div>
+  </div>
+
+  <div class="col-lg-4 container-fluid">
+    <div class="panel panel-default">
+      <div class="panel-heading">
+        Queue Capacities: {{model.selected}}
+      </div>
+      <div class="container-fluid" id="capacity-bar-chart">
+        <br/>
+        {{bar-chart data=model.selectedQueue.capacitiesBarChartData
+        title=""
+        parentId="capacity-bar-chart"
+        textWidth=175
+        ratio=0.55
+        maxHeight=350}}
+      </div>
+    </div>
+  </div>
+
+  <div class="col-lg-4 container-fluid">
+    <div class="panel panel-default">
+      <div class="panel-heading">
+        Running Apps: {{model.selected}}
+      </div>
+      <div class="container-fluid" id="numapplications-donut-chart">
+        {{donut-chart data=model.selectedQueue.numOfApplicationsDonutChartData
+        showLabels=true
+        parentId="numapplications-donut-chart"
+        ratio=0.6
+        maxHeight=350}}
+      </div>
+    </div>
+  </div>
+</div>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fair-queue-conf-table.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fair-queue-conf-table.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fair-queue-conf-table.hbs
new file mode 100644
index 0000000..00fabcc
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fair-queue-conf-table.hbs
@@ -0,0 +1,52 @@
+{{!
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+}}
+
+<table id="queue-configuration-table" class="table table-striped table-bordered" cellspacing="0" width="100%" height="100%">
+  <thead>
+    <tr>
+      <td><b>Configurations</b></td>
+      <td>Value</td>
+    </tr>
+  </thead>
+  <tbody>
+    <tr>
+      <td>Fair Memory, VCores</td>
+      <td>{{queue.fairResources.memory}} MB, {{queue.fairResources.vCores}}</td>
+    </tr>
+    <tr>
+      <td>Minimum Memory, VCores</td>
+      <td>{{queue.minResources.memory}} MB, {{queue.minResources.vCores}}</td>
+    </tr>
+    <tr>
+      <td>Cluster Memory, VCores</td>
+      <td>{{queue.clusterResources.memory}} MB, {{queue.clusterResources.vCores}}</td>
+    </tr>
+    <tr>
+      <td>Pending, Allocated, Reserved Containers</td>
+      <td>{{queue.pendingContainers}} , {{queue.allocatedContainers}} , {{queue.reservedContainers}}</td>
+    </tr>
+    <tr>
+      <td>Scheduling Policy</td>
+      <td>{{queue.schedulingPolicy}}</td>
+    </tr>
+    <tr>
+      <td>Preemption Enabled</td>
+      <td>{{queue.preemptable}}</td>
+    </tr>
+  </tbody>
+</table>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fair-queue-info.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fair-queue-info.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fair-queue-info.hbs
new file mode 100644
index 0000000..a770bfe
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fair-queue-info.hbs
@@ -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.
+}}
+
+<div class="row">
+
+  <div class="col-lg-6 container-fluid">
+    <div class="panel panel-default">
+      <div class="panel-heading">
+        Queue Capacities: {{model.selected}}
+      </div>
+      <div class="container-fluid" id="capacity-bar-chart">
+        <br/>
+        {{bar-chart data=model.selectedQueue.capacitiesBarChartData
+        title=""
+        parentId="capacity-bar-chart"
+        textWidth=170
+        ratio=0.55
+        maxHeight=350}}
+      </div>
+    </div>
+  </div>
+
+  <div class="col-lg-6 container-fluid">
+    <div class="panel panel-default">
+      <div class="panel-heading">
+        Queue Information: {{model.selected}}
+      </div>
+        {{yarn-queue.fair-queue-conf-table queue=model.selectedQueue}}
+    </div>
+  </div>
+
+</div>
+
+<div class="row">
+
+    <div class="col-lg-6 container-fluid">
+      <div class="panel panel-default">
+        <div class="panel-heading">
+          Running Apps: {{model.selected}}
+        </div>
+        <div class="container-fluid" id="numapplications-donut-chart">
+          {{donut-chart data=model.selectedQueue.numOfApplicationsDonutChartData
+          showLabels=true
+          parentId="numapplications-donut-chart"
+          ratio=0.6
+          maxHeight=350}}
+        </div>
+      </div>
+    </div>
+
+</div>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fair-queue.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fair-queue.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fair-queue.hbs
new file mode 100644
index 0000000..0341108
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fair-queue.hbs
@@ -0,0 +1,63 @@
+{{!
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+}}
+
+{{queue-navigator model=model.queues selected=model.selected
+  used="usedResources.memory" max="clusterResources.memory"}}
+
+<div class="row">
+  <div class="col-lg-4 container-fluid">
+    <div class="panel panel-default">
+      <div class="panel-heading">
+        Queue Information: {{model.selected}}
+      </div>
+        {{yarn-queue.fair-queue-conf-table queue=model.selectedQueue}}
+    </div>
+  </div>
+
+  <div class="col-lg-4 container-fluid">
+    <div class="panel panel-default">
+      <div class="panel-heading">
+        Queue Capacities: {{model.selected}}
+      </div>
+      <div class="container-fluid" id="capacity-bar-chart">
+        <br/>
+        {{bar-chart data=model.selectedQueue.capacitiesBarChartData
+        title=""
+        parentId="capacity-bar-chart"
+        textWidth=150
+        ratio=0.55
+        maxHeight=350}}
+      </div>
+    </div>
+  </div>
+
+  <div class="col-lg-4 container-fluid">
+    <div class="panel panel-default">
+      <div class="panel-heading">
+        Running Apps: {{model.selected}}
+      </div>
+      <div class="container-fluid" id="numapplications-donut-chart">
+        {{donut-chart data=model.selectedQueue.numOfApplicationsDonutChartData
+        showLabels=true
+        parentId="numapplications-donut-chart"
+        ratio=0.6
+        maxHeight=350}}
+      </div>
+    </div>
+  </div>
+</div>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fifo-queue-conf-table.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fifo-queue-conf-table.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fifo-queue-conf-table.hbs
new file mode 100644
index 0000000..4ced3e7
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fifo-queue-conf-table.hbs
@@ -0,0 +1,56 @@
+{{!
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+}}
+
+<table id="queue-configuration-table" class="table table-striped table-bordered" cellspacing="0" width="100%" height="100%">
+  <thead>
+    <tr>
+      <td><b>Configurations</b></td>
+      <td>Value</td>
+    </tr>
+  </thead>
+  <tbody>
+    <tr>
+      <td>Configured Capacity</td>
+      <td>{{queue.capacity}}</td>
+    </tr>
+    <tr>
+      <td>Used Capacity</td>
+      <td>{{queue.usedCapacity}}</td>
+    </tr>
+    <tr>
+      <td>State</td>
+      <td>{{queue.state}}</td>
+    </tr>
+    <tr>
+      <td>Minimum Queue Memory Capacity</td>
+      <td>{{queue.minQueueMemoryCapacity}}</td>
+    </tr>
+    <tr>
+      <td>Maximum Queue Memory Capacity</td>
+      <td>{{queue.maxQueueMemoryCapacity}}</td>
+    </tr>
+    <tr>
+      <td>Number of Nodes</td>
+      <td>{{queue.numNodes}}</td>
+    </tr>
+    <tr>
+      <td>Number of Containers</td>
+      <td>{{queue.numContainers}}</td>
+    </tr>
+  </tbody>
+</table>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fifo-queue-info.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fifo-queue-info.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fifo-queue-info.hbs
new file mode 100644
index 0000000..7f4e8a7
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fifo-queue-info.hbs
@@ -0,0 +1,47 @@
+{{!
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+}}
+
+<div class="row">
+
+  <div class="col-lg-6 container-fluid">
+    <div class="panel panel-default">
+      <div class="panel-heading">
+        Queue Capacities: {{model.selected}}
+      </div>
+      <div class="container-fluid" id="capacity-bar-chart">
+        <br/>
+        {{bar-chart data=model.selectedQueue.capacitiesBarChartData
+        title=""
+        parentId="capacity-bar-chart"
+        textWidth=170
+        ratio=0.55
+        maxHeight=350}}
+      </div>
+    </div>
+  </div>
+
+  <div class="col-lg-6 container-fluid">
+    <div class="panel panel-default">
+      <div class="panel-heading">
+        Queue Information: {{model.selected}}
+      </div>
+        {{yarn-queue.fifo-queue-conf-table queue=model.selectedQueue}}
+    </div>
+  </div>
+
+</div>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fifo-queue.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fifo-queue.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fifo-queue.hbs
new file mode 100644
index 0000000..46d79f0
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/fifo-queue.hbs
@@ -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.
+}}
+
+{{queue-navigator model=model.queues selected=model.selected
+  used="usedNodeCapacity" max="totalNodeCapacity"}}
+
+<div class="row">
+  <div class="col-lg-6 container-fluid">
+    <div class="panel panel-default">
+      <div class="panel-heading">
+        Queue Information: {{model.selected}}
+      </div>
+        {{yarn-queue.fifo-queue-conf-table queue=model.selectedQueue}}
+    </div>
+  </div>
+
+  <div class="col-lg-6 container-fluid">
+    <div class="panel panel-default">
+      <div class="panel-heading">
+        Queue Capacities: {{model.selected}}
+      </div>
+      <div class="container-fluid" id="capacity-bar-chart">
+        <br/>
+        {{bar-chart data=model.selectedQueue.capacitiesBarChartData
+        title=""
+        parentId="capacity-bar-chart"
+        textWidth=150
+        ratio=0.55
+        maxHeight=350}}
+      </div>
+    </div>
+  </div>
+</div>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-queue/info.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-queue/info.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-queue/info.hbs
index c112ef9..2f138a7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-queue/info.hbs
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-queue/info.hbs
@@ -16,69 +16,10 @@
  * limitations under the License.
 }}
 
-<div class="row">
-
-  <div class="col-lg-6 container-fluid">
-    <div class="panel panel-default">
-      <div class="panel-heading">
-        Queue Capacities: {{model.selected}}
-      </div>
-      <div class="container-fluid" id="capacity-bar-chart">
-        <br/>
-        {{bar-chart data=model.selectedQueue.capacitiesBarChartData
-        title=""
-        parentId="capacity-bar-chart"
-        textWidth=170
-        ratio=0.55
-        maxHeight=350}}
-      </div>
-    </div>
-  </div>
-
-  <div class="col-lg-6 container-fluid">
-    <div class="panel panel-default">
-      <div class="panel-heading">
-        Queue Information: {{model.selected}}
-      </div>
-      {{queue-configuration-table queue=model.selectedQueue}}
-    </div>
-  </div>
-
-</div>
-
-<div class="row">
-
-  <div class="col-lg-6 container-fluid">
-    <div class="panel panel-default">
-      <div class="panel-heading">
-        Running Apps: {{model.selected}}
-      </div>
-      <div class="container-fluid" id="numapplications-donut-chart">
-        {{donut-chart data=model.selectedQueue.numOfApplicationsDonutChartData
-        showLabels=true
-        parentId="numapplications-donut-chart"
-        ratio=0.6
-        maxHeight=350}}
-      </div>
-    </div>
-  </div>
-
-  {{#if model.selectedQueue.hasUserUsages}}
-    <div class="col-lg-6 container-fluid">
-      <div class="panel panel-default">
-        <div class="panel-heading">
-          User Usages: {{model.selected}}
-        </div>
-        <div class="container-fluid"  id="userusage-donut-chart">
-          {{donut-chart data=model.selectedQueue.userUsagesDonutChartData
-          showLabels=true
-          parentId="userusage-donut-chart"
-          type="memory"
-          ratio=0.6
-          maxHeight=350}}
-        </div>
-      </div>
-    </div>
-  {{/if}}
-
-</div>
+{{#if (eq model.queues.firstObject.type "capacity")}}
+  {{yarn-queue.capacity-queue-info model=model}}
+{{else if (eq model.queues.firstObject.type "fair")}}
+  {{yarn-queue.fair-queue-info model=model}}
+{{else}}
+  {{yarn-queue.fifo-queue-info model=model}}
+{{/if}}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-queues.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-queues.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-queues.hbs
index 6dfb220..fccdb5b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-queues.hbs
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-queues.hbs
@@ -17,54 +17,14 @@
 }}
 
 {{breadcrumb-bar breadcrumbs=breadcrumbs}}
-
 <div class="container-fluid">
-  {{queue-navigator model=model.queues selected=model.selected}}
-
-  <div class="row">
-
-    <div class="col-lg-4 container-fluid">
-      <div class="panel panel-default">
-        <div class="panel-heading">
-          Queue Information: {{model.selected}}
-        </div>
-        {{queue-configuration-table queue=model.selectedQueue}}
-      </div>
-    </div>
-
-    <div class="col-lg-4 container-fluid">
-      <div class="panel panel-default">
-        <div class="panel-heading">
-          Queue Capacities: {{model.selected}}
-        </div>
-        <div class="container-fluid" id="capacity-bar-chart">
-          <br/>
-          {{bar-chart data=model.selectedQueue.capacitiesBarChartData
-          title=""
-          parentId="capacity-bar-chart"
-          textWidth=150
-          ratio=0.55
-          maxHeight=350}}
-        </div>
-      </div>
-    </div>
-
-    <div class="col-lg-4 container-fluid">
-      <div class="panel panel-default">
-        <div class="panel-heading">
-          Running Apps: {{model.selected}}
-        </div>
-        <div class="container-fluid" id="numapplications-donut-chart">
-          {{donut-chart data=model.selectedQueue.numOfApplicationsDonutChartData
-          showLabels=true
-          parentId="numapplications-donut-chart"
-          ratio=0.6
-          maxHeight=350}}
-        </div>
-      </div>
-    </div>
-
-  </div>
+  {{#if (eq model.queues.firstObject.type "capacity")}}
+    {{yarn-queue.capacity-queue model=model}}
+  {{else if (eq model.queues.firstObject.type "fair")}}
+    {{yarn-queue.fair-queue model=model}}
+  {{else}}
+    {{yarn-queue.fifo-queue model=model}}
+  {{/if}}
 </div>
 
 {{outlet}}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dadb0c22/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/utils/color-utils.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/utils/color-utils.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/utils/color-utils.js
index 6c0cfee..af0cdf4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/utils/color-utils.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/utils/color-utils.js
@@ -55,7 +55,6 @@ export default {
       }
     }
 
-    console.log(colors);
     return colors;
   },
 


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


[28/50] [abbrv] hadoop git commit: YARN-7014. Fix off-by-one error causing heap corruption (Jason Lowe via nroberts)

Posted by jh...@apache.org.
YARN-7014. Fix off-by-one error causing heap corruption (Jason Lowe via nroberts)


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

Branch: refs/heads/YARN-5734
Commit: d265459024b8e5f5eccf421627f684ca8f162112
Parents: dadb0c2
Author: Nathan Roberts <nr...@apache.org>
Authored: Tue Aug 15 15:52:48 2017 -0500
Committer: Nathan Roberts <nr...@apache.org>
Committed: Tue Aug 15 15:52:48 2017 -0500

----------------------------------------------------------------------
 .../src/main/native/container-executor/impl/utils/string-utils.c  | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d2654590/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/string-utils.c
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/string-utils.c b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/string-utils.c
index 703d484..063df7e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/string-utils.c
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/string-utils.c
@@ -44,8 +44,7 @@ int validate_container_id(const char* input) {
    * container_e17_1410901177871_0001_01_000005
    * container_1410901177871_0001_01_000005
    */
-  char* input_cpy = malloc(strlen(input));
-  strcpy(input_cpy, input);
+  char* input_cpy = strdup(input);
   char* p = strtok(input_cpy, "_");
   int idx = 0;
   while (p != NULL) {


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


[12/50] [abbrv] hadoop git commit: YARN-6959. RM may allocate wrong AM Container for new attempt. Contributed by Yuqi Wang

Posted by jh...@apache.org.
YARN-6959. RM may allocate wrong AM Container for new attempt. Contributed by Yuqi Wang


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

Branch: refs/heads/YARN-5734
Commit: e2f6299f6f580d7a03f2377d19ac85f55fd4e73b
Parents: ce797a1
Author: Jian He <ji...@apache.org>
Authored: Mon Aug 14 10:51:04 2017 -0700
Committer: Jian He <ji...@apache.org>
Committed: Mon Aug 14 10:51:30 2017 -0700

----------------------------------------------------------------------
 .../scheduler/AbstractYarnScheduler.java        |  1 +
 .../scheduler/capacity/CapacityScheduler.java   | 13 ++++++
 .../scheduler/fair/FairScheduler.java           | 15 ++++++-
 .../scheduler/fifo/FifoScheduler.java           | 15 ++++++-
 .../scheduler/fair/TestFairScheduler.java       | 46 ++++++++++----------
 5 files changed, 63 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e2f6299f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.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/AbstractYarnScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
index d506f4d..79caab0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
@@ -323,6 +323,7 @@ public abstract class AbstractYarnScheduler
 
   }
 
+  // TODO: Rename it to getCurrentApplicationAttempt
   public T getApplicationAttempt(ApplicationAttemptId applicationAttemptId) {
     SchedulerApplication<T> app = applications.get(
         applicationAttemptId.getApplicationId());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e2f6299f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.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/CapacityScheduler.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/CapacityScheduler.java
index 3286982..e4ca003 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/CapacityScheduler.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/CapacityScheduler.java
@@ -903,6 +903,19 @@ public class CapacityScheduler extends
       ContainerUpdates updateRequests) {
     FiCaSchedulerApp application = getApplicationAttempt(applicationAttemptId);
     if (application == null) {
+      LOG.error("Calling allocate on removed or non existent application " +
+          applicationAttemptId.getApplicationId());
+      return EMPTY_ALLOCATION;
+    }
+
+    // The allocate may be the leftover from previous attempt, and it will
+    // impact current attempt, such as confuse the request and allocation for
+    // current attempt's AM container.
+    // Note outside precondition check for the attempt id may be
+    // outdated here, so double check it here is necessary.
+    if (!application.getApplicationAttemptId().equals(applicationAttemptId)) {
+      LOG.error("Calling allocate on previous or removed " +
+          "or non existent application attempt " + applicationAttemptId);
       return EMPTY_ALLOCATION;
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e2f6299f/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 db02bab..0f417c3 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
@@ -835,8 +835,19 @@ public class FairScheduler extends
     // Make sure this application exists
     FSAppAttempt application = getSchedulerApp(appAttemptId);
     if (application == null) {
-      LOG.info("Calling allocate on removed " +
-          "or non existent application " + appAttemptId);
+      LOG.error("Calling allocate on removed or non existent application " +
+          appAttemptId.getApplicationId());
+      return EMPTY_ALLOCATION;
+    }
+
+    // The allocate may be the leftover from previous attempt, and it will
+    // impact current attempt, such as confuse the request and allocation for
+    // current attempt's AM container.
+    // Note outside precondition check for the attempt id may be
+    // outdated here, so double check it here is necessary.
+    if (!application.getApplicationAttemptId().equals(appAttemptId)) {
+      LOG.error("Calling allocate on previous or removed " +
+          "or non existent application attempt " + appAttemptId);
       return EMPTY_ALLOCATION;
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e2f6299f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.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/fifo/FifoScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java
index a8d4f48..92a88b9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java
@@ -329,8 +329,19 @@ public class FifoScheduler extends
       ContainerUpdates updateRequests) {
     FifoAppAttempt application = getApplicationAttempt(applicationAttemptId);
     if (application == null) {
-      LOG.error("Calling allocate on removed " +
-          "or non-existent application " + applicationAttemptId);
+      LOG.error("Calling allocate on removed or non existent application " +
+          applicationAttemptId.getApplicationId());
+      return EMPTY_ALLOCATION;
+    }
+
+    // The allocate may be the leftover from previous attempt, and it will
+    // impact current attempt, such as confuse the request and allocation for
+    // current attempt's AM container.
+    // Note outside precondition check for the attempt id may be
+    // outdated here, so double check it here is necessary.
+    if (!application.getApplicationAttemptId().equals(applicationAttemptId)) {
+      LOG.error("Calling allocate on previous or removed " +
+          "or non existent application attempt " + applicationAttemptId);
       return EMPTY_ALLOCATION;
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e2f6299f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.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/scheduler/fair/TestFairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
index 0d54c33..941c215 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
@@ -2107,49 +2107,49 @@ public class TestFairScheduler extends FairSchedulerTestBase {
     scheduler.init(conf);
     scheduler.start();
     scheduler.reinitialize(conf, resourceManager.getRMContext());
+    int minReqSize =
+        FairSchedulerConfiguration.DEFAULT_RM_SCHEDULER_INCREMENT_ALLOCATION_MB;
 
+    // First ask, queue1 requests 1 large (minReqSize * 2).
     ApplicationAttemptId id11 = createAppAttemptId(1, 1);
     createMockRMApp(id11);
-    scheduler.addApplication(id11.getApplicationId(), "root.queue1", "user1", false);
+    scheduler.addApplication(id11.getApplicationId(),
+        "root.queue1", "user1", false);
     scheduler.addApplicationAttempt(id11, false, false);
-    ApplicationAttemptId id21 = createAppAttemptId(2, 1);
-    createMockRMApp(id21);
-    scheduler.addApplication(id21.getApplicationId(), "root.queue2", "user1", false);
-    scheduler.addApplicationAttempt(id21, false, false);
-    ApplicationAttemptId id22 = createAppAttemptId(2, 2);
-    createMockRMApp(id22);
-
-    scheduler.addApplication(id22.getApplicationId(), "root.queue2", "user1", false);
-    scheduler.addApplicationAttempt(id22, false, false);
-
-    int minReqSize = 
-        FairSchedulerConfiguration.DEFAULT_RM_SCHEDULER_INCREMENT_ALLOCATION_MB;
-    
-    // First ask, queue1 requests 1 large (minReqSize * 2).
     List<ResourceRequest> ask1 = new ArrayList<ResourceRequest>();
-    ResourceRequest request1 =
-        createResourceRequest(minReqSize * 2, ResourceRequest.ANY, 1, 1, true);
+    ResourceRequest request1 = createResourceRequest(minReqSize * 2,
+        ResourceRequest.ANY, 1, 1, true);
     ask1.add(request1);
     scheduler.allocate(id11, ask1, new ArrayList<ContainerId>(),
         null, null, NULL_UPDATE_REQUESTS);
 
     // Second ask, queue2 requests 1 large.
+    ApplicationAttemptId id21 = createAppAttemptId(2, 1);
+    createMockRMApp(id21);
+    scheduler.addApplication(id21.getApplicationId(),
+        "root.queue2", "user1", false);
+    scheduler.addApplicationAttempt(id21, false, false);
     List<ResourceRequest> ask2 = new ArrayList<ResourceRequest>();
-    ResourceRequest request2 = createResourceRequest(2 * minReqSize, "foo", 1, 1,
-        false);
+    ResourceRequest request2 = createResourceRequest(2 * minReqSize,
+        "foo", 1, 1, false);
     ResourceRequest request3 = createResourceRequest(2 * minReqSize,
-            ResourceRequest.ANY, 1, 1, false);
+        ResourceRequest.ANY, 1, 1, false);
     ask2.add(request2);
     ask2.add(request3);
     scheduler.allocate(id21, ask2, new ArrayList<ContainerId>(),
         null, null, NULL_UPDATE_REQUESTS);
 
     // Third ask, queue2 requests 2 small (minReqSize).
+    ApplicationAttemptId id22 = createAppAttemptId(2, 2);
+    createMockRMApp(id22);
+    scheduler.addApplication(id22.getApplicationId(),
+        "root.queue2", "user1", false);
+    scheduler.addApplicationAttempt(id22, false, false);
     List<ResourceRequest> ask3 = new ArrayList<ResourceRequest>();
-    ResourceRequest request4 = createResourceRequest(minReqSize, "bar", 2, 2,
-            true);
+    ResourceRequest request4 = createResourceRequest(minReqSize,
+        "bar", 2, 2, true);
     ResourceRequest request5 = createResourceRequest(minReqSize,
-            ResourceRequest.ANY, 2, 2, true);
+        ResourceRequest.ANY, 2, 2, true);
     ask3.add(request4);
     ask3.add(request5);
     scheduler.allocate(id22, ask3, new ArrayList<ContainerId>(),


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


[11/50] [abbrv] hadoop git commit: HDFS-12221. Replace xcerces in XmlEditsVisitor. (Ajay Kumar via lei)

Posted by jh...@apache.org.
HDFS-12221. Replace xcerces in XmlEditsVisitor. (Ajay Kumar via lei)


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

Branch: refs/heads/YARN-5734
Commit: ce797a170669524224cfeaaf70647047e7626816
Parents: d8f74c3
Author: Lei Xu <le...@apache.org>
Authored: Mon Aug 14 10:27:47 2017 -0700
Committer: Lei Xu <le...@apache.org>
Committed: Mon Aug 14 10:27:47 2017 -0700

----------------------------------------------------------------------
 .../hadoop-client-minicluster/pom.xml           |   6 --
 .../hadoop-client-runtime/pom.xml               |   7 ---
 hadoop-hdfs-project/hadoop-hdfs/pom.xml         |   5 --
 .../offlineEditsViewer/XmlEditsVisitor.java     |  41 ++++++++----
 .../hadoop-hdfs/src/test/resources/editsStored  | Bin 5850 -> 5850 bytes
 .../src/test/resources/editsStored.xml          |  62 +++++++++----------
 .../hadoop-mapreduce-client/pom.xml             |  10 +--
 hadoop-project-dist/pom.xml                     |  10 +--
 hadoop-project/pom.xml                          |   8 ---
 hadoop-yarn-project/hadoop-yarn/pom.xml         |  10 +--
 10 files changed, 62 insertions(+), 97 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ce797a17/hadoop-client-modules/hadoop-client-minicluster/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-client-modules/hadoop-client-minicluster/pom.xml b/hadoop-client-modules/hadoop-client-minicluster/pom.xml
index 5255640..5cf1fad 100644
--- a/hadoop-client-modules/hadoop-client-minicluster/pom.xml
+++ b/hadoop-client-modules/hadoop-client-minicluster/pom.xml
@@ -629,12 +629,6 @@
                       </excludes>
                     </filter>
                     <filter>
-                      <artifact>xerces:xercesImpl</artifact>
-                      <excludes>
-                        <exclude>**/*</exclude>
-                      </excludes>
-                    </filter>
-                    <filter>
                       <artifact>org.apache.hadoop:hadoop-mapreduce-client-jobclient:*</artifact>
                       <excludes>
                         <exclude>testjar/*</exclude>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ce797a17/hadoop-client-modules/hadoop-client-runtime/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-client-modules/hadoop-client-runtime/pom.xml b/hadoop-client-modules/hadoop-client-runtime/pom.xml
index 2f64152..24c6b7a 100644
--- a/hadoop-client-modules/hadoop-client-runtime/pom.xml
+++ b/hadoop-client-modules/hadoop-client-runtime/pom.xml
@@ -174,13 +174,6 @@
                         <exclude>org/apache/jasper/compiler/Localizer.class</exclude>
                       </excludes>
                     </filter>
-                    <!-- We only have xerces as a dependency for XML output for the fsimage edits, we don't need anything specific to it for javax xml support -->
-                    <filter>
-                      <artifact>xerces:xercesImpl</artifact>
-                      <excludes>
-                        <exclude>META-INF/services/*</exclude>
-                      </excludes>
-                    </filter>
                     <!-- We rely on jersey for our web interfaces. We want to use its java services stuff only internal to jersey -->
                     <filter>
                       <artifact>com.sun.jersey:*</artifact>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ce797a17/hadoop-hdfs-project/hadoop-hdfs/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/pom.xml b/hadoop-hdfs-project/hadoop-hdfs/pom.xml
index 1c50d31..fa1044d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/pom.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/pom.xml
@@ -174,11 +174,6 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
       <scope>compile</scope>
     </dependency>
     <dependency>
-      <groupId>xerces</groupId>
-      <artifactId>xercesImpl</artifactId>
-      <scope>compile</scope>
-    </dependency>
-    <dependency>
       <groupId>org.apache.htrace</groupId>
       <artifactId>htrace-core4</artifactId>
     </dependency>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ce797a17/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/XmlEditsVisitor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/XmlEditsVisitor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/XmlEditsVisitor.java
index 7a39ba6..ddf7933 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/XmlEditsVisitor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/XmlEditsVisitor.java
@@ -20,17 +20,21 @@ package org.apache.hadoop.hdfs.tools.offlineEditsViewer;
 import java.io.IOException;
 import java.io.OutputStream;
 
+import javax.xml.transform.OutputKeys;
+import javax.xml.transform.TransformerConfigurationException;
+import javax.xml.transform.sax.SAXTransformerFactory;
+import javax.xml.transform.sax.TransformerHandler;
+import javax.xml.transform.stream.StreamResult;
+import org.xml.sax.ContentHandler;
+import org.xml.sax.SAXException;
+import org.xml.sax.helpers.AttributesImpl;
+
 import org.apache.hadoop.hdfs.util.XMLUtils;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.xml.sax.ContentHandler;
-import org.xml.sax.SAXException;
-import org.xml.sax.helpers.AttributesImpl;
 
-import org.apache.xml.serialize.OutputFormat;
-import org.apache.xml.serialize.XMLSerializer;
 
 /**
  * An XmlEditsVisitor walks over an EditLog structure and writes out
@@ -41,26 +45,37 @@ import org.apache.xml.serialize.XMLSerializer;
 public class XmlEditsVisitor implements OfflineEditsVisitor {
   private final OutputStream out;
   private ContentHandler contentHandler;
+  private final SAXTransformerFactory factory;
+  private final static String XML_INDENTATION_PROP ="{http://xml.apache.org/" +
+          "xslt}indent-amount";
+  private final static String XML_INDENTATION_NUM ="2";
 
   /**
    * Create a processor that writes to the file named and may or may not
    * also output to the screen, as specified.
    *
-   * @param filename Name of file to write output to
-   * @param printToScreen Mirror output to screen?
+   * @param out output stream to write
+   * @throws IOException on any error
    */
   public XmlEditsVisitor(OutputStream out)
       throws IOException {
     this.out = out;
-    OutputFormat outFormat = new OutputFormat("XML", "UTF-8", true);
-    outFormat.setIndenting(true);
-    outFormat.setIndent(2);
-    outFormat.setDoctype(null, null);
-    XMLSerializer serializer = new XMLSerializer(out, outFormat);
-    contentHandler = serializer.asContentHandler();
+    factory =(SAXTransformerFactory)SAXTransformerFactory.newInstance();
     try {
+      TransformerHandler handler = factory.newTransformerHandler();
+      handler.getTransformer().setOutputProperty(OutputKeys.METHOD, "xml");
+      handler.getTransformer().setOutputProperty(OutputKeys.ENCODING, "UTF-8");
+      handler.getTransformer().setOutputProperty(OutputKeys.INDENT, "yes");
+      handler.getTransformer().setOutputProperty(XML_INDENTATION_PROP,
+              XML_INDENTATION_NUM);
+      handler.getTransformer().setOutputProperty(OutputKeys.STANDALONE, "yes");
+      handler.setResult(new StreamResult(out));
+      contentHandler = handler;
+      
       contentHandler.startDocument();
       contentHandler.startElement("", "", "EDITS", new AttributesImpl());
+    } catch (TransformerConfigurationException e) {
+      throw new IOException("SAXTransformer error: " + e.getMessage());
     } catch (SAXException e) {
       throw new IOException("SAX error: " + e.getMessage());
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ce797a17/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored
index a32cd98..e271cb5 100644
Binary files a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored and b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored differ

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ce797a17/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml
index 201db9e..f901145 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml
@@ -1,6 +1,6 @@
-<?xml version="1.0" encoding="UTF-8"?>
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
 <EDITS>
-  <EDITS_VERSION>-63</EDITS_VERSION>
+  <EDITS_VERSION>-64</EDITS_VERSION>
   <RECORD>
     <OPCODE>OP_START_LOG_SEGMENT</OPCODE>
     <DATA>
@@ -63,8 +63,8 @@
       <MTIME>1422406380369</MTIME>
       <ATIME>1422406380345</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME></CLIENT_NAME>
-      <CLIENT_MACHINE></CLIENT_MACHINE>
+      <CLIENT_NAME/>
+      <CLIENT_MACHINE/>
       <OVERWRITE>false</OVERWRITE>
       <PERMISSION_STATUS>
         <USERNAME>xyao</USERNAME>
@@ -96,8 +96,8 @@
       <MTIME>1422406380376</MTIME>
       <ATIME>1422406380345</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME></CLIENT_NAME>
-      <CLIENT_MACHINE></CLIENT_MACHINE>
+      <CLIENT_NAME/>
+      <CLIENT_MACHINE/>
       <OVERWRITE>false</OVERWRITE>
       <PERMISSION_STATUS>
         <USERNAME>xyao</USERNAME>
@@ -238,8 +238,8 @@
       <MTIME>1422406380425</MTIME>
       <ATIME>1422406380423</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME></CLIENT_NAME>
-      <CLIENT_MACHINE></CLIENT_MACHINE>
+      <CLIENT_NAME/>
+      <CLIENT_MACHINE/>
       <OVERWRITE>false</OVERWRITE>
       <PERMISSION_STATUS>
         <USERNAME>xyao</USERNAME>
@@ -360,7 +360,7 @@
         <NUM_BYTES>0</NUM_BYTES>
         <GENSTAMP>1001</GENSTAMP>
       </BLOCK>
-      <RPC_CLIENTID></RPC_CLIENTID>
+      <RPC_CLIENTID/>
       <RPC_CALLID>-2</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -393,7 +393,7 @@
         <NUM_BYTES>0</NUM_BYTES>
         <GENSTAMP>1002</GENSTAMP>
       </BLOCK>
-      <RPC_CLIENTID></RPC_CLIENTID>
+      <RPC_CLIENTID/>
       <RPC_CALLID>-2</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -426,7 +426,7 @@
         <NUM_BYTES>0</NUM_BYTES>
         <GENSTAMP>1003</GENSTAMP>
       </BLOCK>
-      <RPC_CLIENTID></RPC_CLIENTID>
+      <RPC_CLIENTID/>
       <RPC_CALLID>-2</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -441,8 +441,8 @@
       <MTIME>1422406380534</MTIME>
       <ATIME>1422406380446</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME></CLIENT_NAME>
-      <CLIENT_MACHINE></CLIENT_MACHINE>
+      <CLIENT_NAME/>
+      <CLIENT_MACHINE/>
       <OVERWRITE>false</OVERWRITE>
       <BLOCK>
         <BLOCK_ID>1073741825</BLOCK_ID>
@@ -513,7 +513,7 @@
         <NUM_BYTES>0</NUM_BYTES>
         <GENSTAMP>1004</GENSTAMP>
       </BLOCK>
-      <RPC_CLIENTID></RPC_CLIENTID>
+      <RPC_CLIENTID/>
       <RPC_CALLID>-2</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -546,7 +546,7 @@
         <NUM_BYTES>0</NUM_BYTES>
         <GENSTAMP>1005</GENSTAMP>
       </BLOCK>
-      <RPC_CLIENTID></RPC_CLIENTID>
+      <RPC_CLIENTID/>
       <RPC_CALLID>-2</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -579,7 +579,7 @@
         <NUM_BYTES>0</NUM_BYTES>
         <GENSTAMP>1006</GENSTAMP>
       </BLOCK>
-      <RPC_CLIENTID></RPC_CLIENTID>
+      <RPC_CLIENTID/>
       <RPC_CALLID>-2</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -594,8 +594,8 @@
       <MTIME>1422406380558</MTIME>
       <ATIME>1422406380537</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME></CLIENT_NAME>
-      <CLIENT_MACHINE></CLIENT_MACHINE>
+      <CLIENT_NAME/>
+      <CLIENT_MACHINE/>
       <OVERWRITE>false</OVERWRITE>
       <BLOCK>
         <BLOCK_ID>1073741828</BLOCK_ID>
@@ -666,7 +666,7 @@
         <NUM_BYTES>0</NUM_BYTES>
         <GENSTAMP>1007</GENSTAMP>
       </BLOCK>
-      <RPC_CLIENTID></RPC_CLIENTID>
+      <RPC_CLIENTID/>
       <RPC_CALLID>-2</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -699,7 +699,7 @@
         <NUM_BYTES>0</NUM_BYTES>
         <GENSTAMP>1008</GENSTAMP>
       </BLOCK>
-      <RPC_CLIENTID></RPC_CLIENTID>
+      <RPC_CLIENTID/>
       <RPC_CALLID>-2</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -732,7 +732,7 @@
         <NUM_BYTES>0</NUM_BYTES>
         <GENSTAMP>1009</GENSTAMP>
       </BLOCK>
-      <RPC_CLIENTID></RPC_CLIENTID>
+      <RPC_CLIENTID/>
       <RPC_CALLID>-2</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -747,8 +747,8 @@
       <MTIME>1422406380579</MTIME>
       <ATIME>1422406380560</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME></CLIENT_NAME>
-      <CLIENT_MACHINE></CLIENT_MACHINE>
+      <CLIENT_NAME/>
+      <CLIENT_MACHINE/>
       <OVERWRITE>false</OVERWRITE>
       <BLOCK>
         <BLOCK_ID>1073741831</BLOCK_ID>
@@ -834,7 +834,7 @@
         <NUM_BYTES>0</NUM_BYTES>
         <GENSTAMP>1010</GENSTAMP>
       </BLOCK>
-      <RPC_CLIENTID></RPC_CLIENTID>
+      <RPC_CLIENTID/>
       <RPC_CALLID>-2</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -867,7 +867,7 @@
         <NUM_BYTES>0</NUM_BYTES>
         <GENSTAMP>1011</GENSTAMP>
       </BLOCK>
-      <RPC_CLIENTID></RPC_CLIENTID>
+      <RPC_CLIENTID/>
       <RPC_CALLID>-2</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -882,8 +882,8 @@
       <MTIME>1422406380599</MTIME>
       <ATIME>1422406380586</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME></CLIENT_NAME>
-      <CLIENT_MACHINE></CLIENT_MACHINE>
+      <CLIENT_NAME/>
+      <CLIENT_MACHINE/>
       <OVERWRITE>false</OVERWRITE>
       <BLOCK>
         <BLOCK_ID>1073741834</BLOCK_ID>
@@ -979,7 +979,7 @@
         <NUM_BYTES>0</NUM_BYTES>
         <GENSTAMP>1012</GENSTAMP>
       </BLOCK>
-      <RPC_CLIENTID></RPC_CLIENTID>
+      <RPC_CLIENTID/>
       <RPC_CALLID>-2</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -993,7 +993,7 @@
         <NUM_BYTES>11</NUM_BYTES>
         <GENSTAMP>1012</GENSTAMP>
       </BLOCK>
-      <RPC_CLIENTID></RPC_CLIENTID>
+      <RPC_CLIENTID/>
       <RPC_CALLID>-2</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -1024,8 +1024,8 @@
       <MTIME>1422406383261</MTIME>
       <ATIME>1422406380608</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME></CLIENT_NAME>
-      <CLIENT_MACHINE></CLIENT_MACHINE>
+      <CLIENT_NAME/>
+      <CLIENT_MACHINE/>
       <OVERWRITE>false</OVERWRITE>
       <BLOCK>
         <BLOCK_ID>1073741836</BLOCK_ID>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ce797a17/hadoop-mapreduce-project/hadoop-mapreduce-client/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/pom.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/pom.xml
index 4e7a0ae..212078b 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/pom.xml
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/pom.xml
@@ -238,14 +238,6 @@
                       <outputDirectory>${project.build.directory}</outputDirectory>
                       <destFileName>hadoop-annotations.jar</destFileName>
                     </artifactItem>
-                    <artifactItem>
-                      <groupId>xerces</groupId>
-                      <artifactId>xercesImpl</artifactId>
-                      <version>${xerces.version.jdiff}</version>
-                      <overWrite>false</overWrite>
-                      <outputDirectory>${project.build.directory}</outputDirectory>
-                      <destFileName>xerces.jar</destFileName>
-                    </artifactItem>
                   </artifactItems>
                 </configuration>
               </execution>
@@ -283,7 +275,7 @@
                            sourceFiles="${dev-support.relative.dir}/jdiff/Null.java"
                            maxmemory="${jdiff.javadoc.maxmemory}">
                     <doclet name="org.apache.hadoop.classification.tools.IncludePublicAnnotationsJDiffDoclet"
-                            path="${project.build.directory}/hadoop-annotations.jar:${project.build.directory}/jdiff.jar:${project.build.directory}/xerces.jar">
+                            path="${project.build.directory}/hadoop-annotations.jar:${project.build.directory}/jdiff.jar">
                       <param name="-oldapi" value="${project.name} ${jdiff.stable.api}"/>
                       <param name="-newapi" value="${project.name} ${project.version}"/>
                       <param name="-oldapidir" value="${basedir}/${dev-support.relative.dir}/jdiff"/>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ce797a17/hadoop-project-dist/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project-dist/pom.xml b/hadoop-project-dist/pom.xml
index 9da5e53..cf43218 100644
--- a/hadoop-project-dist/pom.xml
+++ b/hadoop-project-dist/pom.xml
@@ -194,14 +194,6 @@
                       <outputDirectory>${project.build.directory}</outputDirectory>
                       <destFileName>hadoop-annotations.jar</destFileName>
                     </artifactItem>
-                    <artifactItem>
-                      <groupId>xerces</groupId>
-                      <artifactId>xercesImpl</artifactId>
-                      <version>${xerces.jdiff.version}</version>
-                      <overWrite>false</overWrite>
-                      <outputDirectory>${project.build.directory}</outputDirectory>
-                      <destFileName>xerces.jar</destFileName>
-                    </artifactItem>
                   </artifactItems>
                 </configuration>
               </execution>
@@ -267,7 +259,7 @@
                              sourceFiles="${basedir}/dev-support/jdiff/Null.java"
                              maxmemory="${jdiff.javadoc.maxmemory}">
                       <doclet name="org.apache.hadoop.classification.tools.IncludePublicAnnotationsJDiffDoclet"
-                              path="${project.build.directory}/hadoop-annotations.jar:${project.build.directory}/jdiff.jar:${project.build.directory}/xerces.jar">
+                              path="${project.build.directory}/hadoop-annotations.jar:${project.build.directory}/jdiff.jar">
                         <param name="-oldapi" value="${project.name} ${jdiff.stable.api}"/>
                         <param name="-newapi" value="${project.name} ${project.version}"/>
                         <param name="-oldapidir" value="${basedir}/dev-support/jdiff"/>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ce797a17/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 8151016..6311cd9 100755
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -45,8 +45,6 @@
     <!-- These 2 versions are defined here because they are used -->
     <!-- JDIFF generation from embedded ant in the antrun plugin -->
     <jdiff.version>1.0.9</jdiff.version>
-    <!-- Version number for xerces used by JDiff -->
-    <xerces.jdiff.version>2.11.0</xerces.jdiff.version>
 
     <kafka.version>0.8.2.1</kafka.version>
     <hbase.version>1.2.6</hbase.version>
@@ -1156,12 +1154,6 @@
      </dependency>
 
      <dependency>
-       <groupId>xerces</groupId>
-       <artifactId>xercesImpl</artifactId>
-       <version>2.9.1</version>
-     </dependency>
-
-     <dependency>
        <groupId>org.apache.curator</groupId>
        <artifactId>curator-recipes</artifactId>
        <version>${curator.version}</version>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ce797a17/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 2e8ec69..12e4604 100644
--- a/hadoop-yarn-project/hadoop-yarn/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/pom.xml
@@ -155,14 +155,6 @@
                       <outputDirectory>${project.build.directory}</outputDirectory>
                       <destFileName>hadoop-annotations.jar</destFileName>
                     </artifactItem>
-                    <artifactItem>
-                      <groupId>xerces</groupId>
-                      <artifactId>xercesImpl</artifactId>
-                      <version>${xerces.version.jdiff}</version>
-                      <overWrite>false</overWrite>
-                      <outputDirectory>${project.build.directory}</outputDirectory>
-                      <destFileName>xerces.jar</destFileName>
-                    </artifactItem>
                   </artifactItems>
                 </configuration>
               </execution>
@@ -201,7 +193,7 @@
                       sourceFiles="${dev-support.relative.dir}/jdiff/Null.java"
                              maxmemory="${jdiff.javadoc.maxmemory}">
                       <doclet name="org.apache.hadoop.classification.tools.IncludePublicAnnotationsJDiffDoclet"
-                              path="${project.build.directory}/hadoop-annotations.jar:${project.build.directory}/jdiff.jar:${project.build.directory}/xerces.jar">
+                              path="${project.build.directory}/hadoop-annotations.jar:${project.build.directory}/jdiff.jar">
                         <param name="-oldapi" value="${project.name} ${jdiff.stable.api}"/>
                         <param name="-newapi" value="${project.name} ${project.version}"/>
                         <param name="-oldapidir" value="${basedir}/${dev-support.relative.dir}/jdiff"/>


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


[09/50] [abbrv] hadoop git commit: HADOOP-14627. Support MSI and DeviceCode token provider in ADLS. Contributed by Atul Sikaria.

Posted by jh...@apache.org.
HADOOP-14627. Support MSI and DeviceCode token provider in ADLS. Contributed by Atul Sikaria.


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

Branch: refs/heads/YARN-5734
Commit: 7769e9614956283a86eda9e4e69aaa592c0ca960
Parents: 8b242f0
Author: John Zhuge <jz...@cloudera.com>
Authored: Thu Aug 10 00:43:40 2017 -0700
Committer: John Zhuge <jz...@apache.org>
Committed: Sun Aug 13 00:22:34 2017 -0700

----------------------------------------------------------------------
 .../src/main/resources/core-default.xml         | 37 +++++++-
 hadoop-tools/hadoop-azure-datalake/pom.xml      |  2 +-
 .../org/apache/hadoop/fs/adl/AdlConfKeys.java   |  8 ++
 .../org/apache/hadoop/fs/adl/AdlFileSystem.java | 21 +++++
 .../apache/hadoop/fs/adl/TokenProviderType.java |  2 +
 .../src/site/markdown/index.md                  | 98 ++++++++++++++++++--
 .../hadoop/fs/adl/TestAzureADTokenProvider.java | 40 ++++++++
 7 files changed, 198 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7769e961/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 ffcab2c..7c4b0f1 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
@@ -2586,11 +2586,16 @@
     <value>ClientCredential</value>
     <description>
       Defines Azure Active Directory OAuth2 access token provider type.
-      Supported types are ClientCredential, RefreshToken, and Custom.
+      Supported types are ClientCredential, RefreshToken, MSI, DeviceCode,
+      and Custom.
       The ClientCredential type requires property fs.adl.oauth2.client.id,
       fs.adl.oauth2.credential, and fs.adl.oauth2.refresh.url.
       The RefreshToken type requires property fs.adl.oauth2.client.id and
       fs.adl.oauth2.refresh.token.
+      The MSI type requires properties fs.adl.oauth2.msi.port and
+      fs.adl.oauth2.msi.tenantguid.
+      The DeviceCode type requires property
+      fs.adl.oauth2.devicecode.clientapp.id.
       The Custom type requires property fs.adl.oauth2.access.token.provider.
     </description>
   </property>
@@ -2627,6 +2632,36 @@
     </description>
   </property>
 
+  <property>
+    <name>fs.adl.oauth2.msi.port</name>
+    <value></value>
+    <description>
+      The localhost port for the MSI token service. This is the port specified
+      when creating the Azure VM.
+      Used by MSI token provider.
+    </description>
+  </property>
+
+  <property>
+    <name>fs.adl.oauth2.msi.tenantguid</name>
+    <value></value>
+    <description>
+      The tenant guid for the Azure AAD tenant under which the azure data lake
+      store account is created.
+      Used by MSI token provider.
+    </description>
+  </property>
+
+  <property>
+    <name>fs.adl.oauth2.devicecode.clientapp.id</name>
+    <value></value>
+    <description>
+      The app id of the AAD native app in whose context the auth request
+      should be made.
+      Used by DeviceCode token provider.
+    </description>
+  </property>
+
   <!-- Azure Data Lake File System Configurations Ends Here-->
 
   <property>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7769e961/hadoop-tools/hadoop-azure-datalake/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/pom.xml b/hadoop-tools/hadoop-azure-datalake/pom.xml
index 3aed5e1..47f12df 100644
--- a/hadoop-tools/hadoop-azure-datalake/pom.xml
+++ b/hadoop-tools/hadoop-azure-datalake/pom.xml
@@ -110,7 +110,7 @@
     <dependency>
       <groupId>com.microsoft.azure</groupId>
       <artifactId>azure-data-lake-store-sdk</artifactId>
-      <version>2.1.4</version>
+      <version>2.2.1</version>
     </dependency>
     <!--  ENDS HERE-->
     <dependency>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7769e961/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlConfKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlConfKeys.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlConfKeys.java
index 31df222..f77d981 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlConfKeys.java
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlConfKeys.java
@@ -54,6 +54,14 @@ public final class AdlConfKeys {
   public static final String TOKEN_PROVIDER_TYPE_CLIENT_CRED =
       "ClientCredential";
 
+  // MSI Auth Configuration
+  public static final String MSI_PORT = "fs.adl.oauth2.msi.port";
+  public static final String MSI_TENANT_GUID = "fs.adl.oauth2.msi.tenantguid";
+
+  // DeviceCode Auth configuration
+  public static final String DEVICE_CODE_CLIENT_APP_ID =
+      "fs.adl.oauth2.devicecode.clientapp.id";
+
   public static final String READ_AHEAD_BUFFER_SIZE_KEY =
       "adl.feature.client.cache.readahead";
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7769e961/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlFileSystem.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlFileSystem.java
index 76ce43e..a5e31e1 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlFileSystem.java
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlFileSystem.java
@@ -34,6 +34,8 @@ import com.microsoft.azure.datalake.store.LatencyTracker;
 import com.microsoft.azure.datalake.store.UserGroupRepresentation;
 import com.microsoft.azure.datalake.store.oauth2.AccessTokenProvider;
 import com.microsoft.azure.datalake.store.oauth2.ClientCredsTokenProvider;
+import com.microsoft.azure.datalake.store.oauth2.DeviceCodeTokenProvider;
+import com.microsoft.azure.datalake.store.oauth2.MsiTokenProvider;
 import com.microsoft.azure.datalake.store.oauth2.RefreshTokenBasedTokenProvider;
 
 import org.apache.commons.lang.StringUtils;
@@ -254,6 +256,12 @@ public class AdlFileSystem extends FileSystem {
     case ClientCredential:
       tokenProvider = getConfCredentialBasedTokenProvider(conf);
       break;
+    case MSI:
+      tokenProvider = getMsiBasedTokenProvider(conf);
+      break;
+    case DeviceCode:
+      tokenProvider = getDeviceCodeTokenProvider(conf);
+      break;
     case Custom:
     default:
       AzureADTokenProvider azureADTokenProvider = getCustomAccessTokenProvider(
@@ -280,6 +288,19 @@ public class AdlFileSystem extends FileSystem {
     return new RefreshTokenBasedTokenProvider(clientId, refreshToken);
   }
 
+  private AccessTokenProvider getMsiBasedTokenProvider(
+          Configuration conf) throws IOException {
+    int port = Integer.parseInt(getNonEmptyVal(conf, MSI_PORT));
+    String tenantGuid = getPasswordString(conf, MSI_TENANT_GUID);
+    return new MsiTokenProvider(port, tenantGuid);
+  }
+
+  private AccessTokenProvider getDeviceCodeTokenProvider(
+          Configuration conf) throws IOException {
+    String clientAppId = getNonEmptyVal(conf, DEVICE_CODE_CLIENT_APP_ID);
+    return new DeviceCodeTokenProvider(clientAppId);
+  }
+
   @VisibleForTesting
   AccessTokenProvider getTokenProvider() {
     return tokenProvider;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7769e961/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/TokenProviderType.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/TokenProviderType.java b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/TokenProviderType.java
index 9fd4f4f..1c11d84 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/TokenProviderType.java
+++ b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/TokenProviderType.java
@@ -21,5 +21,7 @@ package org.apache.hadoop.fs.adl;
 enum TokenProviderType {
   RefreshToken,
   ClientCredential,
+  MSI,
+  DeviceCode,
   Custom
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7769e961/hadoop-tools/hadoop-azure-datalake/src/site/markdown/index.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/site/markdown/index.md b/hadoop-tools/hadoop-azure-datalake/src/site/markdown/index.md
index d4b7d8e..e34da36 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/site/markdown/index.md
+++ b/hadoop-tools/hadoop-azure-datalake/src/site/markdown/index.md
@@ -111,20 +111,24 @@ service associated with the client id. See [*Active Directory Library For Java*]
 ##### Generating the Service Principal
 
 1.  Go to [the portal](https://portal.azure.com)
-2.  Under "Browse", look for Active Directory and click on it.
-3.  Create "Web Application". Remember the name you create here - that is what you will add to your ADL account as authorized user.
+2.  Under services in left nav, look for Azure Active Directory and click it.
+3.  Using "App Registrations" in the menu, create "Web Application". Remember
+    the name you create here - that is what you will add to your ADL account
+    as authorized user.
 4.  Go through the wizard
-5.  Once app is created, Go to app configuration, and find the section on "keys"
+5.  Once app is created, go to "keys" under "settings" for the app
 6.  Select a key duration and hit save. Save the generated keys.
-7. Note down the properties you will need to auth:
-    -  The client ID
+7.  Go back to the App Registrations page, and click on the "Endpoints" button
+    at the top
+    a. Note down the  "Token Endpoint" URL
+8. Note down the properties you will need to auth:
+    -  The "Application ID" of the Web App you created above
     -  The key you just generated above
-    -  The token endpoint (select "View endpoints" at the bottom of the page and copy/paste the OAuth2 .0 Token Endpoint value)
-    -  Resource: Always https://management.core.windows.net/ , for all customers
+    -  The token endpoint
 
 ##### Adding the service principal to your ADL Account
 1.  Go to the portal again, and open your ADL account
-2.  Select Users under Settings
+2.  Select `Access control (IAM)`
 3.  Add your user name you created in Step 6 above (note that it does not show up in the list, but will be found if you searched for the name)
 4.  Add "Owner" role
 
@@ -153,6 +157,84 @@ Add the following properties to your `core-site.xml`
 </property>
 ```
 
+#### Using MSI (Managed Service Identity)
+
+Azure VMs can be provisioned with "service identities" that are managed by the
+Identity extension within the VM. The advantage of doing this is that the
+credentials are managed by the extension, and do not have to be put into
+core-site.xml.
+
+To use MSI, the following two steps are needed:
+1. Modify the VM deployment template to specify the port number of the token
+ service exposed to localhost by the identity extension in the VM.
+2. Get your Azure ActiveDirectory Tenant ID:
+   1. Go to [the portal](https://portal.azure.com)
+   2. Under services in left nav, look for Azure Active Directory and click on it.
+   3. Click on Properties
+   4. Note down the GUID shown under "Directory ID" - this is your AAD tenant ID
+
+
+##### Configure core-site.xml
+Add the following properties to your `core-site.xml`
+
+```xml
+<property>
+  <name>fs.adl.oauth2.access.token.provider.type</name>
+  <value>Msi</value>
+</property>
+
+<property>
+  <name>fs.adl.oauth2.msi.port</name>
+  <value>PORT NUMBER FROM STEP 1 ABOVE</value>
+</property>
+
+<property>
+  <name>fs.adl.oauth2.msi.TenantGuid</name>
+  <value>AAD TENANT ID GUID FROM STEP 2 ABOVE</value>
+</property>
+```
+
+### Using Device Code Auth for interactive login
+
+**Note:** This auth method is suitable for running interactive tools, but will
+not work for jobs submitted to a cluster.
+
+To use user-based login, Azure ActiveDirectory provides login flow using
+device code.
+
+To use device code flow, user must first create a **Native** app registration
+in the Azure portal, and provide the client ID for the app as a config. Here
+are the steps:
+
+1.  Go to [the portal](https://portal.azure.com)
+2.  Under services in left nav, look for Azure Active Directory and click on it.
+3.  Using "App Registrations" in the menu, create "Native Application".
+4.  Go through the wizard
+5.  Once app is created, note down the "Appplication ID" of the app
+6. Grant permissions to the app:
+    1. Click on "Permissions" for the app, and then add "Azure Data Lake" and
+       "Windows Azure Service Management API" permissions
+    2. Click on "Grant Permissions" to add the permissions to the app
+
+Add the following properties to your `core-site.xml`
+
+```xml
+<property>
+  <name>fs.adl.oauth2.devicecode.clientappid</name>
+  <value>APP ID FROM STEP 5 ABOVE</value>
+</property>
+```
+
+It is usually not desirable to add DeviceCode as the default token provider
+type. But it can be used when using a local command:
+```
+ hadoop fs -Dfs.adl.oauth2.access.token.provider.type=DeviceCode -ls ...
+```
+Running this will print a URL and device code that can be used to login from
+any browser (even on a different machine, outside of the ssh session). Once
+the login is done, the command continues.
+
+
 #### Protecting the Credentials with Credential Providers
 
 In many Hadoop clusters, the `core-site.xml` file is world-readable. To protect

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7769e961/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestAzureADTokenProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestAzureADTokenProvider.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestAzureADTokenProvider.java
index 36498c6..929b33a 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestAzureADTokenProvider.java
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestAzureADTokenProvider.java
@@ -23,6 +23,8 @@ import java.io.IOException;
 import java.net.URI;
 import java.net.URISyntaxException;
 
+import com.microsoft.azure.datalake.store.oauth2.DeviceCodeTokenProvider;
+import com.microsoft.azure.datalake.store.oauth2.MsiTokenProvider;
 import org.apache.commons.lang.builder.EqualsBuilder;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.adl.common.CustomMockTokenProvider;
@@ -40,6 +42,9 @@ import static org.apache.hadoop.fs.adl.AdlConfKeys
     .AZURE_AD_TOKEN_PROVIDER_CLASS_KEY;
 import static org.apache.hadoop.fs.adl.AdlConfKeys
     .AZURE_AD_TOKEN_PROVIDER_TYPE_KEY;
+import static org.apache.hadoop.fs.adl.AdlConfKeys.DEVICE_CODE_CLIENT_APP_ID;
+import static org.apache.hadoop.fs.adl.AdlConfKeys.MSI_PORT;
+import static org.apache.hadoop.fs.adl.AdlConfKeys.MSI_TENANT_GUID;
 import static org.apache.hadoop.fs.adl.TokenProviderType.*;
 import static org.junit.Assert.assertEquals;
 
@@ -98,6 +103,41 @@ public class TestAzureADTokenProvider {
   }
 
   @Test
+  public void testMSITokenProvider()
+          throws IOException, URISyntaxException {
+    Configuration conf = new Configuration();
+    conf.setEnum(AZURE_AD_TOKEN_PROVIDER_TYPE_KEY, MSI);
+    conf.set(MSI_PORT, "54321");
+    conf.set(MSI_TENANT_GUID, "TENANT_GUID");
+
+    URI uri = new URI("adl://localhost:8080");
+    AdlFileSystem fileSystem = new AdlFileSystem();
+    fileSystem.initialize(uri, conf);
+    AccessTokenProvider tokenProvider = fileSystem.getTokenProvider();
+    Assert.assertTrue(tokenProvider instanceof MsiTokenProvider);
+  }
+
+  @Test
+  public void testDeviceCodeTokenProvider()
+          throws IOException, URISyntaxException {
+    boolean runTest = false;
+    if (runTest) {
+      // Device code auth method causes an interactive prompt, so run this only
+      // when running the test interactively at a local terminal. Disabling
+      // test by default, to not break any automation.
+      Configuration conf = new Configuration();
+      conf.setEnum(AZURE_AD_TOKEN_PROVIDER_TYPE_KEY, DeviceCode);
+      conf.set(DEVICE_CODE_CLIENT_APP_ID, "CLIENT_APP_ID_GUID");
+
+      URI uri = new URI("adl://localhost:8080");
+      AdlFileSystem fileSystem = new AdlFileSystem();
+      fileSystem.initialize(uri, conf);
+      AccessTokenProvider tokenProvider = fileSystem.getTokenProvider();
+      Assert.assertTrue(tokenProvider instanceof DeviceCodeTokenProvider);
+    }
+  }
+
+  @Test
   public void testCustomCredTokenProvider()
       throws URISyntaxException, IOException {
     Configuration conf = new Configuration();


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


[29/50] [abbrv] hadoop git commit: HDFS-12301. NN File Browser UI: Navigate to a path when enter is pressed

Posted by jh...@apache.org.
HDFS-12301. NN File Browser UI: Navigate to a path when enter is pressed


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

Branch: refs/heads/YARN-5734
Commit: f34646d652310442cb5339aabbbb269f10dfa838
Parents: d265459
Author: Ravi Prakash <ra...@altiscale.com>
Authored: Tue Aug 15 15:44:59 2017 -0700
Committer: Ravi Prakash <ra...@altiscale.com>
Committed: Tue Aug 15 15:44:59 2017 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/src/main/webapps/hdfs/explorer.js              | 6 ++++++
 1 file changed, 6 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f34646d6/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js
index 3e276a9..dae3519 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js
@@ -370,6 +370,12 @@
 
     var b = function() { browse_directory($('#directory').val()); };
     $('#btn-nav-directory').click(b);
+    //Also navigate to the directory when a user presses enter.
+    $('#directory').on('keyup', function (e) {
+      if (e.which == 13) {
+        browse_directory($('#directory').val());
+      }
+    });
     var dir = window.location.hash.slice(1);
     if(dir == "") {
       window.location.hash = "/";


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


[22/50] [abbrv] hadoop git commit: YARN-5978. ContainerScheduler and ContainerManager changes to support ExecType update. (Kartheek Muthyala via asuresh)

Posted by jh...@apache.org.
YARN-5978. ContainerScheduler and ContainerManager changes to support ExecType update. (Kartheek Muthyala via asuresh)


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

Branch: refs/heads/YARN-5734
Commit: 4d7be1d8575e9254c59d41460960708e3718503a
Parents: 0446511
Author: Arun Suresh <as...@apache.org>
Authored: Mon Aug 14 19:46:17 2017 -0700
Committer: Arun Suresh <as...@apache.org>
Committed: Mon Aug 14 19:46:17 2017 -0700

----------------------------------------------------------------------
 .../yarn/client/api/impl/TestAMRMClient.java    | 395 +++++++++++++++++--
 .../yarn/client/api/impl/TestNMClient.java      |   7 +-
 .../containermanager/ContainerManagerImpl.java  | 132 ++++---
 .../containermanager/container/Container.java   |   4 +-
 .../container/ContainerImpl.java                |  37 +-
 .../monitor/ContainersMonitorImpl.java          |  15 -
 .../scheduler/ContainerScheduler.java           |  73 ++++
 .../scheduler/ContainerSchedulerEventType.java  |   1 +
 .../UpdateContainerSchedulerEvent.java          |  85 ++++
 .../nodemanager/TestNodeManagerResync.java      |  11 +-
 .../BaseContainerManagerTest.java               |  33 +-
 .../containermanager/TestContainerManager.java  | 267 ++++++++-----
 .../TestContainerManagerRecovery.java           |   2 +-
 .../TestContainerSchedulerQueuing.java          |  96 +++++
 .../nodemanager/webapp/MockContainer.java       |   2 +-
 .../scheduler/SchedulerApplicationAttempt.java  |   2 +-
 .../security/RMContainerTokenSecretManager.java |  30 +-
 17 files changed, 964 insertions(+), 228 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d7be1d8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java
index 1b2bca3..09b12f2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.yarn.client.api.impl;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 import static org.mockito.Matchers.any;
@@ -36,6 +37,7 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 import java.util.TreeSet;
 
@@ -142,6 +144,10 @@ public class TestAMRMClient {
     // set the minimum allocation so that resource decrease can go under 1024
     conf.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 512);
     conf.setLong(YarnConfiguration.NM_LOG_RETAIN_SECONDS, 1);
+    conf.setBoolean(
+        YarnConfiguration.OPPORTUNISTIC_CONTAINER_ALLOCATION_ENABLED, true);
+    conf.setInt(
+        YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH, 10);
     yarnCluster = new MiniYARNCluster(TestAMRMClient.class.getName(), nodeCount, 1, 1);
     yarnCluster.init(conf);
     yarnCluster.start();
@@ -924,8 +930,8 @@ public class TestAMRMClient {
     // add exp=x to ANY
     client.addContainerRequest(new ContainerRequest(Resource.newInstance(1024,
         1), null, null, Priority.UNDEFINED, true, "x"));
-    Assert.assertEquals(1, client.ask.size());
-    Assert.assertEquals("x", client.ask.iterator().next()
+    assertEquals(1, client.ask.size());
+    assertEquals("x", client.ask.iterator().next()
         .getNodeLabelExpression());
 
     // add exp=x then add exp=a to ANY in same priority, only exp=a should kept
@@ -933,8 +939,8 @@ public class TestAMRMClient {
         1), null, null, Priority.UNDEFINED, true, "x"));
     client.addContainerRequest(new ContainerRequest(Resource.newInstance(1024,
         1), null, null, Priority.UNDEFINED, true, "a"));
-    Assert.assertEquals(1, client.ask.size());
-    Assert.assertEquals("a", client.ask.iterator().next()
+    assertEquals(1, client.ask.size());
+    assertEquals("a", client.ask.iterator().next()
         .getNodeLabelExpression());
     
     // add exp=x to ANY, rack and node, only resource request has ANY resource
@@ -943,10 +949,10 @@ public class TestAMRMClient {
     client.addContainerRequest(new ContainerRequest(Resource.newInstance(1024,
         1), null, null, Priority.UNDEFINED, true,
         "y"));
-    Assert.assertEquals(1, client.ask.size());
+    assertEquals(1, client.ask.size());
     for (ResourceRequest req : client.ask) {
       if (ResourceRequest.ANY.equals(req.getResourceName())) {
-        Assert.assertEquals("y", req.getNodeLabelExpression());
+        assertEquals("y", req.getNodeLabelExpression());
       } else {
         Assert.assertNull(req.getNodeLabelExpression());
       }
@@ -957,7 +963,7 @@ public class TestAMRMClient {
         new String[] { "node1", "node2" }, Priority.UNDEFINED, true, "y"));
     for (ResourceRequest req : client.ask) {
       if (ResourceRequest.ANY.equals(req.getResourceName())) {
-        Assert.assertEquals("y", req.getNodeLabelExpression());
+        assertEquals("y", req.getNodeLabelExpression());
       } else {
         Assert.assertNull(req.getNodeLabelExpression());
       }
@@ -971,7 +977,7 @@ public class TestAMRMClient {
     } catch (InvalidContainerRequestException e) {
       return;
     }
-    Assert.fail();
+    fail();
   }
   
   @Test(timeout=30000)
@@ -1042,7 +1048,8 @@ public class TestAMRMClient {
     // get allocations
     AllocateResponse allocResponse = amClient.allocate(0.1f);
     List<Container> containers = allocResponse.getAllocatedContainers();
-    Assert.assertEquals(num, containers.size());
+    assertEquals(num, containers.size());
+
     // build container launch context
     Credentials ts = new Credentials();
     DataOutputBuffer dob = new DataOutputBuffer();
@@ -1083,14 +1090,14 @@ public class TestAMRMClient {
   private void doContainerResourceChange(
       final AMRMClient<ContainerRequest> amClient, List<Container> containers)
       throws YarnException, IOException {
-    Assert.assertEquals(3, containers.size());
+    assertEquals(3, containers.size());
     // remember the container IDs
     Container container1 = containers.get(0);
     Container container2 = containers.get(1);
     Container container3 = containers.get(2);
     AMRMClientImpl<ContainerRequest> amClientImpl =
         (AMRMClientImpl<ContainerRequest>) amClient;
-    Assert.assertEquals(0, amClientImpl.change.size());
+    assertEquals(0, amClientImpl.change.size());
     // verify newer request overwrites older request for the container1
     amClientImpl.requestContainerUpdate(container1,
         UpdateContainerRequest.newInstance(container1.getVersion(),
@@ -1100,21 +1107,21 @@ public class TestAMRMClient {
         UpdateContainerRequest.newInstance(container1.getVersion(),
             container1.getId(), ContainerUpdateType.INCREASE_RESOURCE,
             Resource.newInstance(4096, 1), null));
-    Assert.assertEquals(Resource.newInstance(4096, 1),
+    assertEquals(Resource.newInstance(4096, 1),
         amClientImpl.change.get(container1.getId()).getValue().getCapability());
     // verify new decrease request cancels old increase request for container1
     amClientImpl.requestContainerUpdate(container1,
         UpdateContainerRequest.newInstance(container1.getVersion(),
             container1.getId(), ContainerUpdateType.DECREASE_RESOURCE,
             Resource.newInstance(512, 1), null));
-    Assert.assertEquals(Resource.newInstance(512, 1),
+    assertEquals(Resource.newInstance(512, 1),
         amClientImpl.change.get(container1.getId()).getValue().getCapability());
     // request resource increase for container2
     amClientImpl.requestContainerUpdate(container2,
         UpdateContainerRequest.newInstance(container2.getVersion(),
             container2.getId(), ContainerUpdateType.INCREASE_RESOURCE,
             Resource.newInstance(2048, 1), null));
-    Assert.assertEquals(Resource.newInstance(2048, 1),
+    assertEquals(Resource.newInstance(2048, 1),
         amClientImpl.change.get(container2.getId()).getValue().getCapability());
     // verify release request will cancel pending change requests for the same
     // container
@@ -1122,27 +1129,357 @@ public class TestAMRMClient {
         UpdateContainerRequest.newInstance(container3.getVersion(),
             container3.getId(), ContainerUpdateType.INCREASE_RESOURCE,
             Resource.newInstance(2048, 1), null));
-    Assert.assertEquals(3, amClientImpl.pendingChange.size());
+    assertEquals(3, amClientImpl.pendingChange.size());
     amClientImpl.releaseAssignedContainer(container3.getId());
-    Assert.assertEquals(2, amClientImpl.pendingChange.size());
+    assertEquals(2, amClientImpl.pendingChange.size());
     // as of now: container1 asks to decrease to (512, 1)
     //            container2 asks to increase to (2048, 1)
     // send allocation requests
     AllocateResponse allocResponse = amClient.allocate(0.1f);
-    Assert.assertEquals(0, amClientImpl.change.size());
+    assertEquals(0, amClientImpl.change.size());
     // we should get decrease confirmation right away
     List<UpdatedContainer> updatedContainers =
         allocResponse.getUpdatedContainers();
-    Assert.assertEquals(1, updatedContainers.size());
+    assertEquals(1, updatedContainers.size());
     // we should get increase allocation after the next NM's heartbeat to RM
     triggerSchedulingWithNMHeartBeat();
     // get allocations
     allocResponse = amClient.allocate(0.1f);
     updatedContainers =
         allocResponse.getUpdatedContainers();
-    Assert.assertEquals(1, updatedContainers.size());
+    assertEquals(1, updatedContainers.size());
+  }
+
+  @Test(timeout=60000)
+  public void testAMRMClientWithContainerPromotion()
+      throws YarnException, IOException {
+    AMRMClientImpl<AMRMClient.ContainerRequest> amClient =
+        (AMRMClientImpl<AMRMClient.ContainerRequest>) AMRMClient
+            .createAMRMClient();
+    //asserting we are not using the singleton instance cache
+    Assert.assertSame(NMTokenCache.getSingleton(),
+        amClient.getNMTokenCache());
+    amClient.init(conf);
+    amClient.start();
+
+    // start am nm client
+    NMClientImpl nmClient = (NMClientImpl) NMClient.createNMClient();
+    Assert.assertNotNull(nmClient);
+    // asserting we are using the singleton instance cache
+    Assert.assertSame(
+        NMTokenCache.getSingleton(), nmClient.getNMTokenCache());
+    nmClient.init(conf);
+    nmClient.start();
+    assertEquals(STATE.STARTED, nmClient.getServiceState());
+
+    amClient.registerApplicationMaster("Host", 10000, "");
+    // setup container request
+    assertEquals(0, amClient.ask.size());
+    assertEquals(0, amClient.release.size());
+
+    // START OPPORTUNISTIC Container, Send allocation request to RM
+    amClient.addContainerRequest(
+        new AMRMClient.ContainerRequest(capability, null, null, priority2, 0,
+            true, null, ExecutionTypeRequest
+            .newInstance(ExecutionType.OPPORTUNISTIC, true)));
+
+    int oppContainersRequestedAny =
+        amClient.getTable(0).get(priority2, ResourceRequest.ANY,
+            ExecutionType.OPPORTUNISTIC, capability).remoteRequest
+            .getNumContainers();
+
+    assertEquals(1, oppContainersRequestedAny);
+    assertEquals(1, amClient.ask.size());
+    assertEquals(0, amClient.release.size());
+
+    // RM should allocate container within 2 calls to allocate()
+    int allocatedContainerCount = 0;
+    Map<ContainerId, Container> allocatedOpportContainers = new HashMap<>();
+    int iterationsLeft = 50;
+
+    amClient.getNMTokenCache().clearCache();
+    assertEquals(0,
+        amClient.getNMTokenCache().numberOfTokensInCache());
+
+    AllocateResponse allocResponse = null;
+    while (allocatedContainerCount < oppContainersRequestedAny
+        && iterationsLeft-- > 0) {
+      allocResponse = amClient.allocate(0.1f);
+      // let NM heartbeat to RM and trigger allocations
+      //triggerSchedulingWithNMHeartBeat();
+      assertEquals(0, amClient.ask.size());
+      assertEquals(0, amClient.release.size());
+
+      allocatedContainerCount +=
+          allocResponse.getAllocatedContainers().size();
+      for (Container container : allocResponse.getAllocatedContainers()) {
+        if (container.getExecutionType() == ExecutionType.OPPORTUNISTIC) {
+          allocatedOpportContainers.put(container.getId(), container);
+        }
+      }
+      if (allocatedContainerCount < oppContainersRequestedAny) {
+        // sleep to let NM's heartbeat to RM and trigger allocations
+        sleep(100);
+      }
+    }
+
+    assertEquals(oppContainersRequestedAny, allocatedContainerCount);
+    assertEquals(oppContainersRequestedAny, allocatedOpportContainers.size());
+
+    startContainer(allocResponse, nmClient);
+
+    // SEND PROMOTION REQUEST TO RM
+    try {
+      Container c = allocatedOpportContainers.values().iterator().next();
+      amClient.requestContainerUpdate(
+          c, UpdateContainerRequest.newInstance(c.getVersion(),
+              c.getId(), ContainerUpdateType.PROMOTE_EXECUTION_TYPE,
+              null, ExecutionType.OPPORTUNISTIC));
+      fail("Should throw Exception..");
+    } catch (IllegalArgumentException e) {
+      System.out.println("## " + e.getMessage());
+      assertTrue(e.getMessage().contains(
+          "target should be GUARANTEED and original should be OPPORTUNISTIC"));
+    }
+
+    Container c = allocatedOpportContainers.values().iterator().next();
+    amClient.requestContainerUpdate(
+        c, UpdateContainerRequest.newInstance(c.getVersion(),
+            c.getId(), ContainerUpdateType.PROMOTE_EXECUTION_TYPE,
+            null, ExecutionType.GUARANTEED));
+    iterationsLeft = 120;
+    Map<ContainerId, UpdatedContainer> updatedContainers = new HashMap<>();
+    // do a few iterations to ensure RM is not going to send new containers
+    while (iterationsLeft-- > 0 && updatedContainers.isEmpty()) {
+      // inform RM of rejection
+      allocResponse = amClient.allocate(0.1f);
+      // RM did not send new containers because AM does not need any
+      if (allocResponse.getUpdatedContainers() != null) {
+        for (UpdatedContainer updatedContainer : allocResponse
+            .getUpdatedContainers()) {
+          System.out.println("Got update..");
+          updatedContainers.put(updatedContainer.getContainer().getId(),
+              updatedContainer);
+        }
+      }
+      if (iterationsLeft > 0) {
+        // sleep to make sure NM's heartbeat
+        sleep(100);
+      }
+    }
+    assertEquals(1, updatedContainers.size());
+
+    for (ContainerId cId : allocatedOpportContainers.keySet()) {
+      Container orig = allocatedOpportContainers.get(cId);
+      UpdatedContainer updatedContainer = updatedContainers.get(cId);
+      assertNotNull(updatedContainer);
+      assertEquals(ExecutionType.GUARANTEED,
+          updatedContainer.getContainer().getExecutionType());
+      assertEquals(orig.getResource(),
+          updatedContainer.getContainer().getResource());
+      assertEquals(orig.getNodeId(),
+          updatedContainer.getContainer().getNodeId());
+      assertEquals(orig.getVersion() + 1,
+          updatedContainer.getContainer().getVersion());
+    }
+    assertEquals(0, amClient.ask.size());
+    assertEquals(0, amClient.release.size());
+
+    // SEND UPDATE EXECTYPE UPDATE TO NM
+    updateContainerExecType(allocResponse, ExecutionType.GUARANTEED, nmClient);
+
+    amClient.ask.clear();
+  }
+
+  @Test(timeout=60000)
+  public void testAMRMClientWithContainerDemotion()
+      throws YarnException, IOException {
+    AMRMClientImpl<AMRMClient.ContainerRequest> amClient =
+        (AMRMClientImpl<AMRMClient.ContainerRequest>) AMRMClient
+            .createAMRMClient();
+    //asserting we are not using the singleton instance cache
+    Assert.assertSame(NMTokenCache.getSingleton(),
+        amClient.getNMTokenCache());
+    amClient.init(conf);
+    amClient.start();
+
+    NMClientImpl nmClient = (NMClientImpl) NMClient.createNMClient();
+    Assert.assertNotNull(nmClient);
+    // asserting we are using the singleton instance cache
+    Assert.assertSame(
+        NMTokenCache.getSingleton(), nmClient.getNMTokenCache());
+    nmClient.init(conf);
+    nmClient.start();
+    assertEquals(STATE.STARTED, nmClient.getServiceState());
+
+    amClient.registerApplicationMaster("Host", 10000, "");
+    assertEquals(0, amClient.ask.size());
+    assertEquals(0, amClient.release.size());
+
+    // START OPPORTUNISTIC Container, Send allocation request to RM
+    amClient.addContainerRequest(
+        new AMRMClient.ContainerRequest(capability, null, null, priority2, 0,
+            true, null, ExecutionTypeRequest
+            .newInstance(ExecutionType.GUARANTEED, true)));
+
+    int oppContainersRequestedAny =
+        amClient.getTable(0).get(priority2, ResourceRequest.ANY,
+            ExecutionType.GUARANTEED, capability).remoteRequest
+            .getNumContainers();
+
+    assertEquals(1, oppContainersRequestedAny);
+    assertEquals(1, amClient.ask.size());
+    assertEquals(0, amClient.release.size());
+
+    // RM should allocate container within 2 calls to allocate()
+    int allocatedContainerCount = 0;
+    Map<ContainerId, Container> allocatedGuaranteedContainers = new HashMap<>();
+    int iterationsLeft = 50;
+
+    amClient.getNMTokenCache().clearCache();
+    assertEquals(0,
+        amClient.getNMTokenCache().numberOfTokensInCache());
+
+    AllocateResponse allocResponse = null;
+    while (allocatedContainerCount < oppContainersRequestedAny
+        && iterationsLeft-- > 0) {
+      allocResponse = amClient.allocate(0.1f);
+      // let NM heartbeat to RM and trigger allocations
+      //triggerSchedulingWithNMHeartBeat();
+      assertEquals(0, amClient.ask.size());
+      assertEquals(0, amClient.release.size());
+
+      allocatedContainerCount +=
+          allocResponse.getAllocatedContainers().size();
+      for (Container container : allocResponse.getAllocatedContainers()) {
+        if (container.getExecutionType() == ExecutionType.GUARANTEED) {
+          allocatedGuaranteedContainers.put(container.getId(), container);
+        }
+      }
+      if (allocatedContainerCount < oppContainersRequestedAny) {
+        // sleep to let NM's heartbeat to RM and trigger allocations
+        sleep(100);
+      }
+    }
+    assertEquals(oppContainersRequestedAny, allocatedContainerCount);
+    assertEquals(oppContainersRequestedAny,
+        allocatedGuaranteedContainers.size());
+    startContainer(allocResponse, nmClient);
+
+    // SEND DEMOTION REQUEST TO RM
+    try {
+      Container c = allocatedGuaranteedContainers.values().iterator().next();
+      amClient.requestContainerUpdate(
+          c, UpdateContainerRequest.newInstance(c.getVersion(),
+              c.getId(), ContainerUpdateType.DEMOTE_EXECUTION_TYPE,
+              null, ExecutionType.GUARANTEED));
+      fail("Should throw Exception..");
+    } catch (IllegalArgumentException e) {
+      System.out.println("## " + e.getMessage());
+      assertTrue(e.getMessage().contains(
+          "target should be OPPORTUNISTIC and original should be GUARANTEED"));
+    }
+
+    Container c = allocatedGuaranteedContainers.values().iterator().next();
+    amClient.requestContainerUpdate(
+        c, UpdateContainerRequest.newInstance(c.getVersion(),
+            c.getId(), ContainerUpdateType.DEMOTE_EXECUTION_TYPE,
+            null, ExecutionType.OPPORTUNISTIC));
+    iterationsLeft = 120;
+    Map<ContainerId, UpdatedContainer> updatedContainers = new HashMap<>();
+    // do a few iterations to ensure RM is not going to send new containers
+    while (iterationsLeft-- > 0 && updatedContainers.isEmpty()) {
+      // inform RM of rejection
+      allocResponse = amClient.allocate(0.1f);
+      // RM did not send new containers because AM does not need any
+      if (allocResponse.getUpdatedContainers() != null) {
+        for (UpdatedContainer updatedContainer : allocResponse
+            .getUpdatedContainers()) {
+          System.out.println("Got update..");
+          updatedContainers.put(updatedContainer.getContainer().getId(),
+              updatedContainer);
+        }
+      }
+      if (iterationsLeft > 0) {
+        // sleep to make sure NM's heartbeat
+        sleep(100);
+      }
+    }
+    assertEquals(1, updatedContainers.size());
+
+    for (ContainerId cId : allocatedGuaranteedContainers.keySet()) {
+      Container orig = allocatedGuaranteedContainers.get(cId);
+      UpdatedContainer updatedContainer = updatedContainers.get(cId);
+      assertNotNull(updatedContainer);
+      assertEquals(ExecutionType.OPPORTUNISTIC,
+          updatedContainer.getContainer().getExecutionType());
+      assertEquals(orig.getResource(),
+          updatedContainer.getContainer().getResource());
+      assertEquals(orig.getNodeId(),
+          updatedContainer.getContainer().getNodeId());
+      assertEquals(orig.getVersion() + 1,
+          updatedContainer.getContainer().getVersion());
+    }
+    assertEquals(0, amClient.ask.size());
+    assertEquals(0, amClient.release.size());
+
+    updateContainerExecType(allocResponse, ExecutionType.OPPORTUNISTIC,
+        nmClient);
+    amClient.ask.clear();
+  }
+
+  private void updateContainerExecType(AllocateResponse allocResponse,
+      ExecutionType expectedExecType, NMClientImpl nmClient)
+      throws IOException, YarnException {
+    for (UpdatedContainer updatedContainer : allocResponse
+        .getUpdatedContainers()) {
+      Container container = updatedContainer.getContainer();
+      nmClient.increaseContainerResource(container);
+      // NodeManager may still need some time to get the stable
+      // container status
+      while (true) {
+        ContainerStatus status = nmClient
+            .getContainerStatus(container.getId(), container.getNodeId());
+        if (status.getExecutionType() == expectedExecType) {
+          break;
+        }
+        sleep(10);
+      }
+    }
+  }
+
+  private void startContainer(AllocateResponse allocResponse,
+      NMClientImpl nmClient) throws IOException, YarnException {
+    // START THE CONTAINER IN NM
+    // build container launch context
+    Credentials ts = new Credentials();
+    DataOutputBuffer dob = new DataOutputBuffer();
+    ts.writeTokenStorageToStream(dob);
+    ByteBuffer securityTokens =
+        ByteBuffer.wrap(dob.getData(), 0, dob.getLength());
+    // start a process long enough for increase/decrease action to take effect
+    ContainerLaunchContext clc = BuilderUtils.newContainerLaunchContext(
+        Collections.<String, LocalResource>emptyMap(),
+        new HashMap<String, String>(), Arrays.asList("sleep", "100"),
+        new HashMap<String, ByteBuffer>(), securityTokens,
+        new HashMap<ApplicationAccessType, String>());
+    // start the containers and make sure they are in RUNNING state
+    for (Container container : allocResponse.getAllocatedContainers()) {
+      nmClient.startContainer(container, clc);
+      // NodeManager may still need some time to get the stable
+      // container status
+      while (true) {
+        ContainerStatus status = nmClient
+            .getContainerStatus(container.getId(), container.getNodeId());
+        if (status.getState() == ContainerState.RUNNING) {
+          break;
+        }
+        sleep(10);
+      }
+    }
   }
 
+
   private void testAllocation(final AMRMClientImpl<ContainerRequest> amClient)
       throws YarnException, IOException {
     // setup container request
@@ -1172,7 +1509,7 @@ public class TestAMRMClient {
     Set<ContainerId> releases = new TreeSet<ContainerId>();
     
     amClient.getNMTokenCache().clearCache();
-    Assert.assertEquals(0, amClient.getNMTokenCache().numberOfTokensInCache());
+    assertEquals(0, amClient.getNMTokenCache().numberOfTokensInCache());
     HashMap<String, Token> receivedNMTokens = new HashMap<String, Token>();
     
     while (allocatedContainerCount < containersRequestedAny
@@ -1192,7 +1529,7 @@ public class TestAMRMClient {
       for (NMToken token : allocResponse.getNMTokens()) {
         String nodeID = token.getNodeId().toString();
         if (receivedNMTokens.containsKey(nodeID)) {
-          Assert.fail("Received token again for : " + nodeID);          
+          fail("Received token again for : " + nodeID);
         }
         receivedNMTokens.put(nodeID, token.getToken());
       }
@@ -1204,7 +1541,7 @@ public class TestAMRMClient {
     }
     
     // Should receive atleast 1 token
-    Assert.assertTrue(receivedNMTokens.size() > 0
+    assertTrue(receivedNMTokens.size() > 0
         && receivedNMTokens.size() <= nodeCount);
     
     assertEquals(allocatedContainerCount, containersRequestedAny);
@@ -1444,7 +1781,7 @@ public class TestAMRMClient {
       org.apache.hadoop.security.token.Token<AMRMTokenIdentifier> amrmToken_1 =
           getAMRMToken();
       Assert.assertNotNull(amrmToken_1);
-      Assert.assertEquals(amrmToken_1.decodeIdentifier().getKeyId(),
+      assertEquals(amrmToken_1.decodeIdentifier().getKeyId(),
         amrmTokenSecretManager.getMasterKey().getMasterKey().getKeyId());
 
       // Wait for enough time and make sure the roll_over happens
@@ -1459,7 +1796,7 @@ public class TestAMRMClient {
       org.apache.hadoop.security.token.Token<AMRMTokenIdentifier> amrmToken_2 =
           getAMRMToken();
       Assert.assertNotNull(amrmToken_2);
-      Assert.assertEquals(amrmToken_2.decodeIdentifier().getKeyId(),
+      assertEquals(amrmToken_2.decodeIdentifier().getKeyId(),
         amrmTokenSecretManager.getMasterKey().getMasterKey().getKeyId());
 
       Assert.assertNotEquals(amrmToken_1, amrmToken_2);
@@ -1474,7 +1811,7 @@ public class TestAMRMClient {
       AMRMTokenIdentifierForTest newVersionTokenIdentifier = 
           new AMRMTokenIdentifierForTest(amrmToken_2.decodeIdentifier(), "message");
       
-      Assert.assertEquals("Message is changed after set to newVersionTokenIdentifier",
+      assertEquals("Message is changed after set to newVersionTokenIdentifier",
           "message", newVersionTokenIdentifier.getMessage());
       org.apache.hadoop.security.token.Token<AMRMTokenIdentifier> newVersionToken = 
           new org.apache.hadoop.security.token.Token<AMRMTokenIdentifier> (
@@ -1530,10 +1867,10 @@ public class TestAMRMClient {
                 .getBindAddress(), conf);
           }
         }).allocate(Records.newRecord(AllocateRequest.class));
-        Assert.fail("The old Token should not work");
+        fail("The old Token should not work");
       } catch (Exception ex) {
-        Assert.assertTrue(ex instanceof InvalidToken);
-        Assert.assertTrue(ex.getMessage().contains(
+        assertTrue(ex instanceof InvalidToken);
+        assertTrue(ex.getMessage().contains(
           "Invalid AMRMToken from "
               + amrmToken_2.decodeIdentifier().getApplicationAttemptId()));
       }
@@ -1560,7 +1897,7 @@ public class TestAMRMClient {
       org.apache.hadoop.security.token.Token<?> token = iter.next();
       if (token.getKind().equals(AMRMTokenIdentifier.KIND_NAME)) {
         if (result != null) {
-          Assert.fail("credentials has more than one AMRM token."
+          fail("credentials has more than one AMRM token."
               + " token1: " + result + " token2: " + token);
         }
         result = (org.apache.hadoop.security.token.Token<AMRMTokenIdentifier>)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d7be1d8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestNMClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestNMClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestNMClient.java
index 6bd0816..9b79e2d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestNMClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestNMClient.java
@@ -301,7 +301,6 @@ public class TestNMClient {
         assertTrue("The thrown exception is not expected",
             e.getMessage().contains("is not handled by this NodeManager"));
       }
-
       // increaseContainerResource shouldn't be called before startContainer,
       // otherwise, NodeManager cannot find the container
       try {
@@ -475,10 +474,10 @@ public class TestNMClient {
     try {
       nmClient.increaseContainerResource(container);
     } catch (YarnException e) {
-      // NM container will only be in SCHEDULED state, so expect the increase
-      // action to fail.
+      // NM container increase container resource should fail without a version
+      // increase action to fail.
       if (!e.getMessage().contains(
-          "can only be changed when a container is in RUNNING state")) {
+          container.getId() + " has update version ")) {
         throw (AssertionError)
             (new AssertionError("Exception is not expected: " + e)
                 .initCause(e));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d7be1d8/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 84ed3c1..a1e8ca0 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
@@ -66,6 +66,7 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.ExecutionType;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
 import org.apache.hadoop.yarn.api.records.LogAggregationContext;
@@ -136,13 +137,14 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.LogHandler;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.NonAggregatingLogHandler;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.event.LogHandlerEventType;
-import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ChangeMonitoringContainerResourceEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitorEventType;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitorImpl;
 
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler.ContainerScheduler;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler.ContainerSchedulerEventType;
+
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler.UpdateContainerSchedulerEvent;
 import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService;
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService.RecoveredApplicationsState;
@@ -410,8 +412,24 @@ public class ContainerManagerImpl extends CompositeService implements
       throws IOException {
     StartContainerRequest req = rcs.getStartRequest();
     ContainerLaunchContext launchContext = req.getContainerLaunchContext();
-    ContainerTokenIdentifier token =
-        BuilderUtils.newContainerTokenIdentifier(req.getContainerToken());
+    ContainerTokenIdentifier token = null;
+    if(rcs.getCapability() != null) {
+      ContainerTokenIdentifier originalToken =
+          BuilderUtils.newContainerTokenIdentifier(req.getContainerToken());
+      token = new ContainerTokenIdentifier(originalToken.getContainerID(),
+          originalToken.getVersion(), originalToken.getNmHostAddress(),
+          originalToken.getApplicationSubmitter(), rcs.getCapability(),
+          originalToken.getExpiryTimeStamp(), originalToken.getMasterKeyId(),
+          originalToken.getRMIdentifier(), originalToken.getPriority(),
+          originalToken.getCreationTime(),
+          originalToken.getLogAggregationContext(),
+          originalToken.getNodeLabelExpression(),
+          originalToken.getContainerType(), originalToken.getExecutionType());
+
+    } else {
+      token = BuilderUtils.newContainerTokenIdentifier(req.getContainerToken());
+    }
+
     ContainerId containerId = token.getContainerID();
     ApplicationId appId =
         containerId.getApplicationAttemptId().getApplicationId();
@@ -1183,9 +1201,7 @@ public class ContainerManagerImpl extends CompositeService implements
           // as container resource increase request will have come with
           // an updated NMToken.
           updateNMTokenIdentifier(nmTokenIdentifier);
-          Resource resource = containerTokenIdentifier.getResource();
-          changeContainerResourceInternal(containerId,
-              containerTokenIdentifier.getVersion(), resource, true);
+          updateContainerInternal(containerId, containerTokenIdentifier);
           successfullyUpdatedContainers.add(containerId);
         } catch (YarnException | InvalidToken e) {
           failedContainers.put(containerId, SerializedException.newInstance(e));
@@ -1199,9 +1215,9 @@ public class ContainerManagerImpl extends CompositeService implements
   }
 
   @SuppressWarnings("unchecked")
-  private void changeContainerResourceInternal(ContainerId containerId,
-      int containerVersion, Resource targetResource, boolean increase)
-          throws YarnException, IOException {
+  private void updateContainerInternal(ContainerId containerId,
+      ContainerTokenIdentifier containerTokenIdentifier)
+      throws YarnException, IOException {
     Container container = context.getContainers().get(containerId);
     // Check container existence
     if (container == null) {
@@ -1213,64 +1229,77 @@ public class ContainerManagerImpl extends CompositeService implements
             + " is not handled by this NodeManager");
       }
     }
+    // Check container version.
+    int currentVersion = container.getContainerTokenIdentifier().getVersion();
+    if (containerTokenIdentifier.getVersion() <= currentVersion) {
+      throw RPCUtil.getRemoteException("Container " + containerId.toString()
+          + " has update version [" + currentVersion + "] >= requested version"
+          + " [" + containerTokenIdentifier.getVersion() + "]");
+    }
+
     // Check container state
     org.apache.hadoop.yarn.server.nodemanager.
         containermanager.container.ContainerState currentState =
         container.getContainerState();
     if (currentState != org.apache.hadoop.yarn.server.
-        nodemanager.containermanager.container.ContainerState.RUNNING) {
+            nodemanager.containermanager.container.ContainerState.RUNNING &&
+        currentState != org.apache.hadoop.yarn.server.
+            nodemanager.containermanager.container.ContainerState.SCHEDULED) {
       throw RPCUtil.getRemoteException("Container " + containerId.toString()
           + " is in " + currentState.name() + " state."
           + " Resource can only be changed when a container is in"
-          + " RUNNING state");
+          + " RUNNING or SCHEDULED state");
     }
+
     // Check validity of the target resource.
     Resource currentResource = container.getResource();
-    if (currentResource.equals(targetResource)) {
-      LOG.warn("Unable to change resource for container "
-          + containerId.toString()
-          + ". The target resource "
-          + targetResource.toString()
-          + " is the same as the current resource");
-      return;
-    }
-    if (increase && !Resources.fitsIn(currentResource, targetResource)) {
-      throw RPCUtil.getRemoteException("Unable to increase resource for "
-          + "container " + containerId.toString()
-          + ". The target resource "
-          + targetResource.toString()
-          + " is smaller than the current resource "
-          + currentResource.toString());
-    }
-    if (!increase &&
-        (!Resources.fitsIn(Resources.none(), targetResource)
-            || !Resources.fitsIn(targetResource, currentResource))) {
-      throw RPCUtil.getRemoteException("Unable to decrease resource for "
-          + "container " + containerId.toString()
-          + ". The target resource "
-          + targetResource.toString()
-          + " is not smaller than the current resource "
-          + currentResource.toString());
-    }
-    if (increase) {
-      org.apache.hadoop.yarn.api.records.Container increasedContainer =
-          org.apache.hadoop.yarn.api.records.Container.newInstance(
-              containerId, null, null, targetResource, null, null);
+    ExecutionType currentExecType =
+        container.getContainerTokenIdentifier().getExecutionType();
+    boolean isResourceChange = false;
+    boolean isExecTypeUpdate = false;
+    Resource targetResource = containerTokenIdentifier.getResource();
+    ExecutionType targetExecType = containerTokenIdentifier.getExecutionType();
+
+    // Is true if either the resources has increased or execution type
+    // updated from opportunistic to guaranteed
+    boolean isIncrease = false;
+    if (!currentResource.equals(targetResource)) {
+      isResourceChange = true;
+      isIncrease = Resources.fitsIn(currentResource, targetResource)
+          && !Resources.fitsIn(targetResource, currentResource);
+    } else if (!currentExecType.equals(targetExecType)) {
+      isExecTypeUpdate = true;
+      isIncrease = currentExecType == ExecutionType.OPPORTUNISTIC &&
+          targetExecType == ExecutionType.GUARANTEED;
+    }
+    if (isIncrease) {
+      org.apache.hadoop.yarn.api.records.Container increasedContainer = null;
+      if (isResourceChange) {
+        increasedContainer =
+            org.apache.hadoop.yarn.api.records.Container.newInstance(
+                containerId, null, null, targetResource, null, null,
+                currentExecType);
+      } else {
+        increasedContainer =
+            org.apache.hadoop.yarn.api.records.Container.newInstance(
+                containerId, null, null, currentResource, null, null,
+                targetExecType);
+      }
       if (context.getIncreasedContainers().putIfAbsent(containerId,
           increasedContainer) != null){
         throw RPCUtil.getRemoteException("Container " + containerId.toString()
-            + " resource is being increased.");
+            + " resource is being increased -or- " +
+            "is undergoing ExecutionType promoted.");
       }
     }
     this.readLock.lock();
     try {
       if (!serviceStopped) {
-        // Persist container resource change for recovery
-        this.context.getNMStateStore().storeContainerResourceChanged(
-            containerId, containerVersion, targetResource);
-        getContainersMonitor().handle(
-            new ChangeMonitoringContainerResourceEvent(
-                containerId, targetResource));
+        // Dispatch message to ContainerScheduler to actually
+        // make the change.
+        dispatcher.getEventHandler().handle(new UpdateContainerSchedulerEvent(
+            container, containerTokenIdentifier, isResourceChange,
+            isExecTypeUpdate, isIncrease));
       } else {
         throw new YarnException(
             "Unable to change container resource as the NodeManager is "
@@ -1571,8 +1600,11 @@ public class ContainerManagerImpl extends CompositeService implements
       for (org.apache.hadoop.yarn.api.records.Container container
           : containersDecreasedEvent.getContainersToDecrease()) {
         try {
-          changeContainerResourceInternal(container.getId(),
-              container.getVersion(), container.getResource(), false);
+          ContainerTokenIdentifier containerTokenIdentifier =
+              BuilderUtils.newContainerTokenIdentifier(
+                  container.getContainerToken());
+          updateContainerInternal(container.getId(),
+              containerTokenIdentifier);
         } catch (YarnException e) {
           LOG.error("Unable to decrease container resource", e);
         } catch (IOException e) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d7be1d8/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 bd3f06d..f6e567c 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
@@ -39,10 +39,10 @@ public interface Container extends EventHandler<ContainerEvent> {
 
   Resource getResource();
 
-  void setResource(Resource targetResource);
-
   ContainerTokenIdentifier getContainerTokenIdentifier();
 
+  void setContainerTokenIdentifier(ContainerTokenIdentifier token);
+
   String getUser();
   
   ContainerState getContainerState();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d7be1d8/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 c0aa6b0..734a27b 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
@@ -148,9 +148,8 @@ public class ContainerImpl implements Container {
   private final Credentials credentials;
   private final NodeManagerMetrics metrics;
   private volatile ContainerLaunchContext launchContext;
-  private final ContainerTokenIdentifier containerTokenIdentifier;
+  private volatile ContainerTokenIdentifier containerTokenIdentifier;
   private final ContainerId containerId;
-  private volatile Resource resource;
   private final String user;
   private int version;
   private int exitCode = ContainerExitStatus.INVALID;
@@ -201,7 +200,6 @@ public class ContainerImpl implements Container {
         YarnConfiguration.DEFAULT_NM_CONTAINER_DIAGNOSTICS_MAXIMUM_SIZE);
     this.containerTokenIdentifier = containerTokenIdentifier;
     this.containerId = containerTokenIdentifier.getContainerID();
-    this.resource = containerTokenIdentifier.getResource();
     this.diagnostics = new StringBuilder();
     this.credentials = creds;
     this.metrics = metrics;
@@ -269,13 +267,6 @@ public class ContainerImpl implements Container {
     this.exitCode = rcs.getExitCode();
     this.recoveredAsKilled = rcs.getKilled();
     this.diagnostics.append(rcs.getDiagnostics());
-    Resource recoveredCapability = rcs.getCapability();
-    if (recoveredCapability != null
-        && !this.resource.equals(recoveredCapability)) {
-      // resource capability had been updated before NM was down
-      this.resource = Resource.newInstance(recoveredCapability.getMemorySize(),
-          recoveredCapability.getVirtualCores());
-    }
     this.version = rcs.getVersion();
     this.remainingRetryAttempts = rcs.getRemainingRetryAttempts();
     this.workDir = rcs.getWorkDir();
@@ -640,14 +631,8 @@ public class ContainerImpl implements Container {
 
   @Override
   public Resource getResource() {
-    return Resources.clone(this.resource);
-  }
-
-  @Override
-  public void setResource(Resource targetResource) {
-    Resource currentResource = getResource();
-    this.resource = Resources.clone(targetResource);
-    this.metrics.changeContainer(currentResource, targetResource);
+    return Resources.clone(
+        this.containerTokenIdentifier.getResource());
   }
 
   @Override
@@ -661,6 +646,16 @@ public class ContainerImpl implements Container {
   }
 
   @Override
+  public void setContainerTokenIdentifier(ContainerTokenIdentifier token) {
+    this.writeLock.lock();
+    try {
+      this.containerTokenIdentifier = token;
+    } finally {
+      this.writeLock.unlock();
+    }
+  }
+
+  @Override
   public String getWorkDir() {
     return workDir;
   }
@@ -833,7 +828,8 @@ public class ContainerImpl implements Container {
             AuditConstants.FINISH_KILLED_CONTAINER, "ContainerImpl",
             container.containerId.getApplicationAttemptId().getApplicationId(),
             container.containerId);
-        container.metrics.releaseContainer(container.resource);
+        container.metrics.releaseContainer(
+            container.containerTokenIdentifier.getResource());
         container.sendFinishedEvents();
         return ContainerState.DONE;
       }
@@ -1517,7 +1513,8 @@ public class ContainerImpl implements Container {
     @Override
     @SuppressWarnings("unchecked")
     public void transition(ContainerImpl container, ContainerEvent event) {
-      container.metrics.releaseContainer(container.resource);
+      container.metrics.releaseContainer(
+          container.containerTokenIdentifier.getResource());
       if (container.containerMetrics != null) {
         container.containerMetrics
             .recordFinishTimeAndExitCode(clock.getTime(), container.exitCode);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d7be1d8/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 6ee60bd..13e7491 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
@@ -741,19 +741,6 @@ public class ContainersMonitorImpl extends AbstractService implements
     }
   }
 
-  private void changeContainerResource(
-      ContainerId containerId, Resource resource) {
-    Container container = context.getContainers().get(containerId);
-    // Check container existence
-    if (container == null) {
-      LOG.warn("Container " + containerId.toString() + "does not exist");
-      return;
-    }
-    // YARN-5860: Route this through the ContainerScheduler to
-    //       fix containerAllocation
-    container.setResource(resource);
-  }
-
   private void updateContainerMetrics(ContainersMonitorEvent monitoringEvent) {
     if (!containerMetricsEnabled || monitoringEvent == null) {
       return;
@@ -902,8 +889,6 @@ public class ContainersMonitorImpl extends AbstractService implements
       int cpuVcores = changeEvent.getResource().getVirtualCores();
       processTreeInfo.setResourceLimit(pmemLimit, vmemLimit, cpuVcores);
     }
-
-    changeContainerResource(containerId, changeEvent.getResource());
   }
 
   private void onStopMonitoringContainer(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d7be1d8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.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/scheduler/ContainerScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java
index 60d6213..19b4505 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java
@@ -31,6 +31,9 @@ import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit;
 import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus;
 import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
+
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor
+    .ChangeMonitoringContainerResourceEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor;
 
 
@@ -136,6 +139,13 @@ public class ContainerScheduler extends AbstractService implements
     case CONTAINER_COMPLETED:
       onContainerCompleted(event.getContainer());
       break;
+    case UPDATE_CONTAINER:
+      if (event instanceof UpdateContainerSchedulerEvent) {
+        onUpdateContainer((UpdateContainerSchedulerEvent) event);
+      } else {
+        LOG.error("Unknown event type on UpdateCOntainer: " + event.getType());
+      }
+      break;
     case SHED_QUEUED_CONTAINERS:
       shedQueuedOpportunisticContainers();
       break;
@@ -146,6 +156,69 @@ public class ContainerScheduler extends AbstractService implements
   }
 
   /**
+   * We assume that the ContainerManager has already figured out what kind
+   * of update this is.
+   */
+  private void onUpdateContainer(UpdateContainerSchedulerEvent updateEvent) {
+    ContainerId containerId = updateEvent.getContainer().getContainerId();
+    if (updateEvent.isResourceChange()) {
+      if (runningContainers.containsKey(containerId)) {
+        this.utilizationTracker.subtractContainerResource(
+            updateEvent.getContainer());
+        updateEvent.getContainer().setContainerTokenIdentifier(
+            updateEvent.getUpdatedToken());
+        this.utilizationTracker.addContainerResources(
+            updateEvent.getContainer());
+        getContainersMonitor().handle(
+            new ChangeMonitoringContainerResourceEvent(containerId,
+                updateEvent.getUpdatedToken().getResource()));
+      } else {
+        updateEvent.getContainer().setContainerTokenIdentifier(
+            updateEvent.getUpdatedToken());
+      }
+      try {
+        // Persist change in the state store.
+        this.context.getNMStateStore().storeContainerResourceChanged(
+            containerId,
+            updateEvent.getUpdatedToken().getVersion(),
+            updateEvent.getUpdatedToken().getResource());
+      } catch (IOException e) {
+        LOG.warn("Could not store container [" + containerId + "] resource " +
+            "change..", e);
+      }
+    }
+
+    if (updateEvent.isExecTypeUpdate()) {
+      updateEvent.getContainer().setContainerTokenIdentifier(
+          updateEvent.getUpdatedToken());
+      // If this is a running container.. just change the execution type
+      // and be done with it.
+      if (!runningContainers.containsKey(containerId)) {
+        // Promotion or not (Increase signifies either a promotion
+        // or container size increase)
+        if (updateEvent.isIncrease()) {
+          // Promotion of queued container..
+          if (queuedOpportunisticContainers.remove(containerId) != null) {
+            queuedGuaranteedContainers.put(containerId,
+                updateEvent.getContainer());
+          }
+          //Kill opportunistic containers if any to make room for
+          // promotion request
+          killOpportunisticContainers(updateEvent.getContainer());
+        } else {
+          // Demotion of queued container.. Should not happen too often
+          // since you should not find too many queued guaranteed
+          // containers
+          if (queuedGuaranteedContainers.remove(containerId) != null) {
+            queuedOpportunisticContainers.put(containerId,
+                updateEvent.getContainer());
+          }
+        }
+      }
+    }
+  }
+
+  /**
    * Return number of queued containers.
    * @return Number of queued containers.
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d7be1d8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerSchedulerEventType.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/scheduler/ContainerSchedulerEventType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerSchedulerEventType.java
index 086cb9b..917eda0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerSchedulerEventType.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerSchedulerEventType.java
@@ -24,6 +24,7 @@ package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler;
 public enum ContainerSchedulerEventType {
   SCHEDULE_CONTAINER,
   CONTAINER_COMPLETED,
+  UPDATE_CONTAINER,
   // Producer: Node HB response - RM has asked to shed the queue
   SHED_QUEUED_CONTAINERS,
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d7be1d8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/UpdateContainerSchedulerEvent.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/scheduler/UpdateContainerSchedulerEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/UpdateContainerSchedulerEvent.java
new file mode 100644
index 0000000..5384b7e
--- /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/scheduler/UpdateContainerSchedulerEvent.java
@@ -0,0 +1,85 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler;
+
+import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container
+    .Container;
+/**
+ * Update Event consumed by the {@link ContainerScheduler}.
+ */
+public class UpdateContainerSchedulerEvent extends ContainerSchedulerEvent {
+
+  private ContainerTokenIdentifier updatedToken;
+  private boolean isResourceChange;
+  private boolean isExecTypeUpdate;
+  private boolean isIncrease;
+
+  /**
+   * Create instance of Event.
+   *
+   * @param originalContainer Original Container.
+   * @param updatedToken Updated Container Token.
+   * @param isResourceChange is this a Resource Change.
+   * @param isExecTypeUpdate is this an ExecTypeUpdate.
+   * @param isIncrease is this a Container Increase.
+   */
+  public UpdateContainerSchedulerEvent(Container originalContainer,
+      ContainerTokenIdentifier updatedToken, boolean isResourceChange,
+      boolean isExecTypeUpdate, boolean isIncrease) {
+    super(originalContainer, ContainerSchedulerEventType.UPDATE_CONTAINER);
+    this.updatedToken = updatedToken;
+    this.isResourceChange = isResourceChange;
+    this.isExecTypeUpdate = isExecTypeUpdate;
+    this.isIncrease = isIncrease;
+  }
+
+  /**
+   * Update Container Token.
+   *
+   * @return Container Token.
+   */
+  public ContainerTokenIdentifier getUpdatedToken() {
+    return updatedToken;
+  }
+
+  /**
+   * isResourceChange.
+   * @return isResourceChange.
+   */
+  public boolean isResourceChange() {
+    return isResourceChange;
+  }
+
+  /**
+   * isExecTypeUpdate.
+   * @return isExecTypeUpdate.
+   */
+  public boolean isExecTypeUpdate() {
+    return isExecTypeUpdate;
+  }
+
+  /**
+   * isIncrease.
+   * @return isIncrease.
+   */
+  public boolean isIncrease() {
+    return isIncrease;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d7be1d8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.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/TestNodeManagerResync.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java
index 0c025ac..b8cd7dd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java
@@ -682,7 +682,7 @@ public class TestNodeManagerResync {
         try{
           try {
             updateBarrier.await();
-            increaseTokens.add(getContainerToken(targetResource));
+            increaseTokens.add(getContainerToken(targetResource, 1));
             ContainerUpdateRequest updateRequest =
                 ContainerUpdateRequest.newInstance(increaseTokens);
             ContainerUpdateResponse updateResponse =
@@ -710,6 +710,15 @@ public class TestNodeManagerResync {
           getNMContext().getNodeId(), user, resource,
           getNMContext().getContainerTokenSecretManager(), null);
     }
+
+    private Token getContainerToken(Resource resource, int version)
+        throws IOException {
+      ContainerId cId = TestContainerManager.createContainerId(0);
+      return TestContainerManager.createContainerToken(
+          cId, version, DUMMY_RM_IDENTIFIER,
+          getNMContext().getNodeId(), user, resource,
+          getNMContext().getContainerTokenSecretManager(), null);
+    }
   }
 
   public static NMContainerStatus createNMContainerStatus(int id,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d7be1d8/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 d266ac1..6c96a47 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
@@ -421,6 +421,20 @@ public abstract class BaseContainerManagerTest {
             containerTokenIdentifier);
   }
 
+  public static Token createContainerToken(ContainerId cId, int version,
+      long rmIdentifier, NodeId nodeId, String user, Resource resource,
+      NMContainerTokenSecretManager containerTokenSecretManager,
+      LogAggregationContext logAggregationContext) throws IOException {
+    ContainerTokenIdentifier containerTokenIdentifier =
+        new ContainerTokenIdentifier(cId, version, nodeId.toString(), user,
+            resource, System.currentTimeMillis() + 100000L, 123, rmIdentifier,
+            Priority.newInstance(0), 0, logAggregationContext, null,
+            ContainerType.TASK, ExecutionType.GUARANTEED);
+    return BuilderUtils.newContainerToken(nodeId,
+        containerTokenSecretManager.retrievePassword(containerTokenIdentifier),
+        containerTokenIdentifier);
+  }
+
   public static Token createContainerToken(ContainerId cId, long rmIdentifier,
       NodeId nodeId, String user, Resource resource,
       NMContainerTokenSecretManager containerTokenSecretManager,
@@ -431,8 +445,23 @@ public abstract class BaseContainerManagerTest {
             System.currentTimeMillis() + 100000L, 123, rmIdentifier,
             Priority.newInstance(0), 0, logAggregationContext, null,
             ContainerType.TASK, executionType);
-    return BuilderUtils.newContainerToken(nodeId, containerTokenSecretManager
-            .retrievePassword(containerTokenIdentifier),
+    return BuilderUtils.newContainerToken(nodeId,
+        containerTokenSecretManager.retrievePassword(containerTokenIdentifier),
+        containerTokenIdentifier);
+  }
+
+  public static Token createContainerToken(ContainerId cId, int version,
+      long rmIdentifier, NodeId nodeId, String user, Resource resource,
+      NMContainerTokenSecretManager containerTokenSecretManager,
+      LogAggregationContext logAggregationContext, ExecutionType executionType)
+      throws IOException {
+    ContainerTokenIdentifier containerTokenIdentifier =
+        new ContainerTokenIdentifier(cId, version, nodeId.toString(), user,
+            resource, System.currentTimeMillis() + 100000L, 123, rmIdentifier,
+            Priority.newInstance(0), 0, logAggregationContext, null,
+            ContainerType.TASK, executionType);
+    return BuilderUtils.newContainerToken(nodeId,
+        containerTokenSecretManager.retrievePassword(containerTokenIdentifier),
         containerTokenIdentifier);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d7be1d8/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 24d46b6..9844225 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
@@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.server.nodemanager.containermanager;
 
 import static org.junit.Assert.assertEquals;
 import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.timeout;
 import static org.mockito.Mockito.verify;
 
@@ -70,6 +71,7 @@ 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.ExecutionType;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LocalResourceType;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
@@ -80,14 +82,15 @@ import org.apache.hadoop.yarn.api.records.SignalContainerCommand;
 import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.api.records.URL;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.ConfigurationException;
 import org.apache.hadoop.yarn.exceptions.InvalidContainerException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
 import org.apache.hadoop.yarn.security.NMTokenIdentifier;
 import org.apache.hadoop.yarn.server.api.ResourceManagerConstants;
 import org.apache.hadoop.yarn.server.nodemanager.CMgrCompletedAppsEvent;
-import org.apache.hadoop.yarn.server.nodemanager.CMgrDecreaseContainersResourceEvent;
 import org.apache.hadoop.yarn.server.nodemanager.CMgrSignalContainersEvent;
+import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
 import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.Signal;
 import org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor;
 import org.apache.hadoop.yarn.server.nodemanager.DeletionService;
@@ -100,6 +103,7 @@ 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.server.nodemanager.executor.ContainerStartContext;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.junit.Assert;
 import org.junit.Before;
@@ -116,10 +120,34 @@ public class TestContainerManager extends BaseContainerManagerTest {
   static {
     LOG = LogFactory.getLog(TestContainerManager.class);
   }
-  
+
+  private boolean delayContainers = false;
+
+  @Override
+  protected ContainerExecutor createContainerExecutor() {
+    DefaultContainerExecutor exec = new DefaultContainerExecutor() {
+      @Override
+      public int launchContainer(ContainerStartContext ctx)
+          throws IOException, ConfigurationException {
+        if (delayContainers) {
+          try {
+            Thread.sleep(10000);
+          } catch (InterruptedException e) {
+            // Nothing..
+          }
+        }
+        return super.launchContainer(ctx);
+      }
+    };
+    exec.setConf(conf);
+    return spy(exec);
+  }
+
   @Override
   @Before
   public void setup() throws IOException {
+    conf.setInt(
+        YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH, 10);
     super.setup();
   }
   
@@ -1468,7 +1496,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
     Assert.assertEquals(strExceptionMsg,
         ContainerManagerImpl.INVALID_NMTOKEN_MSG);
 
-    ContainerManagerImpl spyContainerMgr = Mockito.spy(cMgrImpl);
+    ContainerManagerImpl spyContainerMgr = spy(cMgrImpl);
     UserGroupInformation ugInfo = UserGroupInformation.createRemoteUser("a");
     Mockito.when(spyContainerMgr.getRemoteUgi()).thenReturn(ugInfo);
     Mockito.when(spyContainerMgr.
@@ -1543,7 +1571,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
     // container will have exited, and won't be in RUNNING state
     ContainerId cId0 = createContainerId(0);
     Token containerToken =
-        createContainerToken(cId0, DUMMY_RM_IDENTIFIER,
+        createContainerToken(cId0, 1, DUMMY_RM_IDENTIFIER,
             context.getNodeId(), user,
                 Resource.newInstance(1234, 3),
                     context.getContainerTokenSecretManager(), null);
@@ -1572,7 +1600,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
       if (cId0.equals(entry.getKey())) {
         Assert.assertTrue(entry.getValue().getMessage()
           .contains("Resource can only be changed when a "
-              + "container is in RUNNING state"));
+              + "container is in RUNNING or SCHEDULED state"));
       } else if (cId7.equals(entry.getKey())) {
         Assert.assertTrue(entry.getValue().getMessage()
             .contains("Container " + cId7.toString()
@@ -1585,89 +1613,6 @@ public class TestContainerManager extends BaseContainerManagerTest {
   }
 
   @Test
-  public void testIncreaseContainerResourceWithInvalidResource() throws Exception {
-    containerManager.start();
-    File scriptFile = Shell.appendScriptExtension(tmpDir, "scriptFile");
-    PrintWriter fileWriter = new PrintWriter(scriptFile);
-    // Construct the Container-id
-    ContainerId cId = createContainerId(0);
-    if (Shell.WINDOWS) {
-      fileWriter.println("@ping -n 100 127.0.0.1 >nul");
-    } else {
-      fileWriter.write("\numask 0");
-      fileWriter.write("\nexec sleep 100");
-    }
-    fileWriter.close();
-    ContainerLaunchContext containerLaunchContext =
-        recordFactory.newRecordInstance(ContainerLaunchContext.class);
-    URL resource_alpha =
-        URL.fromPath(localFS
-            .makeQualified(new Path(scriptFile.getAbsolutePath())));
-    LocalResource rsrc_alpha =
-        recordFactory.newRecordInstance(LocalResource.class);
-    rsrc_alpha.setResource(resource_alpha);
-    rsrc_alpha.setSize(-1);
-    rsrc_alpha.setVisibility(LocalResourceVisibility.APPLICATION);
-    rsrc_alpha.setType(LocalResourceType.FILE);
-    rsrc_alpha.setTimestamp(scriptFile.lastModified());
-    String destinationFile = "dest_file";
-    Map<String, LocalResource> localResources =
-        new HashMap<String, LocalResource>();
-    localResources.put(destinationFile, rsrc_alpha);
-    containerLaunchContext.setLocalResources(localResources);
-    List<String> commands =
-        Arrays.asList(Shell.getRunScriptCommand(scriptFile));
-    containerLaunchContext.setCommands(commands);
-
-    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);
-    // Make sure the container reaches RUNNING state
-    BaseContainerManagerTest.waitForNMContainerState(containerManager, cId,
-        org.apache.hadoop.yarn.server.nodemanager.
-            containermanager.container.ContainerState.RUNNING);
-    // Construct container resource increase request,
-    List<Token> increaseTokens = new ArrayList<>();
-    // Add increase request. The increase request should fail
-    // as the current resource does not fit in the target resource
-    Token containerToken =
-        createContainerToken(cId, DUMMY_RM_IDENTIFIER,
-            context.getNodeId(), user,
-            Resource.newInstance(512, 1),
-            context.getContainerTokenSecretManager(), null);
-    increaseTokens.add(containerToken);
-    ContainerUpdateRequest updateRequest =
-        ContainerUpdateRequest.newInstance(increaseTokens);
-    ContainerUpdateResponse updateResponse =
-        containerManager.updateContainer(updateRequest);
-    // Check response
-    Assert.assertEquals(
-        0, updateResponse.getSuccessfullyUpdatedContainers().size());
-    Assert.assertEquals(1, updateResponse.getFailedRequests().size());
-    for (Map.Entry<ContainerId, SerializedException> entry : updateResponse
-        .getFailedRequests().entrySet()) {
-      if (cId.equals(entry.getKey())) {
-        Assert.assertNotNull("Failed message", entry.getValue().getMessage());
-        Assert.assertTrue(entry.getValue().getMessage()
-            .contains("The target resource "
-                + Resource.newInstance(512, 1).toString()
-                + " is smaller than the current resource "
-                + Resource.newInstance(1024, 1)));
-      } else {
-        throw new YarnException("Received failed request from wrong"
-            + " container: " + entry.getKey().toString());
-      }
-    }
-  }
-
-  @Test
   public void testChangeContainerResource() throws Exception {
     containerManager.start();
     File scriptFile = Shell.appendScriptExtension(tmpDir, "scriptFile");
@@ -1720,7 +1665,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
     List<Token> increaseTokens = new ArrayList<>();
     // Add increase request.
     Resource targetResource = Resource.newInstance(4096, 2);
-    Token containerToken = createContainerToken(cId, DUMMY_RM_IDENTIFIER,
+    Token containerToken = createContainerToken(cId, 1, DUMMY_RM_IDENTIFIER,
         context.getNodeId(), user, targetResource,
             context.getContainerTokenSecretManager(), null);
     increaseTokens.add(containerToken);
@@ -1741,15 +1686,19 @@ public class TestContainerManager extends BaseContainerManagerTest {
     // Check status immediately as resource increase is blocking
     assertEquals(targetResource, containerStatus.getCapability());
     // Simulate a decrease request
-    List<org.apache.hadoop.yarn.api.records.Container> containersToDecrease
-        = new ArrayList<>();
+    List<Token> decreaseTokens = new ArrayList<>();
     targetResource = Resource.newInstance(2048, 2);
-    org.apache.hadoop.yarn.api.records.Container decreasedContainer =
-        org.apache.hadoop.yarn.api.records.Container
-            .newInstance(cId, null, null, targetResource, null, null);
-    containersToDecrease.add(decreasedContainer);
-    containerManager.handle(
-        new CMgrDecreaseContainersResourceEvent(containersToDecrease));
+    Token token = createContainerToken(cId, 2, DUMMY_RM_IDENTIFIER,
+        context.getNodeId(), user, targetResource,
+        context.getContainerTokenSecretManager(), null);
+    decreaseTokens.add(token);
+    updateRequest = ContainerUpdateRequest.newInstance(decreaseTokens);
+    updateResponse = containerManager.updateContainer(updateRequest);
+
+    Assert.assertEquals(
+        1, updateResponse.getSuccessfullyUpdatedContainers().size());
+    Assert.assertTrue(updateResponse.getFailedRequests().isEmpty());
+
     // Check status with retry
     containerStatus = containerManager
         .getContainerStatuses(gcsRequest).getContainerStatuses().get(0);
@@ -1879,7 +1828,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
     ContainerLaunchContext containerLaunchContext =
         recordFactory.newRecordInstance(ContainerLaunchContext.class);
     ContainerLaunchContext spyContainerLaunchContext =
-        Mockito.spy(containerLaunchContext);
+        spy(containerLaunchContext);
     Mockito.when(spyContainerLaunchContext.getLocalResources())
         .thenReturn(localResources);
 
@@ -1924,7 +1873,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
     ContainerLaunchContext containerLaunchContext =
         recordFactory.newRecordInstance(ContainerLaunchContext.class);
     ContainerLaunchContext spyContainerLaunchContext =
-        Mockito.spy(containerLaunchContext);
+        spy(containerLaunchContext);
     Mockito.when(spyContainerLaunchContext.getLocalResources())
         .thenReturn(localResources);
 
@@ -1969,7 +1918,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
     ContainerLaunchContext containerLaunchContext =
         recordFactory.newRecordInstance(ContainerLaunchContext.class);
     ContainerLaunchContext spyContainerLaunchContext =
-        Mockito.spy(containerLaunchContext);
+        spy(containerLaunchContext);
     Mockito.when(spyContainerLaunchContext.getLocalResources())
         .thenReturn(localResources);
 
@@ -1996,4 +1945,122 @@ public class TestContainerManager extends BaseContainerManagerTest {
     Assert.assertTrue(response.getFailedRequests().get(cId).getMessage()
         .contains("Null resource visibility for local resource"));
   }
+
+  @Test
+  public void testContainerUpdateExecTypeOpportunisticToGuaranteed()
+      throws IOException, YarnException, InterruptedException {
+    delayContainers = true;
+    containerManager.start();
+    // Construct the Container-id
+    ContainerId cId = createContainerId(0);
+    ContainerLaunchContext containerLaunchContext =
+        recordFactory.newRecordInstance(ContainerLaunchContext.class);
+
+    StartContainerRequest scRequest =
+        StartContainerRequest.newInstance(
+            containerLaunchContext,
+            createContainerToken(cId, DUMMY_RM_IDENTIFIER,
+                context.getNodeId(), user, BuilderUtils.newResource(512, 1),
+                context.getContainerTokenSecretManager(), null,
+                ExecutionType.OPPORTUNISTIC));
+    List<StartContainerRequest> list = new ArrayList<>();
+    list.add(scRequest);
+    StartContainersRequest allRequests =
+        StartContainersRequest.newInstance(list);
+    containerManager.startContainers(allRequests);
+    // Make sure the container reaches RUNNING state
+    BaseContainerManagerTest.waitForNMContainerState(containerManager, cId,
+        org.apache.hadoop.yarn.server.nodemanager.
+            containermanager.container.ContainerState.RUNNING);
+    // Construct container resource increase request,
+    List<Token> updateTokens = new ArrayList<>();
+    Token containerToken =
+        createContainerToken(cId, 1, DUMMY_RM_IDENTIFIER, context.getNodeId(),
+            user, BuilderUtils.newResource(512, 1),
+            context.getContainerTokenSecretManager(), null,
+            ExecutionType.GUARANTEED);
+    updateTokens.add(containerToken);
+    ContainerUpdateRequest updateRequest =
+        ContainerUpdateRequest.newInstance(updateTokens);
+    ContainerUpdateResponse updateResponse =
+        containerManager.updateContainer(updateRequest);
+
+    Assert.assertEquals(
+        1, updateResponse.getSuccessfullyUpdatedContainers().size());
+    Assert.assertTrue(updateResponse.getFailedRequests().isEmpty());
+
+    //Make sure the container is running
+    List<ContainerId> statList = new ArrayList<ContainerId>();
+    statList.add(cId);
+    GetContainerStatusesRequest statRequest =
+        GetContainerStatusesRequest.newInstance(statList);
+    List<ContainerStatus> containerStatuses = containerManager
+        .getContainerStatuses(statRequest).getContainerStatuses();
+    Assert.assertEquals(1, containerStatuses.size());
+    for (ContainerStatus status : containerStatuses) {
+      Assert.assertEquals(
+          org.apache.hadoop.yarn.api.records.ContainerState.RUNNING,
+          status.getState());
+      Assert.assertEquals(ExecutionType.GUARANTEED, status.getExecutionType());
+    }
+  }
+
+  @Test
+  public void testContainerUpdateExecTypeGuaranteedToOpportunistic()
+      throws IOException, YarnException, InterruptedException {
+    delayContainers = true;
+    containerManager.start();
+    // Construct the Container-id
+    ContainerId cId = createContainerId(0);
+    ContainerLaunchContext containerLaunchContext =
+        recordFactory.newRecordInstance(ContainerLaunchContext.class);
+
+    StartContainerRequest scRequest =
+        StartContainerRequest.newInstance(
+            containerLaunchContext,
+            createContainerToken(cId, DUMMY_RM_IDENTIFIER,
+                context.getNodeId(), user, BuilderUtils.newResource(512, 1),
+                context.getContainerTokenSecretManager(), null));
+    List<StartContainerRequest> list = new ArrayList<>();
+    list.add(scRequest);
+    StartContainersRequest allRequests =
+        StartContainersRequest.newInstance(list);
+    containerManager.startContainers(allRequests);
+    // Make sure the container reaches RUNNING state
+    BaseContainerManagerTest.waitForNMContainerState(containerManager, cId,
+        org.apache.hadoop.yarn.server.nodemanager.
+            containermanager.container.ContainerState.RUNNING);
+    // Construct container resource increase request,
+    List<Token> updateTokens = new ArrayList<>();
+    Token containerToken =
+        createContainerToken(cId, 1, DUMMY_RM_IDENTIFIER, context.getNodeId(),
+            user, BuilderUtils.newResource(512, 1),
+            context.getContainerTokenSecretManager(), null,
+            ExecutionType.OPPORTUNISTIC);
+    updateTokens.add(containerToken);
+    ContainerUpdateRequest updateRequest =
+        ContainerUpdateRequest.newInstance(updateTokens);
+    ContainerUpdateResponse updateResponse =
+        containerManager.updateContainer(updateRequest);
+
+    Assert.assertEquals(
+        1, updateResponse.getSuccessfullyUpdatedContainers().size());
+    Assert.assertTrue(updateResponse.getFailedRequests().isEmpty());
+
+    //Make sure the container is running
+    List<ContainerId> statList = new ArrayList<ContainerId>();
+    statList.add(cId);
+    GetContainerStatusesRequest statRequest =
+        GetContainerStatusesRequest.newInstance(statList);
+    List<ContainerStatus> containerStatuses = containerManager
+        .getContainerStatuses(statRequest).getContainerStatuses();
+    Assert.assertEquals(1, containerStatuses.size());
+    for (ContainerStatus status : containerStatuses) {
+      Assert.assertEquals(
+          org.apache.hadoop.yarn.api.records.ContainerState.RUNNING,
+          status.getState());
+      Assert
+          .assertEquals(ExecutionType.OPPORTUNISTIC, status.getExecutionType());
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d7be1d8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.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/TestContainerManagerRecovery.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java
index d2bd79c..224e99c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java
@@ -652,7 +652,7 @@ public class TestContainerManagerRecovery extends BaseContainerManagerTest {
     final List<Token> increaseTokens = new ArrayList<Token>();
     // add increase request
     Token containerToken = TestContainerManager.createContainerToken(
-        cid, 0, context.getNodeId(), user.getShortUserName(),
+        cid, 1, 0, context.getNodeId(), user.getShortUserName(),
         capability, context.getContainerTokenSecretManager(), null);
     increaseTokens.add(containerToken);
     final ContainerUpdateRequest updateRequest =


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


[19/50] [abbrv] hadoop git commit: HADOOP-14732. ProtobufRpcEngine should use Time.monotonicNow to measure durations. Contributed by Hanisha Koneru.

Posted by jh...@apache.org.
HADOOP-14732. ProtobufRpcEngine should use Time.monotonicNow to measure durations. Contributed by Hanisha Koneru.


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

Branch: refs/heads/YARN-5734
Commit: 8bef4eca28a3466707cc4ea0de0330449319a5eb
Parents: 5558792
Author: Arpit Agarwal <ar...@apache.org>
Authored: Mon Aug 14 15:53:35 2017 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Mon Aug 14 15:53:35 2017 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/ipc/ProtobufRpcEngine.java | 14 +++++++-------
 1 file changed, 7 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8bef4eca/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 639bbad..2c0cfe5 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
@@ -190,7 +190,7 @@ public class ProtobufRpcEngine implements RpcEngine {
         throws ServiceException {
       long startTime = 0;
       if (LOG.isDebugEnabled()) {
-        startTime = Time.now();
+        startTime = Time.monotonicNow();
       }
       
       if (args.length != 2) { // RpcController + Message
@@ -245,7 +245,7 @@ public class ProtobufRpcEngine implements RpcEngine {
       }
 
       if (LOG.isDebugEnabled()) {
-        long callTime = Time.now() - startTime;
+        long callTime = Time.monotonicNow() - startTime;
         LOG.debug("Call: " + method.getName() + " took " + callTime + "ms");
       }
       
@@ -373,19 +373,19 @@ public class ProtobufRpcEngine implements RpcEngine {
         this.server = currentCallInfo.get().server;
         this.call = Server.getCurCall().get();
         this.methodName = currentCallInfo.get().methodName;
-        this.setupTime = Time.now();
+        this.setupTime = Time.monotonicNow();
       }
 
       @Override
       public void setResponse(Message message) {
-        long processingTime = Time.now() - setupTime;
+        long processingTime = Time.monotonicNow() - setupTime;
         call.setDeferredResponse(RpcWritable.wrap(message));
         server.updateDeferredMetrics(methodName, processingTime);
       }
 
       @Override
       public void error(Throwable t) {
-        long processingTime = Time.now() - setupTime;
+        long processingTime = Time.monotonicNow() - setupTime;
         String detailedMetricsName = t.getClass().getSimpleName();
         server.updateDeferredMetrics(detailedMetricsName, processingTime);
         call.setDeferredError(t);
@@ -513,7 +513,7 @@ public class ProtobufRpcEngine implements RpcEngine {
         Message param = request.getValue(prototype);
 
         Message result;
-        long startTime = Time.now();
+        long startTime = Time.monotonicNow();
         int qTime = (int) (startTime - receiveTime);
         Exception exception = null;
         boolean isDeferred = false;
@@ -537,7 +537,7 @@ public class ProtobufRpcEngine implements RpcEngine {
           throw e;
         } finally {
           currentCallInfo.set(null);
-          int processingTime = (int) (Time.now() - startTime);
+          int processingTime = (int) (Time.monotonicNow() - startTime);
           if (LOG.isDebugEnabled()) {
             String msg =
                 "Served: " + methodName + (isDeferred ? ", deferred" : "") +


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


[33/50] [abbrv] hadoop git commit: YARN-6900. ZooKeeper based implementation of the FederationStateStore. (Íñigo Goiri via Subru).

Posted by jh...@apache.org.
YARN-6900. ZooKeeper based implementation of the FederationStateStore. (Íñigo Goiri via Subru).


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

Branch: refs/heads/YARN-5734
Commit: de462da04e167a04b89ecf0f40d464cf39dc6549
Parents: 1455306
Author: Subru Krishnan <su...@apache.org>
Authored: Wed Aug 16 11:43:24 2017 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Wed Aug 16 11:43:24 2017 -0700

----------------------------------------------------------------------
 .../hadoop/yarn/conf/YarnConfiguration.java     |   8 +
 .../yarn/conf/TestYarnConfigurationFields.java  |   4 +
 .../hadoop-yarn-server-common/pom.xml           |   5 +
 .../impl/ZookeeperFederationStateStore.java     | 634 +++++++++++++++++++
 .../impl/TestZookeeperFederationStateStore.java |  89 +++
 .../TestFederationStateStoreFacadeRetry.java    |  20 +-
 .../src/site/markdown/Federation.md             |  56 +-
 7 files changed, 785 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/de462da0/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 8acaef8..8515e0a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
@@ -2629,6 +2629,14 @@ public class YarnConfiguration extends Configuration {
 
   public static final String DEFAULT_FEDERATION_POLICY_MANAGER_PARAMS = "";
 
+  public static final String FEDERATION_STATESTORE_ZK_PREFIX =
+      FEDERATION_PREFIX + "zk-state-store.";
+  /** Parent znode path under which ZKRMStateStore will create znodes. */
+  public static final String FEDERATION_STATESTORE_ZK_PARENT_PATH =
+      FEDERATION_STATESTORE_ZK_PREFIX + "parent-path";
+  public static final String DEFAULT_FEDERATION_STATESTORE_ZK_PARENT_PATH =
+      "/federationstore";
+
   private static final String FEDERATION_STATESTORE_SQL_PREFIX =
       FEDERATION_PREFIX + "state-store.sql.";
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/de462da0/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 91a8b0a..c40c2c5 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,6 +96,10 @@ public class TestYarnConfigurationFields extends TestConfigurationFieldsBase {
     configurationPropsToSkipCompare
         .add(YarnConfiguration.DEFAULT_FEDERATION_POLICY_MANAGER_PARAMS);
 
+    // Federation StateStore ZK implementation configs to be ignored
+    configurationPropsToSkipCompare.add(
+        YarnConfiguration.FEDERATION_STATESTORE_ZK_PARENT_PATH);
+
     // Federation StateStore SQL implementation configs to be ignored
     configurationPropsToSkipCompare
         .add(YarnConfiguration.FEDERATION_STATESTORE_SQL_JDBC_CLASS);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/de462da0/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 441a574..e8d3880 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
@@ -130,6 +130,11 @@
         </exclusion>
       </exclusions>
     </dependency>
+    <dependency>
+      <groupId>org.apache.curator</groupId>
+      <artifactId>curator-test</artifactId>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 
   <build>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/de462da0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/ZookeeperFederationStateStore.java
----------------------------------------------------------------------
diff --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/ZookeeperFederationStateStore.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/ZookeeperFederationStateStore.java
new file mode 100644
index 0000000..6ae7d3c
--- /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/ZookeeperFederationStateStore.java
@@ -0,0 +1,634 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.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 static org.apache.hadoop.util.curator.ZKCuratorManager.getNodePath;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.List;
+import java.util.TimeZone;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.curator.ZKCuratorManager;
+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.federation.proto.YarnServerFederationProtos.SubClusterIdProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SubClusterInfoProto;
+import org.apache.hadoop.yarn.federation.proto.YarnServerFederationProtos.SubClusterPolicyConfigurationProto;
+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.GetSubClusterInfoResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterPoliciesConfigurationsRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterPoliciesConfigurationsResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterPolicyConfigurationRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClusterPolicyConfigurationResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClustersInfoRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.GetSubClustersInfoResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.SetSubClusterPolicyConfigurationRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.SetSubClusterPolicyConfigurationResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterDeregisterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterDeregisterResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterHeartbeatRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterHeartbeatResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterPolicyConfiguration;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterRegisterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterRegisterResponse;
+import org.apache.hadoop.yarn.server.federation.store.records.SubClusterState;
+import org.apache.hadoop.yarn.server.federation.store.records.UpdateApplicationHomeSubClusterRequest;
+import org.apache.hadoop.yarn.server.federation.store.records.UpdateApplicationHomeSubClusterResponse;
+import org.apache.hadoop.yarn.server.federation.store.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.SubClusterPolicyConfigurationPBImpl;
+import org.apache.hadoop.yarn.server.federation.store.utils.FederationApplicationHomeSubClusterStoreInputValidator;
+import org.apache.hadoop.yarn.server.federation.store.utils.FederationMembershipStateStoreInputValidator;
+import org.apache.hadoop.yarn.server.federation.store.utils.FederationPolicyStoreInputValidator;
+import org.apache.hadoop.yarn.server.federation.store.utils.FederationStateStoreUtils;
+import org.apache.hadoop.yarn.server.records.Version;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.protobuf.InvalidProtocolBufferException;
+
+/**
+ * ZooKeeper implementation of {@link FederationStateStore}.
+ *
+ * The znode structure is as follows:
+ * ROOT_DIR_PATH
+ * |--- MEMBERSHIP
+ * |     |----- SC1
+ * |     |----- SC2
+ * |--- APPLICATION
+ * |     |----- APP1
+ * |     |----- APP2
+ * |--- POLICY
+ *       |----- QUEUE1
+ *       |----- QUEUE1
+ */
+public class ZookeeperFederationStateStore implements FederationStateStore {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ZookeeperFederationStateStore.class);
+
+  private final static String ROOT_ZNODE_NAME_MEMBERSHIP = "memberships";
+  private final static String ROOT_ZNODE_NAME_APPLICATION = "applications";
+  private final static String ROOT_ZNODE_NAME_POLICY = "policies";
+
+  /** Interface to Zookeeper. */
+  private ZKCuratorManager zkManager;
+
+  /** Directory to store the state store data. */
+  private String baseZNode;
+
+  private String appsZNode;
+  private String membershipZNode;
+  private String policiesZNode;
+
+  @Override
+  public void init(Configuration conf) throws YarnException {
+    LOG.info("Initializing ZooKeeper connection");
+
+    baseZNode = conf.get(
+        YarnConfiguration.FEDERATION_STATESTORE_ZK_PARENT_PATH,
+        YarnConfiguration.DEFAULT_FEDERATION_STATESTORE_ZK_PARENT_PATH);
+    try {
+      this.zkManager = new ZKCuratorManager(conf);
+      this.zkManager.start();
+    } catch (IOException e) {
+      LOG.error("Cannot initialize the ZK connection", e);
+    }
+
+    // Base znodes
+    membershipZNode = getNodePath(baseZNode, ROOT_ZNODE_NAME_MEMBERSHIP);
+    appsZNode = getNodePath(baseZNode, ROOT_ZNODE_NAME_APPLICATION);
+    policiesZNode = getNodePath(baseZNode, ROOT_ZNODE_NAME_POLICY);
+
+    // Create base znode for each entity
+    try {
+      zkManager.createRootDirRecursively(membershipZNode);
+      zkManager.createRootDirRecursively(appsZNode);
+      zkManager.createRootDirRecursively(policiesZNode);
+    } catch (Exception e) {
+      String errMsg = "Cannot create base directories: " + e.getMessage();
+      FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+    }
+
+  }
+
+  @Override
+  public void close() throws Exception {
+    if (zkManager != null) {
+      zkManager.close();
+    }
+  }
+
+  @Override
+  public AddApplicationHomeSubClusterResponse addApplicationHomeSubCluster(
+      AddApplicationHomeSubClusterRequest request) throws YarnException {
+
+    FederationApplicationHomeSubClusterStoreInputValidator.validate(request);
+    ApplicationHomeSubCluster app = request.getApplicationHomeSubCluster();
+    ApplicationId appId = app.getApplicationId();
+
+    // Try to write the subcluster
+    SubClusterId homeSubCluster = app.getHomeSubCluster();
+    try {
+      putApp(appId, homeSubCluster, false);
+    } catch (Exception e) {
+      String errMsg = "Cannot add application home subcluster for " + appId;
+      FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+    }
+
+    // Check for the actual subcluster
+    try {
+      homeSubCluster = getApp(appId);
+    } catch (Exception e) {
+      String errMsg = "Cannot check app home subcluster for " + appId;
+      FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+    }
+
+    return AddApplicationHomeSubClusterResponse
+        .newInstance(homeSubCluster);
+  }
+
+  @Override
+  public UpdateApplicationHomeSubClusterResponse
+      updateApplicationHomeSubCluster(
+          UpdateApplicationHomeSubClusterRequest request)
+              throws YarnException {
+
+    FederationApplicationHomeSubClusterStoreInputValidator.validate(request);
+    ApplicationHomeSubCluster app = request.getApplicationHomeSubCluster();
+    ApplicationId appId = app.getApplicationId();
+    SubClusterId homeSubCluster = getApp(appId);
+    if (homeSubCluster == null) {
+      String errMsg = "Application " + appId + " does not exist";
+      FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+    }
+    SubClusterId newSubClusterId =
+        request.getApplicationHomeSubCluster().getHomeSubCluster();
+    putApp(appId, newSubClusterId, true);
+    return UpdateApplicationHomeSubClusterResponse.newInstance();
+  }
+
+  @Override
+  public GetApplicationHomeSubClusterResponse getApplicationHomeSubCluster(
+      GetApplicationHomeSubClusterRequest request) throws YarnException {
+
+    FederationApplicationHomeSubClusterStoreInputValidator.validate(request);
+    ApplicationId appId = request.getApplicationId();
+    SubClusterId homeSubCluster = getApp(appId);
+    if (homeSubCluster == null) {
+      String errMsg = "Application " + appId + " does not exist";
+      FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+    }
+    return GetApplicationHomeSubClusterResponse.newInstance(
+        ApplicationHomeSubCluster.newInstance(appId, homeSubCluster));
+  }
+
+  @Override
+  public GetApplicationsHomeSubClusterResponse getApplicationsHomeSubCluster(
+      GetApplicationsHomeSubClusterRequest request) throws YarnException {
+    List<ApplicationHomeSubCluster> result = new ArrayList<>();
+
+    try {
+      for (String child : zkManager.getChildren(appsZNode)) {
+        ApplicationId appId = ApplicationId.fromString(child);
+        SubClusterId homeSubCluster = getApp(appId);
+        ApplicationHomeSubCluster app =
+            ApplicationHomeSubCluster.newInstance(appId, homeSubCluster);
+        result.add(app);
+      }
+    } catch (Exception e) {
+      String errMsg = "Cannot get apps: " + e.getMessage();
+      FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+    }
+
+    return GetApplicationsHomeSubClusterResponse.newInstance(result);
+  }
+
+  @Override
+  public DeleteApplicationHomeSubClusterResponse
+      deleteApplicationHomeSubCluster(
+          DeleteApplicationHomeSubClusterRequest request)
+              throws YarnException {
+
+    FederationApplicationHomeSubClusterStoreInputValidator.validate(request);
+    ApplicationId appId = request.getApplicationId();
+    String appZNode = getNodePath(appsZNode, appId.toString());
+
+    boolean exists = false;
+    try {
+      exists = zkManager.exists(appZNode);
+    } catch (Exception e) {
+      String errMsg = "Cannot check app: " + e.getMessage();
+      FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+    }
+    if (!exists) {
+      String errMsg = "Application " + appId + " does not exist";
+      FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+    }
+
+    try {
+      zkManager.delete(appZNode);
+    } catch (Exception e) {
+      String errMsg = "Cannot delete app: " + e.getMessage();
+      FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+    }
+
+    return DeleteApplicationHomeSubClusterResponse.newInstance();
+  }
+
+  @Override
+  public SubClusterRegisterResponse registerSubCluster(
+      SubClusterRegisterRequest request) throws YarnException {
+    FederationMembershipStateStoreInputValidator.validate(request);
+    SubClusterInfo subClusterInfo = request.getSubClusterInfo();
+    SubClusterId subclusterId = subClusterInfo.getSubClusterId();
+
+    // Update the heartbeat time
+    long currentTime = getCurrentTime();
+    subClusterInfo.setLastHeartBeat(currentTime);
+
+    try {
+      putSubclusterInfo(subclusterId, subClusterInfo, true);
+    } catch (Exception e) {
+      String errMsg = "Cannot register subcluster: " + e.getMessage();
+      FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+    }
+    return SubClusterRegisterResponse.newInstance();
+  }
+
+  @Override
+  public SubClusterDeregisterResponse deregisterSubCluster(
+      SubClusterDeregisterRequest request) throws YarnException {
+    FederationMembershipStateStoreInputValidator.validate(request);
+    SubClusterId subClusterId = request.getSubClusterId();
+    SubClusterState state = request.getState();
+
+    // Get the current information and update it
+    SubClusterInfo subClusterInfo = getSubclusterInfo(subClusterId);
+    if (subClusterInfo == null) {
+      String errMsg = "SubCluster " + subClusterId + " not found";
+      FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+    } else {
+      subClusterInfo.setState(state);
+      putSubclusterInfo(subClusterId, subClusterInfo, true);
+    }
+
+    return SubClusterDeregisterResponse.newInstance();
+  }
+
+  @Override
+  public SubClusterHeartbeatResponse subClusterHeartbeat(
+      SubClusterHeartbeatRequest request) throws YarnException {
+
+    FederationMembershipStateStoreInputValidator.validate(request);
+    SubClusterId subClusterId = request.getSubClusterId();
+
+    SubClusterInfo subClusterInfo = getSubclusterInfo(subClusterId);
+    if (subClusterInfo == null) {
+      String errMsg = "SubCluster " + subClusterId
+          + " does not exist; cannot heartbeat";
+      FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+    }
+
+    long currentTime = getCurrentTime();
+    subClusterInfo.setLastHeartBeat(currentTime);
+    subClusterInfo.setState(request.getState());
+    subClusterInfo.setCapability(request.getCapability());
+
+    putSubclusterInfo(subClusterId, subClusterInfo, true);
+
+    return SubClusterHeartbeatResponse.newInstance();
+  }
+
+  @Override
+  public GetSubClusterInfoResponse getSubCluster(
+      GetSubClusterInfoRequest request) throws YarnException {
+
+    FederationMembershipStateStoreInputValidator.validate(request);
+    SubClusterId subClusterId = request.getSubClusterId();
+    SubClusterInfo subClusterInfo = null;
+    try {
+      subClusterInfo = getSubclusterInfo(subClusterId);
+      if (subClusterInfo == null) {
+        LOG.warn("The queried SubCluster: {} does not exist.", subClusterId);
+        return null;
+      }
+    } catch (Exception e) {
+      String errMsg = "Cannot get subcluster: " + e.getMessage();
+      FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+    }
+    return GetSubClusterInfoResponse.newInstance(subClusterInfo);
+  }
+
+  @Override
+  public GetSubClustersInfoResponse getSubClusters(
+      GetSubClustersInfoRequest request) throws YarnException {
+    List<SubClusterInfo> result = new ArrayList<>();
+
+    try {
+      for (String child : zkManager.getChildren(membershipZNode)) {
+        SubClusterId subClusterId = SubClusterId.newInstance(child);
+        SubClusterInfo info = getSubclusterInfo(subClusterId);
+        if (!request.getFilterInactiveSubClusters() ||
+            info.getState().isActive()) {
+          result.add(info);
+        }
+      }
+    } catch (Exception e) {
+      String errMsg = "Cannot get subclusters: " + e.getMessage();
+      FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+    }
+    return GetSubClustersInfoResponse.newInstance(result);
+  }
+
+
+  @Override
+  public GetSubClusterPolicyConfigurationResponse getPolicyConfiguration(
+      GetSubClusterPolicyConfigurationRequest request) throws YarnException {
+
+    FederationPolicyStoreInputValidator.validate(request);
+    String queue = request.getQueue();
+    SubClusterPolicyConfiguration policy = null;
+    try {
+      policy = getPolicy(queue);
+    } catch (Exception e) {
+      String errMsg = "Cannot get policy: " + e.getMessage();
+      FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+    }
+
+    if (policy == null) {
+      LOG.warn("Policy for queue: {} does not exist.", queue);
+      return null;
+    }
+    return GetSubClusterPolicyConfigurationResponse
+        .newInstance(policy);
+  }
+
+  @Override
+  public SetSubClusterPolicyConfigurationResponse setPolicyConfiguration(
+      SetSubClusterPolicyConfigurationRequest request) throws YarnException {
+
+    FederationPolicyStoreInputValidator.validate(request);
+    SubClusterPolicyConfiguration policy =
+        request.getPolicyConfiguration();
+    try {
+      String queue = policy.getQueue();
+      putPolicy(queue, policy, true);
+    } catch (Exception e) {
+      String errMsg = "Cannot set policy: " + e.getMessage();
+      FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+    }
+    return SetSubClusterPolicyConfigurationResponse.newInstance();
+  }
+
+  @Override
+  public GetSubClusterPoliciesConfigurationsResponse getPoliciesConfigurations(
+      GetSubClusterPoliciesConfigurationsRequest request) throws YarnException {
+    List<SubClusterPolicyConfiguration> result = new ArrayList<>();
+
+    try {
+      for (String child : zkManager.getChildren(policiesZNode)) {
+        SubClusterPolicyConfiguration policy = getPolicy(child);
+        result.add(policy);
+      }
+    } catch (Exception e) {
+      String errMsg = "Cannot get policies: " + e.getMessage();
+      FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+    }
+    return GetSubClusterPoliciesConfigurationsResponse.newInstance(result);
+  }
+
+  @Override
+  public Version getCurrentVersion() {
+    return null;
+  }
+
+  @Override
+  public Version loadVersion() {
+    return null;
+  }
+
+  /**
+   * Get the subcluster for an application.
+   * @param appId Application identifier.
+   * @return Subcluster identifier.
+   * @throws Exception If it cannot contact ZooKeeper.
+   */
+  private SubClusterId getApp(final ApplicationId appId) throws YarnException {
+    String appZNode = getNodePath(appsZNode, appId.toString());
+
+    SubClusterId subClusterId = null;
+    byte[] data = get(appZNode);
+    if (data != null) {
+      try {
+        subClusterId = new SubClusterIdPBImpl(
+            SubClusterIdProto.parseFrom(data));
+      } catch (InvalidProtocolBufferException e) {
+        String errMsg = "Cannot parse application at " + appZNode;
+        FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+      }
+    }
+    return subClusterId;
+  }
+
+  /**
+   * Put an application.
+   * @param appId Application identifier.
+   * @param subClusterId Subcluster identifier.
+   * @throws Exception If it cannot contact ZooKeeper.
+   */
+  private void putApp(final ApplicationId appId,
+      final SubClusterId subClusterId, boolean update)
+          throws YarnException {
+    String appZNode = getNodePath(appsZNode, appId.toString());
+    SubClusterIdProto proto =
+        ((SubClusterIdPBImpl)subClusterId).getProto();
+    byte[] data = proto.toByteArray();
+    put(appZNode, data, update);
+  }
+
+  /**
+   * Get the current information for a subcluster from Zookeeper.
+   * @param subclusterId Subcluster identifier.
+   * @return Subcluster information or null if it doesn't exist.
+   * @throws Exception If it cannot contact ZooKeeper.
+   */
+  private SubClusterInfo getSubclusterInfo(final SubClusterId subclusterId)
+      throws YarnException {
+    String memberZNode = getNodePath(membershipZNode, subclusterId.toString());
+
+    SubClusterInfo policy = null;
+    byte[] data = get(memberZNode);
+    if (data != null) {
+      try {
+        policy = new SubClusterInfoPBImpl(
+            SubClusterInfoProto.parseFrom(data));
+      } catch (InvalidProtocolBufferException e) {
+        String errMsg = "Cannot parse subcluster info at " + memberZNode;
+        FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+      }
+    }
+    return policy;
+  }
+
+  /**
+   * Put the subcluster information in Zookeeper.
+   * @param subclusterId Subcluster identifier.
+   * @param subClusterInfo Subcluster information.
+   * @throws Exception If it cannot contact ZooKeeper.
+   */
+  private void putSubclusterInfo(final SubClusterId subclusterId,
+      final SubClusterInfo subClusterInfo, final boolean update)
+          throws YarnException {
+    String memberZNode = getNodePath(membershipZNode, subclusterId.toString());
+    SubClusterInfoProto proto =
+        ((SubClusterInfoPBImpl)subClusterInfo).getProto();
+    byte[] data = proto.toByteArray();
+    put(memberZNode, data, update);
+  }
+
+  /**
+   * Get the queue policy from Zookeeper.
+   * @param queue Name of the queue.
+   * @return Subcluster policy configuration.
+   * @throws YarnException If it cannot contact ZooKeeper.
+   */
+  private SubClusterPolicyConfiguration getPolicy(final String queue)
+      throws YarnException {
+    String policyZNode = getNodePath(policiesZNode, queue);
+
+    SubClusterPolicyConfiguration policy = null;
+    byte[] data = get(policyZNode);
+    if (data != null) {
+      try {
+        policy = new SubClusterPolicyConfigurationPBImpl(
+            SubClusterPolicyConfigurationProto.parseFrom(data));
+      } catch (InvalidProtocolBufferException e) {
+        String errMsg = "Cannot parse policy at " + policyZNode;
+        FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+      }
+    }
+    return policy;
+  }
+
+  /**
+   * Put the subcluster information in Zookeeper.
+   * @param queue Name of the queue.
+   * @param policy Subcluster policy configuration.
+   * @throws YarnException If it cannot contact ZooKeeper.
+   */
+  private void putPolicy(final String queue,
+      final SubClusterPolicyConfiguration policy, boolean update)
+          throws YarnException {
+    String policyZNode = getNodePath(policiesZNode, queue);
+
+    SubClusterPolicyConfigurationProto proto =
+        ((SubClusterPolicyConfigurationPBImpl)policy).getProto();
+    byte[] data = proto.toByteArray();
+    put(policyZNode, data, update);
+  }
+
+  /**
+   * Get data from a znode in Zookeeper.
+   * @param znode Path of the znode.
+   * @return Data in the znode.
+   * @throws YarnException If it cannot contact ZooKeeper.
+   */
+  private byte[] get(String znode) throws YarnException {
+    boolean exists = false;
+    try {
+      exists = zkManager.exists(znode);
+    } catch (Exception e) {
+      String errMsg = "Cannot find znode " + znode;
+      FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+    }
+    if (!exists) {
+      LOG.error("{} does not exist", znode);
+      return null;
+    }
+
+    byte[] data = null;
+    try {
+      data = zkManager.getData(znode);
+    } catch (Exception e) {
+      String errMsg = "Cannot get data from znode " + znode
+          + ": " + e.getMessage();
+      FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+    }
+    return data;
+  }
+
+  /**
+   * Put data into a znode in Zookeeper.
+   * @param znode Path of the znode.
+   * @param data Data to write.
+   * @throws YarnException If it cannot contact ZooKeeper.
+   */
+  private void put(String znode, byte[] data, boolean update)
+      throws YarnException {
+    // Create the znode
+    boolean created = false;
+    try {
+      created = zkManager.create(znode);
+    } catch (Exception e) {
+      String errMsg = "Cannot create znode " + znode + ": " + e.getMessage();
+      FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+    }
+    if (!created) {
+      LOG.debug("{} not created", znode);
+      if (!update) {
+        LOG.info("{} already existed and we are not updating", znode);
+        return;
+      }
+    }
+
+    // Write the data into the znode
+    try {
+      zkManager.setData(znode, data, -1);
+    } catch (Exception e) {
+      String errMsg = "Cannot write data into znode " + znode
+          + ": " + e.getMessage();
+      FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+    }
+  }
+
+  /**
+   * Get the current time.
+   * @return Current time in milliseconds.
+   */
+  private static long getCurrentTime() {
+    Calendar cal = Calendar.getInstance(TimeZone.getTimeZone("UTC"));
+    return cal.getTimeInMillis();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/de462da0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/TestZookeeperFederationStateStore.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/TestZookeeperFederationStateStore.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/TestZookeeperFederationStateStore.java
new file mode 100644
index 0000000..390b803
--- /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/TestZookeeperFederationStateStore.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.impl;
+
+import java.io.IOException;
+
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.retry.RetryNTimes;
+import org.apache.curator.test.TestingServer;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+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.junit.After;
+import org.junit.Before;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Unit tests for ZookeeperFederationStateStore.
+ */
+public class TestZookeeperFederationStateStore
+    extends FederationStateStoreBaseTest {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestZookeeperFederationStateStore.class);
+
+  /** Zookeeper test server. */
+  private static TestingServer curatorTestingServer;
+  private static CuratorFramework curatorFramework;
+
+  @Before
+  public void before() throws IOException, YarnException {
+    try {
+      curatorTestingServer = new TestingServer();
+      curatorTestingServer.start();
+      String connectString = curatorTestingServer.getConnectString();
+      curatorFramework = CuratorFrameworkFactory.builder()
+          .connectString(connectString)
+          .retryPolicy(new RetryNTimes(100, 100))
+          .build();
+      curatorFramework.start();
+
+      Configuration conf = new YarnConfiguration();
+      conf.set(CommonConfigurationKeys.ZK_ADDRESS, connectString);
+      setConf(conf);
+    } catch (Exception e) {
+      LOG.error("Cannot initialize ZooKeeper store", e);
+      throw new IOException(e);
+    }
+
+    super.before();
+  }
+
+  @After
+  public void after() throws Exception {
+    super.after();
+
+    curatorFramework.close();
+    try {
+      curatorTestingServer.stop();
+    } catch (IOException e) {
+    }
+  }
+
+  @Override
+  protected FederationStateStore createStateStore() {
+    Configuration conf = new Configuration();
+    super.setConf(conf);
+    return new ZookeeperFederationStateStore();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/de462da0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/TestFederationStateStoreFacadeRetry.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/TestFederationStateStoreFacadeRetry.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/TestFederationStateStoreFacadeRetry.java
index ea43268..868e771 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/TestFederationStateStoreFacadeRetry.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/TestFederationStateStoreFacadeRetry.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.yarn.server.federation.store.exception.FederationStateS
 import org.apache.hadoop.yarn.server.federation.store.exception.FederationStateStoreInvalidInputException;
 import org.apache.hadoop.yarn.server.federation.store.exception.FederationStateStoreRetriableException;
 import org.junit.Assert;
+import org.junit.Before;
 import org.junit.Test;
 
 import com.zaxxer.hikari.pool.HikariPool.PoolInitializationException;
@@ -40,14 +41,18 @@ public class TestFederationStateStoreFacadeRetry {
   private int maxRetries = 4;
   private Configuration conf;
 
+  @Before
+  public void setup() {
+    conf = new Configuration();
+    conf.setInt(YarnConfiguration.CLIENT_FAILOVER_RETRIES, maxRetries);
+  }
+
   /*
    * Test to validate that FederationStateStoreRetriableException is a retriable
    * exception.
    */
   @Test
   public void testFacadeRetriableException() throws Exception {
-    conf = new Configuration();
-    conf.setInt(YarnConfiguration.CLIENT_FAILOVER_RETRIES, maxRetries);
     RetryPolicy policy = FederationStateStoreFacade.createRetryPolicy(conf);
     RetryAction action = policy.shouldRetry(
         new FederationStateStoreRetriableException(""), 0, 0, false);
@@ -66,9 +71,6 @@ public class TestFederationStateStoreFacadeRetry {
    */
   @Test
   public void testFacadeYarnException() throws Exception {
-
-    conf = new Configuration();
-    conf.setInt(YarnConfiguration.CLIENT_FAILOVER_RETRIES, maxRetries);
     RetryPolicy policy = FederationStateStoreFacade.createRetryPolicy(conf);
     RetryAction action = policy.shouldRetry(new YarnException(), 0, 0, false);
     Assert.assertEquals(RetryAction.FAIL.action, action.action);
@@ -80,8 +82,6 @@ public class TestFederationStateStoreFacadeRetry {
    */
   @Test
   public void testFacadeStateStoreException() throws Exception {
-    conf = new Configuration();
-    conf.setInt(YarnConfiguration.CLIENT_FAILOVER_RETRIES, maxRetries);
     RetryPolicy policy = FederationStateStoreFacade.createRetryPolicy(conf);
     RetryAction action = policy
         .shouldRetry(new FederationStateStoreException("Error"), 0, 0, false);
@@ -94,8 +94,6 @@ public class TestFederationStateStoreFacadeRetry {
    */
   @Test
   public void testFacadeInvalidInputException() throws Exception {
-    conf = new Configuration();
-    conf.setInt(YarnConfiguration.CLIENT_FAILOVER_RETRIES, maxRetries);
     RetryPolicy policy = FederationStateStoreFacade.createRetryPolicy(conf);
     RetryAction action = policy.shouldRetry(
         new FederationStateStoreInvalidInputException(""), 0, 0, false);
@@ -107,8 +105,6 @@ public class TestFederationStateStoreFacadeRetry {
    */
   @Test
   public void testFacadeCacheRetriableException() throws Exception {
-    conf = new Configuration();
-    conf.setInt(YarnConfiguration.CLIENT_FAILOVER_RETRIES, maxRetries);
     RetryPolicy policy = FederationStateStoreFacade.createRetryPolicy(conf);
     RetryAction action =
         policy.shouldRetry(new CacheLoaderException(""), 0, 0, false);
@@ -128,8 +124,6 @@ public class TestFederationStateStoreFacadeRetry {
   @Test
   public void testFacadePoolInitRetriableException() throws Exception {
     // PoolInitializationException is a retriable exception
-    conf = new Configuration();
-    conf.setInt(YarnConfiguration.CLIENT_FAILOVER_RETRIES, maxRetries);
     RetryPolicy policy = FederationStateStoreFacade.createRetryPolicy(conf);
     RetryAction action = policy.shouldRetry(
         new PoolInitializationException(new YarnException()), 0, 0, false);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/de462da0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/Federation.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/Federation.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/Federation.md
index 3e3580c..8a6c137 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/Federation.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/Federation.md
@@ -129,6 +129,7 @@ AMRMProxy, Global Policy Generator (GPG) and Router work together to make this h
 
 
 The figure shows a sequence diagram for the following job execution flow:
+
 1. The Router receives an application submission request that is complaint to the YARN Application Client Protocol.
 2. The router interrogates a routing table / policy to choose the “home RM” for the job (the policy configuration is received from the state-store on heartbeat).
 3. The router queries the membership state to determine the endpoint of the home RM.
@@ -160,15 +161,50 @@ These are common configurations that should appear in the **conf/yarn-site.xml**
 | Property | Example | Description |
 |:---- |:---- |
 |`yarn.federation.enabled` | `true` | Whether federation is enabled or not |
+|`yarn.resourcemanager.cluster-id` | `<unique-subcluster-id>` | The unique subcluster identifier for this RM (same as the one used for HA). |
+
+####State-Store:
+
+Currently, we support ZooKeeper and SQL based implementations of the state-store.
+
+**Note:** The State-Store implementation must always be overwritten with one of the below.
+
+ZooKeeper: one must set the ZooKeeper settings for Hadoop:
+
+| Property | Example | Description |
+|:---- |:---- |
+|`yarn.federation.state-store.class` | `org.apache.hadoop.yarn.server.federation.store.impl.ZookeeperFederationStateStore` | The type of state-store to use. |
+|`hadoop.zk.address` | `host:port` | The address for the ZooKeeper ensemble. |
+
+SQL: one must setup the following parameters:
+
+| Property | Example | Description |
+|:---- |:---- |
 |`yarn.federation.state-store.class` | `org.apache.hadoop.yarn.server.federation.store.impl.SQLFederationStateStore` | The type of state-store to use. |
 |`yarn.federation.state-store.sql.url` | `jdbc:mysql://<host>:<port>/FederationStateStore` | For SQLFederationStateStore the name of the DB where the state is stored. |
 |`yarn.federation.state-store.sql.jdbc-class` | `com.mysql.jdbc.jdbc2.optional.MysqlDataSource` | For SQLFederationStateStore the jdbc class to use. |
 |`yarn.federation.state-store.sql.username` | `<dbuser>` | For SQLFederationStateStore the username for the DB connection. |
 |`yarn.federation.state-store.sql.password` | `<dbpass>` | For SQLFederationStateStore the password for the DB connection. |
-|`yarn.resourcemanager.cluster-id` | `<unique-subcluster-id>` | The unique subcluster identifier for this RM (same as the one used for HA). |
 
+We provide scripts for MySQL and Microsoft SQL Server.
 
-Optional:
+For MySQL, one must download the latest jar version 5.x from [MVN Repository](https://mvnrepository.com/artifact/mysql/mysql-connector-java) and add it to the CLASSPATH.
+Then the DB schema is created by executing the following SQL scripts in the database:
+
+1. **sbin/FederationStateStore/MySQL/FederationStateStoreDatabase.sql**.
+2. **sbin/FederationStateStore/MySQL/FederationStateStoreUser.sql**.
+3. **sbin/FederationStateStore/MySQL/FederationStateStoreTables.sql**.
+4. **sbin/FederationStateStore/MySQL/FederationStateStoreStoredProcs.sql**.
+
+In the same directory we provide scripts to drop the Stored Procedures, the Tables, the User and the Database.
+
+**Note:** the FederationStateStoreUser.sql defines a default user/password for the DB that you are **highly encouraged** to set this to a proper strong password.
+
+For SQL-Server, the process is similar, but the jdbc driver is already included.
+SQL-Server scripts are located in **sbin/FederationStateStore/SQLServer/**.
+
+
+####Optional:
 
 | Property | Example | Description |
 |:---- |:---- |
@@ -236,22 +272,6 @@ Optional:
 |`yarn.federation.statestore.max-connections` | `1` | The maximum number of parallel connections from each AMRMProxy to the state-store. This value is typically lower than the router one, since we have many AMRMProxy that could burn-through many DB connections quickly. |
 |`yarn.federation.cache-ttl.secs` | `300` | The time to leave for the AMRMProxy cache. Typically larger than at the router, as the number of AMRMProxy is large, and we want to limit the load to the centralized state-store. |
 
-###State-Store:
-
-Currently, we support only SQL based implementation of state-store (ZooKeeper is in the works), i.e. either MySQL or Microsoft SQL Server.
-
-For MySQL, one must download the latest jar version 5.x from [MVN Repository](https://mvnrepository.com/artifact/mysql/mysql-connector-java) and add it to the CLASSPATH.
-Then the DB schema is created by executing the following SQL scripts in the database:
-1. **sbin/FederationStateStore/MySQL/FederationStateStoreDatabase.sql**.
-2. **sbin/FederationStateStore/MySQL/FederationStateStoreUser.sql**.
-3. **sbin/FederationStateStore/MySQL/FederationStateStoreTables.sql**.
-4. **sbin/FederationStateStore/MySQL/FederationStateStoreStoredProcs.sql**.
-In the same directory we provide scripts to drop the Stored Procedures, the Tables, the User and the Database.
-**Note:** the FederationStateStoreUser.sql defines a default user/password for the DB that you are **highly encouraged** to set this to a proper strong password.
-
-For SQL-Server, the process is similar, but the jdbc driver is already included in the pom (license allows it).
-SQL-Server scripts are located in **sbin/FederationStateStore/SQLServer/**.
-
 Running a Sample Job
 --------------------
 In order to submit jobs to a Federation cluster one must create a seperate set of configs for the client from which jobs will be submitted. In these, the **conf/yarn-site.xml** should have the following additional configurations:


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


[50/50] [abbrv] hadoop git commit: YARN-5947: Create LeveldbConfigurationStore class using Leveldb as backing store. Contributed by Jonathan Hung

Posted by jh...@apache.org.
YARN-5947: Create LeveldbConfigurationStore class using Leveldb as backing store. Contributed by Jonathan Hung


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

Branch: refs/heads/YARN-5734
Commit: 279456ecedd4a94ec0ee477160bb082176f1cc3f
Parents: b44f573
Author: Xuan <xg...@apache.org>
Authored: Mon Jul 31 16:48:40 2017 -0700
Committer: Jonathan Hung <jh...@linkedin.com>
Committed: Thu Aug 17 13:50:11 2017 -0700

----------------------------------------------------------------------
 .../hadoop/yarn/conf/YarnConfiguration.java     |  13 +
 .../src/main/resources/yarn-default.xml         |  29 ++
 .../scheduler/MutableConfigurationProvider.java |   6 +
 .../scheduler/capacity/CapacityScheduler.java   |   3 +
 .../conf/LeveldbConfigurationStore.java         | 314 +++++++++++++++++++
 .../conf/MutableCSConfigurationProvider.java    |  38 ++-
 .../capacity/conf/YarnConfigurationStore.java   |  14 +-
 .../conf/TestYarnConfigurationStore.java        |   3 +-
 8 files changed, 414 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/279456ec/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 e7cbcfa..73f4d16 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
@@ -638,8 +638,21 @@ public class YarnConfiguration extends Configuration {
   public static final String SCHEDULER_CONFIGURATION_STORE_CLASS =
       YARN_PREFIX + "scheduler.configuration.store.class";
   public static final String MEMORY_CONFIGURATION_STORE = "memory";
+  public static final String LEVELDB_CONFIGURATION_STORE = "leveldb";
   public static final String DEFAULT_CONFIGURATION_STORE =
       MEMORY_CONFIGURATION_STORE;
+  public static final String RM_SCHEDCONF_STORE_PATH = YARN_PREFIX
+      + "scheduler.configuration.leveldb-store.path";
+
+  public static final String RM_SCHEDCONF_LEVELDB_COMPACTION_INTERVAL_SECS =
+      YARN_PREFIX
+          + "scheduler.configuration.leveldb-store.compaction-interval-secs";
+  public static final long
+      DEFAULT_RM_SCHEDCONF_LEVELDB_COMPACTION_INTERVAL_SECS = 60 * 60 * 24L;
+
+  public static final String RM_SCHEDCONF_LEVELDB_MAX_LOGS =
+      YARN_PREFIX + "scheduler.configuration.leveldb-store.max-logs";
+  public static final int DEFAULT_RM_SCHEDCONF_LEVELDB_MAX_LOGS = 1000;
 
   public static final String RM_SCHEDULER_MUTATION_ACL_POLICY_CLASS =
       YARN_PREFIX + "scheduler.configuration.mutation.acl-policy.class";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/279456ec/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 dae3a86..bad0492 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
@@ -3241,4 +3241,33 @@
     <value>org.apache.hadoop.yarn.server.resourcemanager.scheduler.DefaultConfigurationMutationACLPolicy</value>
   </property>
 
+  <property>
+    <description>
+      The storage path for LevelDB implementation of configuration store,
+      when yarn.scheduler.configuration.store.class is configured to be
+      "leveldb".
+    </description>
+    <name>yarn.scheduler.configuration.leveldb-store.path</name>
+    <value>${hadoop.tmp.dir}/yarn/system/confstore</value>
+  </property>
+
+  <property>
+    <description>
+      The compaction interval for LevelDB configuration store in secs,
+      when yarn.scheduler.configuration.store.class is configured to be
+      "leveldb". Default is one day.
+    </description>
+    <name>yarn.scheduler.configuration.leveldb-store.compaction-interval-secs</name>
+    <value>86400</value>
+  </property>
+
+  <property>
+    <description>
+      The max number of configuration change log entries kept in LevelDB config
+      store, when yarn.scheduler.configuration.store.class is configured to be
+      "leveldb". Default is 1000.
+    </description>
+    <name>yarn.scheduler.configuration.leveldb-store.max-logs</name>
+    <value>1000</value>
+  </property>
 </configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/279456ec/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/MutableConfigurationProvider.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/MutableConfigurationProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/MutableConfigurationProvider.java
index 86be7c3..1f13467 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/MutableConfigurationProvider.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/MutableConfigurationProvider.java
@@ -29,6 +29,12 @@ import java.io.IOException;
 public interface MutableConfigurationProvider {
 
   /**
+   * Apply transactions which were not committed.
+   * @throws IOException if recovery fails
+   */
+  void recoverConf() throws IOException;
+
+  /**
    * Update the scheduler configuration with the provided key value pairs.
    * @param user User issuing the request
    * @param confUpdate Key-value pairs for configurations to be updated.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/279456ec/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.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/CapacityScheduler.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/CapacityScheduler.java
index fb5bb83..0d46483 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/CapacityScheduler.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/CapacityScheduler.java
@@ -391,6 +391,9 @@ public class CapacityScheduler extends
   @Override
   public void serviceStart() throws Exception {
     startSchedulerThreads();
+    if (this.csConfProvider instanceof MutableConfigurationProvider) {
+      ((MutableConfigurationProvider) csConfProvider).recoverConf();
+    }
     super.serviceStart();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/279456ec/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/LeveldbConfigurationStore.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/conf/LeveldbConfigurationStore.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/conf/LeveldbConfigurationStore.java
new file mode 100644
index 0000000..1534685
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/LeveldbConfigurationStore.java
@@ -0,0 +1,314 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, 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.scheduler.capacity.conf;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.util.Time;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.fusesource.leveldbjni.JniDBFactory;
+import org.fusesource.leveldbjni.internal.NativeDB;
+import org.iq80.leveldb.DB;
+import org.iq80.leveldb.DBComparator;
+import org.iq80.leveldb.DBException;
+import org.iq80.leveldb.DBIterator;
+import org.iq80.leveldb.Options;
+import org.iq80.leveldb.WriteBatch;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutput;
+import java.io.ObjectOutputStream;
+import java.nio.charset.StandardCharsets;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Timer;
+import java.util.TimerTask;
+
+import static org.fusesource.leveldbjni.JniDBFactory.bytes;
+
+/**
+ * A LevelDB implementation of {@link YarnConfigurationStore}.
+ */
+public class LeveldbConfigurationStore implements YarnConfigurationStore {
+
+  public static final Log LOG =
+      LogFactory.getLog(LeveldbConfigurationStore.class);
+
+  private static final String DB_NAME = "yarn-conf-store";
+  private static final String LOG_PREFIX = "log.";
+  private static final String LOG_COMMITTED_TXN = "committedTxn";
+
+  private DB db;
+  private long txnId = 0;
+  private long minTxn = 0;
+  private long maxLogs;
+  private Configuration conf;
+  private LinkedList<LogMutation> pendingMutations = new LinkedList<>();
+  private Timer compactionTimer;
+  private long compactionIntervalMsec;
+
+  @Override
+  public void initialize(Configuration config, Configuration schedConf)
+      throws IOException {
+    this.conf = config;
+    try {
+      this.db = initDatabase(schedConf);
+      this.txnId = Long.parseLong(new String(db.get(bytes(LOG_COMMITTED_TXN)),
+          StandardCharsets.UTF_8));
+      DBIterator itr = db.iterator();
+      itr.seek(bytes(LOG_PREFIX + txnId));
+      // Seek to first uncommitted log
+      itr.next();
+      while (itr.hasNext()) {
+        Map.Entry<byte[], byte[]> entry = itr.next();
+        if (!new String(entry.getKey(), StandardCharsets.UTF_8)
+            .startsWith(LOG_PREFIX)) {
+          break;
+        }
+        pendingMutations.add(deserLogMutation(entry.getValue()));
+      }
+      // Get the earliest txnId stored in logs
+      itr.seekToFirst();
+      if (itr.hasNext()) {
+        Map.Entry<byte[], byte[]> entry = itr.next();
+        byte[] key = entry.getKey();
+        String logId = new String(key, StandardCharsets.UTF_8);
+        if (logId.startsWith(LOG_PREFIX)) {
+          minTxn = Long.parseLong(logId.substring(logId.indexOf('.') + 1));
+        }
+      }
+      this.maxLogs = config.getLong(
+          YarnConfiguration.RM_SCHEDCONF_LEVELDB_MAX_LOGS,
+          YarnConfiguration.DEFAULT_RM_SCHEDCONF_LEVELDB_MAX_LOGS);
+      this.compactionIntervalMsec = config.getLong(
+          YarnConfiguration.RM_SCHEDCONF_LEVELDB_COMPACTION_INTERVAL_SECS,
+          YarnConfiguration
+              .DEFAULT_RM_SCHEDCONF_LEVELDB_COMPACTION_INTERVAL_SECS) * 1000;
+      startCompactionTimer();
+    } catch (Exception e) {
+      throw new IOException(e);
+    }
+  }
+
+  private DB initDatabase(Configuration config) throws Exception {
+    Path storeRoot = createStorageDir();
+    Options options = new Options();
+    options.createIfMissing(false);
+    options.comparator(new DBComparator() {
+      @Override
+      public int compare(byte[] key1, byte[] key2) {
+        String key1Str = new String(key1, StandardCharsets.UTF_8);
+        String key2Str = new String(key2, StandardCharsets.UTF_8);
+        int key1Txn = Integer.MAX_VALUE;
+        int key2Txn = Integer.MAX_VALUE;
+        if (key1Str.startsWith(LOG_PREFIX)) {
+          key1Txn = Integer.parseInt(key1Str.substring(
+              key1Str.indexOf('.') + 1));
+        }
+        if (key2Str.startsWith(LOG_PREFIX)) {
+          key2Txn = Integer.parseInt(key2Str.substring(
+              key2Str.indexOf('.') + 1));
+        }
+        // TODO txnId could overflow, in theory
+        if (key1Txn == Integer.MAX_VALUE && key2Txn == Integer.MAX_VALUE) {
+          if (key1Str.equals(key2Str) && key1Str.equals(LOG_COMMITTED_TXN)) {
+            return 0;
+          } else if (key1Str.equals(LOG_COMMITTED_TXN)) {
+            return -1;
+          } else if (key2Str.equals(LOG_COMMITTED_TXN)) {
+            return 1;
+          }
+          return key1Str.compareTo(key2Str);
+        }
+        return key1Txn - key2Txn;
+      }
+
+      @Override
+      public String name() {
+        return "logComparator";
+      }
+
+      public byte[] findShortestSeparator(byte[] start, byte[] limit) {
+        return start;
+      }
+
+      public byte[] findShortSuccessor(byte[] key) {
+        return key;
+      }
+    });
+    LOG.info("Using conf database at " + storeRoot);
+    File dbfile = new File(storeRoot.toString());
+    try {
+      db = JniDBFactory.factory.open(dbfile, options);
+    } catch (NativeDB.DBException e) {
+      if (e.isNotFound() || e.getMessage().contains(" does not exist ")) {
+        LOG.info("Creating conf database at " + dbfile);
+        options.createIfMissing(true);
+        try {
+          db = JniDBFactory.factory.open(dbfile, options);
+          // Write the initial scheduler configuration
+          WriteBatch initBatch = db.createWriteBatch();
+          for (Map.Entry<String, String> kv : config) {
+            initBatch.put(bytes(kv.getKey()), bytes(kv.getValue()));
+          }
+          initBatch.put(bytes(LOG_COMMITTED_TXN), bytes("0"));
+          db.write(initBatch);
+        } catch (DBException dbErr) {
+          throw new IOException(dbErr.getMessage(), dbErr);
+        }
+      } else {
+        throw e;
+      }
+    }
+    return db;
+  }
+
+  private Path createStorageDir() throws IOException {
+    Path root = getStorageDir();
+    FileSystem fs = FileSystem.getLocal(conf);
+    fs.mkdirs(root, new FsPermission((short) 0700));
+    return root;
+  }
+
+  private Path getStorageDir() throws IOException {
+    String storePath = conf.get(YarnConfiguration.RM_SCHEDCONF_STORE_PATH);
+    if (storePath == null) {
+      throw new IOException("No store location directory configured in " +
+          YarnConfiguration.RM_SCHEDCONF_STORE_PATH);
+    }
+    return new Path(storePath, DB_NAME);
+  }
+
+  @Override
+  public synchronized long logMutation(LogMutation logMutation)
+      throws IOException {
+    logMutation.setId(++txnId);
+    WriteBatch logBatch = db.createWriteBatch();
+    logBatch.put(bytes(LOG_PREFIX + txnId), serLogMutation(logMutation));
+    if (txnId - minTxn >= maxLogs) {
+      logBatch.delete(bytes(LOG_PREFIX + minTxn));
+      minTxn++;
+    }
+    db.write(logBatch);
+    pendingMutations.add(logMutation);
+    return txnId;
+  }
+
+  @Override
+  public synchronized boolean confirmMutation(long id, boolean isValid)
+      throws IOException {
+    WriteBatch updateBatch = db.createWriteBatch();
+    if (isValid) {
+      LogMutation mutation = deserLogMutation(db.get(bytes(LOG_PREFIX + id)));
+      for (Map.Entry<String, String> changes :
+          mutation.getUpdates().entrySet()) {
+        if (changes.getValue() == null || changes.getValue().isEmpty()) {
+          updateBatch.delete(bytes(changes.getKey()));
+        } else {
+          updateBatch.put(bytes(changes.getKey()), bytes(changes.getValue()));
+        }
+      }
+    }
+    updateBatch.put(bytes(LOG_COMMITTED_TXN), bytes(String.valueOf(id)));
+    db.write(updateBatch);
+    // Assumes logMutation and confirmMutation are done in the same
+    // synchronized method. For example,
+    // {@link MutableCSConfigurationProvider#mutateConfiguration(
+    // UserGroupInformation user, SchedConfUpdateInfo confUpdate)}
+    pendingMutations.removeFirst();
+    return true;
+  }
+
+  private byte[] serLogMutation(LogMutation mutation) throws IOException {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    try (ObjectOutput oos = new ObjectOutputStream(baos)) {
+      oos.writeObject(mutation);
+      oos.flush();
+      return baos.toByteArray();
+    }
+  }
+  private LogMutation deserLogMutation(byte[] mutation) throws IOException {
+    try (ObjectInput input = new ObjectInputStream(
+        new ByteArrayInputStream(mutation))) {
+      return (LogMutation) input.readObject();
+    } catch (ClassNotFoundException e) {
+      throw new IOException(e);
+    }
+  }
+
+  @Override
+  public synchronized Configuration retrieve() {
+    DBIterator itr = db.iterator();
+    itr.seek(bytes(LOG_COMMITTED_TXN));
+    Configuration config = new Configuration(false);
+    itr.next();
+    while (itr.hasNext()) {
+      Map.Entry<byte[], byte[]> entry = itr.next();
+      config.set(new String(entry.getKey(), StandardCharsets.UTF_8),
+          new String(entry.getValue(), StandardCharsets.UTF_8));
+    }
+    return config;
+  }
+
+  @Override
+  public List<LogMutation> getPendingMutations() {
+    return pendingMutations;
+  }
+
+  @Override
+  public List<LogMutation> getConfirmedConfHistory(long fromId) {
+    return null; // unimplemented
+  }
+
+  // TODO below was taken from LeveldbRMStateStore, it can probably be
+  // refactored
+  private void startCompactionTimer() {
+    if (compactionIntervalMsec > 0) {
+      compactionTimer = new Timer(
+          this.getClass().getSimpleName() + " compaction timer", true);
+      compactionTimer.schedule(new CompactionTimerTask(),
+          compactionIntervalMsec, compactionIntervalMsec);
+    }
+  }
+
+  private class CompactionTimerTask extends TimerTask {
+    @Override
+    public void run() {
+      long start = Time.monotonicNow();
+      LOG.info("Starting full compaction cycle");
+      try {
+        db.compactRange(null, null);
+      } catch (DBException e) {
+        LOG.error("Error compacting database", e);
+      }
+      long duration = Time.monotonicNow() - start;
+      LOG.info("Full compaction cycle completed in " + duration + " msec");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/279456ec/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/MutableCSConfigurationProvider.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/conf/MutableCSConfigurationProvider.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/conf/MutableCSConfigurationProvider.java
index 670c0f9..9ccc146 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/conf/MutableCSConfigurationProvider.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/conf/MutableCSConfigurationProvider.java
@@ -19,6 +19,8 @@
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.conf;
 
 import com.google.common.base.Joiner;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -49,6 +51,9 @@ import java.util.Map;
 public class MutableCSConfigurationProvider implements CSConfigurationProvider,
     MutableConfigurationProvider {
 
+  public static final Log LOG =
+      LogFactory.getLog(MutableCSConfigurationProvider.class);
+
   private Configuration schedConf;
   private YarnConfigurationStore confStore;
   private ConfigurationMutationACLPolicy aclMutationPolicy;
@@ -68,6 +73,9 @@ public class MutableCSConfigurationProvider implements CSConfigurationProvider,
     case YarnConfiguration.MEMORY_CONFIGURATION_STORE:
       this.confStore = new InMemoryConfigurationStore();
       break;
+    case YarnConfiguration.LEVELDB_CONFIGURATION_STORE:
+      this.confStore = new LeveldbConfigurationStore();
+      break;
     default:
       this.confStore = YarnConfigurationStoreFactory.getStore(config);
       break;
@@ -82,6 +90,9 @@ public class MutableCSConfigurationProvider implements CSConfigurationProvider,
       schedConf.set(kv.getKey(), kv.getValue());
     }
     confStore.initialize(config, schedConf);
+    // After initializing confStore, the store may already have an existing
+    // configuration. Use this one.
+    schedConf = confStore.retrieve();
     this.aclMutationPolicy = ConfigurationMutationACLPolicyFactory
         .getPolicy(config);
     aclMutationPolicy.init(config, rmContext);
@@ -97,7 +108,7 @@ public class MutableCSConfigurationProvider implements CSConfigurationProvider,
   }
 
   @Override
-  public void mutateConfiguration(UserGroupInformation user,
+  public synchronized void mutateConfiguration(UserGroupInformation user,
       SchedConfUpdateInfo confUpdate) throws IOException {
     if (!aclMutationPolicy.isMutationAllowed(user, confUpdate)) {
       throw new AccessControlException("User is not admin of all modified" +
@@ -124,6 +135,31 @@ public class MutableCSConfigurationProvider implements CSConfigurationProvider,
     confStore.confirmMutation(id, true);
   }
 
+  @Override
+  public void recoverConf() throws IOException {
+    List<LogMutation> uncommittedLogs = confStore.getPendingMutations();
+    Configuration oldConf = new Configuration(schedConf);
+    for (LogMutation mutation : uncommittedLogs) {
+      for (Map.Entry<String, String> kv : mutation.getUpdates().entrySet()) {
+        if (kv.getValue() == null) {
+          schedConf.unset(kv.getKey());
+        } else {
+          schedConf.set(kv.getKey(), kv.getValue());
+        }
+      }
+      try {
+        rmContext.getScheduler().reinitialize(conf, rmContext);
+      } catch (IOException e) {
+        schedConf = oldConf;
+        confStore.confirmMutation(mutation.getId(), false);
+        LOG.info("Configuration mutation " + mutation.getId()
+            + " was rejected", e);
+        continue;
+      }
+      confStore.confirmMutation(mutation.getId(), true);
+      LOG.info("Configuration mutation " + mutation.getId()+ " was accepted");
+    }
+  }
 
   private Map<String, String> constructKeyValueConfUpdate(
       SchedConfUpdateInfo mutationInfo) throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/279456ec/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/YarnConfigurationStore.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/conf/YarnConfigurationStore.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/conf/YarnConfigurationStore.java
index 22c0ef8..065c877 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/conf/YarnConfigurationStore.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/conf/YarnConfigurationStore.java
@@ -21,6 +21,8 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.conf;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
 
+import java.io.IOException;
+import java.io.Serializable;
 import java.util.List;
 import java.util.Map;
 
@@ -43,7 +45,7 @@ public interface YarnConfigurationStore {
    * LogMutation encapsulates the fields needed for configuration mutation
    * audit logging and recovery.
    */
-  class LogMutation {
+  class LogMutation implements Serializable {
     private Map<String, String> updates;
     private String user;
     private long id;
@@ -106,16 +108,19 @@ public interface YarnConfigurationStore {
    * Initialize the configuration store.
    * @param conf configuration to initialize store with
    * @param schedConf Initial key-value configuration to persist
+   * @throws IOException if initialization fails
    */
-  void initialize(Configuration conf, Configuration schedConf);
+  void initialize(Configuration conf, Configuration schedConf)
+      throws IOException;
 
   /**
    * Logs the configuration change to backing store. Generates an id associated
    * with this mutation, sets it in {@code logMutation}, and returns it.
    * @param logMutation configuration change to be persisted in write ahead log
    * @return id which configuration store associates with this mutation
+   * @throws IOException if logging fails
    */
-  long logMutation(LogMutation logMutation);
+  long logMutation(LogMutation logMutation) throws IOException;
 
   /**
    * Should be called after {@code logMutation}. Gets the pending mutation
@@ -130,8 +135,9 @@ public interface YarnConfigurationStore {
    * @param isValid if true, update persisted configuration with mutation
    *                associated with {@code id}.
    * @return true on success
+   * @throws IOException if mutation confirmation fails
    */
-  boolean confirmMutation(long id, boolean isValid);
+  boolean confirmMutation(long id, boolean isValid) throws IOException;
 
   /**
    * Retrieve the persisted configuration.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/279456ec/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/TestYarnConfigurationStore.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/scheduler/capacity/conf/TestYarnConfigurationStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/TestYarnConfigurationStore.java
index dff4e77..631ce65 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/TestYarnConfigurationStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/TestYarnConfigurationStore.java
@@ -23,6 +23,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.conf.Yar
 import org.junit.Before;
 import org.junit.Test;
 
+import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
 
@@ -43,7 +44,7 @@ public class TestYarnConfigurationStore {
   }
 
   @Test
-  public void testInMemoryConfigurationStore() {
+  public void testInMemoryConfigurationStore() throws IOException {
     confStore = new InMemoryConfigurationStore();
     confStore.initialize(new Configuration(), schedConf);
     assertEquals("val1", confStore.retrieve().get("key1"));


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


[32/50] [abbrv] hadoop git commit: YARN-7020. TestAMRMProxy#testAMRMProxyTokenRenewal is flakey. Contributed by Robert Kanter

Posted by jh...@apache.org.
YARN-7020. TestAMRMProxy#testAMRMProxyTokenRenewal is flakey. Contributed by Robert Kanter


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

Branch: refs/heads/YARN-5734
Commit: 14553061be0a341df3e628dcaf06717b4630b05e
Parents: 588c190
Author: Jason Lowe <jl...@yahoo-inc.com>
Authored: Wed Aug 16 13:04:36 2017 -0500
Committer: Jason Lowe <jl...@yahoo-inc.com>
Committed: Wed Aug 16 13:04:36 2017 -0500

----------------------------------------------------------------------
 .../apache/hadoop/yarn/client/api/impl/TestAMRMProxy.java | 10 +++++-----
 1 file changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/14553061/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMProxy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMProxy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMProxy.java
index 14df94a..6a063e6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMProxy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMProxy.java
@@ -151,13 +151,13 @@ public class TestAMRMProxy extends BaseAMRMProxyE2ETest {
            YarnClient rmClient = YarnClient.createYarnClient()) {
       Configuration conf = new YarnConfiguration();
       conf.setBoolean(YarnConfiguration.AMRM_PROXY_ENABLED, true);
-      conf.setInt(YarnConfiguration.RM_NM_EXPIRY_INTERVAL_MS, 1500);
-      conf.setInt(YarnConfiguration.RM_NM_HEARTBEAT_INTERVAL_MS, 1500);
-      conf.setInt(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 1500);
+      conf.setInt(YarnConfiguration.RM_NM_EXPIRY_INTERVAL_MS, 4500);
+      conf.setInt(YarnConfiguration.RM_NM_HEARTBEAT_INTERVAL_MS, 4500);
+      conf.setInt(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 4500);
       // RM_AMRM_TOKEN_MASTER_KEY_ROLLING_INTERVAL_SECS should be at least
       // RM_AM_EXPIRY_INTERVAL_MS * 1.5 *3
       conf.setInt(
-          YarnConfiguration.RM_AMRM_TOKEN_MASTER_KEY_ROLLING_INTERVAL_SECS, 6);
+          YarnConfiguration.RM_AMRM_TOKEN_MASTER_KEY_ROLLING_INTERVAL_SECS, 20);
       cluster.init(conf);
       cluster.start();
       final Configuration yarnConf = cluster.getConfig();
@@ -198,7 +198,7 @@ public class TestAMRMProxy extends BaseAMRMProxyE2ETest {
         lastToken = response.getAMRMToken();
 
         // Time slot to be sure the AMRMProxy renew the token
-        Thread.sleep(1500);
+        Thread.sleep(4500);
 
       }
 


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


[41/50] [abbrv] hadoop git commit: YARN-5951. Changes to allow CapacityScheduler to use configuration store

Posted by jh...@apache.org.
YARN-5951. Changes to allow CapacityScheduler to use configuration store


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

Branch: refs/heads/YARN-5734
Commit: 4b89ce32dadaeed30b77310c06b38d19f8583c12
Parents: dd7916d
Author: Jonathan Hung <jh...@linkedin.com>
Authored: Mon Jan 30 19:03:48 2017 -0800
Committer: Jonathan Hung <jh...@linkedin.com>
Committed: Thu Aug 17 13:48:04 2017 -0700

----------------------------------------------------------------------
 .../scheduler/capacity/CapacityScheduler.java   | 37 +++++------
 .../CapacitySchedulerConfiguration.java         |  9 +++
 .../capacity/conf/CSConfigurationProvider.java  | 46 ++++++++++++++
 .../conf/FileBasedCSConfigurationProvider.java  | 67 ++++++++++++++++++++
 .../scheduler/capacity/conf/package-info.java   | 29 +++++++++
 .../capacity/TestCapacityScheduler.java         |  4 +-
 6 files changed, 170 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4b89ce32/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.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/CapacityScheduler.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/CapacityScheduler.java
index e4ca003..bb42cdc 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/CapacityScheduler.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/CapacityScheduler.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 
 import java.io.IOException;
-import java.io.InputStream;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.EnumSet;
@@ -103,6 +102,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.Activi
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivityDiagnosticConstant;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivityState;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.AllocationState;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.conf.CSConfigurationProvider;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.conf.FileBasedCSConfigurationProvider;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.KillableContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.AssignmentInformation;
@@ -165,6 +166,8 @@ public class CapacityScheduler extends
 
   private int maxAssignPerHeartbeat;
 
+  private CSConfigurationProvider csConfProvider;
+
   @Override
   public void setConf(Configuration conf) {
       yarnConf = conf;
@@ -287,7 +290,18 @@ public class CapacityScheduler extends
       IOException {
     try {
       writeLock.lock();
-      this.conf = loadCapacitySchedulerConfiguration(configuration);
+      String confProviderStr = configuration.get(
+          CapacitySchedulerConfiguration.CS_CONF_PROVIDER,
+          CapacitySchedulerConfiguration.DEFAULT_CS_CONF_PROVIDER);
+      if (confProviderStr.equals(
+          CapacitySchedulerConfiguration.FILE_CS_CONF_PROVIDER)) {
+        this.csConfProvider = new FileBasedCSConfigurationProvider(rmContext);
+      } else {
+        throw new IOException("Invalid CS configuration provider: " +
+            confProviderStr);
+      }
+      this.csConfProvider.init(configuration);
+      this.conf = this.csConfProvider.loadConfiguration(configuration);
       validateConf(this.conf);
       this.minimumAllocation = this.conf.getMinimumAllocation();
       initMaximumResourceCapability(this.conf.getMaximumAllocation());
@@ -397,7 +411,7 @@ public class CapacityScheduler extends
       writeLock.lock();
       Configuration configuration = new Configuration(newConf);
       CapacitySchedulerConfiguration oldConf = this.conf;
-      this.conf = loadCapacitySchedulerConfiguration(configuration);
+      this.conf = csConfProvider.loadConfiguration(configuration);
       validateConf(this.conf);
       try {
         LOG.info("Re-initializing queues...");
@@ -1819,23 +1833,6 @@ public class CapacityScheduler extends
     return true;
   }
 
-  private CapacitySchedulerConfiguration loadCapacitySchedulerConfiguration(
-      Configuration configuration) throws IOException {
-    try {
-      InputStream CSInputStream =
-          this.rmContext.getConfigurationProvider()
-              .getConfigurationInputStream(configuration,
-                  YarnConfiguration.CS_CONFIGURATION_FILE);
-      if (CSInputStream != null) {
-        configuration.addResource(CSInputStream);
-        return new CapacitySchedulerConfiguration(configuration, false);
-      }
-      return new CapacitySchedulerConfiguration(configuration, true);
-    } catch (Exception e) {
-      throw new IOException(e);
-    }
-  }
-
   private String getDefaultReservationQueueName(String planQueueName) {
     return planQueueName + ReservationConstants.DEFAULT_QUEUE_SUFFIX;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4b89ce32/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.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/CapacitySchedulerConfiguration.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/CapacitySchedulerConfiguration.java
index 13b9ff6..f40fb3a 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/CapacitySchedulerConfiguration.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/CapacitySchedulerConfiguration.java
@@ -316,6 +316,15 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
   @Private
   public static final int DEFAULT_MAX_ASSIGN_PER_HEARTBEAT = -1;
 
+  public static final String CS_CONF_PROVIDER = PREFIX
+      + "configuration.provider";
+
+  @Private
+  public static final String FILE_CS_CONF_PROVIDER = "file";
+
+  @Private
+  public static final String DEFAULT_CS_CONF_PROVIDER = FILE_CS_CONF_PROVIDER;
+
   AppPriorityACLConfigurationParser priorityACLConfig = new AppPriorityACLConfigurationParser();
 
   public CapacitySchedulerConfiguration() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4b89ce32/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/CSConfigurationProvider.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/conf/CSConfigurationProvider.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/conf/CSConfigurationProvider.java
new file mode 100644
index 0000000..c9984ac
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/CSConfigurationProvider.java
@@ -0,0 +1,46 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, 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.scheduler.capacity.conf;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
+
+import java.io.IOException;
+
+/**
+ * Configuration provider for {@link CapacityScheduler}.
+ */
+public interface CSConfigurationProvider {
+
+  /**
+   * Initialize the configuration provider with given conf.
+   * @param conf configuration to initialize with
+   */
+  void init(Configuration conf);
+
+  /**
+   * Loads capacity scheduler configuration object.
+   * @param conf initial bootstrap configuration
+   * @return CS configuration
+   * @throws IOException if fail to retrieve configuration
+   */
+  CapacitySchedulerConfiguration loadConfiguration(Configuration conf)
+      throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4b89ce32/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/FileBasedCSConfigurationProvider.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/conf/FileBasedCSConfigurationProvider.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/conf/FileBasedCSConfigurationProvider.java
new file mode 100644
index 0000000..51c64fa
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/FileBasedCSConfigurationProvider.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.resourcemanager.scheduler.capacity.conf;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+/**
+ * {@link CapacityScheduler} configuration provider based on local
+ * {@code capacity-scheduler.xml} file.
+ */
+public class FileBasedCSConfigurationProvider implements
+    CSConfigurationProvider {
+
+  private RMContext rmContext;
+
+  /**
+   * Construct file based CS configuration provider with given context.
+   * @param rmContext the RM context
+   */
+  public FileBasedCSConfigurationProvider(RMContext rmContext) {
+    this.rmContext = rmContext;
+  }
+
+  @Override
+  public void init(Configuration conf) {}
+
+  @Override
+  public CapacitySchedulerConfiguration loadConfiguration(Configuration conf)
+      throws IOException {
+    try {
+      InputStream csInputStream =
+          this.rmContext.getConfigurationProvider()
+              .getConfigurationInputStream(conf,
+                  YarnConfiguration.CS_CONFIGURATION_FILE);
+      if (csInputStream != null) {
+        conf.addResource(csInputStream);
+        return new CapacitySchedulerConfiguration(conf, false);
+      }
+      return new CapacitySchedulerConfiguration(conf, true);
+    } catch (Exception e) {
+      throw new IOException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4b89ce32/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/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/scheduler/capacity/conf/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/scheduler/capacity/conf/package-info.java
new file mode 100644
index 0000000..08d0522
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/package-info.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, 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.scheduler.capacity.conf
+ * contains classes related to capacity scheduler configuration management.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.conf;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4b89ce32/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.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/scheduler/capacity/TestCapacityScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
index a526222..52c7ba2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
@@ -248,13 +248,13 @@ public class TestCapacityScheduler {
 
   @Test (timeout = 30000)
   public void testConfValidation() throws Exception {
-    ResourceScheduler scheduler = new CapacityScheduler();
+    CapacityScheduler scheduler = new CapacityScheduler();
     scheduler.setRMContext(resourceManager.getRMContext());
     Configuration conf = new YarnConfiguration();
     conf.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 2048);
     conf.setInt(YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_MB, 1024);
     try {
-      scheduler.reinitialize(conf, mockContext);
+      scheduler.init(conf);
       fail("Exception is expected because the min memory allocation is" +
         " larger than the max memory allocation.");
     } catch (YarnRuntimeException e) {


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


[42/50] [abbrv] hadoop git commit: YARN-5946: Create YarnConfigurationStore interface and InMemoryConfigurationStore class. Contributed by Jonathan Hung

Posted by jh...@apache.org.
YARN-5946: Create YarnConfigurationStore interface and
InMemoryConfigurationStore class. Contributed by Jonathan Hung


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

Branch: refs/heads/YARN-5734
Commit: 7d0a0f675f0ab827e94847e153268e8352b5df23
Parents: 4b89ce3
Author: Xuan <xg...@apache.org>
Authored: Fri Feb 24 15:58:12 2017 -0800
Committer: Jonathan Hung <jh...@linkedin.com>
Committed: Thu Aug 17 13:48:05 2017 -0700

----------------------------------------------------------------------
 .../conf/InMemoryConfigurationStore.java        |  86 +++++++++++
 .../capacity/conf/YarnConfigurationStore.java   | 154 +++++++++++++++++++
 .../conf/TestYarnConfigurationStore.java        |  70 +++++++++
 3 files changed, 310 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7d0a0f67/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/InMemoryConfigurationStore.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/conf/InMemoryConfigurationStore.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/conf/InMemoryConfigurationStore.java
new file mode 100644
index 0000000..a208fb9
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/InMemoryConfigurationStore.java
@@ -0,0 +1,86 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, 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.scheduler.capacity.conf;
+
+import org.apache.hadoop.conf.Configuration;
+
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A default implementation of {@link YarnConfigurationStore}. Doesn't offer
+ * persistent configuration storage, just stores the configuration in memory.
+ */
+public class InMemoryConfigurationStore implements YarnConfigurationStore {
+
+  private Configuration schedConf;
+  private LinkedList<LogMutation> pendingMutations;
+  private long pendingId;
+
+  @Override
+  public void initialize(Configuration conf, Configuration schedConf) {
+    this.schedConf = schedConf;
+    this.pendingMutations = new LinkedList<>();
+    this.pendingId = 0;
+  }
+
+  @Override
+  public synchronized long logMutation(LogMutation logMutation) {
+    logMutation.setId(++pendingId);
+    pendingMutations.add(logMutation);
+    return pendingId;
+  }
+
+  @Override
+  public synchronized boolean confirmMutation(long id, boolean isValid) {
+    LogMutation mutation = pendingMutations.poll();
+    // If confirmMutation is called out of order, discard mutations until id
+    // is reached.
+    while (mutation != null) {
+      if (mutation.getId() == id) {
+        if (isValid) {
+          Map<String, String> mutations = mutation.getUpdates();
+          for (Map.Entry<String, String> kv : mutations.entrySet()) {
+            schedConf.set(kv.getKey(), kv.getValue());
+          }
+        }
+        return true;
+      }
+      mutation = pendingMutations.poll();
+    }
+    return false;
+  }
+
+  @Override
+  public synchronized Configuration retrieve() {
+    return schedConf;
+  }
+
+  @Override
+  public synchronized List<LogMutation> getPendingMutations() {
+    return pendingMutations;
+  }
+
+  @Override
+  public List<LogMutation> getConfirmedConfHistory(long fromId) {
+    // Unimplemented.
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7d0a0f67/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/YarnConfigurationStore.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/conf/YarnConfigurationStore.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/conf/YarnConfigurationStore.java
new file mode 100644
index 0000000..22c0ef8
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/YarnConfigurationStore.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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, 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.scheduler.capacity.conf;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
+
+import java.util.List;
+import java.util.Map;
+
+/**
+ * YarnConfigurationStore exposes the methods needed for retrieving and
+ * persisting {@link CapacityScheduler} configuration via key-value
+ * using write-ahead logging. When configuration mutation is requested, caller
+ * should first log it with {@code logMutation}, which persists this pending
+ * mutation. This mutation is merged to the persisted configuration only after
+ * {@code confirmMutation} is called.
+ *
+ * On startup/recovery, caller should call {@code retrieve} to get all
+ * confirmed mutations, then get pending mutations which were not confirmed via
+ * {@code getPendingMutations}, and replay/confirm them via
+ * {@code confirmMutation} as in the normal case.
+ */
+public interface YarnConfigurationStore {
+
+  /**
+   * LogMutation encapsulates the fields needed for configuration mutation
+   * audit logging and recovery.
+   */
+  class LogMutation {
+    private Map<String, String> updates;
+    private String user;
+    private long id;
+
+    /**
+     * Create log mutation prior to logging.
+     * @param updates key-value configuration updates
+     * @param user user who requested configuration change
+     */
+    public LogMutation(Map<String, String> updates, String user) {
+      this(updates, user, 0);
+    }
+
+    /**
+     * Create log mutation for recovery.
+     * @param updates key-value configuration updates
+     * @param user user who requested configuration change
+     * @param id transaction id of configuration change
+     */
+    LogMutation(Map<String, String> updates, String user, long id) {
+      this.updates = updates;
+      this.user = user;
+      this.id = id;
+    }
+
+    /**
+     * Get key-value configuration updates.
+     * @return map of configuration updates
+     */
+    public Map<String, String> getUpdates() {
+      return updates;
+    }
+
+    /**
+     * Get user who requested configuration change.
+     * @return user who requested configuration change
+     */
+    public String getUser() {
+      return user;
+    }
+
+    /**
+     * Get transaction id of this configuration change.
+     * @return transaction id
+     */
+    public long getId() {
+      return id;
+    }
+
+    /**
+     * Set transaction id of this configuration change.
+     * @param id transaction id
+     */
+    public void setId(long id) {
+      this.id = id;
+    }
+  }
+
+  /**
+   * Initialize the configuration store.
+   * @param conf configuration to initialize store with
+   * @param schedConf Initial key-value configuration to persist
+   */
+  void initialize(Configuration conf, Configuration schedConf);
+
+  /**
+   * Logs the configuration change to backing store. Generates an id associated
+   * with this mutation, sets it in {@code logMutation}, and returns it.
+   * @param logMutation configuration change to be persisted in write ahead log
+   * @return id which configuration store associates with this mutation
+   */
+  long logMutation(LogMutation logMutation);
+
+  /**
+   * Should be called after {@code logMutation}. Gets the pending mutation
+   * associated with {@code id} and marks the mutation as persisted (no longer
+   * pending). If isValid is true, merge the mutation with the persisted
+   * configuration.
+   *
+   * If {@code confirmMutation} is called with ids in a different order than
+   * was returned by {@code logMutation}, the result is implementation
+   * dependent.
+   * @param id id of mutation to be confirmed
+   * @param isValid if true, update persisted configuration with mutation
+   *                associated with {@code id}.
+   * @return true on success
+   */
+  boolean confirmMutation(long id, boolean isValid);
+
+  /**
+   * Retrieve the persisted configuration.
+   * @return configuration as key-value
+   */
+  Configuration retrieve();
+
+  /**
+   * Get the list of pending mutations, in the order they were logged.
+   * @return list of mutations
+   */
+  List<LogMutation> getPendingMutations();
+
+  /**
+   * Get a list of confirmed configuration mutations starting from a given id.
+   * @param fromId id from which to start getting mutations, inclusive
+   * @return list of configuration mutations
+   */
+  List<LogMutation> getConfirmedConfHistory(long fromId);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7d0a0f67/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/TestYarnConfigurationStore.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/scheduler/capacity/conf/TestYarnConfigurationStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/TestYarnConfigurationStore.java
new file mode 100644
index 0000000..dff4e77
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/TestYarnConfigurationStore.java
@@ -0,0 +1,70 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, 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.scheduler.capacity.conf;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.conf.YarnConfigurationStore.LogMutation;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+public class TestYarnConfigurationStore {
+
+  private YarnConfigurationStore confStore;
+  private Configuration schedConf;
+
+  private static final String testUser = "testUser";
+
+  @Before
+  public void setUp() {
+    schedConf = new Configuration(false);
+    schedConf.set("key1", "val1");
+  }
+
+  @Test
+  public void testInMemoryConfigurationStore() {
+    confStore = new InMemoryConfigurationStore();
+    confStore.initialize(new Configuration(), schedConf);
+    assertEquals("val1", confStore.retrieve().get("key1"));
+
+    Map<String, String> update1 = new HashMap<>();
+    update1.put("keyUpdate1", "valUpdate1");
+    LogMutation mutation1 = new LogMutation(update1, testUser);
+    long id = confStore.logMutation(mutation1);
+    assertEquals(1, confStore.getPendingMutations().size());
+    confStore.confirmMutation(id, true);
+    assertEquals("valUpdate1", confStore.retrieve().get("keyUpdate1"));
+    assertEquals(0, confStore.getPendingMutations().size());
+
+    Map<String, String> update2 = new HashMap<>();
+    update2.put("keyUpdate2", "valUpdate2");
+    LogMutation mutation2 = new LogMutation(update2, testUser);
+    id = confStore.logMutation(mutation2);
+    assertEquals(1, confStore.getPendingMutations().size());
+    confStore.confirmMutation(id, false);
+    assertNull("Configuration should not be updated",
+        confStore.retrieve().get("keyUpdate2"));
+    assertEquals(0, confStore.getPendingMutations().size());
+  }
+}


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


[46/50] [abbrv] hadoop git commit: YARN-5948. Implement MutableConfigurationManager for handling storage into configuration store

Posted by jh...@apache.org.
YARN-5948. Implement MutableConfigurationManager for handling storage into configuration store


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

Branch: refs/heads/YARN-5734
Commit: d4c4f041cf97494e44752e5c23e122d3e75a61f3
Parents: 7d0a0f6
Author: Jonathan Hung <jh...@linkedin.com>
Authored: Wed Mar 1 16:03:01 2017 -0800
Committer: Jonathan Hung <jh...@linkedin.com>
Committed: Thu Aug 17 13:48:55 2017 -0700

----------------------------------------------------------------------
 .../hadoop/yarn/conf/YarnConfiguration.java     |  6 ++
 .../src/main/resources/yarn-default.xml         | 12 +++
 .../scheduler/MutableConfigurationProvider.java | 35 ++++++++
 .../scheduler/capacity/CapacityScheduler.java   | 14 ++-
 .../CapacitySchedulerConfiguration.java         |  3 +
 .../capacity/conf/CSConfigurationProvider.java  |  3 +-
 .../conf/MutableCSConfigurationProvider.java    | 94 ++++++++++++++++++++
 .../conf/YarnConfigurationStoreFactory.java     | 46 ++++++++++
 .../TestMutableCSConfigurationProvider.java     | 83 +++++++++++++++++
 9 files changed, 291 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4c4f041/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 8515e0a..2accb31 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
@@ -635,6 +635,12 @@ public class YarnConfiguration extends Configuration {
   public static final String DEFAULT_RM_CONFIGURATION_PROVIDER_CLASS =
       "org.apache.hadoop.yarn.LocalConfigurationProvider";
 
+  public static final String SCHEDULER_CONFIGURATION_STORE_CLASS =
+      YARN_PREFIX + "scheduler.configuration.store.class";
+  public static final String MEMORY_CONFIGURATION_STORE = "memory";
+  public static final String DEFAULT_CONFIGURATION_STORE =
+      MEMORY_CONFIGURATION_STORE;
+
   public static final String YARN_AUTHORIZATION_PROVIDER = YARN_PREFIX
       + "authorization-provider";
   private static final List<String> RM_SERVICES_ADDRESS_CONF_KEYS_HTTP =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4c4f041/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 dbf115b..49634a5 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
@@ -3218,4 +3218,16 @@
     <value>false</value>
   </property>
 
+  <property>
+    <description>
+      The type of configuration store to use for storing scheduler
+      configurations, if using a mutable configuration provider.
+      Keywords such as "memory" map to certain configuration store
+      implementations. If keyword is not found, try to load this
+      value as a class.
+    </description>
+    <name>yarn.scheduler.configuration.store.class</name>
+    <value>memory</value>
+  </property>
+
 </configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4c4f041/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/MutableConfigurationProvider.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/MutableConfigurationProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/MutableConfigurationProvider.java
new file mode 100644
index 0000000..da30a2b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/MutableConfigurationProvider.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.resourcemanager.scheduler;
+
+import java.util.Map;
+
+/**
+ * Interface for allowing changing scheduler configurations.
+ */
+public interface MutableConfigurationProvider {
+
+  /**
+   * Update the scheduler configuration with the provided key value pairs.
+   * @param user User issuing the request
+   * @param confUpdate Key-value pairs for configurations to be updated.
+   */
+  void mutateConfiguration(String user, Map<String, String> confUpdate);
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4c4f041/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.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/CapacityScheduler.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/CapacityScheduler.java
index bb42cdc..d5abcc0 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/CapacityScheduler.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/CapacityScheduler.java
@@ -104,6 +104,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.Activi
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.AllocationState;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.conf.CSConfigurationProvider;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.conf.FileBasedCSConfigurationProvider;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.conf.MutableCSConfigurationProvider;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.KillableContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.AssignmentInformation;
@@ -293,10 +294,15 @@ public class CapacityScheduler extends
       String confProviderStr = configuration.get(
           CapacitySchedulerConfiguration.CS_CONF_PROVIDER,
           CapacitySchedulerConfiguration.DEFAULT_CS_CONF_PROVIDER);
-      if (confProviderStr.equals(
-          CapacitySchedulerConfiguration.FILE_CS_CONF_PROVIDER)) {
-        this.csConfProvider = new FileBasedCSConfigurationProvider(rmContext);
-      } else {
+      switch (confProviderStr) {
+      case CapacitySchedulerConfiguration.FILE_CS_CONF_PROVIDER:
+        this.csConfProvider =
+            new FileBasedCSConfigurationProvider(rmContext);
+        break;
+      case CapacitySchedulerConfiguration.STORE_CS_CONF_PROVIDER:
+        this.csConfProvider = new MutableCSConfigurationProvider(rmContext);
+        break;
+      default:
         throw new IOException("Invalid CS configuration provider: " +
             confProviderStr);
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4c4f041/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.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/CapacitySchedulerConfiguration.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/CapacitySchedulerConfiguration.java
index f40fb3a..ec5fcdd 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/CapacitySchedulerConfiguration.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/CapacitySchedulerConfiguration.java
@@ -323,6 +323,9 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
   public static final String FILE_CS_CONF_PROVIDER = "file";
 
   @Private
+  public static final String STORE_CS_CONF_PROVIDER = "store";
+
+  @Private
   public static final String DEFAULT_CS_CONF_PROVIDER = FILE_CS_CONF_PROVIDER;
 
   AppPriorityACLConfigurationParser priorityACLConfig = new AppPriorityACLConfigurationParser();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4c4f041/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/CSConfigurationProvider.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/conf/CSConfigurationProvider.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/conf/CSConfigurationProvider.java
index c9984ac..0d2c8bb 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/conf/CSConfigurationProvider.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/conf/CSConfigurationProvider.java
@@ -32,8 +32,9 @@ public interface CSConfigurationProvider {
   /**
    * Initialize the configuration provider with given conf.
    * @param conf configuration to initialize with
+   * @throws IOException if initialization fails due to misconfiguration
    */
-  void init(Configuration conf);
+  void init(Configuration conf) throws IOException;
 
   /**
    * Loads capacity scheduler configuration object.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4c4f041/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/MutableCSConfigurationProvider.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/conf/MutableCSConfigurationProvider.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/conf/MutableCSConfigurationProvider.java
new file mode 100644
index 0000000..267ab6a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/MutableCSConfigurationProvider.java
@@ -0,0 +1,94 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, 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.scheduler.capacity.conf;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.MutableConfigurationProvider;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.conf.YarnConfigurationStore.LogMutation;
+
+import java.io.IOException;
+import java.util.Map;
+
+/**
+ * CS configuration provider which implements
+ * {@link MutableConfigurationProvider} for modifying capacity scheduler
+ * configuration.
+ */
+public class MutableCSConfigurationProvider implements CSConfigurationProvider,
+    MutableConfigurationProvider {
+
+  private Configuration schedConf;
+  private YarnConfigurationStore confStore;
+  private RMContext rmContext;
+  private Configuration conf;
+
+  public MutableCSConfigurationProvider(RMContext rmContext) {
+    this.rmContext = rmContext;
+  }
+
+  @Override
+  public void init(Configuration config) throws IOException {
+    String store = config.get(
+        YarnConfiguration.SCHEDULER_CONFIGURATION_STORE_CLASS,
+        YarnConfiguration.DEFAULT_CONFIGURATION_STORE);
+    switch (store) {
+    case YarnConfiguration.MEMORY_CONFIGURATION_STORE:
+      this.confStore = new InMemoryConfigurationStore();
+      break;
+    default:
+      this.confStore = YarnConfigurationStoreFactory.getStore(config);
+      break;
+    }
+    Configuration initialSchedConf = new Configuration(false);
+    initialSchedConf.addResource(YarnConfiguration.CS_CONFIGURATION_FILE);
+    this.schedConf = initialSchedConf;
+    confStore.initialize(config, initialSchedConf);
+    this.conf = config;
+  }
+
+  @Override
+  public CapacitySchedulerConfiguration loadConfiguration(Configuration
+      configuration) throws IOException {
+    Configuration loadedConf = new Configuration(configuration);
+    loadedConf.addResource(schedConf);
+    return new CapacitySchedulerConfiguration(loadedConf, false);
+  }
+
+  @Override
+  public void mutateConfiguration(String user,
+      Map<String, String> confUpdate) {
+    Configuration oldConf = new Configuration(schedConf);
+    LogMutation log = new LogMutation(confUpdate, user);
+    long id = confStore.logMutation(log);
+    for (Map.Entry<String, String> kv : confUpdate.entrySet()) {
+      schedConf.set(kv.getKey(), kv.getValue());
+    }
+    try {
+      rmContext.getScheduler().reinitialize(conf, rmContext);
+    } catch (IOException e) {
+      schedConf = oldConf;
+      confStore.confirmMutation(id, false);
+      return;
+    }
+    confStore.confirmMutation(id, true);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4c4f041/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/YarnConfigurationStoreFactory.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/conf/YarnConfigurationStoreFactory.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/conf/YarnConfigurationStoreFactory.java
new file mode 100644
index 0000000..60249c8
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/YarnConfigurationStoreFactory.java
@@ -0,0 +1,46 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, 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.scheduler.capacity.conf;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+
+/**
+ * Factory class for creating instances of {@link YarnConfigurationStore}.
+ */
+public final class YarnConfigurationStoreFactory {
+
+  private static final Log LOG = LogFactory.getLog(
+      YarnConfigurationStoreFactory.class);
+
+  private YarnConfigurationStoreFactory() {
+    // Unused.
+  }
+
+  public static YarnConfigurationStore getStore(Configuration conf) {
+    Class<? extends YarnConfigurationStore> storeClass =
+        conf.getClass(YarnConfiguration.SCHEDULER_CONFIGURATION_STORE_CLASS,
+            InMemoryConfigurationStore.class, YarnConfigurationStore.class);
+    LOG.info("Using YarnConfigurationStore implementation - " + storeClass);
+    return ReflectionUtils.newInstance(storeClass, conf);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4c4f041/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/TestMutableCSConfigurationProvider.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/scheduler/capacity/conf/TestMutableCSConfigurationProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/TestMutableCSConfigurationProvider.java
new file mode 100644
index 0000000..3f103b1
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/TestMutableCSConfigurationProvider.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.resourcemanager.scheduler.capacity.conf;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.doNothing;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * Tests {@link MutableCSConfigurationProvider}.
+ */
+public class TestMutableCSConfigurationProvider {
+
+  private MutableCSConfigurationProvider confProvider;
+  private RMContext rmContext;
+  private Map<String, String> goodUpdate;
+  private Map<String, String> badUpdate;
+  private CapacityScheduler cs;
+
+  private static final String TEST_USER = "testUser";
+
+  @Before
+  public void setUp() {
+    cs = mock(CapacityScheduler.class);
+    rmContext = mock(RMContext.class);
+    when(rmContext.getScheduler()).thenReturn(cs);
+    confProvider = new MutableCSConfigurationProvider(rmContext);
+    goodUpdate = new HashMap<>();
+    goodUpdate.put("goodKey", "goodVal");
+    badUpdate = new HashMap<>();
+    badUpdate.put("badKey", "badVal");
+  }
+
+  @Test
+  public void testInMemoryBackedProvider() throws IOException {
+    Configuration conf = new Configuration();
+    confProvider.init(conf);
+    assertNull(confProvider.loadConfiguration(conf)
+        .get("goodKey"));
+
+    doNothing().when(cs).reinitialize(any(Configuration.class),
+        any(RMContext.class));
+    confProvider.mutateConfiguration(TEST_USER, goodUpdate);
+    assertEquals("goodVal", confProvider.loadConfiguration(conf)
+        .get("goodKey"));
+
+    assertNull(confProvider.loadConfiguration(conf).get("badKey"));
+    doThrow(new IOException()).when(cs).reinitialize(any(Configuration.class),
+        any(RMContext.class));
+    confProvider.mutateConfiguration(TEST_USER, badUpdate);
+    assertNull(confProvider.loadConfiguration(conf).get("badKey"));
+  }
+}


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


[21/50] [abbrv] hadoop git commit: YARN-5978. ContainerScheduler and ContainerManager changes to support ExecType update. (Kartheek Muthyala via asuresh)

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d7be1d8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/TestContainerSchedulerQueuing.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/scheduler/TestContainerSchedulerQueuing.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/TestContainerSchedulerQueuing.java
index aeba399..a1c247b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/TestContainerSchedulerQueuing.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/TestContainerSchedulerQueuing.java
@@ -27,6 +27,8 @@ import java.util.List;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.UnsupportedFileSystemException;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.protocolrecords.ContainerUpdateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ContainerUpdateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
@@ -37,6 +39,7 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.ExecutionType;
+import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.ConfigurationException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
@@ -951,4 +954,97 @@ public class TestContainerSchedulerQueuing extends BaseContainerManagerTest {
         map.get(org.apache.hadoop.yarn.api.records.ContainerState.SCHEDULED)
             .getContainerId());
   }
+
+  /**
+   * Starts one OPPORTUNISTIC container that takes up the whole node's
+   * resources, and submit one more that will be queued. Now promote the
+   * queued OPPORTUNISTIC container, which should kill the current running
+   * OPPORTUNISTIC container to make room for the promoted request.
+   * @throws Exception
+   */
+  @Test
+  public void testPromotionOfOpportunisticContainers() throws Exception {
+    containerManager.start();
+
+    ContainerLaunchContext containerLaunchContext =
+        recordFactory.newRecordInstance(ContainerLaunchContext.class);
+
+    List<StartContainerRequest> list = new ArrayList<>();
+    list.add(StartContainerRequest.newInstance(
+        containerLaunchContext,
+        createContainerToken(createContainerId(0), DUMMY_RM_IDENTIFIER,
+            context.getNodeId(),
+            user, BuilderUtils.newResource(2048, 1),
+            context.getContainerTokenSecretManager(), null,
+            ExecutionType.OPPORTUNISTIC)));
+    list.add(StartContainerRequest.newInstance(
+        containerLaunchContext,
+        createContainerToken(createContainerId(1), DUMMY_RM_IDENTIFIER,
+            context.getNodeId(),
+            user, BuilderUtils.newResource(1024, 1),
+            context.getContainerTokenSecretManager(), null,
+            ExecutionType.OPPORTUNISTIC)));
+
+    StartContainersRequest allRequests =
+        StartContainersRequest.newInstance(list);
+    containerManager.startContainers(allRequests);
+
+    Thread.sleep(5000);
+
+    // Ensure first container is running and others are queued.
+    List<ContainerId> statList = new ArrayList<ContainerId>();
+    for (int i = 0; i < 3; i++) {
+      statList.add(createContainerId(i));
+    }
+    GetContainerStatusesRequest statRequest = GetContainerStatusesRequest
+        .newInstance(Arrays.asList(createContainerId(0)));
+    List<ContainerStatus> containerStatuses = containerManager
+        .getContainerStatuses(statRequest).getContainerStatuses();
+    for (ContainerStatus status : containerStatuses) {
+      if (status.getContainerId().equals(createContainerId(0))) {
+        Assert.assertEquals(
+            org.apache.hadoop.yarn.api.records.ContainerState.RUNNING,
+            status.getState());
+      } else {
+        Assert.assertEquals(
+            org.apache.hadoop.yarn.api.records.ContainerState.SCHEDULED,
+            status.getState());
+      }
+    }
+
+    ContainerScheduler containerScheduler =
+        containerManager.getContainerScheduler();
+    // Ensure two containers are properly queued.
+    Assert.assertEquals(1, containerScheduler.getNumQueuedContainers());
+    Assert.assertEquals(0,
+        containerScheduler.getNumQueuedGuaranteedContainers());
+    Assert.assertEquals(1,
+        containerScheduler.getNumQueuedOpportunisticContainers());
+
+    // Promote Queued Opportunistic Container
+    Token updateToken =
+        createContainerToken(createContainerId(1), 1, DUMMY_RM_IDENTIFIER,
+            context.getNodeId(), user, BuilderUtils.newResource(1024, 1),
+            context.getContainerTokenSecretManager(), null,
+            ExecutionType.GUARANTEED);
+    List<Token> updateTokens = new ArrayList<Token>();
+    updateTokens.add(updateToken);
+    ContainerUpdateRequest updateRequest =
+        ContainerUpdateRequest.newInstance(updateTokens);
+    ContainerUpdateResponse updateResponse =
+        containerManager.updateContainer(updateRequest);
+
+    Assert.assertEquals(1,
+        updateResponse.getSuccessfullyUpdatedContainers().size());
+    Assert.assertEquals(0, updateResponse.getFailedRequests().size());
+
+    waitForContainerState(containerManager, createContainerId(0),
+        org.apache.hadoop.yarn.api.records.ContainerState.COMPLETE);
+
+    waitForContainerState(containerManager, createContainerId(1),
+        org.apache.hadoop.yarn.api.records.ContainerState.RUNNING);
+
+    // Ensure no containers are queued.
+    Assert.assertEquals(0, containerScheduler.getNumQueuedContainers());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d7be1d8/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 022baea..4561e85c 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
@@ -140,7 +140,7 @@ public class MockContainer implements Container {
   }
 
   @Override
-  public void setResource(Resource targetResource) {
+  public void setContainerTokenIdentifier(ContainerTokenIdentifier token) {
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d7be1d8/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 8b2f9db..397d507 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
@@ -655,7 +655,7 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
               container.getNodeId(), getUser(), container.getResource(),
               container.getPriority(), rmContainer.getCreationTime(),
               this.logAggregationContext, rmContainer.getNodeLabelExpression(),
-              containerType));
+              containerType, container.getExecutionType()));
       updateNMToken(container);
     } catch (IllegalArgumentException e) {
       // DNS might be down, skip returning this container.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d7be1d8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/RMContainerTokenSecretManager.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/security/RMContainerTokenSecretManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/RMContainerTokenSecretManager.java
index 8c42255..677aa14 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/RMContainerTokenSecretManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/RMContainerTokenSecretManager.java
@@ -187,6 +187,31 @@ public class RMContainerTokenSecretManager extends
   }
 
   /**
+   * Helper function for creating ContainerTokens.
+   *
+   * @param containerId containerId.
+   * @param containerVersion containerVersion.
+   * @param nodeId nodeId.
+   * @param appSubmitter appSubmitter.
+   * @param capability capability.
+   * @param priority priority.
+   * @param createTime createTime.
+   * @param logAggregationContext logAggregationContext.
+   * @param nodeLabelExpression nodeLabelExpression.
+   * @param containerType containerType.
+   * @return the container-token.
+   */
+  public Token createContainerToken(ContainerId containerId,
+      int containerVersion, NodeId nodeId, String appSubmitter,
+      Resource capability, Priority priority, long createTime,
+      LogAggregationContext logAggregationContext, String nodeLabelExpression,
+      ContainerType containerType) {
+    return createContainerToken(containerId, containerVersion, nodeId,
+        appSubmitter, capability, priority, createTime, null, null,
+        ContainerType.TASK, ExecutionType.GUARANTEED);
+  }
+
+  /**
    * Helper function for creating ContainerTokens
    *
    * @param containerId Container Id
@@ -199,13 +224,14 @@ public class RMContainerTokenSecretManager extends
    * @param logAggregationContext Log Aggregation Context
    * @param nodeLabelExpression Node Label Expression
    * @param containerType Container Type
+   * @param execType Execution Type
    * @return the container-token
    */
   public Token createContainerToken(ContainerId containerId,
       int containerVersion, NodeId nodeId, String appSubmitter,
       Resource capability, Priority priority, long createTime,
       LogAggregationContext logAggregationContext, String nodeLabelExpression,
-      ContainerType containerType) {
+      ContainerType containerType, ExecutionType execType) {
     byte[] password;
     ContainerTokenIdentifier tokenIdentifier;
     long expiryTimeStamp =
@@ -220,7 +246,7 @@ public class RMContainerTokenSecretManager extends
               this.currentMasterKey.getMasterKey().getKeyId(),
               ResourceManager.getClusterTimeStamp(), priority, createTime,
               logAggregationContext, nodeLabelExpression, containerType,
-              ExecutionType.GUARANTEED);
+              execType);
       password = this.createPassword(tokenIdentifier);
 
     } finally {


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


[25/50] [abbrv] hadoop git commit: HDFS-12054. FSNamesystem#addErasureCodingPolicies should call checkNameNodeSafeMode() to ensure Namenode is not in safemode. Contributed by lufei.

Posted by jh...@apache.org.
HDFS-12054. FSNamesystem#addErasureCodingPolicies should call checkNameNodeSafeMode() to ensure Namenode is not in safemode. Contributed by lufei.


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

Branch: refs/heads/YARN-5734
Commit: 1040bae6fcbae7079d8126368cdeac60831a4d0c
Parents: 2e43c28
Author: Wei-Chiu Chuang <we...@apache.org>
Authored: Tue Aug 15 07:38:43 2017 -0700
Committer: Wei-Chiu Chuang <we...@apache.org>
Committed: Tue Aug 15 07:38:43 2017 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/server/namenode/FSNamesystem.java   |  2 ++
 .../java/org/apache/hadoop/hdfs/TestSafeMode.java   | 16 ++++++++++++++++
 2 files changed, 18 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1040bae6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index b1639b2..caf73f7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -7081,6 +7081,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       checkOperation(OperationCategory.WRITE);
       for (ErasureCodingPolicy policy : policies) {
         try {
+          checkOperation(OperationCategory.WRITE);
+          checkNameNodeSafeMode("Cannot add erasure coding policy");
           ErasureCodingPolicy newPolicy =
               FSDirErasureCodingOp.addErasureCodePolicy(this, policy);
           addECPolicyName = newPolicy.getName();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1040bae6/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java
index f03b440..bc95ec7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
@@ -48,6 +49,7 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -466,6 +468,20 @@ public class TestSafeMode {
       // expected
     }
 
+    ECSchema toAddSchema = new ECSchema("testcodec", 3, 2);
+    ErasureCodingPolicy newPolicy =
+        new ErasureCodingPolicy(toAddSchema, 128 * 1024);
+    ErasureCodingPolicy[] policyArray =
+        new ErasureCodingPolicy[]{newPolicy};
+    try {
+      dfs.addErasureCodingPolicies(policyArray);
+      fail("AddErasureCodingPolicies should have failed.");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains(
+          "Cannot add erasure coding policy", ioe);
+      // expected
+    }
+
     assertFalse("Could not leave SM",
         dfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE));
   }


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


[07/50] [abbrv] hadoop git commit: YARN-6687. Validate that the duration of the periodic reservation is less than the periodicity. (subru via curino)

Posted by jh...@apache.org.
YARN-6687. Validate that the duration of the periodic reservation is less than the periodicity. (subru via curino)


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

Branch: refs/heads/YARN-5734
Commit: 28d97b79b69bb2be02d9320105e155eeed6f9e78
Parents: cc59b5f
Author: Carlo Curino <cu...@apache.org>
Authored: Fri Aug 11 16:58:04 2017 -0700
Committer: Carlo Curino <cu...@apache.org>
Committed: Fri Aug 11 16:58:04 2017 -0700

----------------------------------------------------------------------
 .../reservation/ReservationInputValidator.java  | 18 ++--
 .../TestReservationInputValidator.java          | 93 ++++++++++++++++++++
 2 files changed, 106 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/28d97b79/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationInputValidator.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/ReservationInputValidator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationInputValidator.java
index 0e9a825..027d066 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationInputValidator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationInputValidator.java
@@ -129,11 +129,12 @@ public class ReservationInputValidator {
               Resources.multiply(rr.getCapability(), rr.getConcurrency()));
     }
     // verify the allocation is possible (skip for ANY)
-    if (contract.getDeadline() - contract.getArrival() < minDuration
+    long duration = contract.getDeadline() - contract.getArrival();
+    if (duration < minDuration
         && type != ReservationRequestInterpreter.R_ANY) {
       message =
           "The time difference ("
-              + (contract.getDeadline() - contract.getArrival())
+              + (duration)
               + ") between arrival (" + contract.getArrival() + ") "
               + "and deadline (" + contract.getDeadline() + ") must "
               + " be greater or equal to the minimum resource duration ("
@@ -158,15 +159,22 @@ public class ReservationInputValidator {
     // check that the recurrence is a positive long value.
     String recurrenceExpression = contract.getRecurrenceExpression();
     try {
-      Long recurrence = Long.parseLong(recurrenceExpression);
+      long recurrence = Long.parseLong(recurrenceExpression);
       if (recurrence < 0) {
         message = "Negative Period : " + recurrenceExpression + ". Please try"
-            + " again with a non-negative long value as period";
+            + " again with a non-negative long value as period.";
+        throw RPCUtil.getRemoteException(message);
+      }
+      // verify duration is less than recurrence for periodic reservations
+      if (recurrence > 0 && duration > recurrence) {
+        message = "Duration of the requested reservation: " + duration
+            + " is greater than the recurrence: " + recurrence
+            + ". Please try again with a smaller duration.";
         throw RPCUtil.getRemoteException(message);
       }
     } catch (NumberFormatException e) {
       message = "Invalid period " + recurrenceExpression + ". Please try"
-          + " again with a non-negative long value as period";
+          + " again with a non-negative long value as period.";
       throw RPCUtil.getRemoteException(message);
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/28d97b79/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestReservationInputValidator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestReservationInputValidator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestReservationInputValidator.java
index 2917cd9..90a681d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestReservationInputValidator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestReservationInputValidator.java
@@ -303,6 +303,7 @@ public class TestReservationInputValidator {
 
   @Test
   public void testSubmitReservationInvalidRecurrenceExpression() {
+    // first check recurrence expression
     ReservationSubmissionRequest request =
         createSimpleReservationSubmissionRequest(1, 1, 1, 5, 3, "123abc");
     plan = null;
@@ -318,6 +319,23 @@ public class TestReservationInputValidator {
           .startsWith("Invalid period "));
       LOG.info(message);
     }
+
+    // now check duration
+    request =
+        createSimpleReservationSubmissionRequest(1, 1, 1, 50, 3, "10");
+    plan = null;
+    try {
+      plan =
+          rrValidator.validateReservationSubmissionRequest(rSystem, request,
+              ReservationSystemTestUtil.getNewReservationId());
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertNull(plan);
+      String message = e.getMessage();
+      Assert.assertTrue(message
+          .startsWith("Duration of the requested reservation:"));
+      LOG.info(message);
+    }
   }
 
   @Test
@@ -500,6 +518,73 @@ public class TestReservationInputValidator {
   }
 
   @Test
+  public void testUpdateReservationValidRecurrenceExpression() {
+    ReservationUpdateRequest request =
+        createSimpleReservationUpdateRequest(1, 1, 1, 5, 3, "600000");
+    plan = null;
+    try {
+      plan =
+          rrValidator.validateReservationUpdateRequest(rSystem, request);
+    } catch (YarnException e) {
+      Assert.fail(e.getMessage());
+    }
+    Assert.assertNotNull(plan);
+  }
+
+  @Test
+  public void testUpdateReservationNegativeRecurrenceExpression() {
+    ReservationUpdateRequest request =
+        createSimpleReservationUpdateRequest(1, 1, 1, 5, 3, "-1234");
+    plan = null;
+    try {
+      plan =
+          rrValidator.validateReservationUpdateRequest(rSystem, request);
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertNull(plan);
+      String message = e.getMessage();
+      Assert.assertTrue(message
+          .startsWith("Negative Period : "));
+      LOG.info(message);
+    }
+  }
+
+  @Test
+  public void testUpdateReservationInvalidRecurrenceExpression() {
+    // first check recurrence expression
+    ReservationUpdateRequest request =
+        createSimpleReservationUpdateRequest(1, 1, 1, 5, 3, "123abc");
+    plan = null;
+    try {
+      plan =
+          rrValidator.validateReservationUpdateRequest(rSystem, request);
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertNull(plan);
+      String message = e.getMessage();
+      Assert.assertTrue(message
+          .startsWith("Invalid period "));
+      LOG.info(message);
+    }
+
+    // now check duration
+    request =
+        createSimpleReservationUpdateRequest(1, 1, 1, 50, 3, "10");
+    plan = null;
+    try {
+      plan =
+          rrValidator.validateReservationUpdateRequest(rSystem, request);
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertNull(plan);
+      String message = e.getMessage();
+      Assert.assertTrue(message
+          .startsWith("Duration of the requested reservation:"));
+      LOG.info(message);
+    }
+  }
+
+  @Test
   public void testDeleteReservationNormal() {
     ReservationDeleteRequest request = new ReservationDeleteRequestPBImpl();
     ReservationId reservationID =
@@ -710,11 +795,19 @@ public class TestReservationInputValidator {
   private ReservationUpdateRequest createSimpleReservationUpdateRequest(
       int numRequests, int numContainers, long arrival, long deadline,
       long duration) {
+    return createSimpleReservationUpdateRequest(numRequests, numContainers,
+        arrival, deadline, duration, "0");
+  }
+
+  private ReservationUpdateRequest createSimpleReservationUpdateRequest(
+      int numRequests, int numContainers, long arrival, long deadline,
+      long duration, String recurrence) {
     // create a request with a single atomic ask
     ReservationUpdateRequest request = new ReservationUpdateRequestPBImpl();
     ReservationDefinition rDef = new ReservationDefinitionPBImpl();
     rDef.setArrival(arrival);
     rDef.setDeadline(deadline);
+    rDef.setRecurrenceExpression(recurrence);
     if (numRequests > 0) {
       ReservationRequests reqs = new ReservationRequestsPBImpl();
       rDef.setReservationRequests(reqs);


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


[03/50] [abbrv] hadoop git commit: YARN-6884. AllocationFileLoaderService.loadQueue() has an if without braces (Contributed by weiyuan via Daniel Templeton)

Posted by jh...@apache.org.
YARN-6884. AllocationFileLoaderService.loadQueue() has an if without braces
(Contributed by weiyuan via Daniel Templeton)


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

Branch: refs/heads/YARN-5734
Commit: c7680d4cc4d9302a5b5efcf2467bd32ecea99585
Parents: 218588b
Author: Daniel Templeton <te...@apache.org>
Authored: Fri Aug 11 14:22:02 2017 -0700
Committer: Daniel Templeton <te...@apache.org>
Committed: Fri Aug 11 14:22:02 2017 -0700

----------------------------------------------------------------------
 .../scheduler/fair/AllocationFileLoaderService.java            | 6 ++++--
 1 file changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c7680d4c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationFileLoaderService.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/AllocationFileLoaderService.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/AllocationFileLoaderService.java
index bc204cb..bf5b4c5 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/AllocationFileLoaderService.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/AllocationFileLoaderService.java
@@ -294,8 +294,9 @@ public class AllocationFileLoaderService extends AbstractService {
           NodeList fields = element.getChildNodes();
           for (int j = 0; j < fields.getLength(); j++) {
             Node fieldNode = fields.item(j);
-            if (!(fieldNode instanceof Element))
+            if (!(fieldNode instanceof Element)) {
               continue;
+            }
             Element field = (Element) fieldNode;
             if ("maxRunningApps".equals(field.getTagName())) {
               String text = ((Text)field.getFirstChild()).getData().trim();
@@ -490,8 +491,9 @@ public class AllocationFileLoaderService extends AbstractService {
 
     for (int j = 0; j < fields.getLength(); j++) {
       Node fieldNode = fields.item(j);
-      if (!(fieldNode instanceof Element))
+      if (!(fieldNode instanceof Element)) {
         continue;
+      }
       Element field = (Element) fieldNode;
       if ("minResources".equals(field.getTagName())) {
         String text = ((Text)field.getFirstChild()).getData().trim();


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


[04/50] [abbrv] hadoop git commit: YARN-6967. Limit application attempt's diagnostic message size thoroughly (Contributed by Chengbing Liu via Daniel Templeton)

Posted by jh...@apache.org.
YARN-6967. Limit application attempt's diagnostic message size thoroughly
(Contributed by Chengbing Liu via Daniel Templeton)


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

Branch: refs/heads/YARN-5734
Commit: 65364defb4a633ca20b39ebc38cd9c0db63a5835
Parents: c7680d4
Author: Daniel Templeton <te...@apache.org>
Authored: Fri Aug 11 14:28:55 2017 -0700
Committer: Daniel Templeton <te...@apache.org>
Committed: Fri Aug 11 14:28:55 2017 -0700

----------------------------------------------------------------------
 .../rmapp/attempt/RMAppAttemptImpl.java             | 16 ++++++++--------
 1 file changed, 8 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/65364def/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
index 4210c54..254768b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
@@ -1315,7 +1315,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
     // AFTER the initial saving on app-attempt-start
     // These fields can be visible from outside only after they are saved in
     // StateStore
-    String diags = null;
+    BoundedAppender diags = new BoundedAppender(diagnostics.limit);
 
     // don't leave the tracking URL pointing to a non-existent AM
     if (conf.getBoolean(YarnConfiguration.APPLICATION_HISTORY_ENABLED,
@@ -1329,15 +1329,15 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
     int exitStatus = ContainerExitStatus.INVALID;
     switch (event.getType()) {
     case LAUNCH_FAILED:
-      diags = event.getDiagnosticMsg();
+      diags.append(event.getDiagnosticMsg());
       break;
     case REGISTERED:
-      diags = getUnexpectedAMRegisteredDiagnostics();
+      diags.append(getUnexpectedAMRegisteredDiagnostics());
       break;
     case UNREGISTERED:
       RMAppAttemptUnregistrationEvent unregisterEvent =
           (RMAppAttemptUnregistrationEvent) event;
-      diags = unregisterEvent.getDiagnosticMsg();
+      diags.append(unregisterEvent.getDiagnosticMsg());
       // reset finalTrackingUrl to url sent by am
       finalTrackingUrl = sanitizeTrackingUrl(unregisterEvent.getFinalTrackingUrl());
       finalStatus = unregisterEvent.getFinalApplicationStatus();
@@ -1345,16 +1345,16 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
     case CONTAINER_FINISHED:
       RMAppAttemptContainerFinishedEvent finishEvent =
           (RMAppAttemptContainerFinishedEvent) event;
-      diags = getAMContainerCrashedDiagnostics(finishEvent);
+      diags.append(getAMContainerCrashedDiagnostics(finishEvent));
       exitStatus = finishEvent.getContainerStatus().getExitStatus();
       break;
     case KILL:
       break;
     case FAIL:
-      diags = event.getDiagnosticMsg();
+      diags.append(event.getDiagnosticMsg());
       break;
     case EXPIRE:
-      diags = getAMExpiredDiagnostics(event);
+      diags.append(getAMExpiredDiagnostics(event));
       break;
     default:
       break;
@@ -1368,7 +1368,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
         ApplicationAttemptStateData.newInstance(
             applicationAttemptId,  getMasterContainer(),
             rmStore.getCredentialsFromAppAttempt(this),
-            startTime, stateToBeStored, finalTrackingUrl, diags,
+            startTime, stateToBeStored, finalTrackingUrl, diags.toString(),
             finalStatus, exitStatus,
           getFinishTime(), resUsage.getMemorySeconds(),
           resUsage.getVcoreSeconds(),


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


[40/50] [abbrv] hadoop git commit: HDFS-12250. Reduce usage of FsPermissionExtension in unit tests. Contributed by Chris Douglas.

Posted by jh...@apache.org.
HDFS-12250. Reduce usage of FsPermissionExtension in unit tests. Contributed by Chris Douglas.


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

Branch: refs/heads/YARN-5734
Commit: dd7916d3cd5d880d0b257d229f43f10feff04c93
Parents: f9a0e23
Author: Andrew Wang <wa...@apache.org>
Authored: Thu Aug 17 09:35:36 2017 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Thu Aug 17 09:35:36 2017 -0700

----------------------------------------------------------------------
 .../hadoop/fs/permission/FsPermission.java      |  2 +-
 .../org/apache/hadoop/fs/shell/AclCommands.java |  6 ++---
 .../hadoop/fs/shell/CommandWithDestination.java |  4 ++--
 .../java/org/apache/hadoop/fs/shell/Ls.java     |  4 ++--
 .../fs/http/client/BaseTestHttpFSWith.java      |  1 +
 .../org/apache/hadoop/hdfs/TestDFSShell.java    | 24 ++++++++++----------
 .../hdfs/server/namenode/FSAclBaseTest.java     |  6 +++++
 .../ClientDistributedCacheManager.java          |  6 ++---
 .../apache/hadoop/fs/adl/TestGetFileStatus.java |  1 +
 .../hadoop/tools/CopyListingFileStatus.java     |  4 ++--
 .../apache/hadoop/tools/util/DistCpUtils.java   |  4 +---
 11 files changed, 33 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/dd7916d3/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsPermission.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsPermission.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsPermission.java
index 23692de..031092b 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsPermission.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsPermission.java
@@ -163,7 +163,7 @@ public class FsPermission implements Writable, Serializable,
    */
   public static FsPermission read(DataInput in) throws IOException {
     FsPermission p = new FsPermission();
-    p.readFields(in);
+    p.fromShort(in.readShort());
     return p;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dd7916d3/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/AclCommands.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/AclCommands.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/AclCommands.java
index a5e386c..701c9de 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/AclCommands.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/AclCommands.java
@@ -86,9 +86,9 @@ class AclCommands extends FsCommand {
           (perm.getOtherAction().implies(FsAction.EXECUTE) ? "t" : "T"));
       }
 
-      AclStatus aclStatus = null;
-      List<AclEntry> entries = null;
-      if (perm.getAclBit()) {
+      final AclStatus aclStatus;
+      final List<AclEntry> entries;
+      if (item.stat.hasAcl()) {
         aclStatus = item.fs.getAclStatus(item.path);
         entries = aclStatus.getEntries();
       } else {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dd7916d3/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CommandWithDestination.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CommandWithDestination.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CommandWithDestination.java
index 2a483c0..0bd4882 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CommandWithDestination.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CommandWithDestination.java
@@ -444,8 +444,8 @@ abstract class CommandWithDestination extends FsCommand {
           src.stat.getPermission());
     }
     if (shouldPreserve(FileAttribute.ACL)) {
-      FsPermission perm = src.stat.getPermission();
-      if (perm.getAclBit()) {
+      if (src.stat.hasAcl()) {
+        FsPermission perm = src.stat.getPermission();
         List<AclEntry> srcEntries =
             src.fs.getAclStatus(src.path).getEntries();
         List<AclEntry> srcFullEntries =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dd7916d3/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Ls.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Ls.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Ls.java
index 221b3cb..a2d5017 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Ls.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Ls.java
@@ -255,7 +255,7 @@ class Ls extends FsCommand {
       ContentSummary contentSummary = item.fs.getContentSummary(item.path);
       String line = String.format(lineFormat,
           (stat.isDirectory() ? "d" : "-"),
-          stat.getPermission() + (stat.getPermission().getAclBit() ? "+" : " "),
+          stat.getPermission() + (stat.hasAcl() ? "+" : " "),
           (stat.isFile() ? stat.getReplication() : "-"),
           stat.getOwner(),
           stat.getGroup(),
@@ -269,7 +269,7 @@ class Ls extends FsCommand {
     } else {
       String line = String.format(lineFormat,
           (stat.isDirectory() ? "d" : "-"),
-          stat.getPermission() + (stat.getPermission().getAclBit() ? "+" : " "),
+          stat.getPermission() + (stat.hasAcl() ? "+" : " "),
           (stat.isFile() ? stat.getReplication() : "-"),
           stat.getOwner(),
           stat.getGroup(),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dd7916d3/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 553bbce..2cd8934 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
@@ -859,6 +859,7 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
     FileStatus expectedFileStatus = expected.getFileStatus(path);
     FileStatus actualFileStatus = actual.getFileStatus(path);
     assertEquals(actualFileStatus.hasAcl(), expectedFileStatus.hasAcl());
+    // backwards compat
     assertEquals(actualFileStatus.getPermission().getAclBit(),
         expectedFileStatus.getPermission().getAclBit());
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dd7916d3/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java
index 27d41b4..9ae49aa 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java
@@ -2189,7 +2189,7 @@ public class TestDFSShell {
       assertTrue(xattrs.isEmpty());
       List<AclEntry> acls = dfs.getAclStatus(target1).getEntries();
       assertTrue(acls.isEmpty());
-      assertFalse(targetPerm.getAclBit());
+      assertFalse(targetStatus.hasAcl());
 
       // -ptop
       Path target2 = new Path(hdfsTestDir, "targetfile2");
@@ -2208,7 +2208,7 @@ public class TestDFSShell {
       assertTrue(xattrs.isEmpty());
       acls = dfs.getAclStatus(target2).getEntries();
       assertTrue(acls.isEmpty());
-      assertFalse(targetPerm.getAclBit());
+      assertFalse(targetStatus.hasAcl());
 
       // -ptopx
       Path target3 = new Path(hdfsTestDir, "targetfile3");
@@ -2229,7 +2229,7 @@ public class TestDFSShell {
       assertArrayEquals(TRUSTED_A1_VALUE, xattrs.get(TRUSTED_A1));
       acls = dfs.getAclStatus(target3).getEntries();
       assertTrue(acls.isEmpty());
-      assertFalse(targetPerm.getAclBit());
+      assertFalse(targetStatus.hasAcl());
 
       // -ptopa
       Path target4 = new Path(hdfsTestDir, "targetfile4");
@@ -2248,7 +2248,7 @@ public class TestDFSShell {
       assertTrue(xattrs.isEmpty());
       acls = dfs.getAclStatus(target4).getEntries();
       assertFalse(acls.isEmpty());
-      assertTrue(targetPerm.getAclBit());
+      assertTrue(targetStatus.hasAcl());
       assertEquals(dfs.getAclStatus(src), dfs.getAclStatus(target4));
 
       // -ptoa (verify -pa option will preserve permissions also)
@@ -2268,7 +2268,7 @@ public class TestDFSShell {
       assertTrue(xattrs.isEmpty());
       acls = dfs.getAclStatus(target5).getEntries();
       assertFalse(acls.isEmpty());
-      assertTrue(targetPerm.getAclBit());
+      assertTrue(targetStatus.hasAcl());
       assertEquals(dfs.getAclStatus(src), dfs.getAclStatus(target5));
     } finally {
       if (null != shell) {
@@ -2480,7 +2480,7 @@ public class TestDFSShell {
       assertTrue(xattrs.isEmpty());
       List<AclEntry> acls = dfs.getAclStatus(targetDir1).getEntries();
       assertTrue(acls.isEmpty());
-      assertFalse(targetPerm.getAclBit());
+      assertFalse(targetStatus.hasAcl());
 
       // -ptop
       Path targetDir2 = new Path(hdfsTestDir, "targetDir2");
@@ -2499,7 +2499,7 @@ public class TestDFSShell {
       assertTrue(xattrs.isEmpty());
       acls = dfs.getAclStatus(targetDir2).getEntries();
       assertTrue(acls.isEmpty());
-      assertFalse(targetPerm.getAclBit());
+      assertFalse(targetStatus.hasAcl());
 
       // -ptopx
       Path targetDir3 = new Path(hdfsTestDir, "targetDir3");
@@ -2520,7 +2520,7 @@ public class TestDFSShell {
       assertArrayEquals(TRUSTED_A1_VALUE, xattrs.get(TRUSTED_A1));
       acls = dfs.getAclStatus(targetDir3).getEntries();
       assertTrue(acls.isEmpty());
-      assertFalse(targetPerm.getAclBit());
+      assertFalse(targetStatus.hasAcl());
 
       // -ptopa
       Path targetDir4 = new Path(hdfsTestDir, "targetDir4");
@@ -2539,7 +2539,7 @@ public class TestDFSShell {
       assertTrue(xattrs.isEmpty());
       acls = dfs.getAclStatus(targetDir4).getEntries();
       assertFalse(acls.isEmpty());
-      assertTrue(targetPerm.getAclBit());
+      assertTrue(targetStatus.hasAcl());
       assertEquals(dfs.getAclStatus(srcDir), dfs.getAclStatus(targetDir4));
 
       // -ptoa (verify -pa option will preserve permissions also)
@@ -2559,7 +2559,7 @@ public class TestDFSShell {
       assertTrue(xattrs.isEmpty());
       acls = dfs.getAclStatus(targetDir5).getEntries();
       assertFalse(acls.isEmpty());
-      assertTrue(targetPerm.getAclBit());
+      assertTrue(targetStatus.hasAcl());
       assertEquals(dfs.getAclStatus(srcDir), dfs.getAclStatus(targetDir5));
     } finally {
       if (shell != null) {
@@ -2615,7 +2615,7 @@ public class TestDFSShell {
       assertTrue(perm.equals(targetPerm));
       List<AclEntry> acls = dfs.getAclStatus(target1).getEntries();
       assertTrue(acls.isEmpty());
-      assertFalse(targetPerm.getAclBit());
+      assertFalse(targetStatus.hasAcl());
 
       // -ptopa preserves both sticky bit and ACL
       Path target2 = new Path(hdfsTestDir, "targetfile2");
@@ -2632,7 +2632,7 @@ public class TestDFSShell {
       assertTrue(perm.equals(targetPerm));
       acls = dfs.getAclStatus(target2).getEntries();
       assertFalse(acls.isEmpty());
-      assertTrue(targetPerm.getAclBit());
+      assertTrue(targetStatus.hasAcl());
       assertEquals(dfs.getAclStatus(src), dfs.getAclStatus(target2));
     } finally {
       if (null != shell) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dd7916d3/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSAclBaseTest.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSAclBaseTest.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSAclBaseTest.java
index 93a83fd..ee92217 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSAclBaseTest.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSAclBaseTest.java
@@ -32,6 +32,7 @@ import java.util.Map;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.AclEntry;
@@ -886,6 +887,11 @@ public abstract class FSAclBaseTest {
     FsPermission perm = inode.getFsPermission();
     assertNotNull(perm);
     assertEquals(0755, perm.toShort());
+    FileStatus stat = fs.getFileStatus(path);
+    assertFalse(stat.hasAcl());
+    assertFalse(stat.isEncrypted());
+    assertFalse(stat.isErasureCoded());
+    // backwards-compat check
     assertEquals(0755, perm.toExtendedShort());
     assertAclFeature(false);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dd7916d3/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/filecache/ClientDistributedCacheManager.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/filecache/ClientDistributedCacheManager.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/filecache/ClientDistributedCacheManager.java
index 9f8edb5..ada14db 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/filecache/ClientDistributedCacheManager.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/filecache/ClientDistributedCacheManager.java
@@ -28,7 +28,6 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsAction;
-import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.security.TokenCache;
 import org.apache.hadoop.security.Credentials;
@@ -293,7 +292,6 @@ public class ClientDistributedCacheManager {
   private static boolean checkPermissionOfOther(FileSystem fs, Path path,
       FsAction action, Map<URI, FileStatus> statCache) throws IOException {
     FileStatus status = getFileStatus(fs, path.toUri(), statCache);
-    FsPermission perms = status.getPermission();
 
     // Encrypted files are always treated as private. This stance has two
     // important side effects.  The first is that the encrypted files will be
@@ -302,8 +300,8 @@ public class ClientDistributedCacheManager {
     // world readable permissions that is stored in an encryption zone from
     // being localized as a publicly shared file with world readable
     // permissions.
-    if (!perms.getEncryptedBit()) {
-      FsAction otherAction = perms.getOtherAction();
+    if (!status.isEncrypted()) {
+      FsAction otherAction = status.getPermission().getOtherAction();
       if (otherAction.implies(action)) {
         return true;
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dd7916d3/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestGetFileStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestGetFileStatus.java b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestGetFileStatus.java
index d9e22db..95c2363 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestGetFileStatus.java
+++ b/hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestGetFileStatus.java
@@ -98,4 +98,5 @@ public class TestGetFileStatus extends AdlMockWebServer {
     Assert.assertFalse(fileStatus.hasAcl());
     Assert.assertFalse(fileStatus.getPermission().getAclBit());
   }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dd7916d3/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/CopyListingFileStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/CopyListingFileStatus.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/CopyListingFileStatus.java
index 29c59ac..138b491 100644
--- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/CopyListingFileStatus.java
+++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/CopyListingFileStatus.java
@@ -280,7 +280,7 @@ public final class CopyListingFileStatus implements Writable {
     out.writeLong(getBlockSize());
     out.writeLong(getModificationTime());
     out.writeLong(getAccessTime());
-    getPermission().write(out);
+    out.writeShort(getPermission().toShort());
     Text.writeString(out, getOwner(), Text.DEFAULT_MAX_LEN);
     Text.writeString(out, getGroup(), Text.DEFAULT_MAX_LEN);
     if (aclEntries != null) {
@@ -330,7 +330,7 @@ public final class CopyListingFileStatus implements Writable {
     blocksize = in.readLong();
     modificationTime = in.readLong();
     accessTime = in.readLong();
-    permission.readFields(in);
+    permission.fromShort(in.readShort());
     owner = Text.readString(in, Text.DEFAULT_MAX_LEN);
     group = Text.readString(in, Text.DEFAULT_MAX_LEN);
     byte aclEntriesSize = in.readByte();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dd7916d3/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/util/DistCpUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/util/DistCpUtils.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/util/DistCpUtils.java
index dbe750a..2b3b529 100644
--- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/util/DistCpUtils.java
+++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/util/DistCpUtils.java
@@ -31,7 +31,6 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclUtil;
-import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.Text;
@@ -403,8 +402,7 @@ public class DistCpUtils {
     CopyListingFileStatus copyListingFileStatus =
         new CopyListingFileStatus(fileStatus, chunkOffset, chunkLength);
     if (preserveAcls) {
-      FsPermission perm = fileStatus.getPermission();
-      if (perm.getAclBit()) {
+      if (fileStatus.hasAcl()) {
         List<AclEntry> aclEntries = fileSystem.getAclStatus(
           fileStatus.getPath()).getEntries();
         copyListingFileStatus.setAclEntries(aclEntries);


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


[39/50] [abbrv] hadoop git commit: YARN-3254. HealthReport should include disk full information. Contributed by Suma Shivaprasad.

Posted by jh...@apache.org.
YARN-3254. HealthReport should include disk full information. Contributed by Suma Shivaprasad.


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

Branch: refs/heads/YARN-5734
Commit: f9a0e2338150f1bd3ba2c29f76979183fd3ed80c
Parents: 1f04cb4
Author: Sunil G <su...@apache.org>
Authored: Thu Aug 17 15:07:15 2017 +0530
Committer: Sunil G <su...@apache.org>
Committed: Thu Aug 17 15:07:15 2017 +0530

----------------------------------------------------------------------
 .../server/nodemanager/DirectoryCollection.java | 61 +++++++++++++++++++-
 .../nodemanager/LocalDirsHandlerService.java    | 59 +++++++++++++++----
 .../nodemanager/TestDirectoryCollection.java    | 23 ++++++++
 3 files changed, 130 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f9a0e233/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DirectoryCollection.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/DirectoryCollection.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DirectoryCollection.java
index ae2a4ef..502485f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DirectoryCollection.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DirectoryCollection.java
@@ -38,6 +38,7 @@ import org.apache.commons.io.FileUtils;
 import org.apache.commons.lang.RandomStringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.Path;
@@ -99,6 +100,7 @@ public class DirectoryCollection {
   private List<String> localDirs;
   private List<String> errorDirs;
   private List<String> fullDirs;
+  private Map<String, DiskErrorInformation> directoryErrorInfo;
 
   // read/write lock for accessing above directories.
   private final ReadLock readLock;
@@ -192,6 +194,7 @@ public class DirectoryCollection {
     localDirs = new CopyOnWriteArrayList<>(dirs);
     errorDirs = new CopyOnWriteArrayList<>();
     fullDirs = new CopyOnWriteArrayList<>();
+    directoryErrorInfo = new ConcurrentHashMap<>();
 
     ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
     this.readLock = lock.readLock();
@@ -248,11 +251,25 @@ public class DirectoryCollection {
   /**
    * @return the directories that have used all disk space
    */
-
   List<String> getFullDirs() {
     this.readLock.lock();
     try {
-      return fullDirs;
+      return Collections.unmodifiableList(fullDirs);
+    } finally {
+      this.readLock.unlock();
+    }
+  }
+
+  /**
+   * @return the directories that have errors - many not have appropriate permissions
+   * or other disk validation checks might have failed in {@link DiskValidator}
+   *
+   */
+  @InterfaceStability.Evolving
+  List<String> getErroredDirs() {
+    this.readLock.lock();
+    try {
+      return Collections.unmodifiableList(errorDirs);
     } finally {
       this.readLock.unlock();
     }
@@ -271,6 +288,39 @@ public class DirectoryCollection {
   }
 
   /**
+   *
+   * @param dirName Absolute path of Directory for which error diagnostics are needed
+   * @return DiskErrorInformation - disk error diagnostics for the specified directory
+   *         null - the disk associated with the directory has passed disk utilization checks
+   *         /error validations in {@link DiskValidator}
+   *
+   */
+  @InterfaceStability.Evolving
+  DiskErrorInformation getDirectoryErrorInfo(String dirName) {
+    this.readLock.lock();
+    try {
+      return directoryErrorInfo.get(dirName);
+    } finally {
+      this.readLock.unlock();
+    }
+  }
+
+  /**
+   *
+   * @param dirName Absolute path of Directory for which the disk has been marked as unhealthy
+   * @return Check if disk associated with the directory is unhealthy
+   */
+  @InterfaceStability.Evolving
+  boolean isDiskUnHealthy(String dirName) {
+    this.readLock.lock();
+    try {
+      return directoryErrorInfo.containsKey(dirName);
+    } finally {
+      this.readLock.unlock();
+    }
+  }
+
+  /**
    * Create any non-existent directories and parent directories, updating the
    * list of valid directories if necessary.
    * @param localFs local file system to use
@@ -297,6 +347,9 @@ public class DirectoryCollection {
         try {
           localDirs.remove(dir);
           errorDirs.add(dir);
+          directoryErrorInfo.put(dir,
+              new DiskErrorInformation(DiskErrorCause.OTHER,
+                  "Cannot create directory : " + dir + ", error " + e.getMessage()));
           numFailures++;
         } finally {
           this.writeLock.unlock();
@@ -343,11 +396,13 @@ public class DirectoryCollection {
       localDirs.clear();
       errorDirs.clear();
       fullDirs.clear();
+      directoryErrorInfo.clear();
 
       for (Map.Entry<String, DiskErrorInformation> entry : dirsFailedCheck
           .entrySet()) {
         String dir = entry.getKey();
         DiskErrorInformation errorInformation = entry.getValue();
+
         switch (entry.getValue().cause) {
         case DISK_FULL:
           fullDirs.add(entry.getKey());
@@ -359,6 +414,8 @@ public class DirectoryCollection {
           LOG.warn(entry.getValue().cause + " is unknown for disk error.");
           break;
         }
+        directoryErrorInfo.put(entry.getKey(), errorInformation);
+
         if (preCheckGoodDirs.contains(dir)) {
           LOG.warn("Directory " + dir + " error, " + errorInformation.message
               + ", removing from list of valid directories");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f9a0e233/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LocalDirsHandlerService.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/LocalDirsHandlerService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LocalDirsHandlerService.java
index f8cb4ee..6e00808 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LocalDirsHandlerService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LocalDirsHandlerService.java
@@ -53,6 +53,8 @@ public class LocalDirsHandlerService extends AbstractService {
 
   private static Log LOG = LogFactory.getLog(LocalDirsHandlerService.class);
 
+  private static final String diskCapacityExceededErrorMsg =  "usable space is below configured utilization percentage/no more usable space";
+
   /**
    * Good local directories, use internally,
    * initial value is the same as NM_LOCAL_DIRS.
@@ -344,21 +346,36 @@ public class LocalDirsHandlerService extends AbstractService {
     }
 
     StringBuilder report = new StringBuilder();
-    List<String> failedLocalDirsList = localDirs.getFailedDirs();
-    List<String> failedLogDirsList = logDirs.getFailedDirs();
+    List<String> erroredLocalDirsList = localDirs.getErroredDirs();
+    List<String> erroredLogDirsList = logDirs.getErroredDirs();
+    List<String> diskFullLocalDirsList = localDirs.getFullDirs();
+    List<String> diskFullLogDirsList = logDirs.getFullDirs();
     List<String> goodLocalDirsList = localDirs.getGoodDirs();
     List<String> goodLogDirsList = logDirs.getGoodDirs();
-    int numLocalDirs = goodLocalDirsList.size() + failedLocalDirsList.size();
-    int numLogDirs = goodLogDirsList.size() + failedLogDirsList.size();
+
+    int numLocalDirs = goodLocalDirsList.size() + erroredLocalDirsList.size() + diskFullLocalDirsList.size();
+    int numLogDirs = goodLogDirsList.size() + erroredLogDirsList.size() + diskFullLogDirsList.size();
     if (!listGoodDirs) {
-      if (!failedLocalDirsList.isEmpty()) {
-        report.append(failedLocalDirsList.size() + "/" + numLocalDirs
-            + " local-dirs are bad: "
-            + StringUtils.join(",", failedLocalDirsList) + "; ");
+      if (!erroredLocalDirsList.isEmpty()) {
+        report.append(erroredLocalDirsList.size() + "/" + numLocalDirs
+            + " local-dirs have errors: "
+            + buildDiskErrorReport(erroredLocalDirsList, localDirs));
+      }
+      if (!diskFullLocalDirsList.isEmpty()) {
+        report.append(diskFullLocalDirsList.size() + "/" + numLocalDirs
+            + " local-dirs " + diskCapacityExceededErrorMsg
+            + buildDiskErrorReport(diskFullLocalDirsList, localDirs) + "; ");
       }
-      if (!failedLogDirsList.isEmpty()) {
-        report.append(failedLogDirsList.size() + "/" + numLogDirs
-            + " log-dirs are bad: " + StringUtils.join(",", failedLogDirsList));
+
+      if (!erroredLogDirsList.isEmpty()) {
+        report.append(erroredLogDirsList.size() + "/" + numLogDirs
+            + " log-dirs have errors: "
+            + buildDiskErrorReport(erroredLogDirsList, logDirs));
+      }
+      if (!diskFullLogDirsList.isEmpty()) {
+        report.append(diskFullLogDirsList.size() + "/" + numLogDirs
+            + " log-dirs " + diskCapacityExceededErrorMsg
+            + buildDiskErrorReport(diskFullLogDirsList, logDirs));
       }
     } else {
       report.append(goodLocalDirsList.size() + "/" + numLocalDirs
@@ -620,4 +637,24 @@ public class LocalDirsHandlerService extends AbstractService {
           logDirs.getGoodDirsDiskUtilizationPercentage());
     }
   }
+
+  private String buildDiskErrorReport(List<String> dirs, DirectoryCollection directoryCollection) {
+    StringBuilder sb = new StringBuilder();
+
+    sb.append(" [ ");
+    for (int i = 0; i < dirs.size(); i++) {
+      final String dirName = dirs.get(i);
+      if ( directoryCollection.isDiskUnHealthy(dirName)) {
+        sb.append(dirName + " : " + directoryCollection.getDirectoryErrorInfo(dirName).message);
+      } else {
+        sb.append(dirName + " : " + "Unknown cause for disk error");
+      }
+
+      if ( i != (dirs.size() - 1)) {
+        sb.append(" , ");
+      }
+    }
+    sb.append(" ] ");
+    return sb.toString();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f9a0e233/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestDirectoryCollection.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/TestDirectoryCollection.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestDirectoryCollection.java
index e529628..095f21a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestDirectoryCollection.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestDirectoryCollection.java
@@ -128,8 +128,12 @@ public class TestDirectoryCollection {
     DirectoryCollection dc = new DirectoryCollection(dirs, 0.0F);
     dc.checkDirs();
     Assert.assertEquals(0, dc.getGoodDirs().size());
+    Assert.assertEquals(0, dc.getErroredDirs().size());
     Assert.assertEquals(1, dc.getFailedDirs().size());
     Assert.assertEquals(1, dc.getFullDirs().size());
+    Assert.assertNotNull(dc.getDirectoryErrorInfo(dirA));
+    Assert.assertEquals(DirectoryCollection.DiskErrorCause.DISK_FULL, dc.getDirectoryErrorInfo(dirA).cause);
+
     // no good dirs
     Assert.assertEquals(0, dc.getGoodDirsDiskUtilizationPercentage());
 
@@ -139,16 +143,21 @@ public class TestDirectoryCollection {
             testDir.getTotalSpace());
     dc.checkDirs();
     Assert.assertEquals(1, dc.getGoodDirs().size());
+    Assert.assertEquals(0, dc.getErroredDirs().size());
     Assert.assertEquals(0, dc.getFailedDirs().size());
     Assert.assertEquals(0, dc.getFullDirs().size());
+    Assert.assertNull(dc.getDirectoryErrorInfo(dirA));
+
     Assert.assertEquals(utilizedSpacePerc,
       dc.getGoodDirsDiskUtilizationPercentage());
 
     dc = new DirectoryCollection(dirs, testDir.getTotalSpace() / (1024 * 1024));
     dc.checkDirs();
     Assert.assertEquals(0, dc.getGoodDirs().size());
+    Assert.assertEquals(0, dc.getErroredDirs().size());
     Assert.assertEquals(1, dc.getFailedDirs().size());
     Assert.assertEquals(1, dc.getFullDirs().size());
+    Assert.assertNotNull(dc.getDirectoryErrorInfo(dirA));
     // no good dirs
     Assert.assertEquals(0, dc.getGoodDirsDiskUtilizationPercentage());
 
@@ -158,8 +167,11 @@ public class TestDirectoryCollection {
             testDir.getTotalSpace());
     dc.checkDirs();
     Assert.assertEquals(1, dc.getGoodDirs().size());
+    Assert.assertEquals(0, dc.getErroredDirs().size());
     Assert.assertEquals(0, dc.getFailedDirs().size());
     Assert.assertEquals(0, dc.getFullDirs().size());
+    Assert.assertNull(dc.getDirectoryErrorInfo(dirA));
+
     Assert.assertEquals(utilizedSpacePerc,
       dc.getGoodDirsDiskUtilizationPercentage());
   }
@@ -209,12 +221,17 @@ public class TestDirectoryCollection {
     Assert.assertEquals(0, dc.getGoodDirs().size());
     Assert.assertEquals(1, dc.getFailedDirs().size());
     Assert.assertEquals(1, dc.getFullDirs().size());
+    Assert.assertEquals(0, dc.getErroredDirs().size());
+    Assert.assertNotNull(dc.getDirectoryErrorInfo(dirA));
+    Assert.assertEquals(DirectoryCollection.DiskErrorCause.DISK_FULL, dc.getDirectoryErrorInfo(dirA).cause);
 
     dc.setDiskUtilizationPercentageCutoff(100.0F, 100.0F);
     dc.checkDirs();
     Assert.assertEquals(1, dc.getGoodDirs().size());
     Assert.assertEquals(0, dc.getFailedDirs().size());
     Assert.assertEquals(0, dc.getFullDirs().size());
+    Assert.assertEquals(0, dc.getErroredDirs().size());
+    Assert.assertNull(dc.getDirectoryErrorInfo(dirA));
 
     conf.set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, "077");
 
@@ -232,12 +249,18 @@ public class TestDirectoryCollection {
     Assert.assertEquals(0, dc.getGoodDirs().size());
     Assert.assertEquals(1, dc.getFailedDirs().size());
     Assert.assertEquals(0, dc.getFullDirs().size());
+    Assert.assertEquals(1, dc.getErroredDirs().size());
+    Assert.assertNotNull(dc.getDirectoryErrorInfo(dirB));
+    Assert.assertEquals(DirectoryCollection.DiskErrorCause.OTHER, dc.getDirectoryErrorInfo(dirB).cause);
+
     permDirB = new FsPermission((short) 0700);
     localFs.setPermission(pathB, permDirB);
     dc.checkDirs();
     Assert.assertEquals(1, dc.getGoodDirs().size());
     Assert.assertEquals(0, dc.getFailedDirs().size());
     Assert.assertEquals(0, dc.getFullDirs().size());
+    Assert.assertEquals(0, dc.getErroredDirs().size());
+    Assert.assertNull(dc.getDirectoryErrorInfo(dirA));
   }
 
   @Test


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


[02/50] [abbrv] hadoop git commit: YARN-6952. Enable scheduling monitor in FS (Contributed by Yufei Gu via Daniel Templeton)

Posted by jh...@apache.org.
YARN-6952. Enable scheduling monitor in FS (Contributed by Yufei Gu via Daniel Templeton)


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

Branch: refs/heads/YARN-5734
Commit: 218588be773123404af4fd26eed5c9e3625feaa7
Parents: bbbf0e2
Author: Daniel Templeton <te...@apache.org>
Authored: Fri Aug 11 14:02:38 2017 -0700
Committer: Daniel Templeton <te...@apache.org>
Committed: Fri Aug 11 14:04:19 2017 -0700

----------------------------------------------------------------------
 .../yarn/server/resourcemanager/ResourceManager.java      |  9 +++------
 .../resourcemanager/monitor/SchedulingEditPolicy.java     |  4 ++--
 .../server/resourcemanager/monitor/SchedulingMonitor.java |  4 +---
 .../capacity/ProportionalCapacityPreemptionPolicy.java    |  4 ++--
 .../monitor/invariants/InvariantsChecker.java             | 10 +++++-----
 .../monitor/invariants/MetricsInvariantChecker.java       |  7 +++----
 6 files changed, 16 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/218588be/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 cb7daf9..5333f25 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
@@ -90,7 +90,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAlloca
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEventType;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.PreemptableResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
@@ -698,8 +697,7 @@ public class ResourceManager extends CompositeService implements Recoverable {
         }
       }
 
-      // creating monitors that handle preemption
-      createPolicyMonitors();
+      createSchedulerMonitors();
 
       masterService = createApplicationMasterService();
       addService(masterService) ;
@@ -800,9 +798,8 @@ public class ResourceManager extends CompositeService implements Recoverable {
 
     }
 
-    protected void createPolicyMonitors() {
-      if (scheduler instanceof PreemptableResourceScheduler
-          && conf.getBoolean(YarnConfiguration.RM_SCHEDULER_ENABLE_MONITORS,
+    protected void createSchedulerMonitors() {
+      if (conf.getBoolean(YarnConfiguration.RM_SCHEDULER_ENABLE_MONITORS,
           YarnConfiguration.DEFAULT_RM_SCHEDULER_ENABLE_MONITORS)) {
         LOG.info("Loading policy monitors");
         List<SchedulingEditPolicy> policies = conf.getInstances(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/218588be/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/SchedulingEditPolicy.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/monitor/SchedulingEditPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/SchedulingEditPolicy.java
index 47458a3..d2550e6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/SchedulingEditPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/SchedulingEditPolicy.java
@@ -19,12 +19,12 @@ package org.apache.hadoop.yarn.server.resourcemanager.monitor;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.PreemptableResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 
 public interface SchedulingEditPolicy {
 
   void init(Configuration config, RMContext context,
-      PreemptableResourceScheduler scheduler);
+      ResourceScheduler scheduler);
 
   /**
    * This method is invoked at regular intervals. Internally the policy is

http://git-wip-us.apache.org/repos/asf/hadoop/blob/218588be/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/SchedulingMonitor.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/monitor/SchedulingMonitor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/SchedulingMonitor.java
index 3def27f..1e3f691 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/SchedulingMonitor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/SchedulingMonitor.java
@@ -29,7 +29,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.PreemptableResourceScheduler;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -59,8 +58,7 @@ public class SchedulingMonitor extends AbstractService {
   }
 
   public void serviceInit(Configuration conf) throws Exception {
-    scheduleEditPolicy.init(conf, rmContext,
-        (PreemptableResourceScheduler) rmContext.getScheduler());
+    scheduleEditPolicy.init(conf, rmContext, rmContext.getScheduler());
     this.monitorInterval = scheduleEditPolicy.getMonitoringInterval();
     super.serviceInit(conf);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/218588be/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.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/monitor/capacity/ProportionalCapacityPreemptionPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.java
index fc8ad2b..c4c98e2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.java
@@ -32,7 +32,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingEditPolicy;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.PreemptableResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
@@ -150,7 +150,7 @@ public class ProportionalCapacityPreemptionPolicy
   }
 
   public void init(Configuration config, RMContext context,
-      PreemptableResourceScheduler sched) {
+      ResourceScheduler sched) {
     LOG.info("Preemption monitor:" + this.getClass().getCanonicalName());
     assert null == scheduler : "Unexpected duplicate call to init";
     if (!(sched instanceof CapacityScheduler)) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/218588be/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/invariants/InvariantsChecker.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/monitor/invariants/InvariantsChecker.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/invariants/InvariantsChecker.java
index 5800162..2c9031f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/invariants/InvariantsChecker.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/invariants/InvariantsChecker.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.monitor.invariants;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingEditPolicy;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.PreemptableResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -39,16 +39,16 @@ public abstract class InvariantsChecker implements SchedulingEditPolicy {
 
   private Configuration conf;
   private RMContext context;
-  private PreemptableResourceScheduler scheduler;
+  private ResourceScheduler scheduler;
   private boolean throwOnInvariantViolation;
   private long monitoringInterval;
 
   @Override
   public void init(Configuration config, RMContext rmContext,
-      PreemptableResourceScheduler preemptableResourceScheduler) {
+      ResourceScheduler scheduler) {
     this.conf = config;
     this.context = rmContext;
-    this.scheduler = preemptableResourceScheduler;
+    this.scheduler = scheduler;
     this.throwOnInvariantViolation =
         conf.getBoolean(InvariantsChecker.THROW_ON_VIOLATION, false);
     this.monitoringInterval =
@@ -89,7 +89,7 @@ public abstract class InvariantsChecker implements SchedulingEditPolicy {
     return context;
   }
 
-  public PreemptableResourceScheduler getScheduler() {
+  public ResourceScheduler getScheduler() {
     return scheduler;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/218588be/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/invariants/MetricsInvariantChecker.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/monitor/invariants/MetricsInvariantChecker.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/invariants/MetricsInvariantChecker.java
index 9fee2bd..ef4b9d0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/invariants/MetricsInvariantChecker.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/invariants/MetricsInvariantChecker.java
@@ -27,8 +27,8 @@ import org.apache.hadoop.metrics2.impl.MetricsCollectorImpl;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.metrics2.source.JvmMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.PreemptableResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -42,7 +42,6 @@ import java.io.IOException;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.stream.Collectors;
 
 /**
  * This policy checks at every invocation that a given set of invariants
@@ -78,9 +77,9 @@ public class MetricsInvariantChecker extends InvariantsChecker {
 
   @Override
   public void init(Configuration config, RMContext rmContext,
-      PreemptableResourceScheduler preemptableResourceScheduler) {
+      ResourceScheduler scheduler) {
 
-    super.init(config, rmContext, preemptableResourceScheduler);
+    super.init(config, rmContext, scheduler);
 
     this.metricsSystem = DefaultMetricsSystem.instance();
     this.queueMetrics =


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


[26/50] [abbrv] hadoop git commit: HDFS-12066. When Namenode is in safemode, may not allowed to remove an user's erasure coding policy. Contributed by lufei.

Posted by jh...@apache.org.
HDFS-12066. When Namenode is in safemode,may not allowed to remove an user's erasure coding policy. Contributed by lufei.


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

Branch: refs/heads/YARN-5734
Commit: e3ae3e26446c2e98b7aebc4ea66256cfdb4a397f
Parents: 1040bae
Author: Wei-Chiu Chuang <we...@apache.org>
Authored: Tue Aug 15 07:41:10 2017 -0700
Committer: Wei-Chiu Chuang <we...@apache.org>
Committed: Tue Aug 15 07:41:43 2017 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hdfs/server/namenode/FSNamesystem.java    | 3 +++
 .../src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java  | 9 +++++++++
 2 files changed, 12 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3ae3e26/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index caf73f7..1cfaa54 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -7113,6 +7113,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     boolean success = false;
     writeLock();
     try {
+      checkOperation(OperationCategory.WRITE);
+      checkNameNodeSafeMode("Cannot remove erasure coding policy "
+          + ecPolicyName);
       FSDirErasureCodingOp.removeErasureCodePolicy(this, ecPolicyName);
       success = true;
     } finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e3ae3e26/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java
index bc95ec7..f25d28f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java
@@ -482,6 +482,15 @@ public class TestSafeMode {
       // expected
     }
 
+    try {
+      dfs.removeErasureCodingPolicy("testECName");
+      fail("RemoveErasureCodingPolicy should have failed.");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains(
+          "Cannot remove erasure coding policy", ioe);
+      // expected
+    }
+
     assertFalse("Could not leave SM",
         dfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE));
   }


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


[37/50] [abbrv] hadoop git commit: HDFS-11082. Provide replicated EC policy to replicate files. Contributed by SammiChen.

Posted by jh...@apache.org.
HDFS-11082. Provide replicated EC policy to replicate files. Contributed by SammiChen.


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

Branch: refs/heads/YARN-5734
Commit: 96b3a6b9721e922d33fadc2459b561a85dbf9b8e
Parents: 08aaa4b
Author: Andrew Wang <wa...@apache.org>
Authored: Wed Aug 16 22:17:06 2017 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Wed Aug 16 22:17:06 2017 -0700

----------------------------------------------------------------------
 .../io/erasurecode/ErasureCodeConstants.java    |  8 ++
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |  3 +-
 .../hadoop/hdfs/DistributedFileSystem.java      |  6 +-
 .../hadoop/hdfs/protocol/ClientProtocol.java    |  6 +-
 .../hdfs/protocol/ErasureCodingPolicy.java      |  5 ++
 .../protocol/SystemErasureCodingPolicies.java   | 14 ++++
 .../namenode/ErasureCodingPolicyManager.java    | 13 ++-
 .../server/namenode/FSDirErasureCodingOp.java   | 13 ++-
 .../hdfs/server/namenode/FSDirWriteFileOp.java  |  2 +-
 .../org/apache/hadoop/hdfs/tools/ECAdmin.java   | 24 +++++-
 .../src/site/markdown/HDFSErasureCoding.md      | 16 ++--
 .../hadoop/hdfs/TestErasureCodingPolicies.java  | 81 ++++++++++++++++++
 .../hdfs/server/namenode/TestFSImage.java       | 87 ++++++++++++++++++++
 .../test/resources/testErasureCodingConf.xml    | 78 +++++++++++++++++-
 14 files changed, 331 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/96b3a6b9/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ErasureCodeConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ErasureCodeConstants.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ErasureCodeConstants.java
index e0d7946..d3c3b6b 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ErasureCodeConstants.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ErasureCodeConstants.java
@@ -30,6 +30,7 @@ public final class ErasureCodeConstants {
   public static final String RS_LEGACY_CODEC_NAME = "rs-legacy";
   public static final String XOR_CODEC_NAME = "xor";
   public static final String HHXOR_CODEC_NAME = "hhxor";
+  public static final String REPLICATION_CODEC_NAME = "replication";
 
   public static final ECSchema RS_6_3_SCHEMA = new ECSchema(
       RS_CODEC_NAME, 6, 3);
@@ -45,4 +46,11 @@ public final class ErasureCodeConstants {
 
   public static final ECSchema RS_10_4_SCHEMA = new ECSchema(
       RS_CODEC_NAME, 10, 4);
+
+  public static final ECSchema REPLICATION_1_2_SCHEMA = new ECSchema(
+      REPLICATION_CODEC_NAME, 1, 2);
+
+  public static final byte USER_DEFINED_POLICY_START_ID = (byte) 64;
+  public static final byte REPLICATION_POLICY_ID = (byte) 63;
+  public static final String REPLICATION_POLICY_NAME = REPLICATION_CODEC_NAME;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96b3a6b9/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index 969522d..47c14e2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -3044,7 +3044,8 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    *
    * @param src path to get the information for
    * @return Returns the policy information if file or directory on the path is
-   * erasure coded, null otherwise
+   * erasure coded, null otherwise. Null will be returned if directory or file
+   * has REPLICATION policy.
    * @throws IOException
    */
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96b3a6b9/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
index 8f82d03..ceec2b3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
@@ -2540,7 +2540,8 @@ public class DistributedFileSystem extends FileSystem {
    *
    * @param path The path of the file or directory
    * @return Returns the policy information if file or directory on the path
-   * is erasure coded, null otherwise
+   * is erasure coded, null otherwise. Null will be returned if directory or
+   * file has REPLICATION policy.
    * @throws IOException
    */
   public ErasureCodingPolicy getErasureCodingPolicy(final Path path)
@@ -2567,7 +2568,8 @@ public class DistributedFileSystem extends FileSystem {
   }
 
   /**
-   * Retrieve all the erasure coding policies supported by this file system.
+   * Retrieve all the erasure coding policies supported by this file system,
+   * excluding REPLICATION policy.
    *
    * @return all erasure coding policies supported by this file system.
    * @throws IOException

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96b3a6b9/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
index eb9380d..b0e85e5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
@@ -1588,7 +1588,8 @@ public interface ClientProtocol {
 
 
   /**
-   * Get the erasure coding policies loaded in Namenode.
+   * Get the erasure coding policies loaded in Namenode, excluding REPLICATION
+   * policy.
    *
    * @throws IOException
    */
@@ -1604,7 +1605,8 @@ public interface ClientProtocol {
   Map<String, String> getErasureCodingCodecs() throws IOException;
 
   /**
-   * Get the information about the EC policy for the path.
+   * Get the information about the EC policy for the path. Null will be returned
+   * if directory or file has REPLICATION policy.
    *
    * @param src path to get the info for
    * @throws IOException

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96b3a6b9/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingPolicy.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingPolicy.java
index 7afc377..501b67c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingPolicy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingPolicy.java
@@ -25,6 +25,7 @@ import org.apache.commons.lang.builder.HashCodeBuilder;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.io.erasurecode.ECSchema;
+import org.apache.hadoop.io.erasurecode.ErasureCodeConstants;
 
 /**
  * A policy about how to write/read/code an erasure coding file.
@@ -107,6 +108,10 @@ public final class ErasureCodingPolicy implements Serializable {
     this.id = id;
   }
 
+  public boolean isReplicationPolicy() {
+    return (id == ErasureCodeConstants.REPLICATION_POLICY_ID);
+  }
+
   @Override
   public boolean equals(Object o) {
     if (o == null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96b3a6b9/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SystemErasureCodingPolicies.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SystemErasureCodingPolicies.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SystemErasureCodingPolicies.java
index 2cd838b..f0efe76 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SystemErasureCodingPolicies.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SystemErasureCodingPolicies.java
@@ -68,6 +68,13 @@ public final class SystemErasureCodingPolicies {
       new ErasureCodingPolicy(ErasureCodeConstants.RS_10_4_SCHEMA,
           DEFAULT_CELLSIZE, RS_10_4_POLICY_ID);
 
+  // REPLICATION policy is always enabled.
+  private static final ErasureCodingPolicy REPLICATION_POLICY =
+      new ErasureCodingPolicy(ErasureCodeConstants.REPLICATION_POLICY_NAME,
+          ErasureCodeConstants.REPLICATION_1_2_SCHEMA,
+          DEFAULT_CELLSIZE,
+          ErasureCodeConstants.REPLICATION_POLICY_ID);
+
   private static final List<ErasureCodingPolicy> SYS_POLICIES =
       Collections.unmodifiableList(Arrays.asList(
           SYS_POLICY1, SYS_POLICY2, SYS_POLICY3, SYS_POLICY4,
@@ -118,4 +125,11 @@ public final class SystemErasureCodingPolicies {
   public static ErasureCodingPolicy getByName(String name) {
     return SYSTEM_POLICIES_BY_NAME.get(name);
   }
+
+  /**
+   * Get the special REPLICATION policy.
+   */
+  public static ErasureCodingPolicy  getReplicationPolicy() {
+    return REPLICATION_POLICY;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96b3a6b9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
index 18b8e8a..404a0aa 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 
 import org.apache.hadoop.io.erasurecode.CodecUtil;
+import org.apache.hadoop.io.erasurecode.ErasureCodeConstants;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -48,7 +49,6 @@ public final class ErasureCodingPolicyManager {
 
   public static Logger LOG = LoggerFactory.getLogger(
       ErasureCodingPolicyManager.class);
-  private static final byte USER_DEFINED_POLICY_START_ID = (byte) 64;
   private int maxCellSize =
       DFSConfigKeys.DFS_NAMENODE_EC_POLICIES_MAX_CELLSIZE_DEFAULT;
 
@@ -157,7 +157,13 @@ public final class ErasureCodingPolicyManager {
    * Get enabled policy by policy name.
    */
   public ErasureCodingPolicy getEnabledPolicyByName(String name) {
-    return enabledPoliciesByName.get(name);
+    ErasureCodingPolicy ecPolicy = enabledPoliciesByName.get(name);
+    if (ecPolicy == null) {
+      if (name.equalsIgnoreCase(ErasureCodeConstants.REPLICATION_POLICY_NAME)) {
+        ecPolicy = SystemErasureCodingPolicies.getReplicationPolicy();
+      }
+    }
+    return ecPolicy;
   }
 
   /**
@@ -257,7 +263,8 @@ public final class ErasureCodingPolicyManager {
 
   private byte getNextAvailablePolicyID() {
     byte currentId = this.userPoliciesByID.keySet().stream()
-        .max(Byte::compareTo).orElse(USER_DEFINED_POLICY_START_ID);
+        .max(Byte::compareTo).orElse(
+            ErasureCodeConstants.USER_DEFINED_POLICY_START_ID);
     return (byte) (currentId + 1);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96b3a6b9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java
index 7895433..426b42b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java
@@ -62,7 +62,7 @@ final class FSDirErasureCodingOp {
 
   /**
    * Check if the ecPolicyName is valid and enabled, return the corresponding
-   * EC policy if is.
+   * EC policy if is, including the REPLICATION EC policy.
    * @param fsn namespace
    * @param ecPolicyName name of EC policy to be checked
    * @return an erasure coding policy if ecPolicyName is valid and enabled
@@ -295,7 +295,12 @@ final class FSDirErasureCodingOp {
     if (iip.getLastINode() == null) {
       throw new FileNotFoundException("Path not found: " + iip.getPath());
     }
-    return getErasureCodingPolicyForPath(fsd, iip);
+
+    ErasureCodingPolicy ecPolicy = getErasureCodingPolicyForPath(fsd, iip);
+    if (ecPolicy != null && ecPolicy.isReplicationPolicy()) {
+      ecPolicy = null;
+    }
+    return ecPolicy;
   }
 
   /**
@@ -312,7 +317,8 @@ final class FSDirErasureCodingOp {
   }
 
   /**
-   * Get the erasure coding policy. This does not do any permission checking.
+   * Get the erasure coding policy, including the REPLICATION policy. This does
+   * not do any permission checking.
    *
    * @param fsn namespace
    * @param iip inodes in the path containing the file
@@ -350,6 +356,7 @@ final class FSDirErasureCodingOp {
     return CodecRegistry.getInstance().getCodec2CoderCompactMap();
   }
 
+  //return erasure coding policy for path, including REPLICATION policy
   private static ErasureCodingPolicy getErasureCodingPolicyForPath(
       FSDirectory fsd, INodesInPath iip) throws IOException {
     Preconditions.checkNotNull(iip, "INodes cannot be null");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96b3a6b9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
index a62cddd..7ab05d7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
@@ -541,7 +541,7 @@ class FSDirWriteFileOp {
           ecPolicy = FSDirErasureCodingOp.unprotectedGetErasureCodingPolicy(
               fsd.getFSNamesystem(), existing);
         }
-        if (ecPolicy != null) {
+        if (ecPolicy != null && (!ecPolicy.isReplicationPolicy())) {
           isStriped = true;
         }
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96b3a6b9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/ECAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/ECAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/ECAdmin.java
index 17a84f9..55d85ff 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/ECAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/ECAdmin.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.protocol.AddECPolicyResponse;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.util.ECPolicyLoader;
+import org.apache.hadoop.io.erasurecode.ErasureCodeConstants;
 import org.apache.hadoop.tools.TableListing;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Tool;
@@ -309,7 +310,8 @@ public class ECAdmin extends Configured implements Tool {
 
     @Override
     public String getShortUsage() {
-      return "[" + getName() + " -path <path> -policy <policy>]\n";
+      return "[" + getName() +
+          " -path <path> [-policy <policy>] [-replicate]]\n";
     }
 
     @Override
@@ -318,9 +320,13 @@ public class ECAdmin extends Configured implements Tool {
       listing.addRow("<path>", "The path of the file/directory to set " +
           "the erasure coding policy");
       listing.addRow("<policy>", "The name of the erasure coding policy");
+      listing.addRow("-replicate",
+          "force 3x replication scheme on the directory");
       return getShortUsage() + "\n" +
           "Set the erasure coding policy for a file/directory.\n\n" +
-          listing.toString();
+          listing.toString() + "\n" +
+          "-replicate and -policy are optional arguments. They cannot been " +
+          "used at the same time";
     }
 
     @Override
@@ -332,14 +338,24 @@ public class ECAdmin extends Configured implements Tool {
         return 1;
       }
 
-      final String ecPolicyName = StringUtils.popOptionWithArgument("-policy",
+      String ecPolicyName = StringUtils.popOptionWithArgument("-policy",
           args);
+      final boolean replicate = StringUtils.popOption("-replicate", args);
 
       if (args.size() > 0) {
         System.err.println(getName() + ": Too many arguments");
         return 1;
       }
 
+      if (replicate) {
+        if (ecPolicyName != null) {
+          System.err.println(getName() +
+              ": -replicate and -policy cannot been used at the same time");
+          return 2;
+        }
+        ecPolicyName = ErasureCodeConstants.REPLICATION_POLICY_NAME;
+      }
+
       final Path p = new Path(path);
       final DistributedFileSystem dfs = AdminHelper.getDFS(p.toUri(), conf);
       try {
@@ -353,7 +369,7 @@ public class ECAdmin extends Configured implements Tool {
         }
       } catch (Exception e) {
         System.err.println(AdminHelper.prettifyException(e));
-        return 2;
+        return 3;
       }
       return 0;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96b3a6b9/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
index 4a48c2a..786b512 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
@@ -65,9 +65,11 @@ Architecture
 
       2. _The size of a striping cell._ This determines the granularity of striped reads and writes, including buffer sizes and encoding work.
 
-    Policies are named *codec*-*num data blocks*-*num parity blocks*-*cell size*. Currently, five built-in policies are supported: `RS-3-2-64k`, `RS-6-3-64k`, `RS-10-4-64k`, `RS-LEGACY-6-3-64k`, and `XOR-2-1-64k`.
+    Policies are named *codec*-*num data blocks*-*num parity blocks*-*cell size*. Currently, six built-in policies are supported: `RS-3-2-64k`, `RS-6-3-64k`, `RS-10-4-64k`, `RS-LEGACY-6-3-64k`, `XOR-2-1-64k` and `REPLICATION`.
 
-    By default, all built-in erasure coding policies are disabled.
+    `REPLICATION` is a special policy. It can only be set on directory, to force the directory to adopt 3x replication scheme, instead of inheriting its ancestor's erasure coding policy. This policy makes it possible to interleave 3x replication scheme directory with erasure coding directory.
+
+    `REPLICATION` policy is always enabled. For other built-in policies, unless they are configured in `dfs.namenode.ec.policies.enabled` property, otherwise they are disabled by default.
 
     Similar to HDFS storage policies, erasure coding policies are set on a directory. When a file is created, it inherits the EC policy of its nearest ancestor directory.
 
@@ -112,7 +114,7 @@ Deployment
   what EC policies can be set by clients. It does not affect the behavior of already set file or directory-level EC policies.
 
   By default, all built-in erasure coding policies are disabled. Typically, the cluster administrator will enable set of policies by including them
-  in the `dfs .namenode.ec.policies.enabled` configuration based on the size of the cluster and the desired fault-tolerance properties. For instance,
+  in the `dfs.namenode.ec.policies.enabled` configuration based on the size of the cluster and the desired fault-tolerance properties. For instance,
   for a cluster with 9 racks, a policy like `RS-10-4-64k` will not preserve rack-level fault-tolerance, and `RS-6-3-64k` or `RS-3-2-64k` might
   be more appropriate. If the administrator only cares about node-level fault-tolerance, `RS-10-4-64k` would still be appropriate as long as
   there are at least 14 DataNodes in the cluster.
@@ -153,7 +155,7 @@ Deployment
   HDFS provides an `ec` subcommand to perform administrative commands related to erasure coding.
 
        hdfs ec [generic options]
-         [-setPolicy -policy <policyName> -path <path>]
+         [-setPolicy -path <path> [-policy <policyName>] [-replicate]]
          [-getPolicy -path <path>]
          [-unsetPolicy -path <path>]
          [-listPolicies]
@@ -165,7 +167,7 @@ Deployment
 
 Below are the details about each command.
 
- *  `[-setPolicy -policy <policyName> -path <path>]`
+ *  `[-setPolicy -path <path> [-policy <policyName>] [-replicate]]`
 
     Sets an erasure coding policy on a directory at the specified path.
 
@@ -175,6 +177,10 @@ Below are the details about each command.
       This parameter can be omitted if a 'dfs.namenode.ec.system.default.policy' configuration is set.
       The EC policy of the path will be set with the default value in configuration.
 
+      `-replicate` apply the special `REPLICATION` policy on the directory, force the directory to adopt 3x replication scheme.
+
+      `-replicate` and `-policy <policyName>` are optional arguments. They cannot be specified at the same time.
+
 
  *  `[-getPolicy -path <path>]`
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96b3a6b9/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
index 22e118f..47cdf23 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
@@ -732,4 +732,85 @@ public class TestErasureCodingPolicies {
       }
     });
   }
+
+  @Test
+  public void testReplicationPolicy() throws Exception {
+    ErasureCodingPolicy replicaPolicy =
+        SystemErasureCodingPolicies.getReplicationPolicy();
+
+    final Path rootDir = new Path("/striped");
+    final Path replicaDir = new Path(rootDir, "replica");
+    final Path subReplicaDir = new Path(replicaDir, "replica");
+    final Path replicaFile = new Path(replicaDir, "file");
+    final Path subReplicaFile = new Path(subReplicaDir, "file");
+
+    fs.mkdirs(rootDir);
+    fs.setErasureCodingPolicy(rootDir, ecPolicy.getName());
+
+    // 1. At first, child directory will inherit parent's EC policy
+    fs.mkdirs(replicaDir);
+    fs.createFile(replicaFile).build().close();
+    HdfsFileStatus fileStatus = (HdfsFileStatus)fs.getFileStatus(replicaFile);
+    assertEquals("File should inherit EC policy.", ecPolicy, fileStatus
+        .getErasureCodingPolicy());
+    assertEquals("File should be a EC file.", true, fileStatus
+        .isErasureCoded());
+    assertEquals("File should have the same EC policy as its ancestor.",
+        ecPolicy, fs.getErasureCodingPolicy(replicaFile));
+    fs.delete(replicaFile, false);
+
+    // 2. Set replication policy on child directory, then get back the policy
+    fs.setErasureCodingPolicy(replicaDir, replicaPolicy.getName());
+    ErasureCodingPolicy temp = fs.getErasureCodingPolicy(replicaDir);
+    assertEquals("Directory should hide replication EC policy.",
+        null, temp);
+
+    // 3. New file will be replication file. Please be noted that replication
+    //    policy only set on directory, not on file
+    fs.createFile(replicaFile).build().close();
+    assertEquals("Replication file should have default replication factor.",
+        fs.getDefaultReplication(),
+        fs.getFileStatus(replicaFile).getReplication());
+    fs.setReplication(replicaFile, (short) 2);
+    assertEquals("File should have replication factor as expected.",
+        2, fs.getFileStatus(replicaFile).getReplication());
+    fileStatus = (HdfsFileStatus)fs.getFileStatus(replicaFile);
+    assertEquals("File should not have EC policy.", null, fileStatus
+        .getErasureCodingPolicy());
+    assertEquals("File should not be a EC file.", false,
+        fileStatus.isErasureCoded());
+    ErasureCodingPolicy ecPolicyOnFile = fs.getErasureCodingPolicy(replicaFile);
+    assertEquals("File should not have EC policy.", null, ecPolicyOnFile);
+    fs.delete(replicaFile, false);
+
+    // 4. New directory under replication directory, is also replication
+    // directory
+    fs.mkdirs(subReplicaDir);
+    assertEquals("Directory should inherit hiding replication EC policy.",
+        null, fs.getErasureCodingPolicy(subReplicaDir));
+    fs.createFile(subReplicaFile).build().close();
+    assertEquals("File should have default replication factor.",
+        fs.getDefaultReplication(),
+        fs.getFileStatus(subReplicaFile).getReplication());
+    fileStatus = (HdfsFileStatus)fs.getFileStatus(subReplicaFile);
+    assertEquals("File should not have EC policy.", null,
+        fileStatus.getErasureCodingPolicy());
+    assertEquals("File should not be a EC file.", false,
+        fileStatus.isErasureCoded());
+    assertEquals("File should not have EC policy.", null,
+        fs.getErasureCodingPolicy(subReplicaFile));
+    fs.delete(subReplicaFile, false);
+
+    // 5. Unset replication policy on directory, new file will be EC file
+    fs.unsetErasureCodingPolicy(replicaDir);
+    fs.createFile(subReplicaFile).build().close();
+    fileStatus = (HdfsFileStatus)fs.getFileStatus(subReplicaFile);
+    assertEquals("File should inherit EC policy.", ecPolicy,
+        fileStatus.getErasureCodingPolicy());
+    assertEquals("File should be a EC file.", true,
+        fileStatus.isErasureCoded());
+    assertEquals("File should have the same EC policy as its ancestor",
+        ecPolicy, fs.getErasureCodingPolicy(subReplicaFile));
+    fs.delete(subReplicaFile, false);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96b3a6b9/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
index 22c40fb..9256056 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
@@ -723,4 +723,91 @@ public class TestFSImage {
         .getBlockType());
     assertEquals(defaultBlockType, BlockType.CONTIGUOUS);
   }
+
+  /**
+   * Test if a INodeFile under a replication EC policy directory
+   * can be saved by FSImageSerialization and loaded by FSImageFormat#Loader.
+   */
+  @Test
+  public void testSaveAndLoadFileUnderReplicationPolicyDir()
+      throws IOException {
+    Configuration conf = new Configuration();
+    DFSTestUtil.enableAllECPolicies(conf);
+    MiniDFSCluster cluster = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).build();
+      cluster.waitActive();
+      FSNamesystem fsn = cluster.getNamesystem();
+      DistributedFileSystem fs = cluster.getFileSystem();
+      ErasureCodingPolicy replicaPolicy =
+          SystemErasureCodingPolicies.getReplicationPolicy();
+      ErasureCodingPolicy defaultEcPolicy =
+          StripedFileTestUtil.getDefaultECPolicy();
+
+      final Path ecDir = new Path("/ec");
+      final Path replicaDir = new Path(ecDir, "replica");
+      final Path replicaFile1 = new Path(replicaDir, "f1");
+      final Path replicaFile2 = new Path(replicaDir, "f2");
+
+      // create root directory
+      fs.mkdir(ecDir, null);
+      fs.setErasureCodingPolicy(ecDir, defaultEcPolicy.getName());
+
+      // create directory, and set replication Policy
+      fs.mkdir(replicaDir, null);
+      fs.setErasureCodingPolicy(replicaDir, replicaPolicy.getName());
+
+      // create an empty file f1
+      fs.create(replicaFile1).close();
+
+      // create an under-construction file f2
+      FSDataOutputStream out = fs.create(replicaFile2, (short) 2);
+      out.writeBytes("hello");
+      ((DFSOutputStream) out.getWrappedStream()).hsync(EnumSet
+          .of(SyncFlag.UPDATE_LENGTH));
+
+      // checkpoint
+      fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+      fs.saveNamespace();
+      fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+
+      cluster.restartNameNode();
+      cluster.waitActive();
+      fs = cluster.getFileSystem();
+
+      assertTrue(fs.getFileStatus(ecDir).isDirectory());
+      assertTrue(fs.getFileStatus(replicaDir).isDirectory());
+      assertTrue(fs.exists(replicaFile1));
+      assertTrue(fs.exists(replicaFile2));
+
+      // check directories
+      assertEquals("Directory should have default EC policy.",
+          defaultEcPolicy, fs.getErasureCodingPolicy(ecDir));
+      assertEquals("Directory should hide replication EC policy.",
+          null, fs.getErasureCodingPolicy(replicaDir));
+
+      // check file1
+      assertEquals("File should not have EC policy.", null,
+          fs.getErasureCodingPolicy(replicaFile1));
+      // check internals of file2
+      INodeFile file2Node =
+          fsn.dir.getINode4Write(replicaFile2.toString()).asFile();
+      assertEquals("hello".length(), file2Node.computeFileSize());
+      assertTrue(file2Node.isUnderConstruction());
+      BlockInfo[] blks = file2Node.getBlocks();
+      assertEquals(1, blks.length);
+      assertEquals(BlockUCState.UNDER_CONSTRUCTION, blks[0].getBlockUCState());
+      assertEquals("File should return expected replication factor.",
+          2, blks[0].getReplication());
+      assertEquals("File should not have EC policy.", null,
+          fs.getErasureCodingPolicy(replicaFile2));
+      // check lease manager
+      Lease lease = fsn.leaseManager.getLease(file2Node);
+      Assert.assertNotNull(lease);
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/96b3a6b9/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml
index c68c6d6..1baf355 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml
@@ -101,7 +101,7 @@
         </comparator>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>[-setPolicy -path &lt;path&gt; -policy &lt;policy&gt;]</expected-output>
+          <expected-output>[-setPolicy -path &lt;path&gt; [-policy &lt;policy&gt;] [-replicate]]</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -238,6 +238,29 @@
     </test>
 
     <test>
+      <description>setPolicy : set replication policy on a directory</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir /ecdir</command>
+        <ec-admin-command>-fs NAMENODE -setPolicy -policy RS-6-3-64k -path /ecdir</ec-admin-command>
+        <command>-fs NAMENODE -mkdir /ecdir/replica</command>
+        <ec-admin-command>-fs NAMENODE -setPolicy -replicate -path /ecdir/replica</ec-admin-command>
+        <command>-fs NAMENODE -touchz /ecdir/replica/file</command>
+        <ec-admin-command>-fs NAMENODE -getPolicy -path /ecdir/replica/file</ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rm /ecdir/replica/file</command>
+        <command>-fs NAMENODE -rmdir /ecdir/replica</command>
+        <command>-fs NAMENODE -rmdir /ecdir</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>is unspecified</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
       <description>unsetPolicy : unset policy and get</description>
       <test-commands>
         <command>-fs NAMENODE -mkdir /ecdir</command>
@@ -453,7 +476,7 @@
 
 <!-- Test illegal parameters -->
     <test>
-      <description>setPolicy : illegal parameters - path is missing</description>
+      <description>setPolicy : illegal parameters - path option is missing</description>
       <test-commands>
         <command>-fs NAMENODE -mkdir /ecdir</command>
         <ec-admin-command>-fs NAMENODE -setPolicy</ec-admin-command>
@@ -470,7 +493,7 @@
     </test>
 
     <test>
-      <description>setPolicy : illegal parameters - policy name is missing</description>
+      <description>setPolicy : illegal parameters - path name is missing</description>
       <test-commands>
         <command>-fs NAMENODE -mkdir /ecdir</command>
         <ec-admin-command>-fs NAMENODE -setPolicy -path</ec-admin-command>
@@ -487,7 +510,7 @@
     </test>
 
     <test>
-      <description>setPolicy : illegal parameters - too many arguments</description>
+      <description>setPolicy : illegal parameters - too many arguments case 1</description>
       <test-commands>
         <command>-fs NAMENODE -mkdir /ecdir</command>
         <ec-admin-command>-fs NAMENODE -setPolicy -path /ecdir1 -policy RS-3-2-64k /ecdir2</ec-admin-command>
@@ -504,6 +527,23 @@
     </test>
 
     <test>
+      <description>setPolicy : illegal parameters - too many arguments case 2</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir /ecdir</command>
+        <ec-admin-command>-fs NAMENODE -setPolicy -path /ecdir1 -policy RS-3-2-64k -replicate /ecdir2</ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rmdir /ecdir</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>-setPolicy: Too many arguments</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
       <description>setPolicy : illegal parameters - invalidpolicy</description>
       <test-commands>
         <command>-fs NAMENODE -mkdir /ecdir</command>
@@ -553,6 +593,36 @@
     </test>
 
     <test>
+      <description>setPolicy : illegal parameters - wrong spelling replicate </description>
+      <test-commands>
+        <ec-admin-command>-fs NAMENODE -setPolicy -path /ecdir -replica</ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>-setPolicy: Too many arguments</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
+      <description>setPolicy : illegal parameters - replicate and policy coexist</description>
+      <test-commands>
+        <ec-admin-command>-fs NAMENODE -setPolicy -path /ecdir -policy RS-3-2-64k -replicate</ec-admin-command>
+      </test-commands>
+      <cleanup-commands>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>-replicate and -policy cannot been used at the same time</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
       <description>setPolicy : set erasure coding policy without given a specific policy name</description>
       <test-commands>
         <command>-fs NAMENODE -mkdir /ecdir</command>


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


[36/50] [abbrv] hadoop git commit: HDFS-12269. Better to return a Map rather than HashMap in getErasureCodingCodecs. Contributed by Huafeng Wang.

Posted by jh...@apache.org.
HDFS-12269. Better to return a Map rather than HashMap in getErasureCodingCodecs. Contributed by Huafeng Wang.


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

Branch: refs/heads/YARN-5734
Commit: 08aaa4b36fab44c3f47878b3c487db3b373ffccf
Parents: ab051bd
Author: Akira Ajisaka <aa...@apache.org>
Authored: Thu Aug 17 13:20:27 2017 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Thu Aug 17 13:20:27 2017 +0900

----------------------------------------------------------------------
 .../java/org/apache/hadoop/io/erasurecode/CodecRegistry.java    | 2 +-
 .../src/main/java/org/apache/hadoop/hdfs/DFSClient.java         | 2 +-
 .../main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java | 3 +--
 .../java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java    | 4 ++--
 .../hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java     | 5 +++--
 .../ClientNamenodeProtocolServerSideTranslatorPB.java           | 3 +--
 .../hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java       | 4 ++--
 .../org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java    | 4 ++--
 .../apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java   | 4 ++--
 .../src/main/java/org/apache/hadoop/hdfs/tools/ECAdmin.java     | 3 +--
 .../java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java  | 4 ++--
 11 files changed, 18 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/08aaa4b3/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/CodecRegistry.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/CodecRegistry.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/CodecRegistry.java
index fcf1349..daf91e2 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/CodecRegistry.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/CodecRegistry.java
@@ -176,7 +176,7 @@ public final class CodecRegistry {
    * @return a map of all codec names, and their corresponding code list
    * separated by ','.
    */
-  public HashMap<String, String> getCodec2CoderCompactMap() {
+  public Map<String, String> getCodec2CoderCompactMap() {
     return coderNameCompactMap;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/08aaa4b3/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index 88b273a..969522d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -2764,7 +2764,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     }
   }
 
-  public HashMap<String, String> getErasureCodingCodecs() throws IOException {
+  public Map<String, String> getErasureCodingCodecs() throws IOException {
     checkOpen();
     try (TraceScope ignored = tracer.newScope("getErasureCodingCodecs")) {
       return namenode.getErasureCodingCodecs();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/08aaa4b3/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
index cd368d4..8f82d03 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
@@ -26,7 +26,6 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.EnumSet;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
@@ -2585,7 +2584,7 @@ public class DistributedFileSystem extends FileSystem {
    * @return all erasure coding codecs and coders supported by this file system.
    * @throws IOException
    */
-  public HashMap<String, String> getAllErasureCodingCodecs()
+  public Map<String, String> getAllErasureCodingCodecs()
       throws IOException {
     return dfs.getErasureCodingCodecs();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/08aaa4b3/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
index 45c6b32..eb9380d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
@@ -19,8 +19,8 @@ package org.apache.hadoop.hdfs.protocol;
 
 import java.io.IOException;
 import java.util.EnumSet;
-import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -1601,7 +1601,7 @@ public interface ClientProtocol {
    * @throws IOException
    */
   @Idempotent
-  HashMap<String, String> getErasureCodingCodecs() throws IOException;
+  Map<String, String> getErasureCodingCodecs() throws IOException;
 
   /**
    * Get the information about the EC policy for the path.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/08aaa4b3/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
index aed4117..ac06c1a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
@@ -26,6 +26,7 @@ import java.util.List;
 
 import com.google.common.collect.Lists;
 
+import java.util.Map;
 import java.util.concurrent.TimeUnit;
 import java.util.stream.Collectors;
 
@@ -1760,11 +1761,11 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
 
   @Override
-  public HashMap<String, String> getErasureCodingCodecs() throws IOException {
+  public Map<String, String> getErasureCodingCodecs() throws IOException {
     try {
       GetErasureCodingCodecsResponseProto response = rpcProxy
           .getErasureCodingCodecs(null, VOID_GET_EC_CODEC_REQUEST);
-      HashMap<String, String> ecCodecs = new HashMap<String, String>();
+      Map<String, String> ecCodecs = new HashMap<>();
       for (CodecProto codec : response.getCodecList()) {
         ecCodecs.put(codec.getCodec(), codec.getCoders());
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/08aaa4b3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
index 38b81c6..a446276 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
@@ -21,7 +21,6 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.EnumSet;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.stream.Collectors;
@@ -1664,7 +1663,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       RpcController controller, GetErasureCodingCodecsRequestProto request)
       throws ServiceException {
     try {
-      HashMap<String, String> codecs = server.getErasureCodingCodecs();
+      Map<String, String> codecs = server.getErasureCodingCodecs();
       GetErasureCodingCodecsResponseProto.Builder resBuilder =
           GetErasureCodingCodecsResponseProto.newBuilder();
       for (Map.Entry<String, String> codec : codecs.entrySet()) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/08aaa4b3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java
index 486503c..7895433 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java
@@ -25,8 +25,8 @@ import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.EnumSet;
-import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.stream.Collectors;
 
 import com.google.common.base.Preconditions;
@@ -344,7 +344,7 @@ final class FSDirErasureCodingOp {
    * @param fsn namespace
    * @return {@link java.util.HashMap} array
    */
-  static HashMap<String, String> getErasureCodingCodecs(final FSNamesystem fsn)
+  static Map<String, String> getErasureCodingCodecs(final FSNamesystem fsn)
       throws IOException {
     assert fsn.hasReadLock();
     return CodecRegistry.getInstance().getCodec2CoderCompactMap();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/08aaa4b3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 1cfaa54..2313335 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -7255,14 +7255,14 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   /**
    * Get available erasure coding codecs and corresponding coders.
    */
-  HashMap<String, String> getErasureCodingCodecs() throws IOException {
+  Map<String, String> getErasureCodingCodecs() throws IOException {
     final String operationName = "getErasureCodingCodecs";
     boolean success = false;
     checkOperation(OperationCategory.READ);
     readLock();
     try {
       checkOperation(OperationCategory.READ);
-      final HashMap<String, String> ret =
+      final Map<String, String> ret =
           FSDirErasureCodingOp.getErasureCodingCodecs(this);
       success = true;
       return ret;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/08aaa4b3/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 d304d3d..7871202 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
@@ -37,9 +37,9 @@ import java.net.InetSocketAddress;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.EnumSet;
-import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.Callable;
 
@@ -2278,7 +2278,7 @@ public class NameNodeRpcServer implements NamenodeProtocols {
   }
 
   @Override // ClientProtocol
-  public HashMap<String, String> getErasureCodingCodecs() throws IOException {
+  public Map<String, String> getErasureCodingCodecs() throws IOException {
     checkNNStartup();
     return namesystem.getErasureCodingCodecs();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/08aaa4b3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/ECAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/ECAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/ECAdmin.java
index 46600a0..17a84f9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/ECAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/ECAdmin.java
@@ -33,7 +33,6 @@ import org.apache.hadoop.util.ToolRunner;
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.Collection;
-import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
@@ -441,7 +440,7 @@ public class ECAdmin extends Configured implements Tool {
 
       final DistributedFileSystem dfs = AdminHelper.getDFS(conf);
       try {
-        HashMap<String, String> codecs =
+        Map<String, String> codecs =
             dfs.getAllErasureCodingCodecs();
         if (codecs.isEmpty()) {
           System.out.println("No erasure coding codecs are supported on the " +

http://git-wip-us.apache.org/repos/asf/hadoop/blob/08aaa4b3/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
index 06edb1a..22e118f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
@@ -50,8 +50,8 @@ import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
 import java.util.Collection;
 import java.util.EnumSet;
-import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 
 import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains;
 import static org.junit.Assert.*;
@@ -647,7 +647,7 @@ public class TestErasureCodingPolicies {
 
   @Test
   public void testGetAllErasureCodingCodecs() throws Exception {
-    HashMap<String, String> allECCodecs = fs
+    Map<String, String> allECCodecs = fs
         .getAllErasureCodingCodecs();
     assertTrue("At least 3 system codecs should be enabled",
         allECCodecs.size() >= 3);


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


[38/50] [abbrv] hadoop git commit: HADOOP-14560. Make HttpServer2 backlog size configurable. Contributed by Alexander Krasheninnikov.

Posted by jh...@apache.org.
HADOOP-14560. Make HttpServer2 backlog size configurable. Contributed by Alexander Krasheninnikov.

This closes #242.


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

Branch: refs/heads/YARN-5734
Commit: 1f04cb45f70648678840cdafbec68d534b03fe95
Parents: 96b3a6b
Author: Alexandr Krasheninnikov <a....@corp.badoo.com>
Authored: Wed Jun 21 12:57:34 2017 +0300
Committer: John Zhuge <jz...@apache.org>
Committed: Thu Aug 17 01:05:19 2017 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/http/HttpServer2.java  |  9 ++++++++-
 .../java/org/apache/hadoop/http/TestHttpServer.java    | 13 +++++++++++++
 2 files changed, 21 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f04cb45/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer2.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer2.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer2.java
index 28b9bb0..a450f66 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer2.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer2.java
@@ -128,6 +128,10 @@ public final class HttpServer2 implements FilterContainer {
   public static final String HTTP_MAX_RESPONSE_HEADER_SIZE_KEY =
       "hadoop.http.max.response.header.size";
   public static final int HTTP_MAX_RESPONSE_HEADER_SIZE_DEFAULT = 65536;
+
+  public static final String HTTP_SOCKET_BACKLOG_SIZE_KEY =
+      "hadoop.http.socket.backlog.size";
+  public static final int HTTP_SOCKET_BACKLOG_SIZE_DEFAULT = 128;
   public static final String HTTP_MAX_THREADS_KEY = "hadoop.http.max.threads";
   public static final String HTTP_TEMP_DIR_KEY = "hadoop.http.temp.dir";
 
@@ -433,6 +437,9 @@ public final class HttpServer2 implements FilterContainer {
       httpConfig.setResponseHeaderSize(responseHeaderSize);
       httpConfig.setSendServerVersion(false);
 
+      int backlogSize = conf.getInt(HTTP_SOCKET_BACKLOG_SIZE_KEY,
+          HTTP_SOCKET_BACKLOG_SIZE_DEFAULT);
+
       for (URI ep : endpoints) {
         final ServerConnector connector;
         String scheme = ep.getScheme();
@@ -448,6 +455,7 @@ public final class HttpServer2 implements FilterContainer {
         }
         connector.setHost(ep.getHost());
         connector.setPort(ep.getPort() == -1 ? 0 : ep.getPort());
+        connector.setAcceptQueueSize(backlogSize);
         server.addListener(connector);
       }
       server.loadListeners();
@@ -640,7 +648,6 @@ public final class HttpServer2 implements FilterContainer {
 
   private static void configureChannelConnector(ServerConnector c) {
     c.setIdleTimeout(10000);
-    c.setAcceptQueueSize(128);
     if(Shell.WINDOWS) {
       // result of setting the SO_REUSEADDR flag is different on Windows
       // http://msdn.microsoft.com/en-us/library/ms740621(v=vs.85).aspx

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1f04cb45/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServer.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServer.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServer.java
index 6ec6e0f..ca7e466 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServer.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServer.java
@@ -682,4 +682,17 @@ public class TestHttpServer extends HttpServerFunctionalTest {
       stopHttpServer(myServer2);
     }
   }
+
+  @Test
+  public void testBacklogSize() throws Exception
+  {
+    final int backlogSize = 2048;
+    Configuration conf = new Configuration();
+    conf.setInt(HttpServer2.HTTP_SOCKET_BACKLOG_SIZE_KEY, backlogSize);
+    HttpServer2 srv = createServer("test", conf);
+    List<?> listeners = (List<?>) Whitebox.getInternalState(srv,
+            "listeners");
+    ServerConnector listener = (ServerConnector)listeners.get(0);
+    assertEquals(backlogSize, listener.getAcceptQueueSize());
+  }
 }


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


[13/50] [abbrv] hadoop git commit: YARN-6996. Change javax.cache library implementation from JSR107 to Apache Geronimo. (Ray Chiang via Subru).

Posted by jh...@apache.org.
YARN-6996. Change javax.cache library implementation from JSR107 to Apache Geronimo. (Ray Chiang via Subru).


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

Branch: refs/heads/YARN-5734
Commit: 18f3603bce37e0e07c9075811b1179afc2c227eb
Parents: e2f6299
Author: Subru Krishnan <su...@apache.org>
Authored: Mon Aug 14 11:10:00 2017 -0700
Committer: Subru Krishnan <su...@apache.org>
Committed: Mon Aug 14 11:10:00 2017 -0700

----------------------------------------------------------------------
 hadoop-project/pom.xml                                         | 6 +++---
 .../hadoop-yarn-server/hadoop-yarn-server-common/pom.xml       | 4 ++--
 2 files changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/18f3603b/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 6311cd9..8c1d374 100755
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -96,7 +96,7 @@
     <apacheds.version>2.0.0-M21</apacheds.version>
     <ldap-api.version>1.0.0-M33</ldap-api.version>
 
-    <jcache.version>1.0.0</jcache.version>
+    <jcache.version>1.0-alpha-1</jcache.version>
     <ehcache.version>3.3.1</ehcache.version>
     <hikari.version>2.4.12</hikari.version>
     <mssql.version>6.2.1.jre7</mssql.version>
@@ -1276,8 +1276,8 @@
           <version>1.0.0</version>
         </dependency>
         <dependency>
-          <groupId>javax.cache</groupId>
-          <artifactId>cache-api</artifactId>
+          <groupId>org.apache.geronimo.specs</groupId>
+          <artifactId>geronimo-jcache_1.0_spec</artifactId>
           <version>${jcache.version}</version>
         </dependency>
         <dependency>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/18f3603b/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 5f85097..441a574 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
@@ -103,8 +103,8 @@
       <artifactId>leveldbjni-all</artifactId>
     </dependency>
     <dependency>
-      <groupId>javax.cache</groupId>
-      <artifactId>cache-api</artifactId>
+      <groupId>org.apache.geronimo.specs</groupId>
+      <artifactId>geronimo-jcache_1.0_spec</artifactId>
     </dependency>
     <dependency>
       <groupId>org.ehcache</groupId>


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


[17/50] [abbrv] hadoop git commit: YARN-6987. Log app attempt during InvalidStateTransition. Contributed by Jonathan Eagles

Posted by jh...@apache.org.
YARN-6987. Log app attempt during InvalidStateTransition. Contributed by Jonathan Eagles


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

Branch: refs/heads/YARN-5734
Commit: 3325ef653d6f364a82dd32485d9ef6d987380ce3
Parents: 6b09c32
Author: Jason Lowe <jl...@yahoo-inc.com>
Authored: Mon Aug 14 14:40:08 2017 -0500
Committer: Jason Lowe <jl...@yahoo-inc.com>
Committed: Mon Aug 14 14:40:08 2017 -0500

----------------------------------------------------------------------
 .../hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java       | 3 ++-
 .../server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java    | 3 ++-
 2 files changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3325ef65/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
index fa2f20c..03be793 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
@@ -888,7 +888,8 @@ public class RMAppImpl implements RMApp, Recoverable {
         /* keep the master in sync with the state machine */
         this.stateMachine.doTransition(event.getType(), event);
       } catch (InvalidStateTransitionException e) {
-        LOG.error("Can't handle this event at current state", e);
+        LOG.error("App: " + appID
+            + " can't handle this event at current state", e);
         /* TODO fail the application on the failed transition */
       }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3325ef65/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
index 254768b..7d453bd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
@@ -911,7 +911,8 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
         /* keep the master in sync with the state machine */
         this.stateMachine.doTransition(event.getType(), event);
       } catch (InvalidStateTransitionException e) {
-        LOG.error("Can't handle this event at current state", e);
+        LOG.error("App attempt: " + appAttemptID
+            + " can't handle this event at current state", e);
         /* TODO fail the application on the failed transition */
       }
 


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


[15/50] [abbrv] hadoop git commit: YARN-6905 Multiple HBaseTimelineStorage test failures due to missing FastNumberFormat (Contributed by Haibo Chen)

Posted by jh...@apache.org.
YARN-6905 Multiple HBaseTimelineStorage test failures due to missing FastNumberFormat (Contributed by Haibo Chen)


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

Branch: refs/heads/YARN-5734
Commit: 608a06cca5d68b3155bd70a94bf29ae0942b9ca0
Parents: d72124a
Author: Vrushali C <vr...@apache.org>
Authored: Mon Aug 14 11:40:27 2017 -0700
Committer: Vrushali C <vr...@apache.org>
Committed: Mon Aug 14 11:41:11 2017 -0700

----------------------------------------------------------------------
 .../storage/TestHBaseTimelineStorageApps.java   |  4 +-
 .../TestHBaseTimelineStorageEntities.java       | 14 ++++---
 .../storage/common/AppIdKeyConverter.java       |  3 +-
 .../common/HBaseTimelineStorageUtils.java       | 33 +++++++++++++++++
 .../TestCustomApplicationIdConversion.java      | 39 ++++++++++++++++++++
 5 files changed, 86 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/608a06cc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageApps.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageApps.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageApps.java
index b3e5197..3948d23 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageApps.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageApps.java
@@ -69,6 +69,7 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.application.Applica
 import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationTable;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.EventColumnName;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.EventColumnNameConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.StringKeyConverter;
@@ -493,7 +494,8 @@ public class TestHBaseTimelineStorageApps {
     event.addInfo(expKey, expVal);
 
     final TimelineEntity entity = new ApplicationEntity();
-    entity.setId(ApplicationId.newInstance(0, 1).toString());
+    entity.setId(HBaseTimelineStorageUtils.convertApplicationIdToString(
+        ApplicationId.newInstance(0, 1)));
     entity.addEvent(event);
 
     TimelineEntities entities = new TimelineEntities();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/608a06cc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageEntities.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageEntities.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageEntities.java
index 4b4c3e1..e18d0d0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageEntities.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageEntities.java
@@ -62,6 +62,7 @@ import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelinePrefi
 import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.EventColumnName;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.EventColumnNameConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.StringKeyConverter;
@@ -202,8 +203,9 @@ public class TestHBaseTimelineStorageEntities {
       String flow = "some_flow_name";
       String flowVersion = "AB7822C10F1111";
       long runid = 1002345678919L;
-      String appName = ApplicationId.newInstance(System.currentTimeMillis() +
-          9000000L, 1).toString();
+      String appName = HBaseTimelineStorageUtils.convertApplicationIdToString(
+          ApplicationId.newInstance(System.currentTimeMillis() + 9000000L, 1)
+      );
       hbi.write(cluster, user, flow, flowVersion, runid, appName, te);
       hbi.stop();
 
@@ -399,8 +401,8 @@ public class TestHBaseTimelineStorageEntities {
       String flow = "other_flow_name";
       String flowVersion = "1111F01C2287BA";
       long runid = 1009876543218L;
-      String appName = ApplicationId.newInstance(System.currentTimeMillis() +
-          9000000L, 1).toString();
+      String appName = HBaseTimelineStorageUtils.convertApplicationIdToString(
+          ApplicationId.newInstance(System.currentTimeMillis() + 9000000L, 1));
       byte[] startRow =
           new EntityRowKeyPrefix(cluster, user, flow, runid, appName)
               .getRowKeyPrefix();
@@ -487,7 +489,9 @@ public class TestHBaseTimelineStorageEntities {
     event.addInfo(expKey, expVal);
 
     final TimelineEntity entity = new ApplicationEntity();
-    entity.setId(ApplicationId.newInstance(0, 1).toString());
+    entity.setId(
+        HBaseTimelineStorageUtils.convertApplicationIdToString(
+            ApplicationId.newInstance(0, 1)));
     entity.addEvent(event);
 
     TimelineEntities entities = new TimelineEntities();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/608a06cc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/AppIdKeyConverter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/AppIdKeyConverter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/AppIdKeyConverter.java
index c165801..51604f0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/AppIdKeyConverter.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/AppIdKeyConverter.java
@@ -82,7 +82,8 @@ public final class AppIdKeyConverter implements KeyConverter<String> {
         Bytes.toLong(appIdBytes, 0, Bytes.SIZEOF_LONG));
     int seqId = HBaseTimelineStorageUtils.invertInt(
         Bytes.toInt(appIdBytes, Bytes.SIZEOF_LONG, Bytes.SIZEOF_INT));
-    return ApplicationId.newInstance(clusterTs, seqId).toString();
+    return HBaseTimelineStorageUtils.convertApplicationIdToString(
+        ApplicationId.newInstance(clusterTs, seqId));
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/608a06cc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/HBaseTimelineStorageUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/HBaseTimelineStorageUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/HBaseTimelineStorageUtils.java
index b6f1157..b8c7029 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/HBaseTimelineStorageUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/HBaseTimelineStorageUtils.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.AggregationCompactionDimension;
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.AggregationOperation;
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute;
@@ -32,6 +33,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.text.NumberFormat;
 import java.util.List;
 import java.util.Map;
 
@@ -240,4 +242,35 @@ public final class HBaseTimelineStorageUtils {
     long dayTimestamp = ts - (ts % MILLIS_ONE_DAY);
     return dayTimestamp;
   }
+
+  private static final ThreadLocal<NumberFormat> APP_ID_FORMAT =
+      new ThreadLocal<NumberFormat>() {
+        @Override
+        public NumberFormat initialValue() {
+          NumberFormat fmt = NumberFormat.getInstance();
+          fmt.setGroupingUsed(false);
+          fmt.setMinimumIntegerDigits(4);
+          return fmt;
+        }
+      };
+
+  /**
+   * A utility method that converts ApplicationId to string without using
+   * FastNumberFormat in order to avoid the incompatibility issue caused
+   * by mixing hadoop-common 2.5.1 and hadoop-yarn-api 3.0 in this module.
+   * This is a work-around implementation as discussed in YARN-6905.
+   *
+   * @param appId application id
+   * @return the string representation of the given application id
+   *
+   */
+  public static String convertApplicationIdToString(ApplicationId appId) {
+    StringBuilder sb = new StringBuilder(64);
+    sb.append(ApplicationId.appIdStrPrefix);
+    sb.append("_");
+    sb.append(appId.getClusterTimestamp());
+    sb.append('_');
+    sb.append(APP_ID_FORMAT.get().format(appId.getId()));
+    return sb.toString();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/608a06cc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestCustomApplicationIdConversion.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestCustomApplicationIdConversion.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestCustomApplicationIdConversion.java
new file mode 100644
index 0000000..73bc29e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestCustomApplicationIdConversion.java
@@ -0,0 +1,39 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.server.timelineservice.storage.common;
+
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test for HBaseTimelineStorageUtils.convertApplicationIdToString(),
+ * a custom conversion from ApplicationId to String that avoids the
+ * incompatibility issue caused by mixing hadoop-common 2.5.1 and
+ * hadoop-yarn-api 3.0. See YARN-6905.
+ */
+public class TestCustomApplicationIdConversion {
+  @Test
+  public void testConvertAplicationIdToString() {
+    ApplicationId applicationId = ApplicationId.newInstance(0, 1);
+    String applicationIdStr =
+        HBaseTimelineStorageUtils.convertApplicationIdToString(applicationId);
+    Assert.assertEquals(applicationId,
+        ApplicationId.fromString(applicationIdStr));
+  }
+}


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


[05/50] [abbrv] hadoop git commit: YARN-6882. AllocationFileLoaderService.reloadAllocations() should use the diamond operator (Contributed by Larry Lo via Daniel Templeton)

Posted by jh...@apache.org.
YARN-6882. AllocationFileLoaderService.reloadAllocations() should use the diamond operator
(Contributed by Larry Lo via Daniel Templeton)


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

Branch: refs/heads/YARN-5734
Commit: 0996acde6c325667aa19ae0740eb6b40bf4a682a
Parents: 65364de
Author: Daniel Templeton <te...@apache.org>
Authored: Fri Aug 11 14:50:46 2017 -0700
Committer: Daniel Templeton <te...@apache.org>
Committed: Fri Aug 11 14:50:46 2017 -0700

----------------------------------------------------------------------
 .../scheduler/fair/AllocationFileLoaderService.java              | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0996acde/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationFileLoaderService.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/AllocationFileLoaderService.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/AllocationFileLoaderService.java
index bf5b4c5..313a27a 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/AllocationFileLoaderService.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/AllocationFileLoaderService.java
@@ -266,7 +266,7 @@ public class AllocationFileLoaderService extends AbstractService {
     Map<FSQueueType, Set<String>> configuredQueues = new HashMap<>();
 
     for (FSQueueType queueType : FSQueueType.values()) {
-      configuredQueues.put(queueType, new HashSet<String>());
+      configuredQueues.put(queueType, new HashSet<>());
     }
 
     // Read and parse the allocations file.
@@ -280,7 +280,7 @@ public class AllocationFileLoaderService extends AbstractService {
       throw new AllocationConfigurationException("Bad fair scheduler config " +
           "file: top-level element not <allocations>");
     NodeList elements = root.getChildNodes();
-    List<Element> queueElements = new ArrayList<Element>();
+    List<Element> queueElements = new ArrayList<>();
     Element placementPolicyElement = null;
     for (int i = 0; i < elements.getLength(); i++) {
       Node node = elements.item(i);


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