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/09/28 02:54:52 UTC

[01/50] [abbrv] hadoop git commit: HDFS-12064. Reuse object mapper in HDFS. Contributed by Hanisha Koneru. [Forced Update!]

Repository: hadoop
Updated Branches:
  refs/heads/YARN-5734 b66896fa8 -> 5c775e61e (forced update)


HDFS-12064. Reuse object mapper in HDFS. 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/36fb90c9
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/36fb90c9
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/36fb90c9

Branch: refs/heads/YARN-5734
Commit: 36fb90c93982a22df7177809c3b0ddae455ebc07
Parents: cda3378
Author: Anu Engineer <ae...@apache.org>
Authored: Sat Sep 23 10:11:00 2017 -0700
Committer: Anu Engineer <ae...@apache.org>
Committed: Sat Sep 23 10:11:00 2017 -0700

----------------------------------------------------------------------
 .../server/blockmanagement/SlowDiskTracker.java     |  9 +++++++--
 .../server/blockmanagement/SlowPeerTracker.java     |  8 ++++++--
 .../hdfs/server/diskbalancer/planner/NodePlan.java  | 15 ++++++++-------
 .../server/blockmanagement/TestSlowDiskTracker.java |  6 ++++--
 .../server/blockmanagement/TestSlowPeerTracker.java | 10 ++++++----
 .../org/apache/hadoop/hdfs/web/TestJsonUtil.java    | 16 ++++++++--------
 6 files changed, 39 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/36fb90c9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SlowDiskTracker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SlowDiskTracker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SlowDiskTracker.java
index 52fce5d..051121e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SlowDiskTracker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SlowDiskTracker.java
@@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonIgnore;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.core.JsonProcessingException;
 import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
@@ -68,6 +69,11 @@ public class SlowDiskTracker {
   private final Timer timer;
 
   /**
+   * ObjectWriter to convert JSON reports to String.
+   */
+  private static final ObjectWriter WRITER = new ObjectMapper().writer();
+
+  /**
    * Number of disks to include in JSON report per operation. We will return
    * disks with the highest latency.
    */
@@ -254,12 +260,11 @@ public class SlowDiskTracker {
    *         serialization failed.
    */
   public String getSlowDiskReportAsJsonString() {
-    ObjectMapper objectMapper = new ObjectMapper();
     try {
       if (slowDisksReport.isEmpty()) {
         return null;
       }
-      return objectMapper.writeValueAsString(slowDisksReport);
+      return WRITER.writeValueAsString(slowDisksReport);
     } catch (JsonProcessingException e) {
       // Failed to serialize. Don't log the exception call stack.
       LOG.debug("Failed to serialize statistics" + e);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/36fb90c9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SlowPeerTracker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SlowPeerTracker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SlowPeerTracker.java
index 7b24370..22983ea 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SlowPeerTracker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SlowPeerTracker.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.core.JsonProcessingException;
 import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.primitives.Ints;
@@ -70,6 +71,10 @@ public class SlowPeerTracker {
   private final Timer timer;
 
   /**
+   * ObjectWriter to convert JSON reports to String.
+   */
+  private static final ObjectWriter WRITER = new ObjectMapper().writer();
+  /**
    * Number of nodes to include in JSON report. We will return nodes with
    * the highest number of votes from peers.
    */
@@ -188,9 +193,8 @@ public class SlowPeerTracker {
   public String getJson() {
     Collection<ReportForJson> validReports = getJsonReports(
         MAX_NODES_TO_REPORT);
-    ObjectMapper objectMapper = new ObjectMapper();
     try {
-      return objectMapper.writeValueAsString(validReports);
+      return WRITER.writeValueAsString(validReports);
     } catch (JsonProcessingException e) {
       // Failed to serialize. Don't log the exception call stack.
       LOG.debug("Failed to serialize statistics" + e);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/36fb90c9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/NodePlan.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/NodePlan.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/NodePlan.java
index c72bb1f..44039ea 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/NodePlan.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/NodePlan.java
@@ -18,8 +18,9 @@
 package org.apache.hadoop.hdfs.server.diskbalancer.planner;
 
 import com.fasterxml.jackson.annotation.JsonTypeInfo;
-import com.fasterxml.jackson.databind.JavaType;
 import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectReader;
+import com.fasterxml.jackson.databind.ObjectWriter;
 import com.google.common.base.Preconditions;
 
 import java.io.IOException;
@@ -38,6 +39,10 @@ public class NodePlan {
   private int port;
   private long timeStamp;
 
+  private static final ObjectMapper MAPPER = new ObjectMapper();
+  private static final ObjectReader READER = MAPPER.readerFor(NodePlan.class);
+  private static final ObjectWriter WRITER = MAPPER.writerFor(
+      MAPPER.constructType(NodePlan.class));
   /**
    * returns timestamp when this plan was created.
    *
@@ -153,8 +158,7 @@ public class NodePlan {
    * @throws IOException
    */
   public static NodePlan parseJson(String json) throws IOException {
-    ObjectMapper mapper = new ObjectMapper();
-    return mapper.readValue(json, NodePlan.class);
+    return READER.readValue(json);
   }
 
   /**
@@ -164,10 +168,7 @@ public class NodePlan {
    * @throws IOException
    */
   public String toJson() throws IOException {
-    ObjectMapper mapper = new ObjectMapper();
-    JavaType planType = mapper.constructType(NodePlan.class);
-    return mapper.writerFor(planType)
-        .writeValueAsString(this);
+    return WRITER.writeValueAsString(this);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/36fb90c9/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSlowDiskTracker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSlowDiskTracker.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSlowDiskTracker.java
index 172400d..78cf927 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSlowDiskTracker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSlowDiskTracker.java
@@ -25,6 +25,7 @@ import static org.junit.Assert.assertTrue;
 
 import com.fasterxml.jackson.core.type.TypeReference;
 import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectReader;
 import com.google.common.collect.ImmutableMap;
 import org.apache.hadoop.conf.Configuration;
 import static org.apache.hadoop.hdfs.DFSConfigKeys
@@ -76,6 +77,8 @@ public class TestSlowDiskTracker {
   private FakeTimer timer;
   private long reportValidityMs;
   private static final long OUTLIERS_REPORT_INTERVAL = 1000;
+  private static final ObjectReader READER = new ObjectMapper().readerFor(
+          new TypeReference<ArrayList<DiskLatency>>() {});
 
   static {
     conf = new HdfsConfiguration();
@@ -416,8 +419,7 @@ public class TestSlowDiskTracker {
 
   private ArrayList<DiskLatency> getAndDeserializeJson(
       final String json) throws IOException {
-    return (new ObjectMapper()).readValue(json,
-        new TypeReference<ArrayList<DiskLatency>>() {});
+    return READER.readValue(json);
   }
 
   private void addSlowDiskForTesting(String dnID, String disk,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/36fb90c9/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSlowPeerTracker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSlowPeerTracker.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSlowPeerTracker.java
index 15eb3a5..fb2928c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSlowPeerTracker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSlowPeerTracker.java
@@ -20,9 +20,11 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
 
 import com.fasterxml.jackson.core.type.TypeReference;
 import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectReader;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
-import org.apache.hadoop.hdfs.server.blockmanagement.SlowPeerTracker.ReportForJson;
+import org.apache.hadoop.hdfs.server.blockmanagement.SlowPeerTracker
+    .ReportForJson;
 import org.apache.hadoop.util.FakeTimer;
 import org.junit.Before;
 import org.junit.Rule;
@@ -39,7 +41,6 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 
-
 /**
  * Tests for {@link SlowPeerTracker}.
  */
@@ -57,6 +58,8 @@ public class TestSlowPeerTracker {
   private SlowPeerTracker tracker;
   private FakeTimer timer;
   private long reportValidityMs;
+  private static final ObjectReader READER =
+      new ObjectMapper().readerFor(new TypeReference<Set<ReportForJson>>() {});
 
   @Before
   public void setup() {
@@ -220,7 +223,6 @@ public class TestSlowPeerTracker {
       throws IOException {
     final String json = tracker.getJson();
     LOG.info("Got JSON: {}", json);
-    return (new ObjectMapper()).readValue(
-        json, new TypeReference<Set<ReportForJson>>() {});
+    return READER.readValue(json);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/36fb90c9/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java
index edb79d3..8c1a549 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java
@@ -52,6 +52,10 @@ import com.fasterxml.jackson.databind.ObjectReader;
 import com.google.common.collect.Lists;
 
 public class TestJsonUtil {
+
+  private static final ObjectReader READER =
+      new ObjectMapper().readerFor(Map.class);
+
   static FileStatus toFileStatus(HdfsFileStatus f, String parent) {
     return new FileStatus(f.getLen(), f.isDirectory(), f.getReplication(),
         f.getBlockSize(), f.getModificationTime(), f.getAccessTime(),
@@ -76,9 +80,8 @@ public class TestJsonUtil {
     System.out.println("fstatus = " + fstatus);
     final String json = JsonUtil.toJsonString(status, true);
     System.out.println("json    = " + json.replace(",", ",\n  "));
-    ObjectReader reader = new ObjectMapper().readerFor(Map.class);
     final HdfsFileStatus s2 =
-        JsonUtilClient.toFileStatus((Map<?, ?>) reader.readValue(json), true);
+        JsonUtilClient.toFileStatus((Map<?, ?>) READER.readValue(json), true);
     final FileStatus fs2 = toFileStatus(s2, parent);
     System.out.println("s2      = " + s2);
     System.out.println("fs2     = " + fs2);
@@ -164,8 +167,7 @@ public class TestJsonUtil {
   public void testToAclStatus() throws IOException {
     String jsonString =
         "{\"AclStatus\":{\"entries\":[\"user::rwx\",\"user:user1:rw-\",\"group::rw-\",\"other::r-x\"],\"group\":\"supergroup\",\"owner\":\"testuser\",\"stickyBit\":false}}";
-    ObjectReader reader = new ObjectMapper().readerFor(Map.class);
-    Map<?, ?> json = reader.readValue(jsonString);
+    Map<?, ?> json = READER.readValue(jsonString);
 
     List<AclEntry> aclSpec =
         Lists.newArrayList(aclEntry(ACCESS, USER, ALL),
@@ -224,8 +226,7 @@ public class TestJsonUtil {
     String jsonString = 
         "{\"XAttrs\":[{\"name\":\"user.a1\",\"value\":\"0x313233\"}," +
         "{\"name\":\"user.a2\",\"value\":\"0x313131\"}]}";
-    ObjectReader reader = new ObjectMapper().readerFor(Map.class);
-    Map<?, ?> json = reader.readValue(jsonString);
+    Map<?, ?> json = READER.readValue(jsonString);
     XAttr xAttr1 = (new XAttr.Builder()).setNameSpace(XAttr.NameSpace.USER).
         setName("a1").setValue(XAttrCodec.decodeValue("0x313233")).build();
     XAttr xAttr2 = (new XAttr.Builder()).setNameSpace(XAttr.NameSpace.USER).
@@ -250,8 +251,7 @@ public class TestJsonUtil {
     String jsonString = 
         "{\"XAttrs\":[{\"name\":\"user.a1\",\"value\":\"0x313233\"}," +
         "{\"name\":\"user.a2\",\"value\":\"0x313131\"}]}";
-    ObjectReader reader = new ObjectMapper().readerFor(Map.class);
-    Map<?, ?> json = reader.readValue(jsonString);
+    Map<?, ?> json = READER.readValue(jsonString);
 
     // Get xattr: user.a2
     byte[] value = JsonUtilClient.getXAttr(json, "user.a2");


---------------------------------------------------------------------
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: HDFS-12536. Add documentation for getconf command with -journalnodes option. Contributed by Bharat Viswanadham.

Posted by jh...@apache.org.
HDFS-12536. Add documentation for getconf command with -journalnodes option. Contributed by  Bharat Viswanadham.


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

Branch: refs/heads/YARN-5734
Commit: 2a81af447f97c56849bbc98dc3fe9ed59753b4cb
Parents: e9a18b6
Author: Anu Engineer <ae...@apache.org>
Authored: Sat Sep 23 10:53:59 2017 -0700
Committer: Anu Engineer <ae...@apache.org>
Committed: Sat Sep 23 10:53:59 2017 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2a81af44/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
index 5903a36..322609a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
@@ -116,6 +116,7 @@ Usage:
        hdfs getconf -namenodes
        hdfs getconf -secondaryNameNodes
        hdfs getconf -backupNodes
+       hdfs getconf -journalNodes
        hdfs getconf -includeFile
        hdfs getconf -excludeFile
        hdfs getconf -nnRpcAddresses
@@ -126,6 +127,7 @@ Usage:
 | `-namenodes` | gets list of namenodes in the cluster. |
 | `-secondaryNameNodes` | gets list of secondary namenodes in the cluster. |
 | `-backupNodes` | gets list of backup nodes in the cluster. |
+| `-journalNodes` | gets list of journal nodes in the cluster. |
 | `-includeFile` | gets the include file path that defines the datanodes that can join the cluster. |
 | `-excludeFile` | gets the exclude file path that defines the datanodes that need to decommissioned. |
 | `-nnRpcAddresses` | gets the namenode rpc addresses |


---------------------------------------------------------------------
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: HDFS-12304. Remove unused parameter from FsDatasetImpl#addVolume. Contributed by Chen Liang.

Posted by jh...@apache.org.
HDFS-12304. Remove unused parameter from FsDatasetImpl#addVolume. Contributed by Chen Liang.


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

Branch: refs/heads/YARN-5734
Commit: 02e2a9b1152b0e144fcf43bec2fce26d8a6c6dbc
Parents: 0807470
Author: Akira Ajisaka <aa...@apache.org>
Authored: Tue Sep 26 01:25:07 2017 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Tue Sep 26 01:25:07 2017 +0900

----------------------------------------------------------------------
 .../hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java      | 5 ++---
 1 file changed, 2 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/02e2a9b1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
index 16df709..41c41e6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
@@ -315,7 +315,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     deletingBlock = new HashMap<String, Set<Long>>();
 
     for (int idx = 0; idx < storage.getNumStorageDirs(); idx++) {
-      addVolume(dataLocations, storage.getStorageDir(idx));
+      addVolume(storage.getStorageDir(idx));
     }
     setupAsyncLazyPersistThreads();
 
@@ -413,8 +413,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     }
   }
 
-  private void addVolume(Collection<StorageLocation> dataLocations,
-      Storage.StorageDirectory sd) throws IOException {
+  private void addVolume(Storage.StorageDirectory sd) throws IOException {
     final StorageLocation storageLocation = sd.getStorageLocation();
 
     // If IOException raises from FsVolumeImpl() or getVolumeMap(), there is


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

Branch: refs/heads/YARN-5734
Commit: 794b77434bfc39a2702132e4c4ce3ad402507bcf
Parents: c85d2ad
Author: Wangda Tan <wa...@apache.org>
Authored: Mon Apr 3 10:12:01 2017 -0700
Committer: Jonathan Hung <jh...@linkedin.com>
Committed: Wed Sep 27 19:52:58 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/794b7743/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/794b7743/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/794b7743/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 11f2f6e..7c5839b 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;
@@ -150,7 +152,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);
 
@@ -2610,4 +2612,16 @@ public class CapacityScheduler extends
     // In seconds
     return ((LeafQueue) queue).getMaximumApplicationLifetime();
   }
+
+  @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/794b7743/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/794b7743/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/794b7743/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 6dc3d9a..215e511 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;
@@ -134,11 +135,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;
@@ -2454,4 +2458,170 @@ public class RMWebServices extends WebServices implements RMWebServiceProtocol {
       GetContainersRequest request) throws YarnException, IOException {
     return rm.getClientRMService().getContainers(request).getContainerList();
   }
+
+  @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/794b7743/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/794b7743/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/794b7743/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/794b7743/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


[25/50] [abbrv] hadoop git commit: Revert "YARN-6570. No logs were found for running application, running"

Posted by jh...@apache.org.
Revert "YARN-6570. No logs were found for running application, running"

This reverts commit 7c732924a889cd280e972882619a1827877fbafa.


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

Branch: refs/heads/YARN-5734
Commit: 14fec04ed335ae5f442abd3e35ec5fe05be62360
Parents: e9b790d
Author: Jason Lowe <jl...@apache.org>
Authored: Tue Sep 26 10:38:03 2017 -0500
Committer: Jason Lowe <jl...@apache.org>
Committed: Tue Sep 26 10:38:03 2017 -0500

----------------------------------------------------------------------
 .../nodemanager/containermanager/container/ContainerImpl.java     | 1 -
 .../org/apache/hadoop/yarn/server/nodemanager/TestEventFlow.java  | 3 +--
 .../nodemanager/containermanager/container/TestContainer.java     | 3 ---
 3 files changed, 1 insertion(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/14fec04e/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 705087b..2bbc267 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
@@ -709,7 +709,6 @@ public class ContainerImpl implements Container {
   public org.apache.hadoop.yarn.api.records.ContainerState getCurrentState() {
     switch (stateMachine.getCurrentState()) {
     case NEW:
-      return org.apache.hadoop.yarn.api.records.ContainerState.NEW;
     case LOCALIZING:
     case LOCALIZATION_FAILED:
     case SCHEDULED:

http://git-wip-us.apache.org/repos/asf/hadoop/blob/14fec04e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestEventFlow.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/TestEventFlow.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestEventFlow.java
index 9e59449..8e4522b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestEventFlow.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestEventFlow.java
@@ -159,8 +159,7 @@ public class TestEventFlow {
     containerManager.startContainers(allRequests);
 
     BaseContainerManagerTest.waitForContainerState(containerManager, cID,
-        Arrays.asList(ContainerState.RUNNING, ContainerState.SCHEDULED,
-            ContainerState.NEW), 20);
+        Arrays.asList(ContainerState.RUNNING, ContainerState.SCHEDULED), 20);
 
     List<ContainerId> containerIds = new ArrayList<ContainerId>();
     containerIds.add(cID);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/14fec04e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/TestContainer.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/container/TestContainer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/TestContainer.java
index b44b500..64e6cf0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/TestContainer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/TestContainer.java
@@ -122,8 +122,6 @@ public class TestContainer {
     try {
       wc = new WrappedContainer(7, 314159265358979L, 4344, "yak");
       assertEquals(ContainerState.NEW, wc.c.getContainerState());
-      ContainerImpl container = (ContainerImpl)wc.c;
-      assertEquals(org.apache.hadoop.yarn.api.records.ContainerState.NEW, container.getCurrentState());
       wc.initContainer();
 
       // Verify request for public/private resources to localizer
@@ -133,7 +131,6 @@ public class TestContainer {
               LocalResourceVisibility.APPLICATION));
       verify(wc.localizerBus).handle(argThat(matchesReq));
       assertEquals(ContainerState.LOCALIZING, wc.c.getContainerState());
-      assertEquals(org.apache.hadoop.yarn.api.records.ContainerState.SCHEDULED, container.getCurrentState());
     }
     finally {
       if (wc != null) {


---------------------------------------------------------------------
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: HADOOP-14881. LoadGenerator should use Time.monotonicNow() to measure durations. Contributed by Bharat Viswanadham

Posted by jh...@apache.org.
HADOOP-14881. LoadGenerator should use Time.monotonicNow() to measure durations. Contributed by Bharat Viswanadham


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

Branch: refs/heads/YARN-5734
Commit: ac05a51bbb2a3fad4e85f9334a3408571967900a
Parents: e928ee5
Author: Jason Lowe <jl...@apache.org>
Authored: Mon Sep 25 15:35:44 2017 -0500
Committer: Jason Lowe <jl...@apache.org>
Committed: Mon Sep 25 15:35:44 2017 -0500

----------------------------------------------------------------------
 .../hadoop/fs/loadGenerator/LoadGenerator.java  | 20 ++++++++++----------
 1 file changed, 10 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac05a51b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/loadGenerator/LoadGenerator.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/loadGenerator/LoadGenerator.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/loadGenerator/LoadGenerator.java
index 6da5182..0bb1b46 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/loadGenerator/LoadGenerator.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/loadGenerator/LoadGenerator.java
@@ -277,9 +277,9 @@ public class LoadGenerator extends Configured implements Tool {
      * the entire file */
     private void read() throws IOException {
       String fileName = files.get(r.nextInt(files.size()));
-      long startTime = Time.now();
+      long startTimestamp = Time.monotonicNow();
       InputStream in = fc.open(new Path(fileName));
-      executionTime[OPEN] += (Time.now()-startTime);
+      executionTime[OPEN] += (Time.monotonicNow() - startTimestamp);
       totalNumOfOps[OPEN]++;
       while (in.read(buffer) != -1) {}
       in.close();
@@ -299,9 +299,9 @@ public class LoadGenerator extends Configured implements Tool {
       double fileSize = 0;
       while ((fileSize = r.nextGaussian()+2)<=0) {}
       genFile(file, (long)(fileSize*BLOCK_SIZE));
-      long startTime = Time.now();
+      long startTimestamp = Time.monotonicNow();
       fc.delete(file, true);
-      executionTime[DELETE] += (Time.now()-startTime);
+      executionTime[DELETE] += (Time.monotonicNow() - startTimestamp);
       totalNumOfOps[DELETE]++;
     }
     
@@ -310,9 +310,9 @@ public class LoadGenerator extends Configured implements Tool {
      */
     private void list() throws IOException {
       String dirName = dirs.get(r.nextInt(dirs.size()));
-      long startTime = Time.now();
+      long startTimestamp = Time.monotonicNow();
       fc.listStatus(new Path(dirName));
-      executionTime[LIST] += (Time.now()-startTime);
+      executionTime[LIST] += (Time.monotonicNow() - startTimestamp);
       totalNumOfOps[LIST]++;
     }
 
@@ -320,14 +320,14 @@ public class LoadGenerator extends Configured implements Tool {
      * The file is filled with 'a'.
      */
     private void genFile(Path file, long fileSize) throws IOException {
-      long startTime = Time.now();
+      long startTimestamp = Time.monotonicNow();
       FSDataOutputStream out = null;
       try {
         out = fc.create(file,
             EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE),
             CreateOpts.createParent(), CreateOpts.bufferSize(4096),
             CreateOpts.repFac((short) 3));
-        executionTime[CREATE] += (Time.now() - startTime);
+        executionTime[CREATE] += (Time.monotonicNow() - startTimestamp);
         numOfOps[CREATE]++;
 
         long i = fileSize;
@@ -337,8 +337,8 @@ public class LoadGenerator extends Configured implements Tool {
           i -= s;
         }
 
-        startTime = Time.now();
-        executionTime[WRITE_CLOSE] += (Time.now() - startTime);
+        startTimestamp = Time.monotonicNow();
+        executionTime[WRITE_CLOSE] += (Time.monotonicNow() - startTimestamp);
         numOfOps[WRITE_CLOSE]++;
       } finally {
         IOUtils.cleanupWithLogger(LOG, out);


---------------------------------------------------------------------
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-6840. Implement zookeeper based store for scheduler configuration updates. (Jonathan Hung via wangda)

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/76746cb8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/TestZKConfigurationStore.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/TestZKConfigurationStore.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/TestZKConfigurationStore.java
new file mode 100644
index 0000000..3cfa8da
--- /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/TestZKConfigurationStore.java
@@ -0,0 +1,312 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, 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.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.ha.HAServiceProtocol;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.service.Service;
+import org.apache.hadoop.yarn.conf.HAUtil;
+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.recovery.ZKRMStateStore;
+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.capacity.CapacityScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
+import org.apache.hadoop.yarn.webapp.dao.SchedConfUpdateInfo;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+/**
+ * Tests {@link ZKConfigurationStore}.
+ */
+public class TestZKConfigurationStore extends ConfigurationStoreBaseTest {
+
+  public static final Log LOG =
+      LogFactory.getLog(TestZKConfigurationStore.class);
+
+  private static final int ZK_TIMEOUT_MS = 10000;
+  private TestingServer curatorTestingServer;
+  private CuratorFramework curatorFramework;
+  private ResourceManager rm;
+
+  public static TestingServer setupCuratorServer() throws Exception {
+    TestingServer curatorTestingServer = new TestingServer();
+    curatorTestingServer.start();
+    return curatorTestingServer;
+  }
+
+  public static CuratorFramework setupCuratorFramework(
+      TestingServer curatorTestingServer) throws Exception {
+    CuratorFramework curatorFramework = CuratorFrameworkFactory.builder()
+        .connectString(curatorTestingServer.getConnectString())
+        .retryPolicy(new RetryNTimes(100, 100))
+        .build();
+    curatorFramework.start();
+    return curatorFramework;
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    super.setUp();
+    curatorTestingServer = setupCuratorServer();
+    curatorFramework = setupCuratorFramework(curatorTestingServer);
+
+    conf.set(CommonConfigurationKeys.ZK_ADDRESS,
+        curatorTestingServer.getConnectString());
+    rm = new MockRM(conf);
+    rm.start();
+    rmContext = rm.getRMContext();
+  }
+
+  @After
+  public void cleanup() throws IOException {
+    rm.stop();
+    curatorFramework.close();
+    curatorTestingServer.stop();
+  }
+
+  @Test
+  public void testVersioning() throws Exception {
+    confStore.initialize(conf, schedConf, rmContext);
+    assertNull(confStore.getConfStoreVersion());
+    confStore.checkVersion();
+    assertEquals(ZKConfigurationStore.CURRENT_VERSION_INFO,
+        confStore.getConfStoreVersion());
+  }
+
+  @Test
+  public void testPersistConfiguration() throws Exception {
+    schedConf.set("key", "val");
+    confStore.initialize(conf, schedConf, rmContext);
+    assertEquals("val", confStore.retrieve().get("key"));
+
+    // Create a new configuration store, and check for old configuration
+    confStore = createConfStore();
+    schedConf.set("key", "badVal");
+    // Should ignore passed-in scheduler configuration.
+    confStore.initialize(conf, schedConf, rmContext);
+    assertEquals("val", confStore.retrieve().get("key"));
+  }
+
+
+  @Test
+  public void testPersistUpdatedConfiguration() throws Exception {
+    confStore.initialize(conf, schedConf, rmContext);
+    assertNull(confStore.retrieve().get("key"));
+
+    Map<String, String> update = new HashMap<>();
+    update.put("key", "val");
+    YarnConfigurationStore.LogMutation mutation =
+        new YarnConfigurationStore.LogMutation(update, TEST_USER);
+    confStore.logMutation(mutation);
+    confStore.confirmMutation(true);
+    assertEquals("val", confStore.retrieve().get("key"));
+
+    // Create a new configuration store, and check for updated configuration
+    confStore = createConfStore();
+    schedConf.set("key", "badVal");
+    // Should ignore passed-in scheduler configuration.
+    confStore.initialize(conf, schedConf, rmContext);
+    assertEquals("val", confStore.retrieve().get("key"));
+  }
+
+  @Test
+  public void testMaxLogs() throws Exception {
+    conf.setLong(YarnConfiguration.RM_SCHEDCONF_MAX_LOGS, 2);
+    confStore.initialize(conf, schedConf, rmContext);
+    LinkedList<YarnConfigurationStore.LogMutation> logs =
+        ((ZKConfigurationStore) confStore).getLogs();
+    assertEquals(0, logs.size());
+
+    Map<String, String> update1 = new HashMap<>();
+    update1.put("key1", "val1");
+    YarnConfigurationStore.LogMutation mutation =
+        new YarnConfigurationStore.LogMutation(update1, TEST_USER);
+    confStore.logMutation(mutation);
+    logs = ((ZKConfigurationStore) confStore).getLogs();
+    assertEquals(1, logs.size());
+    assertEquals("val1", logs.get(0).getUpdates().get("key1"));
+    confStore.confirmMutation(true);
+    assertEquals(1, logs.size());
+    assertEquals("val1", logs.get(0).getUpdates().get("key1"));
+
+    Map<String, String> update2 = new HashMap<>();
+    update2.put("key2", "val2");
+    mutation = new YarnConfigurationStore.LogMutation(update2, TEST_USER);
+    confStore.logMutation(mutation);
+    logs = ((ZKConfigurationStore) confStore).getLogs();
+    assertEquals(2, logs.size());
+    assertEquals("val1", logs.get(0).getUpdates().get("key1"));
+    assertEquals("val2", logs.get(1).getUpdates().get("key2"));
+    confStore.confirmMutation(true);
+    assertEquals(2, logs.size());
+    assertEquals("val1", logs.get(0).getUpdates().get("key1"));
+    assertEquals("val2", logs.get(1).getUpdates().get("key2"));
+
+    // Next update should purge first update from logs.
+    Map<String, String> update3 = new HashMap<>();
+    update3.put("key3", "val3");
+    mutation = new YarnConfigurationStore.LogMutation(update3, TEST_USER);
+    confStore.logMutation(mutation);
+    logs = ((ZKConfigurationStore) confStore).getLogs();
+    assertEquals(2, logs.size());
+    assertEquals("val2", logs.get(0).getUpdates().get("key2"));
+    assertEquals("val3", logs.get(1).getUpdates().get("key3"));
+    confStore.confirmMutation(true);
+    assertEquals(2, logs.size());
+    assertEquals("val2", logs.get(0).getUpdates().get("key2"));
+    assertEquals("val3", logs.get(1).getUpdates().get("key3"));
+  }
+
+  public Configuration createRMHAConf(String rmIds, String rmId,
+      int adminPort) {
+    Configuration conf = new YarnConfiguration();
+    conf.setBoolean(YarnConfiguration.RM_HA_ENABLED, true);
+    conf.set(YarnConfiguration.RM_HA_IDS, rmIds);
+    conf.setBoolean(YarnConfiguration.RECOVERY_ENABLED, true);
+    conf.set(CapacitySchedulerConfiguration.CS_CONF_PROVIDER,
+        CapacitySchedulerConfiguration.STORE_CS_CONF_PROVIDER);
+    conf.set(YarnConfiguration.SCHEDULER_CONFIGURATION_STORE_CLASS,
+        YarnConfiguration.ZK_CONFIGURATION_STORE);
+    conf.set(YarnConfiguration.RM_STORE, ZKRMStateStore.class.getName());
+    conf.set(YarnConfiguration.RM_ZK_ADDRESS,
+        curatorTestingServer.getConnectString());
+    conf.set(YarnConfiguration.RM_HA_ID, rmId);
+    conf.set(YarnConfiguration.RM_WEBAPP_ADDRESS, "localhost:0");
+    conf.setBoolean(YarnConfiguration.AUTO_FAILOVER_ENABLED, false);
+    for (String rpcAddress :
+        YarnConfiguration.getServiceAddressConfKeys(conf)) {
+      for (String id : HAUtil.getRMHAIds(conf)) {
+        conf.set(HAUtil.addSuffix(rpcAddress, id), "localhost:0");
+      }
+    }
+    conf.set(HAUtil.addSuffix(YarnConfiguration.RM_ADMIN_ADDRESS, rmId),
+        "localhost:" + adminPort);
+    return conf;
+  }
+
+  /**
+   * When failing over, new active RM should read from current state of store,
+   * including any updates when the new active RM was in standby.
+   * @throws Exception
+   */
+  @Test
+  public void testFailoverReadsFromUpdatedStore() throws Exception {
+    HAServiceProtocol.StateChangeRequestInfo req =
+        new HAServiceProtocol.StateChangeRequestInfo(
+        HAServiceProtocol.RequestSource.REQUEST_BY_USER);
+
+    Configuration conf1 = createRMHAConf("rm1,rm2", "rm1", 1234);
+    ResourceManager rm1 = new MockRM(conf1);
+    rm1.start();
+    rm1.getRMContext().getRMAdminService().transitionToActive(req);
+    assertEquals("RM with ZKStore didn't start",
+        Service.STATE.STARTED, rm1.getServiceState());
+    assertEquals("RM should be Active",
+        HAServiceProtocol.HAServiceState.ACTIVE,
+        rm1.getRMContext().getRMAdminService().getServiceStatus().getState());
+    assertNull(((MutableConfScheduler) rm1.getResourceScheduler())
+        .getConfiguration().get("key"));
+
+    Configuration conf2 = createRMHAConf("rm1,rm2", "rm2", 5678);
+    ResourceManager rm2 = new MockRM(conf2);
+    rm2.start();
+    assertEquals("RM should be Standby",
+        HAServiceProtocol.HAServiceState.STANDBY,
+        rm2.getRMContext().getRMAdminService().getServiceStatus().getState());
+
+    // Update configuration on RM1
+    SchedConfUpdateInfo schedConfUpdateInfo = new SchedConfUpdateInfo();
+    schedConfUpdateInfo.getGlobalParams().put("key", "val");
+    MutableConfigurationProvider confProvider = ((MutableConfScheduler)
+        rm1.getResourceScheduler()).getMutableConfProvider();
+    UserGroupInformation user = UserGroupInformation
+        .createUserForTesting(TEST_USER, new String[0]);
+    confProvider.logAndApplyMutation(user, schedConfUpdateInfo);
+    rm1.getResourceScheduler().reinitialize(conf1, rm1.getRMContext());
+    assertEquals("val", ((MutableConfScheduler) rm1.getResourceScheduler())
+        .getConfiguration().get("key"));
+    confProvider.confirmPendingMutation(true);
+    assertEquals("val", ((MutableCSConfigurationProvider) confProvider)
+        .getConfStore().retrieve().get("key"));
+    // Next update is not persisted, it should not be recovered
+    schedConfUpdateInfo.getGlobalParams().put("key", "badVal");
+    confProvider.logAndApplyMutation(user, schedConfUpdateInfo);
+
+    // Start RM2 and verifies it starts with updated configuration
+    rm2.getRMContext().getRMAdminService().transitionToActive(req);
+    assertEquals("RM with ZKStore didn't start",
+        Service.STATE.STARTED, rm2.getServiceState());
+    assertEquals("RM should be Active",
+        HAServiceProtocol.HAServiceState.ACTIVE,
+        rm2.getRMContext().getRMAdminService().getServiceStatus().getState());
+
+    for (int i = 0; i < ZK_TIMEOUT_MS / 50; i++) {
+      if (HAServiceProtocol.HAServiceState.ACTIVE ==
+          rm1.getRMContext().getRMAdminService().getServiceStatus()
+              .getState()) {
+        Thread.sleep(100);
+      }
+    }
+    assertEquals("RM should have been fenced",
+        HAServiceProtocol.HAServiceState.STANDBY,
+        rm1.getRMContext().getRMAdminService().getServiceStatus().getState());
+    assertEquals("RM should be Active",
+        HAServiceProtocol.HAServiceState.ACTIVE,
+        rm2.getRMContext().getRMAdminService().getServiceStatus().getState());
+
+    assertEquals("val", ((MutableCSConfigurationProvider) (
+        (CapacityScheduler) rm2.getResourceScheduler())
+        .getMutableConfProvider()).getConfStore().retrieve().get("key"));
+    assertEquals("val", ((MutableConfScheduler) rm2.getResourceScheduler())
+        .getConfiguration().get("key"));
+    // Transition to standby will set RM's HA status and then reinitialize in
+    // a separate thread. Despite asserting for STANDBY state, it's
+    // possible for reinitialization to be unfinished. Wait here for it to
+    // finish, otherwise closing rm1 will close zkManager and the unfinished
+    // reinitialization will throw an exception.
+    Thread.sleep(10000);
+    rm1.close();
+    rm2.close();
+  }
+
+  @Override
+  public YarnConfigurationStore createConfStore() {
+    return new ZKConfigurationStore();
+  }
+}


---------------------------------------------------------------------
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: HDFS-12375. Fail to start/stop journalnodes using start-dfs.sh/stop-dfs.sh. Contributed by Bharat Viswanadham.

Posted by jh...@apache.org.
HDFS-12375. Fail to start/stop journalnodes using start-dfs.sh/stop-dfs.sh. Contributed by  Bharat Viswanadham.


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

Branch: refs/heads/YARN-5734
Commit: 415e5a157a2c97835be02db0d155404273c7713d
Parents: ecc8562
Author: Anu Engineer <ae...@apache.org>
Authored: Sat Sep 23 15:05:25 2017 -0700
Committer: Anu Engineer <ae...@apache.org>
Committed: Sat Sep 23 15:05:25 2017 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/src/main/bin/start-dfs.sh       | 25 ++++++++----------
 .../hadoop-hdfs/src/main/bin/stop-dfs.sh        | 27 +++++++++-----------
 2 files changed, 23 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/415e5a15/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/start-dfs.sh
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/start-dfs.sh b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/start-dfs.sh
index 9b6a61d..12d5209 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/start-dfs.sh
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/start-dfs.sh
@@ -144,22 +144,19 @@ fi
 #---------------------------------------------------------
 # quorumjournal nodes (if any)
 
-SHARED_EDITS_DIR=$("${HADOOP_HDFS_HOME}/bin/hdfs" getconf -confKey dfs.namenode.shared.edits.dir 2>&-)
+JOURNAL_NODES=$("${HADOOP_HDFS_HOME}/bin/hdfs" getconf -journalNodes 2>&-)
 
-case "${SHARED_EDITS_DIR}" in
-  qjournal://*)
-    JOURNAL_NODES=$(echo "${SHARED_EDITS_DIR}" | sed 's,qjournal://\([^/]*\)/.*,\1,g; s/;/ /g; s/:[0-9]*//g')
-    echo "Starting journal nodes [${JOURNAL_NODES}]"
+if [[ "{$JOURNAL_NODES-}" != $'\n' ]]; then
+  echo "Starting journal nodes [${JOURNAL_NODES}]"
 
-    hadoop_uservar_su hdfs journalnode "${HADOOP_HDFS_HOME}/bin/hdfs" \
-      --workers \
-      --config "${HADOOP_CONF_DIR}" \
-      --hostnames "${JOURNAL_NODES}" \
-      --daemon start \
-      journalnode
-    (( HADOOP_JUMBO_RETCOUNTER=HADOOP_JUMBO_RETCOUNTER + $? ))
-  ;;
-esac
+  hadoop_uservar_su hdfs journalnode "${HADOOP_HDFS_HOME}/bin/hdfs" \
+    --workers \
+    --config "${HADOOP_CONF_DIR}" \
+    --hostnames "${JOURNAL_NODES}" \
+    --daemon start \
+    journalnode
+   (( HADOOP_JUMBO_RETCOUNTER=HADOOP_JUMBO_RETCOUNTER + $? ))
+fi
 
 #---------------------------------------------------------
 # ZK Failover controllers, if auto-HA is enabled

http://git-wip-us.apache.org/repos/asf/hadoop/blob/415e5a15/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/stop-dfs.sh
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/stop-dfs.sh b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/stop-dfs.sh
index 5d414c8..92974de 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/stop-dfs.sh
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/stop-dfs.sh
@@ -100,21 +100,18 @@ fi
 #---------------------------------------------------------
 # quorumjournal nodes (if any)
 
-SHARED_EDITS_DIR=$("${HADOOP_HDFS_HOME}/bin/hdfs" getconf -confKey dfs.namenode.shared.edits.dir 2>&-)
-
-case "${SHARED_EDITS_DIR}" in
-  qjournal://*)
-    JOURNAL_NODES=$(echo "${SHARED_EDITS_DIR}" | sed 's,qjournal://\([^/]*\)/.*,\1,g; s/;/ /g; s/:[0-9]*//g')
-    echo "Stopping journal nodes [${JOURNAL_NODES}]"
-
-    hadoop_uservar_su hdfs journalnode "${HADOOP_HDFS_HOME}/bin/hdfs" \
-      --workers \
-      --config "${HADOOP_CONF_DIR}" \
-      --hostnames "${JOURNAL_NODES}" \
-      --daemon stop \
-      journalnode
-  ;;
-esac
+JOURNAL_NODES=$("${HADOOP_HDFS_HOME}/bin/hdfs" getconf -journalNodes 2>&-)
+
+if [[ "{$JOURNAL_NODES-}" != $'\n' ]]; then
+  echo "Stopping journal nodes [${JOURNAL_NODES}]"
+
+  hadoop_uservar_su hdfs journalnode "${HADOOP_HDFS_HOME}/bin/hdfs" \
+    --workers \
+    --config "${HADOOP_CONF_DIR}" \
+    --hostnames "${JOURNAL_NODES}" \
+    --daemon stop \
+    journalnode
+fi
 
 #---------------------------------------------------------
 # ZK Failover controllers, if auto-HA is enabled


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


[35/50] [abbrv] hadoop git commit: YARN-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/44907bf1
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/44907bf1
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/44907bf1

Branch: refs/heads/YARN-5734
Commit: 44907bf11451d634006b0b67f130e716a23719f7
Parents: 44475c4
Author: Xuan <xg...@apache.org>
Authored: Tue Aug 1 08:48:04 2017 -0700
Committer: Jonathan Hung <jh...@linkedin.com>
Committed: Wed Sep 27 19:52:58 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/44907bf1/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/44907bf1/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/44907bf1/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/44907bf1/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 6d2de7e..51ee6a7 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
@@ -2620,8 +2620,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 {
@@ -2629,4 +2629,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/44907bf1/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/44907bf1/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 588f16d..620ca58 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
@@ -198,6 +198,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/44907bf1/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


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

Branch: refs/heads/YARN-5734
Commit: 44475c4fef3685189177a72bd9c1886e2a93c493
Parents: a210907
Author: Xuan <xg...@apache.org>
Authored: Mon Jul 31 16:48:40 2017 -0700
Committer: Jonathan Hung <jh...@linkedin.com>
Committed: Wed Sep 27 19:52:58 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/44475c4f/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 609f689..612e3d1 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
@@ -677,8 +677,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/44475c4f/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 5fcb27c..63c6f09 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
@@ -3390,4 +3390,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/44475c4f/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/44475c4f/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 da395b7..6d2de7e 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
@@ -393,6 +393,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/44475c4f/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/44475c4f/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/44475c4f/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/44475c4f/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


[02/50] [abbrv] hadoop git commit: HDFS-12320. Add quantiles for transactions batched in Journal sync. Contributed by Hanisha Koneru.

Posted by jh...@apache.org.
HDFS-12320. Add quantiles for transactions batched in Journal sync. 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/423ebcaf
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/423ebcaf
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/423ebcaf

Branch: refs/heads/YARN-5734
Commit: 423ebcaf51099268a1c219c42ce339f86d4da111
Parents: 36fb90c
Author: Anu Engineer <ae...@apache.org>
Authored: Sat Sep 23 10:34:13 2017 -0700
Committer: Anu Engineer <ae...@apache.org>
Committed: Sat Sep 23 10:34:13 2017 -0700

----------------------------------------------------------------------
 .../hadoop-common/src/site/markdown/Metrics.md    | 18 ++++++++++++++++++
 .../server/namenode/metrics/NameNodeMetrics.java  | 10 ++++++++++
 2 files changed, 28 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/423ebcaf/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md b/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md
index 367d9e0..98d4dfa 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md
@@ -268,6 +268,24 @@ The server-side metrics for a journal from the JournalNode's perspective. Each m
 | `Syncs3600s90thPercentileLatencyMicros` | The 90th percentile of sync latency in microseconds (1 hour granularity) |
 | `Syncs3600s95thPercentileLatencyMicros` | The 95th percentile of sync latency in microseconds (1 hour granularity) |
 | `Syncs3600s99thPercentileLatencyMicros` | The 99th percentile of sync latency in microseconds (1 hour granularity) |
+| `NumTransactionsBatchedInSync60sNumOps` | Number of times transactions were batched in sync operation (1 minute granularity) |
+| `NumTransactionsBatchedInSync60s50thPercentileLatencyMicros` | The 50th percentile of transactions batched in sync count (1 minute granularity) |
+| `NumTransactionsBatchedInSync60s75thPercentileLatencyMicros` | The 75th percentile of transactions batched in sync count (1 minute granularity) |
+| `NumTransactionsBatchedInSync60s90thPercentileLatencyMicros` | The 90th percentile of transactions batched in sync count (1 minute granularity) |
+| `NumTransactionsBatchedInSync60s95thPercentileLatencyMicros` | The 95th percentile of transactions batched in sync count (1 minute granularity) |
+| `NumTransactionsBatchedInSync60s99thPercentileLatencyMicros` | The 99th percentile of transactions batched in sync count (1 minute granularity) |
+| `NumTransactionsBatchedInSync300sNumOps` | Number of times transactions were batched in sync operation (5 minutes granularity) |
+| `NumTransactionsBatchedInSync300s50thPercentileLatencyMicros` | The 50th percentile of transactions batched in sync count (5 minutes granularity) |
+| `NumTransactionsBatchedInSync300s75thPercentileLatencyMicros` | The 75th percentile of transactions batched in sync count (5 minutes granularity) |
+| `NumTransactionsBatchedInSync300s90thPercentileLatencyMicros` | The 90th percentile of transactions batched in sync count (5 minutes granularity) |
+| `NumTransactionsBatchedInSync300s95thPercentileLatencyMicros` | The 95th percentile of transactions batched in sync count (5 minutes granularity) |
+| `NumTransactionsBatchedInSync300s99thPercentileLatencyMicros` | The 99th percentile of transactions batched in sync count (5 minutes granularity) |
+| `NumTransactionsBatchedInSync3600sNumOps` | Number of times transactions were batched in sync operation (1 hour granularity) |
+| `NumTransactionsBatchedInSync3600s50thPercentileLatencyMicros` | The 50th percentile of transactions batched in sync count (1 hour granularity) |
+| `NumTransactionsBatchedInSync3600s75thPercentileLatencyMicros` | The 75th percentile of transactions batched in sync count (1 hour granularity) |
+| `NumTransactionsBatchedInSync3600s90thPercentileLatencyMicros` | The 90th percentile of transactions batched in sync count (1 hour granularity) |
+| `NumTransactionsBatchedInSync3600s95thPercentileLatencyMicros` | The 95th percentile of transactions batched in sync count (1 hour granularity) |
+| `NumTransactionsBatchedInSync3600s99thPercentileLatencyMicros` | The 99th percentile of transactions batched in sync count (1 hour granularity) |
 | `BatchesWritten` | Total number of batches written since startup |
 | `TxnsWritten` | Total number of transactions written since startup |
 | `BytesWritten` | Total number of bytes written since startup |

http://git-wip-us.apache.org/repos/asf/hadoop/blob/423ebcaf/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/NameNodeMetrics.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/NameNodeMetrics.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/NameNodeMetrics.java
index f2534e4..94c5e9e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/NameNodeMetrics.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/NameNodeMetrics.java
@@ -115,6 +115,8 @@ public class NameNodeMetrics {
   final MutableQuantiles[] syncsQuantiles;
   @Metric("Journal transactions batched in sync")
   MutableCounterLong transactionsBatchedInSync;
+  @Metric("Journal transactions batched in sync")
+  final MutableQuantiles[] numTransactionsBatchedInSync;
   @Metric("Number of blockReports from individual storages")
   MutableRate storageBlockReport;
   final MutableQuantiles[] storageBlockReportQuantiles;
@@ -148,6 +150,7 @@ public class NameNodeMetrics {
     
     final int len = intervals.length;
     syncsQuantiles = new MutableQuantiles[len];
+    numTransactionsBatchedInSync = new MutableQuantiles[len];
     storageBlockReportQuantiles = new MutableQuantiles[len];
     cacheReportQuantiles = new MutableQuantiles[len];
     generateEDEKTimeQuantiles = new MutableQuantiles[len];
@@ -159,6 +162,10 @@ public class NameNodeMetrics {
       syncsQuantiles[i] = registry.newQuantiles(
           "syncs" + interval + "s",
           "Journal syncs", "ops", "latency", interval);
+      numTransactionsBatchedInSync[i] = registry.newQuantiles(
+          "numTransactionsBatchedInSync" + interval + "s",
+          "Number of Transactions batched in sync", "ops",
+          "count", interval);
       storageBlockReportQuantiles[i] = registry.newQuantiles(
           "storageBlockReport" + interval + "s",
           "Storage block report", "ops", "latency", interval);
@@ -304,6 +311,9 @@ public class NameNodeMetrics {
 
   public void incrTransactionsBatchedInSync(long count) {
     transactionsBatchedInSync.incr(count);
+    for (MutableQuantiles q : numTransactionsBatchedInSync) {
+      q.add(count);
+    }
   }
 
   public void incSuccessfulReReplications() {


---------------------------------------------------------------------
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: HADOOP-14220 Enhance S3GuardTool with bucket-info and set-capacity commands, tests. Contributed by Steve Loughran

Posted by jh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/47011d7d/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolLocal.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolLocal.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolLocal.java
index 181cdfb..43cbe93 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolLocal.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolLocal.java
@@ -21,11 +21,12 @@ package org.apache.hadoop.fs.s3a.s3guard;
 import java.io.BufferedReader;
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InputStreamReader;
-import java.io.PrintStream;
 import java.util.HashSet;
 import java.util.Set;
+import java.util.concurrent.Callable;
 
 import org.junit.Test;
 
@@ -34,13 +35,16 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.s3a.S3AFileSystem;
 import org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.Diff;
 
-import static org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.SUCCESS;
+import static org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.*;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 
 /**
  * Test S3Guard related CLI commands against a LocalMetadataStore.
  */
 public class ITestS3GuardToolLocal extends AbstractS3GuardToolTestBase {
 
+  private static final String LOCAL_METADATA = "local://metadata";
+
   @Override
   protected MetadataStore newMetadataStore() {
     return new LocalMetadataStore();
@@ -65,10 +69,7 @@ public class ITestS3GuardToolLocal extends AbstractS3GuardToolTestBase {
 
     S3GuardTool.Import cmd = new S3GuardTool.Import(fs.getConf());
     cmd.setStore(ms);
-
-    expectSuccess("Import command did not exit successfully - see output",
-        cmd,
-        "import", parent.toString());
+    exec(cmd, "import", parent.toString());
 
     DirListingMetadata children =
         ms.listChildren(dir);
@@ -80,7 +81,7 @@ public class ITestS3GuardToolLocal extends AbstractS3GuardToolTestBase {
   }
 
   @Test
-  public void testDiffCommand() throws IOException {
+  public void testDiffCommand() throws Exception {
     S3AFileSystem fs = getFileSystem();
     MetadataStore ms = getMetadataStore();
     Set<Path> filesOnS3 = new HashSet<>(); // files on S3.
@@ -108,13 +109,10 @@ public class ITestS3GuardToolLocal extends AbstractS3GuardToolTestBase {
     }
 
     ByteArrayOutputStream buf = new ByteArrayOutputStream();
-    PrintStream out = new PrintStream(buf);
     Diff cmd = new Diff(fs.getConf());
     cmd.setStore(ms);
-    assertEquals("Diff command did not exit successfully - see output", SUCCESS,
-        cmd.run(new String[]{"diff", "-meta", "local://metadata",
-            testPath.toString()}, out));
-    out.close();
+    exec(cmd, buf, "diff", "-meta", LOCAL_METADATA,
+            testPath.toString());
 
     Set<Path> actualOnS3 = new HashSet<>();
     Set<Path> actualOnMS = new HashSet<>();
@@ -140,10 +138,128 @@ public class ITestS3GuardToolLocal extends AbstractS3GuardToolTestBase {
         }
       }
     }
-    String actualOut = out.toString();
+    String actualOut = buf.toString();
     assertEquals("Mismatched metadata store outputs: " + actualOut,
         filesOnMS, actualOnMS);
     assertEquals("Mismatched s3 outputs: " + actualOut, filesOnS3, actualOnS3);
     assertFalse("Diff contained duplicates", duplicates);
   }
+
+  @Test
+  public void testDestroyBucketExistsButNoTable() throws Throwable {
+    run(Destroy.NAME,
+        "-meta", LOCAL_METADATA,
+        getLandsatCSVFile());
+  }
+
+  @Test
+  public void testImportNoFilesystem() throws Throwable {
+    final Import importer =
+        new S3GuardTool.Import(getConfiguration());
+    importer.setStore(getMetadataStore());
+    intercept(IOException.class,
+        new Callable<Integer>() {
+          @Override
+          public Integer call() throws Exception {
+            return importer.run(
+                new String[]{
+                    "import",
+                    "-meta", LOCAL_METADATA,
+                    S3A_THIS_BUCKET_DOES_NOT_EXIST
+                });
+          }
+        });
+  }
+
+  @Test
+  public void testInfoBucketAndRegionNoFS() throws Throwable {
+    intercept(FileNotFoundException.class,
+        new Callable<Integer>() {
+          @Override
+          public Integer call() throws Exception {
+            return run(BucketInfo.NAME, "-meta",
+                LOCAL_METADATA, "-region",
+                "any-region", S3A_THIS_BUCKET_DOES_NOT_EXIST);
+          }
+        });
+  }
+
+  @Test
+  public void testInitNegativeRead() throws Throwable {
+    runToFailure(INVALID_ARGUMENT,
+        Init.NAME, "-meta", LOCAL_METADATA, "-region",
+        "eu-west-1",
+        READ_FLAG, "-10");
+  }
+
+  @Test
+  public void testInit() throws Throwable {
+    run(Init.NAME,
+        "-meta", LOCAL_METADATA,
+        "-region", "us-west-1");
+  }
+
+  @Test
+  public void testInitTwice() throws Throwable {
+    run(Init.NAME,
+        "-meta", LOCAL_METADATA,
+        "-region", "us-west-1");
+    run(Init.NAME,
+        "-meta", LOCAL_METADATA,
+        "-region", "us-west-1");
+  }
+
+  @Test
+  public void testLandsatBucketUnguarded() throws Throwable {
+    run(BucketInfo.NAME,
+        "-" + BucketInfo.UNGUARDED_FLAG,
+        getLandsatCSVFile());
+  }
+
+  @Test
+  public void testLandsatBucketRequireGuarded() throws Throwable {
+    runToFailure(E_BAD_STATE,
+        BucketInfo.NAME,
+        "-" + BucketInfo.GUARDED_FLAG,
+        ITestS3GuardToolLocal.this.getLandsatCSVFile());
+  }
+
+  @Test
+  public void testLandsatBucketRequireUnencrypted() throws Throwable {
+    run(BucketInfo.NAME,
+        "-" + BucketInfo.ENCRYPTION_FLAG, "none",
+        getLandsatCSVFile());
+  }
+
+  @Test
+  public void testLandsatBucketRequireEncrypted() throws Throwable {
+    runToFailure(E_BAD_STATE,
+        BucketInfo.NAME,
+        "-" + BucketInfo.ENCRYPTION_FLAG,
+        "AES256", ITestS3GuardToolLocal.this.getLandsatCSVFile());
+  }
+
+  @Test
+  public void testStoreInfo() throws Throwable {
+    S3GuardTool.BucketInfo cmd = new S3GuardTool.BucketInfo(
+        getFileSystem().getConf());
+    cmd.setStore(getMetadataStore());
+    String output = exec(cmd, cmd.getName(),
+        "-" + S3GuardTool.BucketInfo.GUARDED_FLAG,
+        getFileSystem().getUri().toString());
+    LOG.info("Exec output=\n{}", output);
+  }
+
+  @Test
+  public void testSetCapacity() throws Throwable {
+    S3GuardTool cmd = new S3GuardTool.SetCapacity(getFileSystem().getConf());
+    cmd.setStore(getMetadataStore());
+    String output = exec(cmd, cmd.getName(),
+        "-" + READ_FLAG, "100",
+        "-" + WRITE_FLAG, "100",
+        getFileSystem().getUri().toString());
+    LOG.info("Exec output=\n{}", output);
+  }
+
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/47011d7d/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestS3GuardCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestS3GuardCLI.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestS3GuardCLI.java
new file mode 100644
index 0000000..43256b9
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestS3GuardCLI.java
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a.s3guard;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.s3a.S3ATestConstants;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.apache.hadoop.util.ExitUtil;
+
+import static org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.*;
+
+/**
+ * Test the S3Guard CLI entry point.
+ */
+public class TestS3GuardCLI extends Assert {
+
+  /**
+   * Run a S3GuardTool command from a varags list.
+   * @param args argument list
+   * @return the return code
+   * @throws Exception any exception
+   */
+  protected int run(String... args)
+      throws Exception {
+    Configuration conf = new Configuration(false);
+    return S3GuardTool.run(conf, args);
+  }
+
+  /**
+   * Run a S3GuardTool command from a varags list, catch any raised
+   * ExitException and verify the status code matches that expected.
+   * @param status expected status code of an exception
+   * @param args argument list
+   * @throws Exception any exception
+   */
+  protected void runToFailure(int status, String... args)
+      throws Exception {
+    ExitUtil.ExitException ex =
+        LambdaTestUtils.intercept(ExitUtil.ExitException.class,
+            () -> run(args));
+    if (ex.status != status) {
+      throw ex;
+    }
+  }
+
+  @Test
+  public void testInfoNoArgs() throws Throwable {
+    runToFailure(INVALID_ARGUMENT, BucketInfo.NAME);
+  }
+
+  @Test
+  public void testInfoWrongFilesystem() throws Throwable {
+    runToFailure(INVALID_ARGUMENT,
+        BucketInfo.NAME, "file://");
+  }
+
+  @Test
+  public void testNoCommand() throws Throwable {
+    runToFailure(E_USAGE);
+  }
+
+  @Test
+  public void testUnknownCommand() throws Throwable {
+    runToFailure(E_USAGE, "unknown");
+  }
+
+  @Test
+  public void testPruneNoArgs() throws Throwable {
+    runToFailure(INVALID_ARGUMENT, Prune.NAME);
+  }
+
+  @Test
+  public void testDiffNoArgs() throws Throwable {
+    runToFailure(INVALID_ARGUMENT, Diff.NAME);
+  }
+
+  @Test
+  public void testImportNoArgs() throws Throwable {
+    runToFailure(INVALID_ARGUMENT, Import.NAME);
+  }
+
+  @Test
+  public void testDestroyNoArgs() throws Throwable {
+    runToFailure(INVALID_ARGUMENT, Destroy.NAME);
+  }
+
+  @Test
+  public void testDestroyUnknownTableNoRegion() throws Throwable {
+    runToFailure(INVALID_ARGUMENT, Destroy.NAME,
+        "-meta", "dynamodb://ireland-team");
+  }
+
+  @Test
+  public void testInitBucketAndRegion() throws Throwable {
+    runToFailure(INVALID_ARGUMENT, Init.NAME,
+        "-meta", "dynamodb://ireland-team",
+        "-region", "eu-west-1",
+        S3ATestConstants.DEFAULT_CSVTEST_FILE
+    );
+  }
+
+}


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

Branch: refs/heads/YARN-5734
Commit: e7b476241cbd4a7adb8f1442521f1eb7ef3a3822
Parents: 28c4957
Author: Jonathan Hung <jh...@linkedin.com>
Authored: Mon Jan 30 19:03:48 2017 -0800
Committer: Jonathan Hung <jh...@linkedin.com>
Committed: Wed Sep 27 19:52:58 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/e7b47624/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 7f50272..e74cbe1 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;
@@ -167,6 +168,8 @@ public class CapacityScheduler extends
 
   private int maxAssignPerHeartbeat;
 
+  private CSConfigurationProvider csConfProvider;
+
   @Override
   public void setConf(Configuration conf) {
       yarnConf = conf;
@@ -289,7 +292,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 = super.getMinimumAllocation();
       initMaximumResourceCapability(super.getMaximumAllocation());
@@ -399,7 +413,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...");
@@ -1831,23 +1845,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/e7b47624/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 3a519ec..3821e24 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
@@ -315,6 +315,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();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e7b47624/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/e7b47624/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/e7b47624/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/e7b47624/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 8440519..1dea4ee 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
@@ -241,13 +241,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


[48/50] [abbrv] hadoop git commit: YARN-7238. Documentation for API based scheduler configuration management

Posted by jh...@apache.org.
YARN-7238. Documentation for API based scheduler configuration management


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

Branch: refs/heads/YARN-5734
Commit: 48202a5c6a606e5392a8af7a86df67ed64d2dff8
Parents: c7948fe
Author: Jonathan Hung <jh...@linkedin.com>
Authored: Mon Sep 25 17:47:32 2017 -0700
Committer: Jonathan Hung <jh...@linkedin.com>
Committed: Wed Sep 27 19:52:59 2017 -0700

----------------------------------------------------------------------
 .../src/site/markdown/CapacityScheduler.md      |  24 ++-
 .../src/site/markdown/ResourceManagerRest.md    | 186 +++++++++++++++++++
 .../src/site/markdown/YarnCommands.md           |  13 ++
 3 files changed, 221 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/48202a5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/CapacityScheduler.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/CapacityScheduler.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/CapacityScheduler.md
index 6bb8489..d70f891 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/CapacityScheduler.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/CapacityScheduler.md
@@ -295,9 +295,29 @@ The `ReservationSystem` is integrated with the `CapacityScheduler` queue hierach
 Changing Queue Configuration
 ----------------------------
 
-Changing queue properties and adding new queues is very simple. You need to edit **conf/capacity-scheduler.xml** and run *yarn rmadmin -refreshQueues*.
+Changing queue/scheduler properties and adding/removing queues can be done in two ways, via file or via API.
+
+### Changing queue configuration via file
+
+  To edit by file, you need to edit **conf/capacity-scheduler.xml** and run *yarn rmadmin -refreshQueues*.
 
     $ vi $HADOOP_CONF_DIR/capacity-scheduler.xml
     $ $HADOOP_YARN_HOME/bin/yarn rmadmin -refreshQueues
 
-**Note:** Queues cannot be *deleted*, only addition of new queues is supported - the updated queue configuration should be a valid one i.e. queue-capacity at each *level* should be equal to 100%.
+### Changing queue configuration via API
+
+  Editing by API uses a backing store for the scheduler configuration. To enable this, the following parameters can be configured in yarn-site.xml.
+
+  | Property | Description |
+  |:---- |:---- |
+  | `yarn.scheduler.capacity.configuration.provider` | The type of configuration provider to use for capacity scheduler. To enable changing queue configuration via API, this should be set to *store*. Default value is *file*, which disables the API and reverts back to changing queue configuration via file. |
+  | `yarn.scheduler.configuration.store.class` | The type of backing store to use. Default value is *memory*, which stores the scheduler configuration in memory (and does not persist configuration changes across restarts). Other values are *leveldb* (using a leveldb-based implementation), and *zk* (using a zookeeper-based implementation). |
+  | `yarn.scheduler.configuration.mutation.acl-policy.class` | An ACL policy can be configured to restrict which users can modify which queues. Default value is *org.apache.hadoop.yarn.server.resourcemanager.scheduler.DefaultConfigurationMutationACLPolicy*, which only allows YARN admins to make any configuration modifications. Another value is *org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.conf.QueueAdminConfigurationMutationACLPolicy*, which only allows queue modifications if the caller is an admin of the queue. |
+  | `yarn.scheduler.configuration.store.max-logs` | Configuration changes are audit logged in the backing store, if using leveldb or zookeeper. This configuration controls the maximum number of audit logs to store, dropping the oldest logs when exceeded. Default is 1000. |
+  | `yarn.scheduler.configuration.leveldb-store.path` | The storage path of the configuration store when using leveldb. Default value is *${hadoop.tmp.dir}/yarn/system/confstore*. |
+  | `yarn.scheduler.configuration.leveldb-store.compaction-interval-secs` | The interval for compacting the configuration store in seconds, when using leveldb. Default value is 86400, or one day. |
+  | `yarn.scheduler.configuration.zk-store.parent-path` | The zookeeper root node path for configuration store related information, when using zookeeper. Default value is */confstore*. |
+
+  **Note:** When enabling backing store for scheduler configuration, *yarn rmadmin -refreshQueues* will be disabled, i.e. it will no longer be possible to update configuration via file.
+
+  See the [YARN Resource Manager REST API](ResourceManagerRest.html#Scheduler_Configuration_Mutation_API) for examples on how to change scheduler configuration via REST, and [YARN Commands Reference](YarnCommands.html#schedconf) for examples on how to change scheduler configuration via command line.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/48202a5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md
index 82f80f3..025e9a7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md
@@ -4430,3 +4430,189 @@ Response Body:
     <remainingTimeInSeconds>90</remainingTimeInSeconds>
 </timeout>
 ```
+
+Scheduler Configuration Mutation API
+--------------------------------
+
+The scheduler configuration mutation API provides a way to modify scheduler/queue configuration and queue hierarchy.
+
+
+### URI
+
+      * http://rm-http-address:port/ws/v1/cluster/sched-conf
+
+### HTTP Operations Supported
+
+      * PUT
+
+### Elements of the *schedConf* object
+
+| Item | Data Type | Description |
+|:---- |:---- |:---- |
+| update-queue | object | A queue whose configurations should be updated |
+| add-queue | object | A queue to add to the scheduler along with this queue's configurations |
+| remove-queue | string | Full path name of a queue to remove |
+| global-updates | map | Map of key value pairs to update scheduler's global configuration |
+
+### PUT Request Examples
+
+Put requests are used to modify the scheduler configuration. A successful mutation results in a 200 response. A malformed request or one which resulted in an invalid scheduler configuration results in a 400 response.
+
+**Updating queue configuration(s)**
+
+Request for updating queue configurations.
+
+*Elements of the* update-queue *object*
+
+| Item | Data Type | Description |
+|:---- |:---- |:---- |
+| queueName | string | Full path name of the queue to update |
+| params | map | A map of key value configuration pairs to update for this queue |
+
+Assuming we are using the capacity scheduler and the current queue configuration is a single queue *root.default*, this example sets *root.default*'s maximum applications to 100 and its minimum user limit percent to 10.
+
+HTTP Request:
+
+```xml
+      Accept: application/xml
+      PUT http://rm-http-address:port/ws/v1/cluster/sched-conf
+      Content-Type: application/xml
+      <?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+      <schedConf>
+        <update-queue>
+          <queueName>root.default</queueName>
+          <params>
+            <entry>
+              <key>maximum-applications</key>
+              <value>100</value>
+            </entry>
+            <entry>
+              <key>minimum-user-limit-percent</key>
+              <value>10</value>
+            </entry>
+          </params>
+        </update-queue>
+      </schedConf>
+```
+
+
+Response Header:
+
+      HTTP/1.1 200 OK
+      Content-Type: application/xml
+      Transfer-Encoding: chunked
+
+
+**Adding a queue**
+
+Request for adding queues/updating queue configurations.
+
+*Elements of the* add-queue *object*
+
+| Item | Data Type | Description |
+|:---- |:---- |:---- |
+| queueName | string | Full path name of the queue to add |
+| params | map | A map of key value configuration pairs to set for this queue |
+
+Assuming we are using the capacity scheduler and the current queue configuration is a single queue *root.default*, this example adds a queue *root.a* with capacity/maximum-capacity 10, and adjusts *root.default*'s capacity/maximum-capacity to 90. (More complex examples include adding a queue whose parent is also being added in the same request, or adding multiple sibling queues.)
+
+HTTP Request:
+
+```xml
+      Accept: application/xml
+      PUT http://rm-http-address:port/ws/v1/cluster/sched-conf
+      Content-Type: application/xml
+      <?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+      <schedConf>
+        <add-queue>
+          <queueName>root.a</queueName>
+          <params>
+            <entry>
+              <key>capacity</key>
+              <value>10</value>
+            </entry>
+            <entry>
+              <key>maximum-capacity</key>
+              <value>10</value>
+            </entry>
+          </params>
+        </add-queue>
+        <update-queue>
+          <queueName>root.default</queueName>
+          <params>
+            <entry>
+              <key>capacity</key>
+              <value>90</value>
+            </entry>
+            <entry>
+              <key>maximum-capacity</key>
+              <value>90</value>
+            </entry>
+          </params>
+        </update-queue>
+      </schedConf>
+```
+
+
+Response Header:
+
+      HTTP/1.1 200 OK
+      Content-Type: application/xml
+      Transfer-Encoding: chunked
+
+**Removing queues**
+
+Request for removing queues from the queue hierarchy.
+
+Assuming we are using the capacity scheduler and the current queue configuration is three queues *root.default*, *root.a*, and *root.b*, this example removes both *root.a* and *root.b*. (More complex examples include removing a parent queue and its children.)
+
+**Note:** Queues must be put into `STOPPED` state before they are deleted. Any updated queue configuration should be a valid one i.e. queue-capacity at each *level* should be equal to 100%.
+
+
+HTTP Request:
+
+```xml
+      Accept: application/xml
+      PUT http://rm-http-address:port/ws/v1/cluster/sched-conf
+      Content-Type: application/xml
+      <?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+      <schedConf>
+        <remove-queue>root.a</remove-queue>
+        <remove-queue>root.b</remove-queue>
+      </schedConf>
+```
+
+
+Response Header:
+
+      HTTP/1.1 200 OK
+      Content-Type: application/xml
+      Transfer-Encoding: chunked
+
+**Updating global scheduler configurations**
+
+Request for updating global scheduler configurations. Assuming we are using the capacity scheduler, this example enables queue mappings. For global configuration updates, the full configuration key must be specified.
+
+HTTP Request:
+
+```xml
+      Accept: application/xml
+      PUT http://rm-http-address:port/ws/v1/cluster/sched-conf
+      Content-Type: application/xml
+      <?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+      <schedConf>
+        <global-updates>
+          <entry>
+            <key>yarn.scheduler.capacity.queue-mappings-override.enable</key>
+            <value>true</value>
+          </entry>
+        </global-updates>
+      </schedConf>
+```
+
+
+Response Header:
+
+      HTTP/1.1 200 OK
+      Content-Type: application/xml
+      Transfer-Encoding: chunked

http://git-wip-us.apache.org/repos/asf/hadoop/blob/48202a5c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/YarnCommands.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/YarnCommands.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/YarnCommands.md
index 5f430ec..2fc65bc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/YarnCommands.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/YarnCommands.md
@@ -237,6 +237,19 @@ Usage:
 
 Runs ResourceManager admin client
 
+### schedconf
+
+Usage: `yarn schedconf [options]`
+
+| COMMAND\_OPTIONS | Description |
+|:---- |:---- |
+| -add <queuePath1,key1=val1,key2=val2;queuePath2,key3=val3> | Semicolon separated values of queues to add and their queue configurations. This example adds queue "queuePath1" (a full path name), which has queue configurations key1=val1 and key2=val2. It also adds queue "queuePath2", which has queue configuration key3=val3. |
+| -remove <queuePath1,queuePath2> | Comma-separated queues to remove. This example removes queuePath1 and queuePath2 queues (full path names). **Note:** Queues must be put into `STOPPED` state before they are deleted. |
+| -update <queuePath1,key1=val1,key2=val2;queuePath2,key3=val3> | Semicolon separated values of queues whose configurations should be updated. This example sets key1=val1 and key2=val2 for queue configuration of queuePath1 (full path name), and sets key3=val3 for queue configuration of queuePath2. |
+| -global <key1=val1,key2=val2> | Update scheduler global configurations. This example sets key1=val1 and key2=val2 for scheduler's global configuration. |
+
+Updates scheduler configuration
+
 ### scmadmin
 
 Usage: `yarn scmadmin [options] `


---------------------------------------------------------------------
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-7253. Shared Cache Manager daemon command listed as admin subcmd in yarn script.

Posted by jh...@apache.org.
YARN-7253. Shared Cache Manager daemon command listed as admin subcmd in yarn script.


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

Branch: refs/heads/YARN-5734
Commit: c87db8d154ab2501e786b4f1669b205759ece5c3
Parents: 8facf1f
Author: Chris Trezzo <ct...@apache.org>
Authored: Wed Sep 27 11:32:09 2017 -0700
Committer: Chris Trezzo <ct...@apache.org>
Committed: Wed Sep 27 11:32:09 2017 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/hadoop-yarn/bin/yarn | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c87db8d1/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..52138c5 100755
--- a/hadoop-yarn-project/hadoop-yarn/bin/yarn
+++ b/hadoop-yarn-project/hadoop-yarn/bin/yarn
@@ -48,7 +48,7 @@ function hadoop_usage
   hadoop_add_subcommand "rmadmin" admin "admin tools"
   hadoop_add_subcommand "router" daemon "run the Router daemon"
   hadoop_add_subcommand "scmadmin" admin "SharedCacheManager admin tools"
-  hadoop_add_subcommand "sharedcachemanager" admin "run the SharedCacheManager daemon"
+  hadoop_add_subcommand "sharedcachemanager" daemon "run the SharedCacheManager daemon"
   hadoop_add_subcommand "timelinereader" client "run the timeline reader server"
   hadoop_add_subcommand "timelineserver" daemon "run the timeline server"
   hadoop_add_subcommand "top" client "view cluster information"


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


[28/50] [abbrv] hadoop git commit: MAPREDUCE-6968. Staging directory erasure coding config property has a typo. Contributed by Jason Lowe

Posted by jh...@apache.org.
MAPREDUCE-6968. Staging directory erasure coding config property has a typo. Contributed by Jason Lowe


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

Branch: refs/heads/YARN-5734
Commit: 9df05005ac258441eeceec25ecef7b28f9f50a02
Parents: 1267ff2
Author: Jason Lowe <jl...@apache.org>
Authored: Tue Sep 26 14:52:27 2017 -0500
Committer: Jason Lowe <jl...@apache.org>
Committed: Tue Sep 26 14:52:27 2017 -0500

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java     | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9df05005/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
index 86abb42..cf59730 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
@@ -1039,7 +1039,7 @@ public interface MRJobConfig {
   boolean DEFAULT_FINISH_JOB_WHEN_REDUCERS_DONE = true;
 
   String MR_AM_STAGING_DIR_ERASURECODING_ENABLED =
-      MR_AM_STAGING_DIR + "erasurecoding.enabled";
+      MR_AM_STAGING_DIR + ".erasurecoding.enabled";
 
   boolean DEFAULT_MR_AM_STAGING_ERASURECODING_ENABLED = false;
 }


---------------------------------------------------------------------
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-6871. Add additional deSelects params in RMWebServices#getAppReport. Contributed by Tanuj Nayak.

Posted by jh...@apache.org.
YARN-6871. Add additional deSelects params in RMWebServices#getAppReport. Contributed by Tanuj Nayak.


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

Branch: refs/heads/YARN-5734
Commit: 8facf1f976d7e12a846f12baabf54be1b7a49f9d
Parents: dd37877
Author: Sunil G <su...@apache.org>
Authored: Wed Sep 27 14:37:32 2017 +0530
Committer: Sunil G <su...@apache.org>
Committed: Wed Sep 27 14:37:32 2017 +0530

----------------------------------------------------------------------
 .../resourcemanager/webapp/DeSelectFields.java  |  11 +-
 .../resourcemanager/webapp/dao/AppInfo.java     | 108 +++++++++++++------
 .../webapp/TestRMWebServicesApps.java           |  77 ++++++++++++-
 3 files changed, 160 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8facf1f9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/DeSelectFields.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/DeSelectFields.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/DeSelectFields.java
index 258bbfa..c991766 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/DeSelectFields.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/DeSelectFields.java
@@ -91,7 +91,16 @@ public class DeSelectFields {
      * <code>RESOURCE_REQUESTS</code> is the first
      * supported type from YARN-6280.
      */
-    RESOURCE_REQUESTS("resourceRequests");
+    RESOURCE_REQUESTS("resourceRequests"),
+    /**
+     * <code>APP_TIMEOUTS, APP_NODE_LABEL_EXPRESSION, AM_NODE_LABEL_EXPRESSION,
+     * RESOURCE_INFO</code> are additionally supported parameters added in
+     * YARN-6871.
+     */
+    TIMEOUTS("timeouts"),
+    APP_NODE_LABEL_EXPRESSION("appNodeLabelExpression"),
+    AM_NODE_LABEL_EXPRESSION("amNodeLabelExpression"),
+    RESOURCE_INFO("resourceInfo");
 
     private final String literals;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8facf1f9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppInfo.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/AppInfo.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/AppInfo.java
index 236c467..40d7b4c 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/AppInfo.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/AppInfo.java
@@ -123,7 +123,7 @@ public class AppInfo {
   protected String amNodeLabelExpression;
 
   protected ResourcesInfo resourceInfo = null;
-  protected AppTimeoutsInfo timeouts = new AppTimeoutsInfo();
+  private AppTimeoutsInfo timeouts;
 
   public AppInfo() {
   } // JAXB needs this
@@ -247,47 +247,87 @@ public class AppInfo {
       unmanagedApplication = appSubmissionContext.getUnmanagedAM();
       appNodeLabelExpression =
           app.getApplicationSubmissionContext().getNodeLabelExpression();
-      amNodeLabelExpression = (unmanagedApplication) ? null
-          : app.getAMResourceRequests().get(0).getNodeLabelExpression();
+      /*
+       * When the deSelects parameter contains "amNodeLabelExpression", objects
+       * pertaining to the amNodeLabelExpression are not returned. By default,
+       * this is not skipped. (YARN-6871)
+       */
+      if(!deSelects.contains(DeSelectType.AM_NODE_LABEL_EXPRESSION)) {
+        amNodeLabelExpression = (unmanagedApplication) ?
+            null :
+            app.getAMResourceRequests().get(0).getNodeLabelExpression();
+      }
+      /*
+       * When the deSelects parameter contains "appNodeLabelExpression", objects
+       * pertaining to the appNodeLabelExpression are not returned. By default,
+       * this is not skipped. (YARN-6871)
+       */
+      if (!deSelects.contains(DeSelectType.APP_NODE_LABEL_EXPRESSION)) {
+        appNodeLabelExpression =
+            app.getApplicationSubmissionContext().getNodeLabelExpression();
+      }
+      /*
+       * When the deSelects parameter contains "amNodeLabelExpression", objects
+       * pertaining to the amNodeLabelExpression are not returned. By default,
+       * this is not skipped. (YARN-6871)
+       */
+      if (!deSelects.contains(DeSelectType.AM_NODE_LABEL_EXPRESSION)) {
+        amNodeLabelExpression = (unmanagedApplication) ?
+            null :
+            app.getAMResourceRequests().get(0).getNodeLabelExpression();
+      }
 
+      /*
+       * When the deSelects parameter contains "resourceInfo", ResourceInfo
+       * objects are not returned. Default behavior is no skipping. (YARN-6871)
+       */
       // Setting partition based resource usage of application
-      ResourceScheduler scheduler = rm.getRMContext().getScheduler();
-      if (scheduler instanceof CapacityScheduler) {
-        RMAppAttempt attempt = app.getCurrentAppAttempt();
-        if (null != attempt) {
-          FiCaSchedulerApp ficaAppAttempt = ((CapacityScheduler) scheduler)
-              .getApplicationAttempt(attempt.getAppAttemptId());
-          resourceInfo = null != ficaAppAttempt
-              ? new ResourcesInfo(ficaAppAttempt.getSchedulingResourceUsage())
-              : null;
+      if (!deSelects.contains(DeSelectType.RESOURCE_INFO)) {
+        ResourceScheduler scheduler = rm.getRMContext().getScheduler();
+        if (scheduler instanceof CapacityScheduler) {
+          RMAppAttempt attempt = app.getCurrentAppAttempt();
+          if (null != attempt) {
+            FiCaSchedulerApp ficaAppAttempt = ((CapacityScheduler) scheduler)
+                .getApplicationAttempt(attempt.getAppAttemptId());
+            resourceInfo = null != ficaAppAttempt ?
+                new ResourcesInfo(ficaAppAttempt.getSchedulingResourceUsage()) :
+                null;
+          }
         }
       }
 
-      Map<ApplicationTimeoutType, Long> applicationTimeouts =
-          app.getApplicationTimeouts();
-      if (applicationTimeouts.isEmpty()) {
-        // If application is not set timeout, lifetime should be sent as default
-        // with expiryTime=UNLIMITED and remainingTime=-1
-        AppTimeoutInfo timeoutInfo = new AppTimeoutInfo();
-        timeoutInfo.setTimeoutType(ApplicationTimeoutType.LIFETIME);
-        timeouts.add(timeoutInfo);
-      } else {
-        for (Map.Entry<ApplicationTimeoutType, Long> entry : app
-            .getApplicationTimeouts().entrySet()) {
-          AppTimeoutInfo timeout = new AppTimeoutInfo();
-          timeout.setTimeoutType(entry.getKey());
-          long timeoutInMillis = entry.getValue().longValue();
-          timeout.setExpiryTime(Times.formatISO8601(timeoutInMillis));
-          if (app.isAppInCompletedStates()) {
-            timeout.setRemainingTime(0);
-          } else {
-            timeout.setRemainingTime(Math
-                .max((timeoutInMillis - System.currentTimeMillis()) / 1000, 0));
+      /*
+       * When the deSelects parameter contains "appTimeouts", objects pertaining
+       * to app timeouts are not returned. By default, this is not skipped.
+       * (YARN-6871)
+       */
+      if (!deSelects.contains(DeSelectType.TIMEOUTS)) {
+        Map<ApplicationTimeoutType, Long> applicationTimeouts =
+            app.getApplicationTimeouts();
+        if (applicationTimeouts.isEmpty()) {
+          // If application is not set timeout, lifetime should be sent
+          // as default with expiryTime=UNLIMITED and remainingTime=-1
+          AppTimeoutInfo timeoutInfo = new AppTimeoutInfo();
+          timeoutInfo.setTimeoutType(ApplicationTimeoutType.LIFETIME);
+          timeouts = new AppTimeoutsInfo();
+          timeouts.add(timeoutInfo);
+        } else {
+          for (Map.Entry<ApplicationTimeoutType, Long> entry : app
+              .getApplicationTimeouts().entrySet()) {
+            AppTimeoutInfo timeout = new AppTimeoutInfo();
+            timeout.setTimeoutType(entry.getKey());
+            long timeoutInMillis = entry.getValue().longValue();
+            timeout.setExpiryTime(Times.formatISO8601(timeoutInMillis));
+            if (app.isAppInCompletedStates()) {
+              timeout.setRemainingTime(0);
+            } else {
+              timeout.setRemainingTime(Math.max(
+                  (timeoutInMillis - System.currentTimeMillis()) / 1000, 0));
+            }
+            timeouts.add(timeout);
           }
-          timeouts.add(timeout);
         }
       }
-
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8facf1f9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.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/TestRMWebServicesApps.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java
index 07f74a3..f0704ac 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java
@@ -1113,8 +1113,83 @@ public class TestRMWebServicesApps extends JerseyTestBase {
     JSONArray array = apps.getJSONArray("app");
     assertEquals("incorrect number of elements", 1, array.length());
     JSONObject app = array.getJSONObject(0);
-    assertTrue("resource requests shouldn't exits",
+    assertTrue("resource requests shouldn't exist",
         !app.has("resourceRequests"));
+
+    params.clear();
+    params.add("deSelects",
+        DeSelectFields.DeSelectType.AM_NODE_LABEL_EXPRESSION.toString());
+    response =
+        r.path("ws").path("v1").path("cluster").path("apps").queryParams(params)
+            .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
+
+    json = response.getEntity(JSONObject.class);
+    assertEquals("incorrect number of elements", 1, json.length());
+    apps = json.getJSONObject("apps");
+    assertEquals("incorrect number of elements", 1, apps.length());
+    array = apps.getJSONArray("app");
+    assertEquals("incorrect number of elements", 1, array.length());
+    app = array.getJSONObject(0);
+    assertTrue("AMNodeLabelExpression shouldn't exist",
+        !app.has("amNodeLabelExpression"));
+
+    params.clear();
+    params.add("deSelects", DeSelectFields.DeSelectType.TIMEOUTS.toString());
+    response =
+        r.path("ws").path("v1").path("cluster").path("apps").queryParams(params)
+            .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
+
+    json = response.getEntity(JSONObject.class);
+    assertEquals("incorrect number of elements", 1, json.length());
+    apps = json.getJSONObject("apps");
+    assertEquals("incorrect number of elements", 1, apps.length());
+    array = apps.getJSONArray("app");
+    assertEquals("incorrect number of elements", 1, array.length());
+    app = array.getJSONObject(0);
+    assertTrue("Timeouts shouldn't exist", !app.has("timeouts"));
+    rm.stop();
+
+    params.clear();
+    params.add("deSelects",
+        DeSelectFields.DeSelectType.APP_NODE_LABEL_EXPRESSION.toString());
+    response =
+        r.path("ws").path("v1").path("cluster").path("apps").queryParams(params)
+            .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
+
+    json = response.getEntity(JSONObject.class);
+    assertEquals("incorrect number of elements", 1, json.length());
+    apps = json.getJSONObject("apps");
+    assertEquals("incorrect number of elements", 1, apps.length());
+    array = apps.getJSONArray("app");
+    assertEquals("incorrect number of elements", 1, array.length());
+    app = array.getJSONObject(0);
+    assertTrue("AppNodeLabelExpression shouldn't exist",
+        !app.has("appNodeLabelExpression"));
+    rm.stop();
+
+    params.clear();
+    params
+        .add("deSelects", DeSelectFields.DeSelectType.RESOURCE_INFO.toString());
+    response =
+        r.path("ws").path("v1").path("cluster").path("apps").queryParams(params)
+            .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class);
+    assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
+        response.getType().toString());
+
+    json = response.getEntity(JSONObject.class);
+    assertEquals("incorrect number of elements", 1, json.length());
+    apps = json.getJSONObject("apps");
+    assertEquals("incorrect number of elements", 1, apps.length());
+    array = apps.getJSONArray("app");
+    assertEquals("incorrect number of elements", 1, array.length());
+    app = array.getJSONObject(0);
+    assertTrue("Resource info shouldn't exist", !app.has("resourceInfo"));
     rm.stop();
   }
 


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


[06/50] [abbrv] hadoop git commit: HADOOP-14905. Fix javadocs issues in Hadoop HDFS-NFS. Contributed by Mukul Kumar Singh.

Posted by jh...@apache.org.
HADOOP-14905. Fix javadocs issues in Hadoop HDFS-NFS. Contributed by Mukul Kumar Singh.


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

Branch: refs/heads/YARN-5734
Commit: 11af907c644a5e0b7929497b6f7538f3bc7bdaa0
Parents: 1c4a798
Author: Anu Engineer <ae...@apache.org>
Authored: Sat Sep 23 12:12:12 2017 -0700
Committer: Anu Engineer <ae...@apache.org>
Committed: Sat Sep 23 12:12:12 2017 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3Utils.java    | 5 +++++
 1 file changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/11af907c/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 abaa5ca..e376ebd 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
@@ -88,6 +88,8 @@ public class Nfs3Utils {
   /**
    * HDFS directory size is always zero. Try to return something meaningful
    * here. Assume each child take 32bytes.
+   * @param childNum number of children of the directory
+   * @return total size of the directory
    */
   public static long getDirSize(int childNum) {
     return (childNum + 2) * 32;
@@ -122,6 +124,9 @@ public class Nfs3Utils {
 
   /**
    * Send a write response to the netty network socket channel
+   * @param channel channel to which the buffer needs to be written
+   * @param out xdr object to be written to the channel
+   * @param xid transaction identifier
    */
   public static void writeChannel(Channel channel, XDR out, int xid) {
     if (channel == null) {


---------------------------------------------------------------------
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-5040.Audit log for admin commands/ logging output of all DFS admin commands. Contributed by Kuhu Shukla.

Posted by jh...@apache.org.
HDFS-5040.Audit log for admin commands/ logging output of all DFS admin commands. Contributed by Kuhu Shukla.


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

Branch: refs/heads/YARN-5734
Commit: 9d3e4cccf9cd0ffb60ee0e7c65cea5ae3c8015c2
Parents: 14fec04
Author: Brahma Reddy Battula <br...@apache.org>
Authored: Tue Sep 26 21:59:22 2017 +0530
Committer: Brahma Reddy Battula <br...@apache.org>
Committed: Tue Sep 26 21:59:22 2017 +0530

----------------------------------------------------------------------
 .../hdfs/server/namenode/FSNamesystem.java      | 149 ++--
 .../hdfs/server/namenode/NameNodeRpcServer.java |  26 +-
 .../hdfs/server/namenode/TestAuditLogger.java   |  21 +-
 .../namenode/TestAuditLoggerWithCommands.java   | 692 ++++++++++++++++++-
 .../server/namenode/TestNameNodeMXBean.java     |   9 +-
 5 files changed, 815 insertions(+), 82 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9d3e4ccc/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 217df94..1534850 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
@@ -87,6 +87,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PERMISSIONS_SUPERUSERGROU
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PERMISSIONS_SUPERUSERGROUP_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_KEY;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import static org.apache.hadoop.hdfs.server.namenode.FSDirStatAndListingOp.*;
 
 import org.apache.hadoop.hdfs.protocol.ReplicatedBlockStats;
@@ -351,7 +352,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         && !auditLoggers.isEmpty();
   }
 
-  private void logAuditEvent(boolean succeeded, String cmd, String src)
+  void logAuditEvent(boolean succeeded, String cmd, String src)
       throws IOException {
     logAuditEvent(succeeded, cmd, src, null, null);
   }
@@ -1715,7 +1716,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * Dump all metadata into specified file
    */
   void metaSave(String filename) throws IOException {
-    checkSuperuserPrivilege();
+    String operationName = "metaSave";
+    checkSuperuserPrivilege(operationName);
     checkOperation(OperationCategory.UNCHECKED);
     writeLock();
     try {
@@ -1727,8 +1729,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       out.flush();
       out.close();
     } finally {
-      writeUnlock("metaSave");
+      writeUnlock(operationName);
     }
+    logAuditEvent(true, operationName, null);
   }
 
   private void metaSave(PrintWriter out) {
@@ -3123,7 +3126,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       requireEffectiveLayoutVersionForFeature(Feature.QUOTA_BY_STORAGE_TYPE);
     }
     checkOperation(OperationCategory.WRITE);
-    final String operationName = "setQuota";
+    final String operationName = getQuotaCommand(nsQuota, ssQuota);
     writeLock();
     boolean success = false;
     try {
@@ -4207,30 +4210,34 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     }
   }
 
-  DatanodeInfo[] datanodeReport(final DatanodeReportType type
-      ) throws AccessControlException, StandbyException {
-    checkSuperuserPrivilege();
+  DatanodeInfo[] datanodeReport(final DatanodeReportType type)
+      throws IOException {
+    String operationName = "datanodeReport";
+    DatanodeInfo[] arr;
+    checkSuperuserPrivilege(operationName);
     checkOperation(OperationCategory.UNCHECKED);
     readLock();
     try {
       checkOperation(OperationCategory.UNCHECKED);
       final DatanodeManager dm = getBlockManager().getDatanodeManager();      
       final List<DatanodeDescriptor> results = dm.getDatanodeListForReport(type);
-
-      DatanodeInfo[] arr = new DatanodeInfo[results.size()];
+      arr = new DatanodeInfo[results.size()];
       for (int i=0; i<arr.length; i++) {
         arr[i] = new DatanodeInfoBuilder().setFrom(results.get(i))
             .build();
       }
-      return arr;
     } finally {
-      readUnlock("datanodeReport");
+      readUnlock(operationName);
     }
+    logAuditEvent(true, operationName, null);
+    return arr;
   }
 
   DatanodeStorageReport[] getDatanodeStorageReport(final DatanodeReportType type
-      ) throws AccessControlException, StandbyException {
-    checkSuperuserPrivilege();
+      ) throws IOException {
+    String operationName = "getDatanodeStorageReport";
+    DatanodeStorageReport[] reports;
+    checkSuperuserPrivilege(operationName);
     checkOperation(OperationCategory.UNCHECKED);
     readLock();
     try {
@@ -4238,17 +4245,18 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       final DatanodeManager dm = getBlockManager().getDatanodeManager();      
       final List<DatanodeDescriptor> datanodes = dm.getDatanodeListForReport(type);
 
-      DatanodeStorageReport[] reports = new DatanodeStorageReport[datanodes.size()];
+      reports = new DatanodeStorageReport[datanodes.size()];
       for (int i = 0; i < reports.length; i++) {
         final DatanodeDescriptor d = datanodes.get(i);
         reports[i] = new DatanodeStorageReport(
             new DatanodeInfoBuilder().setFrom(d).build(),
             d.getStorageReports());
       }
-      return reports;
     } finally {
       readUnlock("getDatanodeStorageReport");
     }
+    logAuditEvent(true, operationName, null);
+    return reports;
   }
 
   /**
@@ -4258,8 +4266,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    */
   boolean saveNamespace(final long timeWindow, final long txGap)
       throws IOException {
+    String operationName = "saveNamespace";
     checkOperation(OperationCategory.UNCHECKED);
-    checkSuperuserPrivilege();
+    checkSuperuserPrivilege(operationName);
 
     boolean saved = false;
     cpLock();  // Block if a checkpointing is in progress on standby.
@@ -4273,12 +4282,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       }
       saved = getFSImage().saveNamespace(timeWindow, txGap, this);
     } finally {
-      readUnlock("saveNamespace");
+      readUnlock(operationName);
       cpUnlock();
     }
     if (saved) {
       LOG.info("New namespace image has been created");
     }
+    logAuditEvent(true, operationName, null);
     return saved;
   }
   
@@ -4288,9 +4298,10 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * 
    * @throws AccessControlException if superuser privilege is violated.
    */
-  boolean restoreFailedStorage(String arg) throws AccessControlException,
-      StandbyException {
-    checkSuperuserPrivilege();
+  boolean restoreFailedStorage(String arg) throws IOException {
+    String operationName = getFailedStorageCommand(arg);
+    boolean val = false;
+    checkSuperuserPrivilege(operationName);
     checkOperation(OperationCategory.UNCHECKED);
     cpLock();  // Block if a checkpointing is in progress on standby.
     writeLock();
@@ -4298,17 +4309,18 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       checkOperation(OperationCategory.UNCHECKED);
       
       // if it is disabled - enable it and vice versa.
-      if(arg.equals("check"))
-        return getFSImage().getStorage().getRestoreFailedStorage();
-      
-      boolean val = arg.equals("true");  // false if not
-      getFSImage().getStorage().setRestoreFailedStorage(val);
-      
-      return val;
+      if(arg.equals("check")) {
+        val = getFSImage().getStorage().getRestoreFailedStorage();
+      } else {
+        val = arg.equals("true");  // false if not
+        getFSImage().getStorage().setRestoreFailedStorage(val);
+      }
     } finally {
-      writeUnlock("restoreFailedStorage");
+      writeUnlock(operationName);
       cpUnlock();
     }
+    logAuditEvent(true, operationName, null);
+    return val;
   }
 
   Date getStartTime() {
@@ -4316,7 +4328,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
     
   void finalizeUpgrade() throws IOException {
-    checkSuperuserPrivilege();
+    String operationName = "finalizeUpgrade";
+    checkSuperuserPrivilege(operationName);
     checkOperation(OperationCategory.UNCHECKED);
     cpLock();  // Block if a checkpointing is in progress on standby.
     writeLock();
@@ -4324,26 +4337,33 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       checkOperation(OperationCategory.UNCHECKED);
       getFSImage().finalizeUpgrade(this.isHaEnabled() && inActiveState());
     } finally {
-      writeUnlock("finalizeUpgrade");
+      writeUnlock(operationName);
       cpUnlock();
     }
+    logAuditEvent(true, operationName, null);
   }
 
   void refreshNodes() throws IOException {
+    String operationName = "refreshNodes";
     checkOperation(OperationCategory.UNCHECKED);
-    checkSuperuserPrivilege();
+    checkSuperuserPrivilege(operationName);
     getBlockManager().getDatanodeManager().refreshNodes(new HdfsConfiguration());
+    logAuditEvent(true, operationName, null);
   }
 
   void setBalancerBandwidth(long bandwidth) throws IOException {
+    String operationName = "setBalancerBandwidth";
     checkOperation(OperationCategory.UNCHECKED);
-    checkSuperuserPrivilege();
+    checkSuperuserPrivilege(operationName);
     getBlockManager().getDatanodeManager().setBalancerBandwidth(bandwidth);
+    logAuditEvent(true, operationName, null);
   }
 
   boolean setSafeMode(SafeModeAction action) throws IOException {
+    String operationName = action.toString().toLowerCase();
+    boolean error = false;
     if (action != SafeModeAction.SAFEMODE_GET) {
-      checkSuperuserPrivilege();
+      checkSuperuserPrivilege(operationName);
       switch(action) {
       case SAFEMODE_LEAVE: // leave safe mode
         leaveSafeMode(false);
@@ -4356,8 +4376,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         break;
       default:
         LOG.error("Unexpected safe mode action");
+        error = true;
       }
     }
+    if (!error) {
+      logAuditEvent(true, operationName, null);
+    }
     return isInSafeMode();
   }
 
@@ -4493,7 +4517,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   CheckpointSignature rollEditLog() throws IOException {
-    checkSuperuserPrivilege();
+    String operationName = "rollEditLog";
+    CheckpointSignature result = null;
+    checkSuperuserPrivilege(operationName);
     checkOperation(OperationCategory.JOURNAL);
     writeLock();
     try {
@@ -4502,10 +4528,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       if (Server.isRpcInvocation()) {
         LOG.info("Roll Edit Log from " + Server.getRemoteAddress());
       }
-      return getFSImage().rollEditLog(getEffectiveLayoutVersion());
+      result = getFSImage().rollEditLog(getEffectiveLayoutVersion());
     } finally {
-      writeUnlock("rollEditLog");
+      writeUnlock(operationName);
     }
+    logAuditEvent(true, operationName, null);
+    return result;
   }
 
   NamenodeCommand startCheckpoint(NamenodeRegistration backupNode,
@@ -6194,11 +6222,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     checkOperation(OperationCategory.WRITE);
     final String operationName = "allowSnapshot";
     boolean success = false;
+    checkSuperuserPrivilege(operationName);
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot allow snapshot for " + path);
-      checkSuperuserPrivilege();
       FSDirSnapshotOp.allowSnapshot(dir, snapshotManager, path);
       success = true;
     } finally {
@@ -6212,12 +6240,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   void disallowSnapshot(String path) throws IOException {
     checkOperation(OperationCategory.WRITE);
     final String operationName = "disallowSnapshot";
+    checkSuperuserPrivilege(operationName);
     boolean success = false;
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot disallow snapshot for " + path);
-      checkSuperuserPrivilege();
       FSDirSnapshotOp.disallowSnapshot(dir, snapshotManager, path);
       success = true;
     } finally {
@@ -6410,7 +6438,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   RollingUpgradeInfo queryRollingUpgrade() throws IOException {
-    checkSuperuserPrivilege();
+    final String operationName = "queryRollingUpgrade";
+    checkSuperuserPrivilege(operationName);
     checkOperation(OperationCategory.READ);
     readLock();
     try {
@@ -6420,15 +6449,16 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       Preconditions.checkNotNull(rollingUpgradeInfo);
       boolean hasRollbackImage = this.getFSImage().hasRollbackFSImage();
       rollingUpgradeInfo.setCreatedRollbackImages(hasRollbackImage);
-      return rollingUpgradeInfo;
     } finally {
-      readUnlock("queryRollingUpgrade");
+      readUnlock(operationName);
     }
+    logAuditEvent(true, operationName, null, null, null);
+    return rollingUpgradeInfo;
   }
 
   RollingUpgradeInfo startRollingUpgrade() throws IOException {
     final String operationName = "startRollingUpgrade";
-    checkSuperuserPrivilege();
+    checkSuperuserPrivilege(operationName);
     checkOperation(OperationCategory.WRITE);
     writeLock();
     try {
@@ -6619,7 +6649,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
   RollingUpgradeInfo finalizeRollingUpgrade() throws IOException {
     final String operationName = "finalizeRollingUpgrade";
-    checkSuperuserPrivilege();
+    checkSuperuserPrivilege(operationName);
     checkOperation(OperationCategory.WRITE);
     writeLock();
     try {
@@ -7744,5 +7774,38 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         .size();
   }
 
+  void checkSuperuserPrivilege(String operationName)
+      throws IOException {
+    try {
+      checkSuperuserPrivilege();
+    } catch (AccessControlException ace) {
+      logAuditEvent(false, operationName, null);
+      throw ace;
+    }
+  }
+
+  String getQuotaCommand(long nsQuota, long dsQuota) {
+    if (nsQuota == HdfsConstants.QUOTA_RESET
+        && dsQuota == HdfsConstants.QUOTA_DONT_SET) {
+      return "clearQuota";
+    } else if (nsQuota == HdfsConstants.QUOTA_DONT_SET
+        && dsQuota == HdfsConstants.QUOTA_RESET) {
+      return "clearSpaceQuota";
+    } else if (dsQuota == HdfsConstants.QUOTA_DONT_SET) {
+      return "setQuota";
+    } else {
+      return "setSpaceQuota";
+    }
+  }
+
+  String getFailedStorageCommand(String mode) {
+    if(mode.equals("check")) {
+      return "checkRestoreFailedStorage";
+    } else if (mode.equals("true")) {
+      return "enableRestoreFailedStorage";
+    } else {
+      return "disableRestoreFailedStorage";
+    }
+  }
 }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9d3e4ccc/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 c07b521..2512b57 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
@@ -1624,6 +1624,7 @@ public class NameNodeRpcServer implements NamenodeProtocols {
     if (this.serviceRpcServer != null) {
       this.serviceRpcServer.refreshServiceAcl(new Configuration(), new HDFSPolicyProvider());
     }
+    namesystem.logAuditEvent(true, "refreshServiceAcl", null);
   }
 
   @Override // RefreshAuthorizationPolicyProtocol
@@ -1631,17 +1632,19 @@ public class NameNodeRpcServer implements NamenodeProtocols {
     LOG.info("Refreshing all user-to-groups mappings. Requested by user: " +
         getRemoteUser().getShortUserName());
     Groups.getUserToGroupsMappingService().refresh();
+    namesystem.logAuditEvent(true, "refreshUserToGroupsMappings", null);
   }
 
   @Override // RefreshAuthorizationPolicyProtocol
-  public void refreshSuperUserGroupsConfiguration() {
+  public void refreshSuperUserGroupsConfiguration() throws IOException {
     LOG.info("Refreshing SuperUser proxy group mapping list ");
 
     ProxyUsers.refreshSuperUserGroupsConfiguration();
+    namesystem.logAuditEvent(true, "refreshSuperUserGroupsConfiguration", null);
   }
 
   @Override // RefreshCallQueueProtocol
-  public void refreshCallQueue() {
+  public void refreshCallQueue() throws IOException {
     LOG.info("Refreshing call queue.");
 
     Configuration conf = new Configuration();
@@ -1649,6 +1652,7 @@ public class NameNodeRpcServer implements NamenodeProtocols {
     if (this.serviceRpcServer != null) {
       serviceRpcServer.refreshCallQueue(conf);
     }
+    namesystem.logAuditEvent(true, "refreshCallQueue", null);
   }
 
   @Override // GenericRefreshProtocol
@@ -2412,22 +2416,30 @@ public class NameNodeRpcServer implements NamenodeProtocols {
   @Override // ReconfigurationProtocol
   public void startReconfiguration() throws IOException {
     checkNNStartup();
-    namesystem.checkSuperuserPrivilege();
+    String operationName = "startNamenodeReconfiguration";
+    namesystem.checkSuperuserPrivilege(operationName);
     nn.startReconfigurationTask();
+    namesystem.logAuditEvent(true, operationName, null);
   }
 
   @Override // ReconfigurationProtocol
   public ReconfigurationTaskStatus getReconfigurationStatus()
       throws IOException {
     checkNNStartup();
-    namesystem.checkSuperuserPrivilege();
-    return nn.getReconfigurationTaskStatus();
+    String operationName = "getNamenodeReconfigurationStatus";
+    namesystem.checkSuperuserPrivilege(operationName);
+    ReconfigurationTaskStatus status = nn.getReconfigurationTaskStatus();
+    namesystem.logAuditEvent(true, operationName, null);
+    return status;
   }
 
   @Override // ReconfigurationProtocol
   public List<String> listReconfigurableProperties() throws IOException {
     checkNNStartup();
-    namesystem.checkSuperuserPrivilege();
-    return Lists.newArrayList(nn.getReconfigurableProperties());
+    String operationName = "listNamenodeReconfigurableProperties";
+    namesystem.checkSuperuserPrivilege(operationName);
+    List<String> result = Lists.newArrayList(nn.getReconfigurableProperties());
+    namesystem.logAuditEvent(true, operationName, null);
+    return result;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9d3e4ccc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLogger.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLogger.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLogger.java
index 0e3cc8d..c422f32 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLogger.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLogger.java
@@ -152,7 +152,6 @@ public class TestAuditLogger {
     conf.set(DFS_NAMENODE_AUDIT_LOGGERS_KEY,
         DummyAuditLogger.class.getName());
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
-    
     GetOpParam.Op op = GetOpParam.Op.GETFILESTATUS;
     try {
       cluster.waitClusterUp();
@@ -168,7 +167,8 @@ public class TestAuditLogger {
       conn.connect();
       assertEquals(200, conn.getResponseCode());
       conn.disconnect();
-      assertEquals(1, DummyAuditLogger.logCount);
+      assertEquals("getfileinfo", DummyAuditLogger.lastCommand);
+      DummyAuditLogger.resetLogCount();
       assertEquals("127.0.0.1", DummyAuditLogger.remoteAddr);
       
       // non-trusted proxied request
@@ -178,7 +178,9 @@ public class TestAuditLogger {
       conn.connect();
       assertEquals(200, conn.getResponseCode());
       conn.disconnect();
-      assertEquals(2, DummyAuditLogger.logCount);
+      assertEquals("getfileinfo", DummyAuditLogger.lastCommand);
+      assertTrue(DummyAuditLogger.logCount == 1);
+      DummyAuditLogger.resetLogCount();
       assertEquals("127.0.0.1", DummyAuditLogger.remoteAddr);
       
       // trusted proxied request
@@ -190,7 +192,8 @@ public class TestAuditLogger {
       conn.connect();
       assertEquals(200, conn.getResponseCode());
       conn.disconnect();
-      assertEquals(3, DummyAuditLogger.logCount);
+      assertEquals("getfileinfo", DummyAuditLogger.lastCommand);
+      assertTrue(DummyAuditLogger.logCount == 1);
       assertEquals("1.1.1.1", DummyAuditLogger.remoteAddr);
     } finally {
       cluster.shutdown();
@@ -547,6 +550,7 @@ public class TestAuditLogger {
     static int unsuccessfulCount;
     static short foundPermission;
     static String remoteAddr;
+    private static String lastCommand;
     
     public void initialize(Configuration conf) {
       initialized = true;
@@ -565,11 +569,16 @@ public class TestAuditLogger {
       if (!succeeded) {
         unsuccessfulCount++;
       }
+      lastCommand = cmd;
       if (stat != null) {
         foundPermission = stat.getPermission().toShort();
       }
     }
 
+    public static String getLastCommand() {
+      return lastCommand;
+    }
+
   }
 
   public static class BrokenAuditLogger implements AuditLogger {
@@ -581,7 +590,9 @@ public class TestAuditLogger {
     public void logAuditEvent(boolean succeeded, String userName,
         InetAddress addr, String cmd, String src, String dst,
         FileStatus stat) {
-      throw new RuntimeException("uh oh");
+      if (!cmd.equals("datanodeReport")) {
+        throw new RuntimeException("uh oh");
+      }
     }
 
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9d3e4ccc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLoggerWithCommands.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLoggerWithCommands.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLoggerWithCommands.java
index 2adf470..4eda88f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLoggerWithCommands.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLoggerWithCommands.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BatchedRemoteIterator;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -31,24 +32,30 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
 import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
 
-import org.junit.BeforeClass;
-import org.junit.AfterClass;
-import org.junit.Ignore;
+import org.junit.After;
+import static org.junit.Assert.assertEquals;
+import org.junit.Before;
 import org.junit.Test;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
-import org.mockito.Mock;
 import org.mockito.Mockito;
 import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.when;
 
 public class TestAuditLoggerWithCommands {
 
@@ -65,13 +72,15 @@ public class TestAuditLoggerWithCommands {
   static UserGroupInformation user2;
   private static NamenodeProtocols proto;
 
-  @BeforeClass
-  public static void initialize() throws Exception {
+  @Before
+  public void initialize() throws Exception {
     // start a cluster
     conf = new HdfsConfiguration();
     conf.setBoolean(DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY, true);
     conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY,true);
     conf.setBoolean(DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY, true);
+    conf.setBoolean(
+        CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION, true);
     cluster =
         new MiniDFSCluster.Builder(conf).numDataNodes(NUM_DATA_NODES).build();
     cluster.waitActive();
@@ -88,8 +97,9 @@ public class TestAuditLoggerWithCommands {
     fs = cluster.getFileSystem();
   }
 
-  @AfterClass
-  public static void tearDown() throws Exception {
+  @After
+  public void tearDown() throws Exception {
+    Server.getCurCall().set(null);
     fs.close();
     fs2.close();
     fileSys.close();
@@ -126,22 +136,29 @@ public class TestAuditLoggerWithCommands {
     Path path = new Path("/testdir/testdir1");
     fs.mkdirs(path);
     fileSys = DFSTestUtil.getFileSystemAs(user1, conf);
-    try {
-      ((DistributedFileSystem)fileSys).setQuota(path, 10l, 10l);
-      fail("The operation should have failed with AccessControlException");
-    } catch (AccessControlException ace) {
-    }
-    String acePattern =
-        ".*allowed=false.*ugi=theDoctor.*cmd=setQuota.*";
-    int length = verifyAuditLogs(acePattern);
+    verifySetQuota(path, HdfsConstants.QUOTA_RESET,
+        HdfsConstants.QUOTA_DONT_SET);
+    verifySetQuota(path, HdfsConstants.QUOTA_DONT_SET,
+        HdfsConstants.QUOTA_RESET);
+    verifySetQuota(path, HdfsConstants.QUOTA_DONT_SET,
+        HdfsConstants.BYTES_IN_INTEGER);
+    verifySetQuota(path, HdfsConstants.BYTES_IN_INTEGER,
+        HdfsConstants.BYTES_IN_INTEGER);
     fileSys.close();
+  }
+
+  private void verifySetQuota(Path path, long nsQuota, long ssQuota)
+      throws IOException {
+    String operationName = cluster.getNamesystem().getQuotaCommand(
+        nsQuota, ssQuota);
+    String acePattern =
+        ".*allowed=false.*ugi=theDoctor.*cmd=.*" + operationName + ".*";
     try {
-      ((DistributedFileSystem)fileSys).setQuota(path, 10l, 10l);
-      fail("The operation should have failed with IOException");
-    } catch (IOException ace) {
+      ((DistributedFileSystem) fileSys).setQuota(path, nsQuota, ssQuota);
+      fail("The operation should have failed");
+    } catch (AccessControlException ace) {
     }
-    assertTrue("Unexpected log from getContentSummary",
-        length == auditlog.getOutput().split("\n").length);
+    verifyAuditLogs(acePattern);
   }
 
   @Test
@@ -180,7 +197,7 @@ public class TestAuditLoggerWithCommands {
         ".*allowed=false.*ugi=theDoctor.*cmd=renameSnapshot.*";
     fs.mkdirs(srcDir);
     fileSys = DFSTestUtil.getFileSystemAs(user1, conf);
-    cluster.getNamesystem().allowSnapshot(srcDir.toString());
+    ((DistributedFileSystem)fs).allowSnapshot(srcDir);
     try {
       fileSys.createSnapshot(srcDir);
       fail("The operation should have failed with AccessControlException");
@@ -215,7 +232,7 @@ public class TestAuditLoggerWithCommands {
     Path s1;
     fs.mkdirs(srcDir);
     fileSys = DFSTestUtil.getFileSystemAs(user1, conf);
-    cluster.getNamesystem().allowSnapshot(srcDir.toString());
+    ((DistributedFileSystem)fs).allowSnapshot(srcDir);
     try {
       s1 = fs.createSnapshot(srcDir);
       fileSys.deleteSnapshot(srcDir, s1.getName());
@@ -237,12 +254,65 @@ public class TestAuditLoggerWithCommands {
   }
 
   @Test
+  public void testAllowSnapshot() throws Exception {
+    Path srcDir = new Path(System.getProperty("user.dir"), "/src");
+    fs.mkdirs(srcDir);
+    String pattern =
+        ".*allowed=true.*ugi=" +
+            System.getProperty("user.name")+".*cmd=allowSnapshot.*";
+    try {
+      ((DistributedFileSystem)fs).allowSnapshot(srcDir);
+      verifyAuditLogs(pattern);
+    } catch (Exception e) {
+      fail("The operation should not have failed with Exception");
+    }
+    fileSys = DFSTestUtil.getFileSystemAs(user1, conf);
+    try {
+      ((DistributedFileSystem)fileSys).allowSnapshot(srcDir);
+      fail("The operation should have failed with AccessControlException");
+    } catch (AccessControlException ace) {
+    }
+    pattern =
+        ".*allowed=false.*ugi=theDoctor.*cmd=allowSnapshot.*";
+    verifyAuditLogs(pattern);
+    fs.delete(srcDir, true);
+    fileSys.close();
+  }
+
+  @Test
+  public void testDisallowSnapshot() throws Exception {
+    Path srcDir = new Path(System.getProperty("user.dir"), "/src");
+    fs.mkdirs(srcDir);
+    cluster.getNamesystem().allowSnapshot(srcDir.toString());
+    String pattern =
+        ".*allowed=true.*ugi=" +
+            System.getProperty("user.name")+".*cmd=disallowSnapshot.*";
+    try {
+      ((DistributedFileSystem)fs).disallowSnapshot(srcDir);
+      verifyAuditLogs(pattern);
+    } catch (Exception e) {
+      fail("The operation should not have failed with Exception");
+    }
+    cluster.getNamesystem().allowSnapshot(srcDir.toString());
+    fileSys = DFSTestUtil.getFileSystemAs(user1, conf);
+    try {
+      ((DistributedFileSystem)fileSys).disallowSnapshot(srcDir);
+      fail("The operation should have failed with AccessControlException");
+    } catch (AccessControlException ace) {
+      pattern =
+          ".*allowed=false.*ugi=theDoctor.*cmd=disallowSnapshot.*";
+      verifyAuditLogs(pattern);
+    }
+    fileSys.close();
+  }
+
+  @Test
   public void testAddCacheDirective() throws Exception {
     removeExistingCachePools(null);
     proto.addCachePool(new CachePoolInfo("pool1").
         setMode(new FsPermission((short) 0)));
     CacheDirectiveInfo alpha = new CacheDirectiveInfo.Builder().
-        setPath(new Path("/alpha")).
+        setPath(new Path(System.getProperty("user.dir"), "/alpha")).
         setPool("pool1").
         build();
     fileSys = DFSTestUtil.getFileSystemAs(user1, conf);
@@ -618,6 +688,579 @@ public class TestAuditLoggerWithCommands {
     return verifyAuditLogs(".*allowed=" + allowed + pattern);
   }
 
+  @Test
+  public void testMetaSave() throws Exception {
+    String aceMetaSave =
+        ".*allowed=true.*cmd=metaSave.*";
+    try {
+      ((DistributedFileSystem)fs).metaSave("test.log");
+      verifyAuditLogs(aceMetaSave);
+    } catch (Exception e) {
+      fail("The operation should not have failed with Exception");
+    }
+    try {
+      ((DistributedFileSystem)fileSys).metaSave("test.log");
+      fail("The operation should have failed with AccessControlException");
+    } catch (IOException ace) {
+      GenericTestUtils.assertExceptionContains("Access denied", ace);
+      aceMetaSave =
+          ".*allowed=false.*cmd=metaSave.*";
+      verifyAuditLogs(aceMetaSave);
+    }
+  }
+
+  @Test
+  public void testStartReconfiguration() throws Exception {
+    String auditLogString =
+        ".*allowed=true.*cmd=startNamenodeReconfiguration.*";
+    FSNamesystem fsNamesystem = spy(cluster.getNamesystem());
+    when(fsNamesystem.isExternalInvocation()).thenReturn(true);
+    Server.Call call = spy(new Server.Call(
+        1, 1, null, null, RPC.RpcKind.RPC_BUILTIN, new byte[] {1, 2, 3}));
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser(System.getProperty("user.name")));
+    Server.getCurCall().set(call);
+    try {
+      ((NameNodeRpcServer) cluster.getNameNodeRpc()).startReconfiguration();
+      verifyAuditLogs(auditLogString);
+    } catch (Exception e) {
+      fail("StartConfiguration should have passed!");
+    }
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser("theDoctor"));
+    try {
+      ((NameNodeRpcServer)cluster.getNameNodeRpc()).startReconfiguration();
+      fail(
+          "startNameNodeReconfiguration should throw AccessControlException!");
+    } catch (AccessControlException ace) {
+      auditLogString =
+          ".*allowed=false.*cmd=startNamenodeReconfiguration.*";
+      verifyAuditLogs(auditLogString);
+    }
+  }
+
+  @Test
+  public void testGetReconfigurationStatus() throws Exception {
+    String auditLogString =
+        ".*allowed=true.*cmd=getNamenodeReconfigurationStatus.*";
+    FSNamesystem fsNamesystem = spy(cluster.getNamesystem());
+    when(fsNamesystem.isExternalInvocation()).thenReturn(true);
+    Server.Call call = spy(new Server.Call(
+        1, 1, null, null, RPC.RpcKind.RPC_BUILTIN, new byte[] {1, 2, 3}));
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser(System.getProperty("user.name")));
+    Server.getCurCall().set(call);
+    try {
+      ((NameNodeRpcServer) cluster.getNameNodeRpc()).getReconfigurationStatus();
+      verifyAuditLogs(auditLogString);
+    } catch (Exception e) {
+      fail("getNamenodeReconfigurationStatus " +
+          " threw Exception!");
+    }
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser("theDoctor"));
+    try {
+      ((NameNodeRpcServer)cluster.getNameNodeRpc()).getReconfigurationStatus();
+      fail("getNamenodeReconfigurationStatus " +
+          " did not throw AccessControlException!");
+    } catch (AccessControlException ace) {
+      auditLogString =
+          ".*allowed=false.*cmd=getNamenodeReconfigurationStatus.*";
+      verifyAuditLogs(auditLogString);
+    }
+  }
+
+  @Test
+  public void testListReconfigurableProperties() throws Exception {
+    String auditLogString =
+        ".*allowed=true.*cmd=listNamenodeReconfigurableProperties.*";
+    FSNamesystem fsNamesystem = spy(cluster.getNamesystem());
+    when(fsNamesystem.isExternalInvocation()).thenReturn(true);
+    Server.Call call = spy(new Server.Call(
+        1, 1, null, null, RPC.RpcKind.RPC_BUILTIN, new byte[] {1, 2, 3}));
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser(System.getProperty("user.name")));
+    Server.getCurCall().set(call);
+    try {
+      ((NameNodeRpcServer) cluster.getNameNodeRpc()).
+          listReconfigurableProperties();
+      verifyAuditLogs(auditLogString);
+    } catch (Exception e) {
+      fail("listReconfigurableProperties " +
+          " threw Exception!");
+    }
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser("theDoctor"));
+    try {
+      ((NameNodeRpcServer)cluster.getNameNodeRpc()).
+          listReconfigurableProperties();
+      fail("getNamenodeReconfigurationStatus " +
+          " did not throw AccessControlException!");
+    } catch (AccessControlException ace) {
+      auditLogString =
+          ".*allowed=false.*cmd=listNamenodeReconfigurableProperties.*";
+      verifyAuditLogs(auditLogString);
+    }
+  }
+
+  @Test
+  public void testRefreshUserToGroupsMappings() throws Exception {
+    String auditLogString =
+        ".*allowed=true.*cmd=refreshUserToGroupsMappings.*";
+    FSNamesystem fsNamesystem = spy(cluster.getNamesystem());
+    when(fsNamesystem.isExternalInvocation()).thenReturn(true);
+    Server.Call call = spy(new Server.Call(
+        1, 1, null, null, RPC.RpcKind.RPC_BUILTIN, new byte[] {1, 2, 3}));
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser(System.getProperty("user.name")));
+    Server.getCurCall().set(call);
+    ((NameNodeRpcServer)cluster.getNameNodeRpc()).
+        refreshUserToGroupsMappings();
+    verifyAuditLogs(auditLogString);
+  }
+
+  @Test
+  public void testRefreshSuperUserGroupsConfiguration() throws Exception {
+    String auditLogString =
+        ".*allowed=true.*cmd=refreshSuperUserGroupsConfiguration.*";
+    FSNamesystem fsNamesystem = spy(cluster.getNamesystem());
+    when(fsNamesystem.isExternalInvocation()).thenReturn(true);
+    Server.Call call = spy(new Server.Call(
+        1, 1, null, null, RPC.RpcKind.RPC_BUILTIN, new byte[] {1, 2, 3}));
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser(System.getProperty("user.name")));
+    Server.getCurCall().set(call);
+    try {
+      ((NameNodeRpcServer) cluster.getNameNodeRpc()).
+          refreshSuperUserGroupsConfiguration();
+      verifyAuditLogs(auditLogString);
+    } catch (Exception e) {
+      fail(" The operation threw an exception");
+    }
+  }
+
+  @Test
+  public void testRefreshQueue() throws Exception {
+    String auditLogString =
+        ".*allowed=true.*cmd=refreshCallQueue.*";
+    FSNamesystem fsNamesystem = spy(cluster.getNamesystem());
+    when(fsNamesystem.isExternalInvocation()).thenReturn(true);
+    Server.Call call = spy(new Server.Call(
+        1, 1, null, null, RPC.RpcKind.RPC_BUILTIN, new byte[] {1, 2, 3}));
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser(System.getProperty("user.name")));
+    Server.getCurCall().set(call);
+    try {
+      ((NameNodeRpcServer) cluster.getNameNodeRpc()).refreshCallQueue();
+      verifyAuditLogs(auditLogString);
+    } catch (Exception e) {
+      fail(" The operation threw an exception");
+    }
+  }
+
+  @Test
+  public void testRefreshServiceAcl() throws Exception {
+    String auditLogString =
+        ".*allowed=true.*cmd=refreshServiceAcl.*";
+    FSNamesystem fsNamesystem = spy(cluster.getNamesystem());
+    when(fsNamesystem.isExternalInvocation()).thenReturn(true);
+    Server.Call call = spy(new Server.Call(
+        1, 1, null, null, RPC.RpcKind.RPC_BUILTIN, new byte[] {1, 2, 3}));
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser(System.getProperty("user.name")));
+    Server.getCurCall().set(call);
+    try {
+      ((NameNodeRpcServer) cluster.getNameNodeRpc()).refreshServiceAcl();
+      verifyAuditLogs(auditLogString);
+    } catch (Exception e) {
+      fail(" The operation threw an exception" + e);
+    }
+  }
+
+  @Test
+  public void testFinalizeRollingUpgrade() throws Exception {
+    String auditLogString =
+        ".*allowed=true.*cmd=finalizeRollingUpgrade.*";
+    FSNamesystem fsNamesystem = spy(cluster.getNamesystem());
+    when(fsNamesystem.isExternalInvocation()).thenReturn(true);
+    Server.Call call = spy(new Server.Call(
+        1, 1, null, null, RPC.RpcKind.RPC_BUILTIN, new byte[] {1, 2, 3}));
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser(System.getProperty("user.name")));
+    Server.getCurCall().set(call);
+    fsNamesystem.setRollingUpgradeInfo(false, System.currentTimeMillis());
+    try {
+      fsNamesystem.finalizeRollingUpgrade();
+      verifyAuditLogs(auditLogString);
+    } catch (Exception e) {
+      fail("finalizeRollingUpgrade threw Exception");
+    }
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser("theDoctor"));
+    try {
+      fsNamesystem.finalizeRollingUpgrade();
+      fail("finalizeRollingUpgrade should throw AccessControlException!");
+    } catch (AccessControlException ace) {
+      auditLogString =
+          ".*allowed=false.*cmd=finalizeRollingUpgrade.*";
+      verifyAuditLogs(auditLogString);
+    }
+  }
+
+  @Test
+  public void testQueryRollingUpgrade() throws Exception {
+    String auditLogString =
+        ".*allowed=true.*cmd=queryRollingUpgrade.*";
+    FSNamesystem fsNamesystem = spy(cluster.getNamesystem());
+    when(fsNamesystem.isExternalInvocation()).thenReturn(true);
+    Server.Call call = spy(new Server.Call(
+        1, 1, null, null, RPC.RpcKind.RPC_BUILTIN, new byte[] {1, 2, 3}));
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser(System.getProperty("user.name")));
+    Server.getCurCall().set(call);
+    fsNamesystem.setRollingUpgradeInfo(false, System.currentTimeMillis());
+    try {
+      fsNamesystem.queryRollingUpgrade();
+      verifyAuditLogs(auditLogString);
+    } catch (Exception e) {
+      fail("queryRollingUpgrade threw Exception");
+    }
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser("theDoctor"));
+    try {
+      fsNamesystem.queryRollingUpgrade();
+      fail("queryRollingUpgrade should have thrown an AccessControlException!");
+    } catch (AccessControlException ace) {
+      auditLogString =
+          ".*allowed=false.*cmd=queryRollingUpgrade.*";
+      verifyAuditLogs(auditLogString);
+    }
+  }
+
+  @Test
+  public void testRollEditLog() throws Exception {
+    String auditLogString =
+        ".*allowed=true.*cmd=rollEditLog.*";
+    FSNamesystem fsNamesystem = spy(cluster.getNamesystem());
+    when(fsNamesystem.isExternalInvocation()).thenReturn(true);
+    Server.Call call = spy(new Server.Call(
+        1, 1, null, null, RPC.RpcKind.RPC_BUILTIN, new byte[] {1, 2, 3}));
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser(System.getProperty("user.name")));
+    Server.getCurCall().set(call);
+    try {
+      fsNamesystem.rollEditLog();
+    } catch (Exception e) {
+      fail("rollEditLog threw Exception");
+    }
+    verifyAuditLogs(auditLogString);
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser("theDoctor"));
+    try {
+      fsNamesystem.rollEditLog();
+      fail("rollEditLog should have thrown an AccessControlException!");
+    } catch (AccessControlException ace) {
+      auditLogString =
+          ".*allowed=false.*cmd=rollEditLog.*";
+      verifyAuditLogs(auditLogString);
+    }
+  }
+
+  @Test
+  public void testSetSafeMode() throws Exception {
+    FSNamesystem fsNamesystem = spy(cluster.getNamesystem());
+    when(fsNamesystem.isExternalInvocation()).thenReturn(true);
+    Server.Call call = spy(new Server.Call(
+        1, 1, null, null, RPC.RpcKind.RPC_BUILTIN, new byte[] {1, 2, 3}));
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser(System.getProperty("user.name")));
+    Server.getCurCall().set(call);
+    verifySuccessfulSetSafeMode(fsNamesystem,
+        HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
+    verifySuccessfulSetSafeMode(fsNamesystem,
+        HdfsConstants.SafeModeAction.SAFEMODE_GET);
+    verifySuccessfulSetSafeMode(fsNamesystem,
+        HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
+    verifySuccessfulSetSafeMode(fsNamesystem,
+        HdfsConstants.SafeModeAction.SAFEMODE_FORCE_EXIT);
+    String auditLogString;
+    auditLogString =
+        ".*allowed=true.*cmd=safemode_get.*";
+    fsNamesystem.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_GET);
+    verifyAuditLogs(auditLogString);
+    auditLogString =
+        ".*allowed=true.*cmd=safemode_leave.*";
+    fsNamesystem.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
+    verifyAuditLogs(auditLogString);
+    auditLogString =
+        ".*allowed=true.*cmd=safemode_force_exit.*";
+    fsNamesystem.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_FORCE_EXIT);
+    verifyAuditLogs(auditLogString);
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser("theDoctor"));
+    verifySafeModeAction(fsNamesystem,
+        HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
+    verifySafeModeAction(fsNamesystem,
+        HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
+    verifySafeModeAction(fsNamesystem,
+        HdfsConstants.SafeModeAction.SAFEMODE_FORCE_EXIT);
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser(System.getProperty("user.name")));
+  }
+
+  @Test
+  public void testSetBalancerBandwidth() throws Exception {
+    String auditLogString =
+        ".*allowed=true.*cmd=setBalancerBandwidth.*";
+    FSNamesystem fsNamesystem = spy(cluster.getNamesystem());
+    when(fsNamesystem.isExternalInvocation()).thenReturn(true);
+    Server.Call call = spy(new Server.Call(
+        1, 1, null, null, RPC.RpcKind.RPC_BUILTIN, new byte[] {1, 2, 3}));
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser(System.getProperty("user.name")));
+    Server.getCurCall().set(call);
+    try {
+      fsNamesystem.setBalancerBandwidth(10);
+      verifyAuditLogs(auditLogString);
+    } catch (Exception e) {
+      fail("setBalancerBandwidth threw exception!");
+    }
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser("theDoctor"));
+    try {
+      fsNamesystem.setBalancerBandwidth(10);
+      fail(
+          "setBalancerBandwidth should have thrown AccessControlException!");
+    } catch (AccessControlException ace) {
+      auditLogString =
+          ".*allowed=false.*cmd=setBalancerBandwidth.*";
+      verifyAuditLogs(auditLogString);
+    }
+  }
+
+  @Test
+  public void testRefreshNodes() throws Exception {
+    String auditLogString =
+        ".*allowed=true.*cmd=refreshNodes.*";
+    FSNamesystem fsNamesystem = spy(cluster.getNamesystem());
+    when(fsNamesystem.isExternalInvocation()).thenReturn(true);
+    Server.Call call = spy(new Server.Call(
+        1, 1, null, null, RPC.RpcKind.RPC_BUILTIN, new byte[] {1, 2, 3}));
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser(System.getProperty("user.name")));
+    Server.getCurCall().set(call);
+    try {
+      fsNamesystem.refreshNodes();
+      verifyAuditLogs(auditLogString);
+    } catch (Exception e) {
+      fail("refreshNodes threw exception!");
+    }
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser("theDoctor"));
+    try {
+      fsNamesystem.refreshNodes();
+      fail(
+          "refreshNodes should have thrown an AccessControlException!");
+    } catch (AccessControlException ace) {
+      auditLogString =
+          ".*allowed=false.*cmd=refreshNodes.*";
+      verifyAuditLogs(auditLogString);
+    }
+  }
+
+  @Test
+  public void testFinalizeUpgrade() throws Exception {
+    String auditLogString =
+        ".*allowed=true.*cmd=finalizeUpgrade.*";
+    FSNamesystem fsNamesystem = spy(cluster.getNamesystem());
+    when(fsNamesystem.isExternalInvocation()).thenReturn(true);
+    Server.Call call = spy(new Server.Call(
+        1, 1, null, null, RPC.RpcKind.RPC_BUILTIN, new byte[] {1, 2, 3}));
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser(System.getProperty("user.name")));
+    Server.getCurCall().set(call);
+    try {
+      fsNamesystem.finalizeUpgrade();
+      verifyAuditLogs(auditLogString);
+    } catch (Exception e) {
+      fail("finalizeUpgrade threw Exception");
+    }
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser("theDoctor"));
+    try {
+      fsNamesystem.finalizeUpgrade();
+      fail("finalizeUpgrade should have thrown an AccessControlException!");
+    } catch (AccessControlException ace) {
+      auditLogString =
+          ".*allowed=false.*cmd=finalizeUpgrade.*";
+      verifyAuditLogs(auditLogString);
+    }
+  }
+
+  @Test
+  public void testSaveNamespace() throws Exception {
+    String auditLogString =
+        ".*allowed=true.*cmd=saveNamespace.*";
+    FSNamesystem fsNamesystem = spy(cluster.getNamesystem());
+    when(fsNamesystem.isExternalInvocation()).thenReturn(true);
+    Server.Call call = spy(new Server.Call(
+        1, 1, null, null, RPC.RpcKind.RPC_BUILTIN, new byte[] {1, 2, 3}));
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser(System.getProperty("user.name")));
+    Server.getCurCall().set(call);
+    fsNamesystem.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
+    try {
+      fsNamesystem.saveNamespace(10, 10);
+      verifyAuditLogs(auditLogString);
+    } catch (Exception e) {
+      fail("saveNamespace threw Exception");
+    }
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser("theDoctor"));
+    try {
+      fsNamesystem.saveNamespace(10, 10);
+      fail("saveNamespace should have thrown an AccessControlException!");
+    } catch (AccessControlException ace) {
+      auditLogString =
+          ".*allowed=false.*cmd=saveNamespace.*";
+      verifyAuditLogs(auditLogString);
+    }
+  }
+
+  @Test
+  public void testDatanodeReport() throws Exception {
+    String auditLogString =
+        ".*allowed=true.*cmd=datanodeReport.*";
+    FSNamesystem fsNamesystem = spy(cluster.getNamesystem());
+    when(fsNamesystem.isExternalInvocation()).thenReturn(true);
+    Server.Call call = spy(new Server.Call(
+        1, 1, null, null, RPC.RpcKind.RPC_BUILTIN, new byte[] {1, 2, 3}));
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser(System.getProperty("user.name")));
+    Server.getCurCall().set(call);
+    try {
+      fsNamesystem.datanodeReport(HdfsConstants.DatanodeReportType.ALL);
+      verifyAuditLogs(auditLogString);
+    } catch (Exception e) {
+      fail("datanodeReport threw Exception");
+    }
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser("theDoctor"));
+    try {
+      fsNamesystem.datanodeReport(HdfsConstants.DatanodeReportType.ALL);
+      fail(
+          "datanodeReport should have thrown an AccessControlException!");
+    } catch (AccessControlException ace) {
+      auditLogString =
+          ".*allowed=false.*cmd=datanodeReport.*";
+      verifyAuditLogs(auditLogString);
+    }
+  }
+
+  @Test
+  public void testRestoreFailedStorage() throws Exception {
+    FSNamesystem fsNamesystem = spy(cluster.getNamesystem());
+    when(fsNamesystem.isExternalInvocation()).thenReturn(true);
+    Server.Call call = spy(new Server.Call(
+        1, 1, null, null, RPC.RpcKind.RPC_BUILTIN, new byte[] {1, 2, 3}));
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser(System.getProperty("user.name")));
+    Server.getCurCall().set(call);
+    verifyAuditRestoreFailedStorage(fsNamesystem, "check");
+    verifyAuditRestoreFailedStorage(fsNamesystem, "true");
+    verifyAuditRestoreFailedStorage(fsNamesystem, "false");
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser("theDoctor"));
+    verifyAuditRestoreFailedStorageACE(fsNamesystem, "check");
+    verifyAuditRestoreFailedStorageACE(fsNamesystem, "true");
+    verifyAuditRestoreFailedStorageACE(fsNamesystem, "false");
+  }
+
+  @Test
+  public void testGetDatanodeStorageReport() throws Exception {
+    FSNamesystem fsNamesystem = spy(cluster.getNamesystem());
+    when(fsNamesystem.isExternalInvocation()).thenReturn(true);
+    Server.Call call = spy(new Server.Call(
+        1, 1, null, null, RPC.RpcKind.RPC_BUILTIN, new byte[] {1, 2, 3}));
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser(System.getProperty("user.name")));
+    Server.getCurCall().set(call);
+    DatanodeStorageReport[] reports  = fsNamesystem.getDatanodeStorageReport(
+        HdfsConstants.DatanodeReportType.ALL);
+    String auditLogString =
+        ".*allowed=true.*cmd=" + "getDatanodeStorageReport" + ".*";
+    verifyAuditLogs(auditLogString);
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser("theDoctor"));
+    auditLogString =
+        ".*allowed=false.*cmd=" + "getDatanodeStorageReport" + ".*";
+    try {
+      fsNamesystem.getDatanodeStorageReport(
+          HdfsConstants.DatanodeReportType.ALL);
+      fail("Should have thrown an AccessControlException!");
+    } catch (AccessControlException ace) {
+      verifyAuditLogs(auditLogString);
+    }
+  }
+
+  private void verifyAuditRestoreFailedStorageACE(
+      FSNamesystem fsNamesystem, String arg) throws IOException {
+    String operationName = fsNamesystem.getFailedStorageCommand(arg);
+    try {
+      fsNamesystem.restoreFailedStorage(arg);
+      fail(
+          "RestoreFailedStorage should have thrown AccessControlException!");
+    } catch (IOException ace) {
+      assertEquals("Unexpected Exception!",
+          ace.getClass(), AccessControlException.class);
+      String auditLogString =
+          ".*allowed=false.*cmd=" + operationName + ".*";
+      verifyAuditLogs(auditLogString);
+    }
+  }
+
+  private void verifyAuditRestoreFailedStorage(
+      FSNamesystem fsNamesystem, String arg) throws IOException {
+    String operationName = fsNamesystem.getFailedStorageCommand(arg);
+    String auditLogString =
+        ".*allowed=true.*cmd=" + operationName + ".*";
+    try {
+      fsNamesystem.restoreFailedStorage(arg);
+      verifyAuditLogs(auditLogString);
+    } catch (Exception e) {
+      fail(
+          "The operation should not have failed with Exception");
+    }
+  }
+
+  private void verifySuccessfulSetSafeMode(FSNamesystem fsNamesystem,
+      HdfsConstants.SafeModeAction safeModeAction) throws IOException {
+    String operationName = safeModeAction.toString().toLowerCase();
+    String auditLogString =
+        ".*allowed=true.*cmd=" + operationName +".*";
+    try {
+      fsNamesystem.setSafeMode(safeModeAction);
+      verifyAuditLogs(auditLogString);
+    } catch (Exception e) {
+      fail("The operation should not have failed with Exception");
+    }
+  }
+
+  private void verifySafeModeAction(
+      FSNamesystem fsNamesystem, HdfsConstants.SafeModeAction safeModeAction)
+      throws IOException {
+    String operationName = safeModeAction.toString().toLowerCase();
+    String auditLogString;
+    try {
+      fsNamesystem.setSafeMode(safeModeAction);
+      fail("setSafeMode should have thrown an AccessControlException!");
+    } catch (AccessControlException ace) {
+      auditLogString =
+          ".*allowed=false.*cmd=" + operationName +".*";
+      verifyAuditLogs(auditLogString);
+    }
+  }
+
   private int verifyAuditLogs(String pattern) {
     int length = auditlog.getOutput().split("\n").length;
     String lastAudit = auditlog.getOutput().split("\n")[length - 1];
@@ -633,4 +1276,3 @@ public class TestAuditLoggerWithCommands {
     }
   }
 }
-

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9d3e4ccc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java
index 36638e0..93a7525 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java
@@ -556,13 +556,18 @@ public class TestNameNodeMXBean {
       assertEquals("Unexpected num windows", 3, windows.size());
       for (Map<String, List<Map<String, Object>>> window : windows) {
         final List<Map<String, Object>> ops = window.get("ops");
-        assertEquals("Unexpected num ops", 3, ops.size());
+        assertEquals("Unexpected num ops", 4, ops.size());
         for (Map<String, Object> op: ops) {
+          if (op.get("opType").equals("datanodeReport")) {
+            continue;
+          }
           final long count = Long.parseLong(op.get("totalCount").toString());
           final String opType = op.get("opType").toString();
           final int expected;
           if (opType.equals(TopConf.ALL_CMDS)) {
-            expected = 2*NUM_OPS;
+            expected = 2 * NUM_OPS + 2;
+          } else if (opType.equals("datanodeReport")) {
+            expected = 2;
           } else {
             expected = NUM_OPS;
           }


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


[36/50] [abbrv] hadoop git commit: YARN-7024: Fix issues on recovery in LevelDB store. Contributed by Jonathan Hung

Posted by jh...@apache.org.
YARN-7024: Fix issues on recovery in LevelDB 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/a3aec1cc
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/a3aec1cc
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/a3aec1cc

Branch: refs/heads/YARN-5734
Commit: a3aec1cc12693642edcecb53830db6a80002c1ff
Parents: 44907bf
Author: Xuan <xg...@apache.org>
Authored: Wed Aug 23 11:11:41 2017 -0700
Committer: Jonathan Hung <jh...@linkedin.com>
Committed: Wed Sep 27 19:52:58 2017 -0700

----------------------------------------------------------------------
 .../scheduler/capacity/conf/InMemoryConfigurationStore.java      | 2 +-
 .../scheduler/capacity/conf/LeveldbConfigurationStore.java       | 4 +++-
 2 files changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a3aec1cc/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 b97be1b..c63734d 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
@@ -79,7 +79,7 @@ public class InMemoryConfigurationStore implements YarnConfigurationStore {
 
   @Override
   public synchronized List<LogMutation> getPendingMutations() {
-    return pendingMutations;
+    return new LinkedList<>(pendingMutations);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a3aec1cc/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
index 1534685..1280fab 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/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
@@ -65,6 +65,7 @@ public class LeveldbConfigurationStore implements YarnConfigurationStore {
   private static final String LOG_COMMITTED_TXN = "committedTxn";
 
   private DB db;
+  // Txnid for the last transaction logged to the store.
   private long txnId = 0;
   private long minTxn = 0;
   private long maxLogs;
@@ -92,6 +93,7 @@ public class LeveldbConfigurationStore implements YarnConfigurationStore {
           break;
         }
         pendingMutations.add(deserLogMutation(entry.getValue()));
+        txnId++;
       }
       // Get the earliest txnId stored in logs
       itr.seekToFirst();
@@ -278,7 +280,7 @@ public class LeveldbConfigurationStore implements YarnConfigurationStore {
 
   @Override
   public List<LogMutation> getPendingMutations() {
-    return pendingMutations;
+    return new LinkedList<>(pendingMutations);
   }
 
   @Override


---------------------------------------------------------------------
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-7257. AggregatedLogsBlock reports a bad 'end' value as a bad 'start' value. Contributed by Jason Lowe

Posted by jh...@apache.org.
YARN-7257. AggregatedLogsBlock reports a bad 'end' value as a bad
'start' value. Contributed by Jason Lowe


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

Branch: refs/heads/YARN-5734
Commit: 28c4957fccebe2d7e63ec9fe9af58313b4f21d4f
Parents: c87db8d
Author: Xuan <xg...@apache.org>
Authored: Wed Sep 27 15:46:18 2017 -0700
Committer: Xuan <xg...@apache.org>
Committed: Wed Sep 27 15:46:18 2017 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/yarn/webapp/log/AggregatedLogsBlock.java     | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/28c4957f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/log/AggregatedLogsBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/log/AggregatedLogsBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/log/AggregatedLogsBlock.java
index f6fea74..ef6876a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/log/AggregatedLogsBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/log/AggregatedLogsBlock.java
@@ -67,7 +67,7 @@ public class AggregatedLogsBlock extends HtmlBlock {
       LogAggregationWebUtils.getLogEndIndex(
           html, $("end"));
     } catch (NumberFormatException ne) {
-      html.h1().__("Invalid log start value: " + $("end")).__();
+      html.h1().__("Invalid log end value: " + $("end")).__();
       isValid = false;
     }
 


---------------------------------------------------------------------
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: HDFS-12529. Get source for config tags from file name. Contributed by Ajay Kumar.

Posted by jh...@apache.org.
HDFS-12529. Get source for config tags from file name. 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/0889e5a8
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/0889e5a8
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/0889e5a8

Branch: refs/heads/YARN-5734
Commit: 0889e5a8b7102ca1b64af6806537ad99c2018dfd
Parents: ac05a51
Author: Anu Engineer <ae...@apache.org>
Authored: Mon Sep 25 13:55:09 2017 -0700
Committer: Anu Engineer <ae...@apache.org>
Committed: Mon Sep 25 13:55:09 2017 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/conf/Configuration.java   | 52 +++++++++++---------
 .../apache/hadoop/conf/TestConfiguration.java   | 11 ++++-
 2 files changed, 39 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0889e5a8/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
index a339dac..2890853 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
@@ -81,6 +81,7 @@ import javax.xml.transform.stream.StreamResult;
 
 import com.google.common.base.Charsets;
 import org.apache.commons.collections.map.UnmodifiableMap;
+import org.apache.commons.io.FilenameUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
@@ -2811,6 +2812,7 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
       String confName = null;
       String confValue = null;
       String confInclude = null;
+      String confTag = null;
       boolean confFinal = false;
       boolean fallbackAllowed = false;
       boolean fallbackEntered = false;
@@ -2825,6 +2827,7 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
             confName = null;
             confValue = null;
             confFinal = false;
+            confTag = null;
             confSource.clear();
 
             // First test for short format configuration
@@ -2843,9 +2846,8 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
                 confSource.add(StringInterner.weakIntern(
                     reader.getAttributeValue(i)));
               } else if ("tag".equals(propertyAttr)) {
-                //Read tags and put them in propertyTagsMap
-                readTagFromConfig(reader.getAttributeValue(i), confName,
-                    confValue, confSource);
+                confTag = StringInterner
+                    .weakIntern(reader.getAttributeValue(i));
               }
             }
             break;
@@ -2937,9 +2939,7 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
             break;
           case "tag":
             if (token.length() > 0) {
-              //Read tags and put them in propertyTagsMap
-              readTagFromConfig(token.toString(), confName,
-                  confValue, confSource);
+              confTag = StringInterner.weakIntern(token.toString());
             }
             break;
           case "include":
@@ -2956,6 +2956,11 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
               break;
             }
             confSource.add(name);
+            //Read tags and put them in propertyTagsMap
+            if (confTag != null) {
+              readTagFromConfig(confTag, confName, confValue, confSource);
+            }
+
             DeprecatedKeyInfo keyInfo =
                 deprecations.getDeprecatedKeyMap().get(confName);
             if (keyInfo != null) {
@@ -3001,21 +3006,24 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
         if (confSource.size() > 0) {
           for (String source : confSource) {
             PropertyTag tag1 = this.getPropertyTag(tagStr,
-                source.split("-")[0]);
-            if (propertyTagsMap.containsKey(tag1)) {
-              propertyTagsMap.get(tag1)
-                  .setProperty(confName, confValue);
-            } else {
-              Properties props = new Properties();
-              props.setProperty(confName, confValue);
-              propertyTagsMap.put(tag1, props);
+                FilenameUtils.getName(source).split("-")[0]);
+            if (tag1 != null) {
+              //Handle property with no/null value
+              if (confValue == null) {
+                confValue = "";
+              }
+              if (propertyTagsMap.containsKey(tag1)) {
+                propertyTagsMap.get(tag1).setProperty(confName, confValue);
+              } else {
+                Properties props = new Properties();
+                props.setProperty(confName, confValue);
+                propertyTagsMap.put(tag1, props);
+              }
             }
           }
         } else {
-          //If no source is set try to find tag in CorePropertyTag
-          if (propertyTagsMap
-              .containsKey(CorePropertyTag.valueOf(tagStr)
-              )) {
+          // If no source is set try to find tag in CorePropertyTag
+          if (propertyTagsMap.containsKey(CorePropertyTag.valueOf(tagStr))) {
             propertyTagsMap.get(CorePropertyTag.valueOf(tagStr))
                 .setProperty(confName, confValue);
           } else {
@@ -3025,11 +3033,11 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
                 props);
           }
         }
-      } catch (IllegalArgumentException iae) {
-        //Log the invalid tag and continue to parse rest of the
-        // properties.
+      } catch (Exception ex) {
+        // Log the invalid tag and continue to parse rest of the properties.
         LOG.info("Invalid tag '" + tagStr + "' found for "
-            + "property:" + confName, iae);
+            + "property:" + confName + " Source:" + Arrays
+            .toString(confSource.toArray()), ex);
       }
 
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0889e5a8/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java
index 4cd1666..a806b8c 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java
@@ -77,6 +77,8 @@ public class TestConfiguration {
   private Configuration conf;
   final static String CONFIG = new File("./test-config-TestConfiguration.xml").getAbsolutePath();
   final static String CONFIG2 = new File("./test-config2-TestConfiguration.xml").getAbsolutePath();
+  final static String CONFIG_CORE = new File("./core-site.xml")
+      .getAbsolutePath();
   final static String CONFIG_FOR_ENUM = new File("./test-config-enum-TestConfiguration.xml").getAbsolutePath();
   final static String CONFIG_FOR_URI = "file://"
       + new File("./test-config-uri-TestConfiguration.xml").getAbsolutePath();
@@ -114,6 +116,7 @@ public class TestConfiguration {
     new File(new URI(CONFIG_FOR_URI)).delete();
     new File(CONFIG_MULTI_BYTE).delete();
     new File(CONFIG_MULTI_BYTE_SAVED).delete();
+    new File(CONFIG_CORE).delete();
   }
 
   private void startConfig() throws IOException{
@@ -2248,14 +2251,14 @@ public class TestConfiguration {
   @Test
   public void testGetAllPropertiesByTags() throws Exception {
 
-    out = new BufferedWriter(new FileWriter(CONFIG));
+    out = new BufferedWriter(new FileWriter(CONFIG_CORE));
     startConfig();
     appendPropertyByTag("dfs.cblock.trace.io", "false", "DEBUG");
     appendPropertyByTag("dfs.replication", "1", "PERFORMANCE,REQUIRED");
     appendPropertyByTag("dfs.namenode.logging.level", "INFO", "CLIENT,DEBUG");
     endConfig();
 
-    Path fileResource = new Path(CONFIG);
+    Path fileResource = new Path(CONFIG_CORE);
     conf.addResource(fileResource);
     conf.getProps();
 
@@ -2266,6 +2269,10 @@ public class TestConfiguration {
     tagList.add(CorePropertyTag.CLIENT);
 
     Properties properties = conf.getAllPropertiesByTags(tagList);
+    String[] sources = conf.getPropertySources("dfs.replication");
+    assertTrue(sources.length == 1);
+    assertTrue(Arrays.toString(sources).contains("core-site.xml"));
+
     assertEq(3, properties.size());
     assertEq(true, properties.containsKey("dfs.namenode.logging.level"));
     assertEq(true, properties.containsKey("dfs.replication"));


---------------------------------------------------------------------
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-7251. Misc changes to YARN-5734

Posted by jh...@apache.org.
YARN-7251. Misc changes to YARN-5734


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

Branch: refs/heads/YARN-5734
Commit: 4bbedd02415d43abf3fa7ff96ed530dba7d9d2c6
Parents: 48202a5
Author: Jonathan Hung <jh...@linkedin.com>
Authored: Mon Sep 25 17:52:40 2017 -0700
Committer: Jonathan Hung <jh...@linkedin.com>
Committed: Wed Sep 27 19:52:59 2017 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/hadoop-yarn/bin/yarn        |  3 +-
 hadoop-yarn-project/hadoop-yarn/bin/yarn.cmd    |  3 +-
 .../hadoop/yarn/conf/YarnConfiguration.java     | 33 ++++++++++++++-
 .../hadoop/yarn/client/cli/SchedConfCLI.java    | 42 ++++++++++++++------
 .../yarn/client/cli/TestSchedConfCLI.java       | 20 +++++-----
 .../hadoop/yarn/webapp/dao/QueueConfigInfo.java |  2 +-
 .../yarn/webapp/dao/SchedConfUpdateInfo.java    |  2 +-
 .../src/main/resources/yarn-default.xml         | 15 ++++---
 .../scheduler/capacity/CapacityScheduler.java   | 12 +++---
 .../CapacitySchedulerConfiguration.java         | 12 ------
 .../conf/MutableCSConfigurationProvider.java    |  2 +-
 .../resourcemanager/webapp/RMWebServices.java   |  2 +-
 .../resourcemanager/TestRMAdminService.java     |  4 +-
 .../conf/TestLeveldbConfigurationStore.java     |  2 -
 .../TestMutableCSConfigurationProvider.java     |  3 ++
 .../capacity/conf/TestZKConfigurationStore.java |  2 -
 .../TestRMWebServicesConfigurationMutation.java | 28 ++++++-------
 .../src/site/markdown/CapacityScheduler.md      | 11 ++---
 .../src/site/markdown/ResourceManagerRest.md    | 40 ++++++++++---------
 .../src/site/markdown/YarnCommands.md           | 12 +++---
 20 files changed, 146 insertions(+), 104 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4bbedd02/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 0063596..0f19989 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 "schedulerconf" client "Updates scheduler configuration"
   hadoop_add_subcommand "scmadmin" admin "SharedCacheManager admin tools"
   hadoop_add_subcommand "sharedcachemanager" daemon "run the SharedCacheManager daemon"
   hadoop_add_subcommand "timelinereader" client "run the timeline reader server"
@@ -142,7 +143,7 @@ function yarncmd_case
       HADOOP_SUBCMD_SUPPORTDAEMONIZATION="true"
       HADOOP_CLASSNAME='org.apache.hadoop.yarn.server.router.Router'
     ;;
-    schedconf)
+    schedulerconf)
     HADOOP_CLASSNAME='org.apache.hadoop.yarn.client.cli.SchedConfCLI'
     ;;
     scmadmin)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4bbedd02/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 7ec9848..fed3d90 100644
--- a/hadoop-yarn-project/hadoop-yarn/bin/yarn.cmd
+++ b/hadoop-yarn-project/hadoop-yarn/bin/yarn.cmd
@@ -295,7 +295,7 @@ goto :eof
   set YARN_OPTS=%YARN_OPTS% %YARN_CLIENT_OPTS%
   goto :eof
 
-:schedconf
+:schedulerconf
   set CLASS=org.apache.hadoop.yarn.client.cli.SchedConfCLI
   set YARN_OPTS=%YARN_OPTS% %YARN_CLIENT_OPTS%
   goto :eof
@@ -345,6 +345,7 @@ goto :eof
   @echo   node                 prints node report(s)
   @echo   queue                prints queue information
   @echo   logs                 dump container logs
+  @echo   schedulerconf        updates scheduler configuration
   @echo   classpath            prints the class path needed to get the
   @echo                        Hadoop jar and the required libraries
   @echo   daemonlog            get/set the log level for each daemon

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4bbedd02/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 e3ce3ecc..69e89df 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
@@ -674,33 +674,64 @@ public class YarnConfiguration extends Configuration {
   public static final String DEFAULT_RM_CONFIGURATION_PROVIDER_CLASS =
       "org.apache.hadoop.yarn.LocalConfigurationProvider";
 
+  @Private
+  @Unstable
   public static final String SCHEDULER_CONFIGURATION_STORE_CLASS =
       YARN_PREFIX + "scheduler.configuration.store.class";
+  @Private
+  @Unstable
+  public static final String FILE_CONFIGURATION_STORE = "file";
+  @Private
+  @Unstable
   public static final String MEMORY_CONFIGURATION_STORE = "memory";
+  @Private
+  @Unstable
   public static final String LEVELDB_CONFIGURATION_STORE = "leveldb";
+  @Private
+  @Unstable
   public static final String ZK_CONFIGURATION_STORE = "zk";
+  @Private
+  @Unstable
   public static final String DEFAULT_CONFIGURATION_STORE =
-      MEMORY_CONFIGURATION_STORE;
+      FILE_CONFIGURATION_STORE;
+  @Private
+  @Unstable
   public static final String RM_SCHEDCONF_STORE_PATH = YARN_PREFIX
       + "scheduler.configuration.leveldb-store.path";
 
+  @Private
+  @Unstable
   public static final String RM_SCHEDCONF_LEVELDB_COMPACTION_INTERVAL_SECS =
       YARN_PREFIX
           + "scheduler.configuration.leveldb-store.compaction-interval-secs";
+  @Private
+  @Unstable
   public static final long
       DEFAULT_RM_SCHEDCONF_LEVELDB_COMPACTION_INTERVAL_SECS = 60 * 60 * 24L;
 
+  @Private
+  @Unstable
   public static final String RM_SCHEDCONF_MAX_LOGS =
       YARN_PREFIX + "scheduler.configuration.store.max-logs";
+  @Private
+  @Unstable
   public static final long DEFAULT_RM_SCHEDCONF_LEVELDB_MAX_LOGS = 1000;
+  @Private
+  @Unstable
   public static final long DEFAULT_RM_SCHEDCONF_ZK_MAX_LOGS = 1000;
 
   /** Parent znode path under which ZKConfigurationStore will create znodes. */
+  @Private
+  @Unstable
   public static final String RM_SCHEDCONF_STORE_ZK_PARENT_PATH = YARN_PREFIX
       + "scheduler.configuration.zk-store.parent-path";
+  @Private
+  @Unstable
   public static final String DEFAULT_RM_SCHEDCONF_STORE_ZK_PARENT_PATH =
       "/confstore";
 
+  @Private
+  @Unstable
   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/4bbedd02/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
index e17062e..11bfdd7 100644
--- 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
@@ -27,7 +27,7 @@ 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.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
@@ -48,7 +48,7 @@ import java.util.Map;
  * CLI for modifying scheduler configuration.
  */
 @Public
-@Evolving
+@Unstable
 public class SchedConfCLI extends Configured implements Tool {
 
   private static final String ADD_QUEUES_OPTION = "addQueues";
@@ -135,7 +135,7 @@ public class SchedConfCLI extends Configured implements Tool {
     WebResource webResource = webServiceClient.resource(WebAppUtils.
         getRMWebAppURLWithScheme(getConf()));
     ClientResponse response = webResource.path("ws").path("v1").path("cluster")
-        .path("sched-conf").accept(MediaType.APPLICATION_JSON)
+        .path("scheduler-conf").accept(MediaType.APPLICATION_JSON)
         .entity(YarnWebServiceUtils.toJson(updateInfo,
             SchedConfUpdateInfo.class), MediaType.APPLICATION_JSON)
         .put(ClientResponse.class);
@@ -170,7 +170,7 @@ public class SchedConfCLI extends Configured implements Tool {
     if (args == null) {
       return;
     }
-    List<String> queuesToRemove = Arrays.asList(args.split(","));
+    List<String> queuesToRemove = Arrays.asList(args.split(";"));
     updateInfo.setRemoveQueueInfo(new ArrayList<>(queuesToRemove));
   }
 
@@ -199,11 +199,14 @@ public class SchedConfCLI extends Configured implements Tool {
   }
 
   private QueueConfigInfo getQueueConfigInfo(String arg) {
-    String[] queueArgs = arg.split(",");
-    String queuePath = queueArgs[0];
+    String[] args = arg.split(":");
+    String queuePath = args[0];
     Map<String, String> queueConfigs = new HashMap<>();
-    for (int i = 1; i < queueArgs.length; ++i) {
-      putKeyValuePair(queueConfigs, queueArgs[i]);
+    if (args.length > 1) {
+      String[] queueArgs = args[1].split(",");
+      for (int i = 0; i < queueArgs.length; ++i) {
+        putKeyValuePair(queueConfigs, queueArgs[i]);
+      }
     }
     return new QueueConfigInfo(queuePath, queueConfigs);
   }
@@ -228,11 +231,24 @@ public class SchedConfCLI extends Configured implements Tool {
   }
 
   private void printUsage() {
-    System.out.println("yarn schedconf [-add queueAddPath1,confKey1=confVal1,"
-        + "confKey2=confVal2;queueAddPath2,confKey3=confVal3] "
-        + "[-remove queueRemovePath1,queueRemovePath2] "
-        + "[-update queueUpdatePath1,confKey1=confVal1] "
+    System.out.println("yarn schedulerconf [-add "
+        + "\"queueAddPath1:confKey1=confVal1,confKey2=confVal2;"
+        + "queueAddPath2:confKey3=confVal3\"] "
+        + "[-remove \"queueRemovePath1;queueRemovePath2\"] "
+        + "[-update \"queueUpdatePath1:confKey1=confVal1\"] "
         + "[-global globalConfKey1=globalConfVal1,"
-        + "globalConfKey2=globalConfVal2]");
+        + "globalConfKey2=globalConfVal2]\n"
+        + "Example (adding queues): yarn schedulerconf -add "
+        + "\"root.a.a1:capacity=100,maximum-capacity=100;root.a.a2:capacity=0,"
+        + "maximum-capacity=0\"\n"
+        + "Example (removing queues): yarn schedulerconf -remove \"root.a.a1;"
+        + "root.a.a2\"\n"
+        + "Example (updating queues): yarn schedulerconf -update \"root.a.a1"
+        + ":capacity=25,maximum-capacity=25;root.a.a2:capacity=75,"
+        + "maximum-capacity=75\"\n"
+        + "Example (global scheduler update): yarn schedulerconf "
+        + "-global yarn.scheduler.capacity.maximum-applications=10000\n"
+        + "Note: This is an alpha feature, the syntax/options are subject to "
+        + "change, please run at your own risk.");
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4bbedd02/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
index d2f0639..5364e83 100644
--- 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
@@ -61,20 +61,20 @@ public class TestSchedConfCLI {
   @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"});
+    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"});
+    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"});
+    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"});
+    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."));
@@ -83,8 +83,7 @@ public class TestSchedConfCLI {
   @Test(timeout = 10000)
   public void testAddQueues() {
     SchedConfUpdateInfo schedUpdateInfo = new SchedConfUpdateInfo();
-    cli.addQueues("root.a,a1=aVal1,a2=aVal2," +
-        "a3=", schedUpdateInfo);
+    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();
@@ -94,7 +93,7 @@ public class TestSchedConfCLI {
     assertNull(params.get("a3"));
 
     schedUpdateInfo = new SchedConfUpdateInfo();
-    cli.addQueues("root.b,b1=bVal1;root.c,c1=cVal1", schedUpdateInfo);
+    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());
@@ -111,7 +110,7 @@ public class TestSchedConfCLI {
   @Test(timeout = 10000)
   public void testRemoveQueues() {
     SchedConfUpdateInfo schedUpdateInfo = new SchedConfUpdateInfo();
-    cli.removeQueues("root.a,root.b,root.c.c1", schedUpdateInfo);
+    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));
@@ -122,8 +121,7 @@ public class TestSchedConfCLI {
   @Test(timeout = 10000)
   public void testUpdateQueues() {
     SchedConfUpdateInfo schedUpdateInfo = new SchedConfUpdateInfo();
-    cli.updateQueues("root.a,a1=aVal1,a2=aVal2," +
-        "a3=", schedUpdateInfo);
+    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();
@@ -133,7 +131,7 @@ public class TestSchedConfCLI {
     assertNull(params.get("a3"));
 
     schedUpdateInfo = new SchedConfUpdateInfo();
-    cli.updateQueues("root.b,b1=bVal1;root.c,c1=cVal1", schedUpdateInfo);
+    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());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4bbedd02/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
index d1d91c2..6d4e0cf 100644
--- 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
@@ -34,7 +34,7 @@ import javax.xml.bind.annotation.XmlRootElement;
 @XmlAccessorType(XmlAccessType.FIELD)
 public class QueueConfigInfo {
 
-  @XmlElement(name = "queueName")
+  @XmlElement(name = "queue-name")
   private String queue;
 
   private HashMap<String, String> params = new HashMap<>();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4bbedd02/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
index bb84096..4546291 100644
--- 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
@@ -31,7 +31,7 @@ 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")
+@XmlRootElement(name = "sched-conf")
 @XmlAccessorType(XmlAccessType.FIELD)
 public class SchedConfUpdateInfo {
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4bbedd02/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 a6521de..985463d 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
@@ -3369,14 +3369,17 @@
 
   <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.
+      The type of configuration store to use for scheduler configurations.
+      Default is "file", which uses file based capacity-scheduler.xml to
+      retrieve and change scheduler configuration. To enable API based
+      scheduler configuration, use either "memory" (in memory storage, no
+      persistence across restarts), "leveldb" (leveldb based storage), or
+      "zk" (zookeeper based storage). API based configuration is only useful
+      when using a scheduler which supports mutable configuration. Currently
+      only capacity scheduler supports this.
     </description>
     <name>yarn.scheduler.configuration.store.class</name>
-    <value>memory</value>
+    <value>file</value>
   </property>
 
   <property>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4bbedd02/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 de95179..d91aa55 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
@@ -296,18 +296,20 @@ public class CapacityScheduler extends
     try {
       writeLock.lock();
       String confProviderStr = configuration.get(
-          CapacitySchedulerConfiguration.CS_CONF_PROVIDER,
-          CapacitySchedulerConfiguration.DEFAULT_CS_CONF_PROVIDER);
+          YarnConfiguration.SCHEDULER_CONFIGURATION_STORE_CLASS,
+          YarnConfiguration.DEFAULT_CONFIGURATION_STORE);
       switch (confProviderStr) {
-      case CapacitySchedulerConfiguration.FILE_CS_CONF_PROVIDER:
+      case YarnConfiguration.FILE_CONFIGURATION_STORE:
         this.csConfProvider =
             new FileBasedCSConfigurationProvider(rmContext);
         break;
-      case CapacitySchedulerConfiguration.STORE_CS_CONF_PROVIDER:
+      case YarnConfiguration.MEMORY_CONFIGURATION_STORE:
+      case YarnConfiguration.LEVELDB_CONFIGURATION_STORE:
+      case YarnConfiguration.ZK_CONFIGURATION_STORE:
         this.csConfProvider = new MutableCSConfigurationProvider(rmContext);
         break;
       default:
-        throw new IOException("Invalid CS configuration provider: " +
+        throw new IOException("Invalid configuration store class: " +
             confProviderStr);
       }
       this.csConfProvider.init(configuration);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4bbedd02/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 40cb893..3a519ec 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
@@ -315,18 +315,6 @@ 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 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/4bbedd02/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 ccadf76..40a19a4 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
@@ -66,7 +66,7 @@ public class MutableCSConfigurationProvider implements CSConfigurationProvider,
   public void init(Configuration config) throws IOException {
     String store = config.get(
         YarnConfiguration.SCHEDULER_CONFIGURATION_STORE_CLASS,
-        YarnConfiguration.DEFAULT_CONFIGURATION_STORE);
+        YarnConfiguration.MEMORY_CONFIGURATION_STORE);
     switch (store) {
     case YarnConfiguration.MEMORY_CONFIGURATION_STORE:
       this.confStore = new InMemoryConfigurationStore();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4bbedd02/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 d264c10..e6a0cae 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
@@ -2461,7 +2461,7 @@ public class RMWebServices extends WebServices implements RMWebServiceProtocol {
   }
 
   @PUT
-  @Path("/sched-conf")
+  @Path("/scheduler-conf")
   @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
       MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   @Consumes({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4bbedd02/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 620ca58..03fc081 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
@@ -199,8 +199,8 @@ public class TestRMAdminService {
 
   @Test
   public void testAdminRefreshQueuesWithMutableSchedulerConfiguration() {
-    configuration.set(CapacitySchedulerConfiguration.CS_CONF_PROVIDER,
-        CapacitySchedulerConfiguration.STORE_CS_CONF_PROVIDER);
+    configuration.set(YarnConfiguration.SCHEDULER_CONFIGURATION_STORE_CLASS,
+        YarnConfiguration.MEMORY_CONFIGURATION_STORE);
 
     try {
       rm = new MockRM(configuration);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4bbedd02/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/TestLeveldbConfigurationStore.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/TestLeveldbConfigurationStore.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/TestLeveldbConfigurationStore.java
index 779208a..324cbee 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/TestLeveldbConfigurationStore.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/TestLeveldbConfigurationStore.java
@@ -59,8 +59,6 @@ public class TestLeveldbConfigurationStore extends ConfigurationStoreBaseTest {
   public void setUp() throws Exception {
     super.setUp();
     FileUtil.fullyDelete(TEST_DIR);
-    conf.set(CapacitySchedulerConfiguration.CS_CONF_PROVIDER,
-        CapacitySchedulerConfiguration.STORE_CS_CONF_PROVIDER);
     conf.set(YarnConfiguration.SCHEDULER_CONFIGURATION_STORE_CLASS,
         YarnConfiguration.LEVELDB_CONFIGURATION_STORE);
     conf.set(YarnConfiguration.RM_SCHEDCONF_STORE_PATH, TEST_DIR.toString());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4bbedd02/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 9b080cd..5d43ebb 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,7 @@ 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.conf.YarnConfiguration;
 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;
@@ -80,6 +81,8 @@ public class TestMutableCSConfigurationProvider {
   @Test
   public void testInMemoryBackedProvider() throws Exception {
     Configuration conf = new Configuration();
+    conf.set(YarnConfiguration.SCHEDULER_CONFIGURATION_STORE_CLASS,
+        YarnConfiguration.MEMORY_CONFIGURATION_STORE);
     confProvider.init(conf);
     assertNull(confProvider.loadConfiguration(conf)
         .get("yarn.scheduler.capacity.root.a.goodKey"));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4bbedd02/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/TestZKConfigurationStore.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/TestZKConfigurationStore.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/TestZKConfigurationStore.java
index 3cfa8da..355f741 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/TestZKConfigurationStore.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/TestZKConfigurationStore.java
@@ -199,8 +199,6 @@ public class TestZKConfigurationStore extends ConfigurationStoreBaseTest {
     conf.setBoolean(YarnConfiguration.RM_HA_ENABLED, true);
     conf.set(YarnConfiguration.RM_HA_IDS, rmIds);
     conf.setBoolean(YarnConfiguration.RECOVERY_ENABLED, true);
-    conf.set(CapacitySchedulerConfiguration.CS_CONF_PROVIDER,
-        CapacitySchedulerConfiguration.STORE_CS_CONF_PROVIDER);
     conf.set(YarnConfiguration.SCHEDULER_CONFIGURATION_STORE_CLASS,
         YarnConfiguration.ZK_CONFIGURATION_STORE);
     conf.set(YarnConfiguration.RM_STORE, ZKRMStateStore.class.getName());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4bbedd02/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 26ef1b7..3d28f12 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
@@ -88,8 +88,8 @@ public class TestRMWebServicesConfigurationMutation extends JerseyTestBase {
       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.SCHEDULER_CONFIGURATION_STORE_CLASS,
+          YarnConfiguration.MEMORY_CONFIGURATION_STORE);
       conf.set(YarnConfiguration.YARN_ADMIN_ACL, userName);
       try {
         if (CONF_FILE.exists()) {
@@ -179,7 +179,7 @@ public class TestRMWebServicesConfigurationMutation extends JerseyTestBase {
     updateInfo.getAddQueueInfo().add(d);
     response =
         r.path("ws").path("v1").path("cluster")
-            .path("sched-conf").queryParam("user.name", userName)
+            .path("scheduler-conf").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON)
             .entity(YarnWebServiceUtils.toJson(updateInfo,
                 SchedConfUpdateInfo.class), MediaType.APPLICATION_JSON)
@@ -214,7 +214,7 @@ public class TestRMWebServicesConfigurationMutation extends JerseyTestBase {
     updateInfo.getUpdateQueueInfo().add(b);
     response =
         r.path("ws").path("v1").path("cluster")
-            .path("sched-conf").queryParam("user.name", userName)
+            .path("scheduler-conf").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON)
             .entity(YarnWebServiceUtils.toJson(updateInfo,
                 SchedConfUpdateInfo.class), MediaType.APPLICATION_JSON)
@@ -240,7 +240,7 @@ public class TestRMWebServicesConfigurationMutation extends JerseyTestBase {
     updateInfo.getRemoveQueueInfo().add("root.a.a2");
     response =
         r.path("ws").path("v1").path("cluster")
-            .path("sched-conf").queryParam("user.name", userName)
+            .path("scheduler-conf").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON)
             .entity(YarnWebServiceUtils.toJson(updateInfo,
                 SchedConfUpdateInfo.class), MediaType.APPLICATION_JSON)
@@ -265,7 +265,7 @@ public class TestRMWebServicesConfigurationMutation extends JerseyTestBase {
     updateInfo.getRemoveQueueInfo().add("root.c");
     response =
         r.path("ws").path("v1").path("cluster")
-            .path("sched-conf").queryParam("user.name", userName)
+            .path("scheduler-conf").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON)
             .entity(YarnWebServiceUtils.toJson(updateInfo,
                 SchedConfUpdateInfo.class), MediaType.APPLICATION_JSON)
@@ -296,7 +296,7 @@ public class TestRMWebServicesConfigurationMutation extends JerseyTestBase {
     updateInfo.getUpdateQueueInfo().add(b);
     response =
         r.path("ws").path("v1").path("cluster")
-            .path("sched-conf").queryParam("user.name", userName)
+            .path("scheduler-conf").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON)
             .entity(YarnWebServiceUtils.toJson(updateInfo,
                 SchedConfUpdateInfo.class), MediaType.APPLICATION_JSON)
@@ -328,7 +328,7 @@ public class TestRMWebServicesConfigurationMutation extends JerseyTestBase {
     updateInfo.getUpdateQueueInfo().add(configInfo);
     response =
         r.path("ws").path("v1").path("cluster")
-            .path("sched-conf").queryParam("user.name", userName)
+            .path("scheduler-conf").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON)
             .entity(YarnWebServiceUtils.toJson(updateInfo,
                 SchedConfUpdateInfo.class), MediaType.APPLICATION_JSON)
@@ -356,7 +356,7 @@ public class TestRMWebServicesConfigurationMutation extends JerseyTestBase {
     }
     response =
         r.path("ws").path("v1").path("cluster")
-            .path("sched-conf").queryParam("user.name", userName)
+            .path("scheduler-conf").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON)
             .entity(YarnWebServiceUtils.toJson(updateInfo,
                 SchedConfUpdateInfo.class), MediaType.APPLICATION_JSON)
@@ -391,7 +391,7 @@ public class TestRMWebServicesConfigurationMutation extends JerseyTestBase {
         0.001f);
     response =
         r.path("ws").path("v1").path("cluster")
-            .path("sched-conf").queryParam("user.name", userName)
+            .path("scheduler-conf").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON)
             .entity(YarnWebServiceUtils.toJson(updateInfo,
                 SchedConfUpdateInfo.class), MediaType.APPLICATION_JSON)
@@ -409,7 +409,7 @@ public class TestRMWebServicesConfigurationMutation extends JerseyTestBase {
     updateInfo.getUpdateQueueInfo().add(aUpdateInfo);
     response =
         r.path("ws").path("v1").path("cluster")
-            .path("sched-conf").queryParam("user.name", userName)
+            .path("scheduler-conf").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON)
             .entity(YarnWebServiceUtils.toJson(updateInfo,
                 SchedConfUpdateInfo.class), MediaType.APPLICATION_JSON)
@@ -439,7 +439,7 @@ public class TestRMWebServicesConfigurationMutation extends JerseyTestBase {
 
     response =
         r.path("ws").path("v1").path("cluster")
-            .path("sched-conf").queryParam("user.name", userName)
+            .path("scheduler-conf").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON)
             .entity(YarnWebServiceUtils.toJson(updateInfo,
                 SchedConfUpdateInfo.class), MediaType.APPLICATION_JSON)
@@ -464,7 +464,7 @@ public class TestRMWebServicesConfigurationMutation extends JerseyTestBase {
 
     response =
         r.path("ws").path("v1").path("cluster")
-            .path("sched-conf").queryParam("user.name", userName)
+            .path("scheduler-conf").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON)
             .entity(YarnWebServiceUtils.toJson(updateInfo,
                 SchedConfUpdateInfo.class), MediaType.APPLICATION_JSON)
@@ -479,7 +479,7 @@ public class TestRMWebServicesConfigurationMutation extends JerseyTestBase {
     // Unset maximum-applications. Should be set to default.
     response =
         r.path("ws").path("v1").path("cluster")
-            .path("sched-conf").queryParam("user.name", userName)
+            .path("scheduler-conf").queryParam("user.name", userName)
             .accept(MediaType.APPLICATION_JSON)
             .entity(YarnWebServiceUtils.toJson(updateInfo,
                 SchedConfUpdateInfo.class), MediaType.APPLICATION_JSON)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4bbedd02/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/CapacityScheduler.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/CapacityScheduler.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/CapacityScheduler.md
index d70f891..17a6ab9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/CapacityScheduler.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/CapacityScheduler.md
@@ -295,7 +295,7 @@ The `ReservationSystem` is integrated with the `CapacityScheduler` queue hierach
 Changing Queue Configuration
 ----------------------------
 
-Changing queue/scheduler properties and adding/removing queues can be done in two ways, via file or via API.
+Changing queue/scheduler properties and adding/removing queues can be done in two ways, via file or via API. This behavior can be changed via `yarn.scheduler.configuration.store.class` in yarn-site.xml. Possible values are *file*, which allows modifying properties via file; *memory*, which allows modifying properties via API, but does not persist changes across restart; *leveldb*, which allows modifying properties via API and stores changes in leveldb backing store; and *zk*, which allows modifying properties via API and stores changes in zookeeper backing store. The default value is *file*.
 
 ### Changing queue configuration via file
 
@@ -308,16 +308,17 @@ Changing queue/scheduler properties and adding/removing queues can be done in tw
 
   Editing by API uses a backing store for the scheduler configuration. To enable this, the following parameters can be configured in yarn-site.xml.
 
+  **Note:** This feature is in alpha phase and is subject to change.
+
   | Property | Description |
   |:---- |:---- |
-  | `yarn.scheduler.capacity.configuration.provider` | The type of configuration provider to use for capacity scheduler. To enable changing queue configuration via API, this should be set to *store*. Default value is *file*, which disables the API and reverts back to changing queue configuration via file. |
-  | `yarn.scheduler.configuration.store.class` | The type of backing store to use. Default value is *memory*, which stores the scheduler configuration in memory (and does not persist configuration changes across restarts). Other values are *leveldb* (using a leveldb-based implementation), and *zk* (using a zookeeper-based implementation). |
+  | `yarn.scheduler.configuration.store.class` | The type of backing store to use, as described [above](CapacityScheduler.html#Changing_Queue_Configuration). |
   | `yarn.scheduler.configuration.mutation.acl-policy.class` | An ACL policy can be configured to restrict which users can modify which queues. Default value is *org.apache.hadoop.yarn.server.resourcemanager.scheduler.DefaultConfigurationMutationACLPolicy*, which only allows YARN admins to make any configuration modifications. Another value is *org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.conf.QueueAdminConfigurationMutationACLPolicy*, which only allows queue modifications if the caller is an admin of the queue. |
   | `yarn.scheduler.configuration.store.max-logs` | Configuration changes are audit logged in the backing store, if using leveldb or zookeeper. This configuration controls the maximum number of audit logs to store, dropping the oldest logs when exceeded. Default is 1000. |
   | `yarn.scheduler.configuration.leveldb-store.path` | The storage path of the configuration store when using leveldb. Default value is *${hadoop.tmp.dir}/yarn/system/confstore*. |
   | `yarn.scheduler.configuration.leveldb-store.compaction-interval-secs` | The interval for compacting the configuration store in seconds, when using leveldb. Default value is 86400, or one day. |
   | `yarn.scheduler.configuration.zk-store.parent-path` | The zookeeper root node path for configuration store related information, when using zookeeper. Default value is */confstore*. |
 
-  **Note:** When enabling backing store for scheduler configuration, *yarn rmadmin -refreshQueues* will be disabled, i.e. it will no longer be possible to update configuration via file.
+  **Note:** When enabling scheduler configuration mutations via `yarn.scheduler.configuration.store.class`, *yarn rmadmin -refreshQueues* will be disabled, i.e. it will no longer be possible to update configuration via file.
 
-  See the [YARN Resource Manager REST API](ResourceManagerRest.html#Scheduler_Configuration_Mutation_API) for examples on how to change scheduler configuration via REST, and [YARN Commands Reference](YarnCommands.html#schedconf) for examples on how to change scheduler configuration via command line.
+  See the [YARN Resource Manager REST API](ResourceManagerRest.html#Scheduler_Configuration_Mutation_API) for examples on how to change scheduler configuration via REST, and [YARN Commands Reference](YarnCommands.html#schedulerconf) for examples on how to change scheduler configuration via command line.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4bbedd02/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md
index 025e9a7..f3a1907 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md
@@ -4436,16 +4436,18 @@ Scheduler Configuration Mutation API
 
 The scheduler configuration mutation API provides a way to modify scheduler/queue configuration and queue hierarchy.
 
+Please note that this feature is currently in the alpha stage and is subject to change.
+
 
 ### URI
 
-      * http://rm-http-address:port/ws/v1/cluster/sched-conf
+      * http://rm-http-address:port/ws/v1/cluster/scheduler-conf
 
 ### HTTP Operations Supported
 
       * PUT
 
-### Elements of the *schedConf* object
+### Elements of the *sched-conf* object
 
 | Item | Data Type | Description |
 |:---- |:---- |:---- |
@@ -4466,7 +4468,7 @@ Request for updating queue configurations.
 
 | Item | Data Type | Description |
 |:---- |:---- |:---- |
-| queueName | string | Full path name of the queue to update |
+| queue-name | string | Full path name of the queue to update |
 | params | map | A map of key value configuration pairs to update for this queue |
 
 Assuming we are using the capacity scheduler and the current queue configuration is a single queue *root.default*, this example sets *root.default*'s maximum applications to 100 and its minimum user limit percent to 10.
@@ -4475,12 +4477,12 @@ HTTP Request:
 
 ```xml
       Accept: application/xml
-      PUT http://rm-http-address:port/ws/v1/cluster/sched-conf
+      PUT http://rm-http-address:port/ws/v1/cluster/scheduler-conf
       Content-Type: application/xml
       <?xml version="1.0" encoding="UTF-8" standalone="yes"?>
-      <schedConf>
+      <sched-conf>
         <update-queue>
-          <queueName>root.default</queueName>
+          <queue-name>root.default</queue-name>
           <params>
             <entry>
               <key>maximum-applications</key>
@@ -4492,7 +4494,7 @@ HTTP Request:
             </entry>
           </params>
         </update-queue>
-      </schedConf>
+      </sched-conf>
 ```
 
 
@@ -4511,7 +4513,7 @@ Request for adding queues/updating queue configurations.
 
 | Item | Data Type | Description |
 |:---- |:---- |:---- |
-| queueName | string | Full path name of the queue to add |
+| queue-name | string | Full path name of the queue to add |
 | params | map | A map of key value configuration pairs to set for this queue |
 
 Assuming we are using the capacity scheduler and the current queue configuration is a single queue *root.default*, this example adds a queue *root.a* with capacity/maximum-capacity 10, and adjusts *root.default*'s capacity/maximum-capacity to 90. (More complex examples include adding a queue whose parent is also being added in the same request, or adding multiple sibling queues.)
@@ -4520,12 +4522,12 @@ HTTP Request:
 
 ```xml
       Accept: application/xml
-      PUT http://rm-http-address:port/ws/v1/cluster/sched-conf
+      PUT http://rm-http-address:port/ws/v1/cluster/scheduler-conf
       Content-Type: application/xml
       <?xml version="1.0" encoding="UTF-8" standalone="yes"?>
-      <schedConf>
+      <sched-conf>
         <add-queue>
-          <queueName>root.a</queueName>
+          <queue-name>root.a</queue-name>
           <params>
             <entry>
               <key>capacity</key>
@@ -4538,7 +4540,7 @@ HTTP Request:
           </params>
         </add-queue>
         <update-queue>
-          <queueName>root.default</queueName>
+          <queue-name>root.default</queue-name>
           <params>
             <entry>
               <key>capacity</key>
@@ -4550,7 +4552,7 @@ HTTP Request:
             </entry>
           </params>
         </update-queue>
-      </schedConf>
+      </sched-conf>
 ```
 
 
@@ -4573,13 +4575,13 @@ HTTP Request:
 
 ```xml
       Accept: application/xml
-      PUT http://rm-http-address:port/ws/v1/cluster/sched-conf
+      PUT http://rm-http-address:port/ws/v1/cluster/scheduler-conf
       Content-Type: application/xml
       <?xml version="1.0" encoding="UTF-8" standalone="yes"?>
-      <schedConf>
+      <sched-conf>
         <remove-queue>root.a</remove-queue>
         <remove-queue>root.b</remove-queue>
-      </schedConf>
+      </sched-conf>
 ```
 
 
@@ -4597,17 +4599,17 @@ HTTP Request:
 
 ```xml
       Accept: application/xml
-      PUT http://rm-http-address:port/ws/v1/cluster/sched-conf
+      PUT http://rm-http-address:port/ws/v1/cluster/scheduler-conf
       Content-Type: application/xml
       <?xml version="1.0" encoding="UTF-8" standalone="yes"?>
-      <schedConf>
+      <sched-conf>
         <global-updates>
           <entry>
             <key>yarn.scheduler.capacity.queue-mappings-override.enable</key>
             <value>true</value>
           </entry>
         </global-updates>
-      </schedConf>
+      </sched-conf>
 ```
 
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4bbedd02/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/YarnCommands.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/YarnCommands.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/YarnCommands.md
index 2fc65bc..fa33b8b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/YarnCommands.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/YarnCommands.md
@@ -237,18 +237,18 @@ Usage:
 
 Runs ResourceManager admin client
 
-### schedconf
+### schedulerconf
 
-Usage: `yarn schedconf [options]`
+Usage: `yarn schedulerconf [options]`
 
 | COMMAND\_OPTIONS | Description |
 |:---- |:---- |
-| -add <queuePath1,key1=val1,key2=val2;queuePath2,key3=val3> | Semicolon separated values of queues to add and their queue configurations. This example adds queue "queuePath1" (a full path name), which has queue configurations key1=val1 and key2=val2. It also adds queue "queuePath2", which has queue configuration key3=val3. |
-| -remove <queuePath1,queuePath2> | Comma-separated queues to remove. This example removes queuePath1 and queuePath2 queues (full path names). **Note:** Queues must be put into `STOPPED` state before they are deleted. |
-| -update <queuePath1,key1=val1,key2=val2;queuePath2,key3=val3> | Semicolon separated values of queues whose configurations should be updated. This example sets key1=val1 and key2=val2 for queue configuration of queuePath1 (full path name), and sets key3=val3 for queue configuration of queuePath2. |
+| -add <"queuePath1:key1=val1,key2=val2;queuePath2:key3=val3"> | Semicolon separated values of queues to add and their queue configurations. This example adds queue "queuePath1" (a full path name), which has queue configurations key1=val1 and key2=val2. It also adds queue "queuePath2", which has queue configuration key3=val3. |
+| -remove <"queuePath1;queuePath2"> | Semicolon separated queues to remove. This example removes queuePath1 and queuePath2 queues (full path names). **Note:** Queues must be put into `STOPPED` state before they are deleted. |
+| -update <"queuePath1:key1=val1,key2=val2;queuePath2:key3=val3"> | Semicolon separated values of queues whose configurations should be updated. This example sets key1=val1 and key2=val2 for queue configuration of queuePath1 (full path name), and sets key3=val3 for queue configuration of queuePath2. |
 | -global <key1=val1,key2=val2> | Update scheduler global configurations. This example sets key1=val1 and key2=val2 for scheduler's global configuration. |
 
-Updates scheduler configuration
+Updates scheduler configuration. Note, this feature is in alpha phase and is subject to change.
 
 ### scmadmin
 


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


[20/50] [abbrv] hadoop git commit: YARN-7240. Add more states and transitions to stabilize the NM Container state machine. (Kartheek Muthyala via asuresh)

Posted by jh...@apache.org.
YARN-7240. Add more states and transitions to stabilize the NM Container state machine. (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/df800f6c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/df800f6c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/df800f6c

Branch: refs/heads/YARN-5734
Commit: df800f6cf3ea663daf4081ebe784808b08d9366d
Parents: 47011d7
Author: Arun Suresh <as...@apache.org>
Authored: Mon Sep 25 14:11:55 2017 -0700
Committer: Arun Suresh <as...@apache.org>
Committed: Mon Sep 25 16:02:55 2017 -0700

----------------------------------------------------------------------
 .../containermanager/ContainerManagerImpl.java  |  41 +---
 .../container/ContainerEventType.java           |   6 +-
 .../container/ContainerImpl.java                | 174 ++++++++++++--
 .../container/ContainerState.java               |   3 +-
 .../container/UpdateContainerTokenEvent.java    |  86 +++++++
 .../scheduler/ContainerScheduler.java           | 114 ++++-----
 .../UpdateContainerSchedulerEvent.java          |  46 ++--
 .../BaseContainerManagerTest.java               |   2 +
 .../containermanager/TestContainerManager.java  | 229 ++++++++++++++++++-
 .../TestContainerSchedulerQueuing.java          | 101 ++++++++
 10 files changed, 660 insertions(+), 142 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/df800f6c/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 e497f62..d12892e 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
@@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.server.nodemanager.containermanager;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.protobuf.ByteString;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.UpdateContainerTokenEvent;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
@@ -144,7 +145,6 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.Contai
 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;
@@ -1251,29 +1251,6 @@ public class ContainerManagerImpl extends CompositeService implements
           + " [" + containerTokenIdentifier.getVersion() + "]");
     }
 
-    // Check container state
-    org.apache.hadoop.yarn.server.nodemanager.
-        containermanager.container.ContainerState currentState =
-        container.getContainerState();
-    EnumSet<org.apache.hadoop.yarn.server.nodemanager.containermanager
-        .container.ContainerState> allowedStates = EnumSet.of(
-        org.apache.hadoop.yarn.server.nodemanager.containermanager.container
-            .ContainerState.RUNNING,
-        org.apache.hadoop.yarn.server.nodemanager.containermanager.container
-            .ContainerState.SCHEDULED,
-        org.apache.hadoop.yarn.server.nodemanager.containermanager.container
-            .ContainerState.LOCALIZING,
-        org.apache.hadoop.yarn.server.nodemanager.containermanager.container
-            .ContainerState.REINITIALIZING,
-        org.apache.hadoop.yarn.server.nodemanager.containermanager.container
-            .ContainerState.RELAUNCHING);
-    if (!allowedStates.contains(currentState)) {
-      throw RPCUtil.getRemoteException("Container " + containerId.toString()
-          + " is in " + currentState.name() + " state."
-          + " Resource can only be changed when a container is in"
-          + " RUNNING or SCHEDULED state");
-    }
-
     // Check validity of the target resource.
     Resource currentResource = container.getResource();
     ExecutionType currentExecType =
@@ -1313,11 +1290,11 @@ public class ContainerManagerImpl extends CompositeService implements
     this.readLock.lock();
     try {
       if (!serviceStopped) {
-        // Dispatch message to ContainerScheduler to actually
+        // Dispatch message to Container to actually
         // make the change.
-        dispatcher.getEventHandler().handle(new UpdateContainerSchedulerEvent(
-            container, containerTokenIdentifier, isResourceChange,
-            isExecTypeUpdate, isIncrease));
+        dispatcher.getEventHandler().handle(new UpdateContainerTokenEvent(
+            container.getContainerId(), containerTokenIdentifier,
+            isResourceChange, isExecTypeUpdate, isIncrease));
       } else {
         throw new YarnException(
             "Unable to change container resource as the NodeManager is "
@@ -1816,10 +1793,14 @@ public class ContainerManagerImpl extends CompositeService implements
     if (container == null) {
       throw new YarnException("Specified " + containerId + " does not exist!");
     }
-    if (!container.isRunning() || container.isReInitializing()) {
+    if (!container.isRunning() || container.isReInitializing()
+        || container.getContainerTokenIdentifier().getExecutionType()
+        == ExecutionType.OPPORTUNISTIC) {
       throw new YarnException("Cannot perform " + op + " on [" + containerId
           + "]. Current state is [" + container.getContainerState() + ", " +
-          "isReInitializing=" + container.isReInitializing() + "].");
+          "isReInitializing=" + container.isReInitializing() + "]. Container"
+          + " Execution Type is [" + container.getContainerTokenIdentifier()
+          .getExecutionType() + "].");
     }
     return container;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df800f6c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerEventType.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerEventType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerEventType.java
index 1475435..e28b37d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerEventType.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerEventType.java
@@ -29,6 +29,7 @@ public enum ContainerEventType {
   ROLLBACK_REINIT,
   PAUSE_CONTAINER,
   RESUME_CONTAINER,
+  UPDATE_CONTAINER_TOKEN,
 
   // DownloadManager
   CONTAINER_INITED,
@@ -42,5 +43,8 @@ public enum ContainerEventType {
   CONTAINER_EXITED_WITH_FAILURE,
   CONTAINER_KILLED_ON_REQUEST,
   CONTAINER_PAUSED,
-  CONTAINER_RESUMED
+  CONTAINER_RESUMED,
+
+  // Producer: ContainerScheduler
+  CONTAINER_TOKEN_UPDATED
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df800f6c/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 836e70e..705087b 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
@@ -33,6 +33,8 @@ import java.util.Set;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler.UpdateContainerSchedulerEvent;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -308,8 +310,8 @@ public class ContainerImpl implements Container {
         UPDATE_DIAGNOSTICS_TRANSITION)
     .addTransition(ContainerState.NEW, ContainerState.DONE,
         ContainerEventType.KILL_CONTAINER, new KillOnNewTransition())
-    .addTransition(ContainerState.NEW, ContainerState.DONE,
-            ContainerEventType.PAUSE_CONTAINER, new KillOnPauseTransition())
+    .addTransition(ContainerState.NEW, ContainerState.NEW,
+        ContainerEventType.UPDATE_CONTAINER_TOKEN, new UpdateTransition())
 
     // From LOCALIZING State
     .addTransition(ContainerState.LOCALIZING,
@@ -325,8 +327,9 @@ public class ContainerImpl implements Container {
     .addTransition(ContainerState.LOCALIZING, ContainerState.KILLING,
         ContainerEventType.KILL_CONTAINER,
         new KillBeforeRunningTransition())
-    .addTransition(ContainerState.LOCALIZING, ContainerState.KILLING,
-        ContainerEventType.PAUSE_CONTAINER, new KillOnPauseTransition())
+    .addTransition(ContainerState.LOCALIZING, ContainerState.LOCALIZING,
+        ContainerEventType.UPDATE_CONTAINER_TOKEN, new UpdateTransition())
+
 
     // From LOCALIZATION_FAILED State
     .addTransition(ContainerState.LOCALIZATION_FAILED,
@@ -351,6 +354,9 @@ public class ContainerImpl implements Container {
     .addTransition(ContainerState.LOCALIZATION_FAILED,
         ContainerState.LOCALIZATION_FAILED,
         ContainerEventType.RESOURCE_FAILED)
+    .addTransition(ContainerState.LOCALIZATION_FAILED,
+        ContainerState.LOCALIZATION_FAILED,
+        ContainerEventType.UPDATE_CONTAINER_TOKEN, new UpdateTransition())
 
     // From SCHEDULED State
     .addTransition(ContainerState.SCHEDULED, ContainerState.RUNNING,
@@ -364,6 +370,9 @@ public class ContainerImpl implements Container {
     .addTransition(ContainerState.SCHEDULED, ContainerState.KILLING,
         ContainerEventType.KILL_CONTAINER,
         new KillBeforeRunningTransition())
+    .addTransition(ContainerState.SCHEDULED, ContainerState.SCHEDULED,
+        ContainerEventType.UPDATE_CONTAINER_TOKEN,
+        new NotifyContainerSchedulerOfUpdateTransition())
 
     // From RUNNING State
     .addTransition(ContainerState.RUNNING,
@@ -376,10 +385,16 @@ public class ContainerImpl implements Container {
             ContainerState.EXITED_WITH_FAILURE),
         ContainerEventType.CONTAINER_EXITED_WITH_FAILURE,
         new RetryFailureTransition())
-    .addTransition(ContainerState.RUNNING, ContainerState.REINITIALIZING,
+    .addTransition(ContainerState.RUNNING,
+        EnumSet.of(ContainerState.RUNNING,
+            ContainerState.REINITIALIZING,
+            ContainerState.REINITIALIZING_AWAITING_KILL),
         ContainerEventType.REINITIALIZE_CONTAINER,
         new ReInitializeContainerTransition())
-    .addTransition(ContainerState.RUNNING, ContainerState.REINITIALIZING,
+    .addTransition(ContainerState.RUNNING,
+        EnumSet.of(ContainerState.RUNNING,
+            ContainerState.REINITIALIZING,
+            ContainerState.REINITIALIZING_AWAITING_KILL),
         ContainerEventType.ROLLBACK_REINIT,
         new RollbackContainerTransition())
     .addTransition(ContainerState.RUNNING, ContainerState.RUNNING,
@@ -398,9 +413,16 @@ public class ContainerImpl implements Container {
         ContainerEventType.CONTAINER_KILLED_ON_REQUEST,
         new KilledExternallyTransition())
     .addTransition(ContainerState.RUNNING, ContainerState.PAUSING,
-    ContainerEventType.PAUSE_CONTAINER, new PauseContainerTransition())
+        ContainerEventType.PAUSE_CONTAINER, new PauseContainerTransition())
+    .addTransition(ContainerState.RUNNING, ContainerState.RUNNING,
+        ContainerEventType.UPDATE_CONTAINER_TOKEN,
+        new NotifyContainerSchedulerOfUpdateTransition())
+
 
     // From PAUSING State
+    .addTransition(ContainerState.PAUSING, ContainerState.PAUSING,
+        ContainerEventType.RESOURCE_LOCALIZED,
+        new ResourceLocalizedWhileRunningTransition())
     .addTransition(ContainerState.PAUSING, ContainerState.KILLING,
         ContainerEventType.KILL_CONTAINER, new KillTransition())
     .addTransition(ContainerState.PAUSING, ContainerState.PAUSING,
@@ -420,6 +442,12 @@ public class ContainerImpl implements Container {
     .addTransition(ContainerState.PAUSING, ContainerState.EXITED_WITH_FAILURE,
         ContainerEventType.CONTAINER_KILLED_ON_REQUEST,
         new KilledExternallyTransition())
+    .addTransition(ContainerState.PAUSING, ContainerState.PAUSING,
+        ContainerEventType.RESOURCE_LOCALIZED,
+        new ResourceLocalizedWhileRunningTransition())
+    .addTransition(ContainerState.PAUSING, ContainerState.PAUSING,
+        ContainerEventType.UPDATE_CONTAINER_TOKEN,
+        new NotifyContainerSchedulerOfUpdateTransition())
 
     // From PAUSED State
     .addTransition(ContainerState.PAUSED, ContainerState.KILLING,
@@ -429,6 +457,10 @@ public class ContainerImpl implements Container {
         UPDATE_DIAGNOSTICS_TRANSITION)
     .addTransition(ContainerState.PAUSED, ContainerState.PAUSED,
         ContainerEventType.PAUSE_CONTAINER)
+    // This can happen during re-initialization.
+    .addTransition(ContainerState.PAUSED, ContainerState.PAUSED,
+        ContainerEventType.RESOURCE_LOCALIZED,
+        new ResourceLocalizedWhileRunningTransition())
     .addTransition(ContainerState.PAUSED, ContainerState.RESUMING,
         ContainerEventType.RESUME_CONTAINER, new ResumeContainerTransition())
     // In case something goes wrong then container will exit from the
@@ -444,6 +476,9 @@ public class ContainerImpl implements Container {
         ContainerState.EXITED_WITH_SUCCESS,
         ContainerEventType.CONTAINER_EXITED_WITH_SUCCESS,
         new ExitedWithSuccessTransition(true))
+    .addTransition(ContainerState.PAUSED, ContainerState.PAUSED,
+        ContainerEventType.UPDATE_CONTAINER_TOKEN,
+        new NotifyContainerSchedulerOfUpdateTransition())
 
     // From RESUMING State
     .addTransition(ContainerState.RESUMING, ContainerState.KILLING,
@@ -453,6 +488,10 @@ public class ContainerImpl implements Container {
     .addTransition(ContainerState.RESUMING, ContainerState.RESUMING,
         ContainerEventType.UPDATE_DIAGNOSTICS_MSG,
         UPDATE_DIAGNOSTICS_TRANSITION)
+    // This can happen during re-initialization
+    .addTransition(ContainerState.RESUMING, ContainerState.RESUMING,
+        ContainerEventType.RESOURCE_LOCALIZED,
+        new ResourceLocalizedWhileRunningTransition())
     // In case something goes wrong then container will exit from the
     // RESUMING state
     .addTransition(ContainerState.RESUMING,
@@ -467,6 +506,10 @@ public class ContainerImpl implements Container {
         ContainerState.EXITED_WITH_SUCCESS,
         ContainerEventType.CONTAINER_EXITED_WITH_SUCCESS,
         new ExitedWithSuccessTransition(true))
+    .addTransition(ContainerState.RESUMING, ContainerState.RESUMING,
+        ContainerEventType.UPDATE_CONTAINER_TOKEN,
+        new NotifyContainerSchedulerOfUpdateTransition())
+    // NOTE - We cannot get a PAUSE_CONTAINER while in RESUMING state.
 
     // From REINITIALIZING State
     .addTransition(ContainerState.REINITIALIZING,
@@ -478,7 +521,8 @@ public class ContainerImpl implements Container {
         ContainerEventType.CONTAINER_EXITED_WITH_FAILURE,
         new ExitedWithFailureTransition(true))
     .addTransition(ContainerState.REINITIALIZING,
-        ContainerState.REINITIALIZING,
+        EnumSet.of(ContainerState.REINITIALIZING,
+            ContainerState.REINITIALIZING_AWAITING_KILL),
         ContainerEventType.RESOURCE_LOCALIZED,
         new ResourceLocalizedWhileReInitTransition())
     .addTransition(ContainerState.REINITIALIZING, ContainerState.RUNNING,
@@ -490,12 +534,39 @@ public class ContainerImpl implements Container {
         UPDATE_DIAGNOSTICS_TRANSITION)
     .addTransition(ContainerState.REINITIALIZING, ContainerState.KILLING,
         ContainerEventType.KILL_CONTAINER, new KillTransition())
-    .addTransition(ContainerState.REINITIALIZING, ContainerState.KILLING,
-        ContainerEventType.PAUSE_CONTAINER, new KillOnPauseTransition())
+    .addTransition(ContainerState.REINITIALIZING, ContainerState.PAUSING,
+        ContainerEventType.PAUSE_CONTAINER, new PauseContainerTransition())
     .addTransition(ContainerState.REINITIALIZING,
+        ContainerState.REINITIALIZING,
+        ContainerEventType.UPDATE_CONTAINER_TOKEN,
+        new NotifyContainerSchedulerOfUpdateTransition())
+
+    // from REINITIALIZING_AWAITING_KILL
+    .addTransition(ContainerState.REINITIALIZING_AWAITING_KILL,
+        ContainerState.EXITED_WITH_SUCCESS,
+        ContainerEventType.CONTAINER_EXITED_WITH_SUCCESS,
+        new ExitedWithSuccessTransition(true))
+    .addTransition(ContainerState.REINITIALIZING_AWAITING_KILL,
+        ContainerState.EXITED_WITH_FAILURE,
+        ContainerEventType.CONTAINER_EXITED_WITH_FAILURE,
+        new ExitedWithFailureTransition(true))
+    .addTransition(ContainerState.REINITIALIZING_AWAITING_KILL,
+        ContainerState.REINITIALIZING_AWAITING_KILL,
+        ContainerEventType.UPDATE_DIAGNOSTICS_MSG,
+        UPDATE_DIAGNOSTICS_TRANSITION)
+    .addTransition(ContainerState.REINITIALIZING_AWAITING_KILL,
+        ContainerState.KILLING,
+        ContainerEventType.KILL_CONTAINER, new KillTransition())
+    .addTransition(ContainerState.REINITIALIZING_AWAITING_KILL,
+        ContainerState.SCHEDULED, ContainerEventType.PAUSE_CONTAINER)
+    .addTransition(ContainerState.REINITIALIZING_AWAITING_KILL,
         ContainerState.SCHEDULED,
         ContainerEventType.CONTAINER_KILLED_ON_REQUEST,
         new KilledForReInitializationTransition())
+    .addTransition(ContainerState.REINITIALIZING_AWAITING_KILL,
+        ContainerState.REINITIALIZING_AWAITING_KILL,
+        ContainerEventType.UPDATE_CONTAINER_TOKEN,
+        new NotifyContainerSchedulerOfUpdateTransition())
 
     // From RELAUNCHING State
     .addTransition(ContainerState.RELAUNCHING, ContainerState.RUNNING,
@@ -511,6 +582,10 @@ public class ContainerImpl implements Container {
         ContainerEventType.KILL_CONTAINER, new KillTransition())
     .addTransition(ContainerState.RELAUNCHING, ContainerState.KILLING,
         ContainerEventType.PAUSE_CONTAINER, new KillOnPauseTransition())
+    .addTransition(ContainerState.RELAUNCHING, ContainerState.RELAUNCHING,
+        ContainerEventType.UPDATE_CONTAINER_TOKEN,
+        new NotifyContainerSchedulerOfUpdateTransition())
+
 
     // From CONTAINER_EXITED_WITH_SUCCESS State
     .addTransition(ContainerState.EXITED_WITH_SUCCESS, ContainerState.DONE,
@@ -524,6 +599,10 @@ public class ContainerImpl implements Container {
         ContainerState.EXITED_WITH_SUCCESS,
         EnumSet.of(ContainerEventType.KILL_CONTAINER,
             ContainerEventType.PAUSE_CONTAINER))
+    // No transition - assuming container is on its way to completion
+    .addTransition(ContainerState.EXITED_WITH_SUCCESS,
+        ContainerState.EXITED_WITH_SUCCESS,
+        ContainerEventType.UPDATE_CONTAINER_TOKEN)
 
     // From EXITED_WITH_FAILURE State
     .addTransition(ContainerState.EXITED_WITH_FAILURE, ContainerState.DONE,
@@ -537,6 +616,10 @@ public class ContainerImpl implements Container {
                    ContainerState.EXITED_WITH_FAILURE,
         EnumSet.of(ContainerEventType.KILL_CONTAINER,
             ContainerEventType.PAUSE_CONTAINER))
+    // No transition - assuming container is on its way to completion
+    .addTransition(ContainerState.EXITED_WITH_FAILURE,
+        ContainerState.EXITED_WITH_FAILURE,
+        ContainerEventType.UPDATE_CONTAINER_TOKEN)
 
     // From KILLING State.
     .addTransition(ContainerState.KILLING,
@@ -572,6 +655,9 @@ public class ContainerImpl implements Container {
         ContainerState.KILLING,
         EnumSet.of(ContainerEventType.CONTAINER_LAUNCHED,
             ContainerEventType.PAUSE_CONTAINER))
+    // No transition - assuming container is on its way to completion
+    .addTransition(ContainerState.KILLING, ContainerState.KILLING,
+        ContainerEventType.UPDATE_CONTAINER_TOKEN)
 
     // From CONTAINER_CLEANEDUP_AFTER_KILL State.
     .addTransition(ContainerState.CONTAINER_CLEANEDUP_AFTER_KILL,
@@ -589,6 +675,10 @@ public class ContainerImpl implements Container {
             ContainerEventType.CONTAINER_EXITED_WITH_SUCCESS,
             ContainerEventType.CONTAINER_EXITED_WITH_FAILURE,
             ContainerEventType.PAUSE_CONTAINER))
+    // No transition - assuming container is on its way to completion
+    .addTransition(ContainerState.CONTAINER_CLEANEDUP_AFTER_KILL,
+        ContainerState.CONTAINER_CLEANEDUP_AFTER_KILL,
+        ContainerEventType.UPDATE_CONTAINER_TOKEN)
 
     // From DONE
     .addTransition(ContainerState.DONE, ContainerState.DONE,
@@ -606,6 +696,9 @@ public class ContainerImpl implements Container {
         EnumSet.of(ContainerEventType.RESOURCE_FAILED,
             ContainerEventType.CONTAINER_EXITED_WITH_SUCCESS,
             ContainerEventType.CONTAINER_EXITED_WITH_FAILURE))
+    // No transition - assuming container is on its way to completion
+    .addTransition(ContainerState.DONE, ContainerState.DONE,
+        ContainerEventType.UPDATE_CONTAINER_TOKEN)
 
     // create the topology tables
     .installTopology();
@@ -626,6 +719,7 @@ public class ContainerImpl implements Container {
     case RUNNING:
     case RELAUNCHING:
     case REINITIALIZING:
+    case REINITIALIZING_AWAITING_KILL:
     case EXITED_WITH_SUCCESS:
     case EXITED_WITH_FAILURE:
     case KILLING:
@@ -929,6 +1023,45 @@ public class ContainerImpl implements Container {
 
   }
 
+  static class UpdateTransition extends ContainerTransition {
+    @Override
+    public void transition(
+        ContainerImpl container, ContainerEvent event) {
+      UpdateContainerTokenEvent updateEvent = (UpdateContainerTokenEvent)event;
+      // Update the container token
+      container.setContainerTokenIdentifier(updateEvent.getUpdatedToken());
+      if (updateEvent.isResourceChange()) {
+        try {
+          // Persist change in the state store.
+          container.context.getNMStateStore().storeContainerResourceChanged(
+              container.containerId,
+              container.getContainerTokenIdentifier().getVersion(),
+              container.getResource());
+        } catch (IOException e) {
+          LOG.warn("Could not store container [" + container.containerId
+              + "] resource change..", e);
+        }
+      }
+    }
+  }
+
+  static class NotifyContainerSchedulerOfUpdateTransition extends
+      UpdateTransition {
+    @Override
+    public void transition(
+        ContainerImpl container, ContainerEvent event) {
+
+      UpdateContainerTokenEvent updateEvent = (UpdateContainerTokenEvent)event;
+      // Save original token
+      ContainerTokenIdentifier originalToken =
+          container.containerTokenIdentifier;
+      super.transition(container, updateEvent);
+      container.dispatcher.getEventHandler().handle(
+          new UpdateContainerSchedulerEvent(container,
+              originalToken, updateEvent));
+    }
+  }
+
   /**
    * State transition when a NEW container receives the INIT_CONTAINER
    * message.
@@ -1074,12 +1207,15 @@ public class ContainerImpl implements Container {
   /**
    * Transition to start the Re-Initialization process.
    */
-  static class ReInitializeContainerTransition extends ContainerTransition {
+  static class ReInitializeContainerTransition implements
+      MultipleArcTransition<ContainerImpl, ContainerEvent, ContainerState> {
 
     @SuppressWarnings("unchecked")
     @Override
-    public void transition(ContainerImpl container, ContainerEvent event) {
+    public ContainerState transition(
+        ContainerImpl container, ContainerEvent event) {
       container.reInitContext = createReInitContext(container, event);
+      boolean resourcesPresent = false;
       try {
         // 'reInitContext.newResourceSet' can be
         // a) current container resourceSet (In case of Restart)
@@ -1101,6 +1237,7 @@ public class ContainerImpl implements Container {
           container.dispatcher.getEventHandler().handle(
               new ContainersLauncherEvent(container,
                   ContainersLauncherEventType.CLEANUP_CONTAINER_FOR_REINIT));
+          resourcesPresent = true;
         }
         container.metrics.reInitingContainer();
         NMAuditLogger.logSuccess(container.user,
@@ -1112,7 +1249,11 @@ public class ContainerImpl implements Container {
             " re-initialization failure..", e);
         container.addDiagnostics("Error re-initializing due to" +
             "[" + e.getMessage() + "]");
+        return ContainerState.RUNNING;
       }
+      return resourcesPresent ?
+          ContainerState.REINITIALIZING_AWAITING_KILL :
+          ContainerState.REINITIALIZING;
     }
 
     protected ReInitializationContext createReInitContext(
@@ -1164,11 +1305,14 @@ public class ContainerImpl implements Container {
    * If all dependencies are met, then restart Container with new bits.
    */
   static class ResourceLocalizedWhileReInitTransition
-      extends ContainerTransition {
+      implements MultipleArcTransition
+      <ContainerImpl, ContainerEvent, ContainerState> {
+
 
     @SuppressWarnings("unchecked")
     @Override
-    public void transition(ContainerImpl container, ContainerEvent event) {
+    public ContainerState transition(
+        ContainerImpl container, ContainerEvent event) {
       ContainerResourceLocalizedEvent rsrcEvent =
           (ContainerResourceLocalizedEvent) event;
       container.reInitContext.newResourceSet.resourceLocalized(
@@ -1180,7 +1324,9 @@ public class ContainerImpl implements Container {
         container.dispatcher.getEventHandler().handle(
             new ContainersLauncherEvent(container,
                 ContainersLauncherEventType.CLEANUP_CONTAINER_FOR_REINIT));
+        return ContainerState.REINITIALIZING_AWAITING_KILL;
       }
+      return ContainerState.REINITIALIZING;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df800f6c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerState.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerState.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerState.java
index 7c3fea8..5644d03 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerState.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerState.java
@@ -20,7 +20,8 @@ package org.apache.hadoop.yarn.server.nodemanager.containermanager.container;
 
 public enum ContainerState {
   NEW, LOCALIZING, LOCALIZATION_FAILED, SCHEDULED, RUNNING, RELAUNCHING,
-  REINITIALIZING, EXITED_WITH_SUCCESS, EXITED_WITH_FAILURE, KILLING,
+  REINITIALIZING, REINITIALIZING_AWAITING_KILL,
+  EXITED_WITH_SUCCESS, EXITED_WITH_FAILURE, KILLING,
   CONTAINER_CLEANEDUP_AFTER_KILL, CONTAINER_RESOURCES_CLEANINGUP, DONE,
   PAUSING, PAUSED, RESUMING
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df800f6c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/UpdateContainerTokenEvent.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/UpdateContainerTokenEvent.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/UpdateContainerTokenEvent.java
new file mode 100644
index 0000000..c9dc97e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/UpdateContainerTokenEvent.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
+ * <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.nodemanager.containermanager.container;
+
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
+
+/**
+ * Update Event consumed by the Container.
+ */
+public class UpdateContainerTokenEvent extends ContainerEvent {
+  private final ContainerTokenIdentifier updatedToken;
+  private final boolean isResourceChange;
+  private final boolean isExecTypeUpdate;
+  private final boolean isIncrease;
+
+  /**
+   * Create Update event.
+   *
+   * @param cID Container Id.
+   * @param updatedToken Updated Container Token.
+   * @param isResourceChange Is Resource change.
+   * @param isExecTypeUpdate Is ExecutionType Update.
+   * @param isIncrease Is container increase.
+   */
+  public UpdateContainerTokenEvent(ContainerId cID,
+      ContainerTokenIdentifier updatedToken, boolean isResourceChange,
+      boolean isExecTypeUpdate, boolean isIncrease) {
+    super(cID, ContainerEventType.UPDATE_CONTAINER_TOKEN);
+    this.updatedToken = updatedToken;
+    this.isResourceChange = isResourceChange;
+    this.isExecTypeUpdate = isExecTypeUpdate;
+    this.isIncrease = isIncrease;
+  }
+
+  /**
+   * Update Container Token.
+   *
+   * @return Container Token.
+   */
+  public ContainerTokenIdentifier getUpdatedToken() {
+    return updatedToken;
+  }
+
+  /**
+   * Is this update a ResourceChange.
+   *
+   * @return isResourceChange.
+   */
+  public boolean isResourceChange() {
+    return isResourceChange;
+  }
+
+  /**
+   * Is this update an ExecType Update.
+   *
+   * @return isExecTypeUpdate.
+   */
+  public boolean isExecTypeUpdate() {
+    return isExecTypeUpdate;
+  }
+
+  /**
+   * Is this a container Increase.
+   *
+   * @return isIncrease.
+   */
+  public boolean isIncrease() {
+    return isIncrease;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df800f6c/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 830a06d..e436822 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
@@ -33,6 +33,7 @@ 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.container.ContainerImpl;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor
     .ChangeMonitoringContainerResourceEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerState;
@@ -151,7 +152,9 @@ public class ContainerScheduler extends AbstractService implements
     case SCHEDULE_CONTAINER:
       scheduleContainer(event.getContainer());
       break;
+    // NOTE: Is sent only after container state has changed to PAUSED...
     case CONTAINER_PAUSED:
+    // NOTE: Is sent only after container state has changed to DONE...
     case CONTAINER_COMPLETED:
       onResourcesReclaimed(event.getContainer());
       break;
@@ -180,58 +183,38 @@ public class ContainerScheduler extends AbstractService implements
     if (updateEvent.isResourceChange()) {
       if (runningContainers.containsKey(containerId)) {
         this.utilizationTracker.subtractContainerResource(
-            updateEvent.getContainer());
-        updateEvent.getContainer().setContainerTokenIdentifier(
-            updateEvent.getUpdatedToken());
+            new ContainerImpl(getConfig(), null, null, null, null,
+                updateEvent.getOriginalToken(), context));
         this.utilizationTracker.addContainerResources(
             updateEvent.getContainer());
         getContainersMonitor().handle(
             new ChangeMonitoringContainerResourceEvent(containerId,
                 updateEvent.getUpdatedToken().getResource()));
-      } else {
-        // Is Queued or localizing..
-        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());
-          }
+      // 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/pause opportunistic containers if any to make room for
           // promotion request
           reclaimOpportunisticContainerResources(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());
-          }
+        }
+      } 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());
         }
       }
+      startPendingContainers(maxOppQueueLength <= 0);
     }
   }
 
@@ -290,6 +273,16 @@ public class ContainerScheduler extends AbstractService implements
       queuedGuaranteedContainers.remove(container.getContainerId());
     }
 
+    // Requeue PAUSED containers
+    if (container.getContainerState() == ContainerState.PAUSED) {
+      if (container.getContainerTokenIdentifier().getExecutionType() ==
+          ExecutionType.GUARANTEED) {
+        queuedGuaranteedContainers.put(container.getContainerId(), container);
+      } else {
+        queuedOpportunisticContainers.put(
+            container.getContainerId(), container);
+      }
+    }
     // decrement only if it was a running container
     Container completedContainer = runningContainers.remove(container
         .getContainerId());
@@ -301,7 +294,8 @@ public class ContainerScheduler extends AbstractService implements
           ExecutionType.OPPORTUNISTIC) {
         this.metrics.completeOpportunisticContainer(container.getResource());
       }
-      startPendingContainers(false);
+      boolean forceStartGuaranteedContainers = (maxOppQueueLength <= 0);
+      startPendingContainers(forceStartGuaranteedContainers);
     }
   }
 
@@ -311,26 +305,9 @@ public class ContainerScheduler extends AbstractService implements
    *        container without looking at available resource
    */
   private void startPendingContainers(boolean forceStartGuaranteedContaieners) {
-    // Start pending guaranteed containers, if resources available.
+    // Start guaranteed containers that are paused, if resources available.
     boolean resourcesAvailable = startContainers(
-        queuedGuaranteedContainers.values(), forceStartGuaranteedContaieners);
-    // Resume opportunistic containers, if resource available.
-    if (resourcesAvailable) {
-      List<Container> pausedContainers = new ArrayList<Container>();
-      Map<ContainerId, Container> containers =
-          context.getContainers();
-      for (Map.Entry<ContainerId, Container>entry : containers.entrySet()) {
-        ContainerId contId = entry.getKey();
-        // Find containers that were not already started and are in paused state
-        if(false == runningContainers.containsKey(contId)) {
-          if(containers.get(contId).getContainerState()
-              == ContainerState.PAUSED) {
-            pausedContainers.add(containers.get(contId));
-          }
-        }
-      }
-      resourcesAvailable = startContainers(pausedContainers, false);
-    }
+          queuedGuaranteedContainers.values(), forceStartGuaranteedContaieners);
     // Start opportunistic containers, if resources available.
     if (resourcesAvailable) {
       startContainers(queuedOpportunisticContainers.values(), false);
@@ -590,16 +567,19 @@ public class ContainerScheduler extends AbstractService implements
         queuedOpportunisticContainers.values().iterator();
     while (containerIter.hasNext()) {
       Container container = containerIter.next();
-      if (numAllowed <= 0) {
-        container.sendKillEvent(
-            ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER,
-            "Container De-queued to meet NM queuing limits.");
-        containerIter.remove();
-        LOG.info(
-            "Opportunistic container {} will be killed to meet NM queuing" +
-                " limits.", container.getContainerId());
+      // Do not shed PAUSED containers
+      if (container.getContainerState() != ContainerState.PAUSED) {
+        if (numAllowed <= 0) {
+          container.sendKillEvent(
+              ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER,
+              "Container De-queued to meet NM queuing limits.");
+          containerIter.remove();
+          LOG.info(
+              "Opportunistic container {} will be killed to meet NM queuing" +
+                  " limits.", container.getContainerId());
+        }
+        numAllowed--;
       }
-      numAllowed--;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df800f6c/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
index 5384b7e..2473982 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/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
@@ -21,33 +21,37 @@ 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;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.UpdateContainerTokenEvent;
+
 /**
  * Update Event consumed by the {@link ContainerScheduler}.
  */
 public class UpdateContainerSchedulerEvent extends ContainerSchedulerEvent {
 
-  private ContainerTokenIdentifier updatedToken;
-  private boolean isResourceChange;
-  private boolean isExecTypeUpdate;
-  private boolean isIncrease;
+  private final UpdateContainerTokenEvent containerEvent;
+  private final ContainerTokenIdentifier originalToken;
 
   /**
    * 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.
+   * @param container Container.
+   * @param origToken The Original Container Token.
+   * @param event The Container Event.
+   */
+  public UpdateContainerSchedulerEvent(Container container,
+      ContainerTokenIdentifier origToken, UpdateContainerTokenEvent event) {
+    super(container, ContainerSchedulerEventType.UPDATE_CONTAINER);
+    this.containerEvent = event;
+    this.originalToken = origToken;
+  }
+
+  /**
+   * Original Token before update.
+   *
+   * @return Container Token.
    */
-  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;
+  public ContainerTokenIdentifier getOriginalToken() {
+    return this.originalToken;
   }
 
   /**
@@ -56,7 +60,7 @@ public class UpdateContainerSchedulerEvent extends ContainerSchedulerEvent {
    * @return Container Token.
    */
   public ContainerTokenIdentifier getUpdatedToken() {
-    return updatedToken;
+    return containerEvent.getUpdatedToken();
   }
 
   /**
@@ -64,7 +68,7 @@ public class UpdateContainerSchedulerEvent extends ContainerSchedulerEvent {
    * @return isResourceChange.
    */
   public boolean isResourceChange() {
-    return isResourceChange;
+    return containerEvent.isResourceChange();
   }
 
   /**
@@ -72,7 +76,7 @@ public class UpdateContainerSchedulerEvent extends ContainerSchedulerEvent {
    * @return isExecTypeUpdate.
    */
   public boolean isExecTypeUpdate() {
-    return isExecTypeUpdate;
+    return containerEvent.isExecTypeUpdate();
   }
 
   /**
@@ -80,6 +84,6 @@ public class UpdateContainerSchedulerEvent extends ContainerSchedulerEvent {
    * @return isIncrease.
    */
   public boolean isIncrease() {
-    return isIncrease;
+    return containerEvent.isIncrease();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df800f6c/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 3cafcbd..fc9e6c4 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
@@ -208,6 +208,8 @@ public abstract class BaseContainerManagerTest {
     containerManager.init(conf);
     nodeStatusUpdater.start();
     ((NMContext)context).setNodeStatusUpdater(nodeStatusUpdater);
+    ((NMContext)context).setContainerStateTransitionListener(
+        new NodeManager.DefaultContainerStateListener());
   }
 
   protected ContainerManagerImpl

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df800f6c/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 38df208..6e8c005 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
@@ -90,12 +90,16 @@ import org.apache.hadoop.yarn.server.nodemanager.CMgrCompletedAppsEvent;
 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.ContainerStateTransitionListener;
+import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor;
 import org.apache.hadoop.yarn.server.nodemanager.DeletionService;
 import org.apache.hadoop.yarn.server.nodemanager.NodeManager;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.TestAuxServices.ServiceA;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationState;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEvent;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEventType;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerImpl;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ContainerLocalizer;
@@ -119,6 +123,41 @@ public class TestContainerManager extends BaseContainerManagerTest {
     LOG = LoggerFactory.getLogger(TestContainerManager.class);
   }
 
+  private static class Listener implements ContainerStateTransitionListener {
+
+    private final Map<ContainerId,
+        List<org.apache.hadoop.yarn.server.nodemanager.containermanager.
+            container.ContainerState>> states = new HashMap<>();
+    private final Map<ContainerId, List<ContainerEventType>> events =
+        new HashMap<>();
+
+    @Override
+    public void init(Context context) {}
+
+    @Override
+    public void preTransition(ContainerImpl op,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState beforeState,
+        ContainerEvent eventToBeProcessed) {
+      if (!states.containsKey(op.getContainerId())) {
+        states.put(op.getContainerId(), new ArrayList<>());
+        states.get(op.getContainerId()).add(beforeState);
+        events.put(op.getContainerId(), new ArrayList<>());
+      }
+    }
+
+    @Override
+    public void postTransition(ContainerImpl op,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState beforeState,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState afterState,
+        ContainerEvent processedEvent) {
+      states.get(op.getContainerId()).add(afterState);
+      events.get(op.getContainerId()).add(processedEvent.getType());
+    }
+  }
+
   private boolean delayContainers = false;
 
   @Override
@@ -144,7 +183,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
   @Override
   protected ContainerManagerImpl
       createContainerManager(DeletionService delSrvc) {
-    return new ContainerManagerImpl(context, exec, delSrvc, nodeStatusUpdater,
+    return  new ContainerManagerImpl(context, exec, delSrvc, nodeStatusUpdater,
       metrics, dirsHandler) {
 
       @Override
@@ -496,6 +535,9 @@ public class TestContainerManager extends BaseContainerManagerTest {
   @Test
   public void testContainerUpgradeSuccessAutoCommit() throws IOException,
       InterruptedException, YarnException {
+    Listener listener = new Listener();
+    ((NodeManager.DefaultContainerStateListener)containerManager.context.
+        getContainerStateTransitionListener()).addListener(listener);
     testContainerReInitSuccess(true);
     // Should not be able to Commit (since already auto committed)
     try {
@@ -504,6 +546,41 @@ public class TestContainerManager extends BaseContainerManagerTest {
     } catch (Exception e) {
       Assert.assertTrue(e.getMessage().contains("Nothing to Commit"));
     }
+
+    List<org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+        ContainerState> containerStates =
+        listener.states.get(createContainerId(0));
+    Assert.assertEquals(Arrays.asList(
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.NEW,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.LOCALIZING,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.SCHEDULED,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.RUNNING,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.REINITIALIZING,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.REINITIALIZING_AWAITING_KILL,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.REINITIALIZING_AWAITING_KILL,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.SCHEDULED,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.RUNNING), containerStates);
+
+    List<ContainerEventType> containerEventTypes =
+        listener.events.get(createContainerId(0));
+    Assert.assertEquals(Arrays.asList(
+        ContainerEventType.INIT_CONTAINER,
+        ContainerEventType.RESOURCE_LOCALIZED,
+        ContainerEventType.CONTAINER_LAUNCHED,
+        ContainerEventType.REINITIALIZE_CONTAINER,
+        ContainerEventType.RESOURCE_LOCALIZED,
+        ContainerEventType.UPDATE_DIAGNOSTICS_MSG,
+        ContainerEventType.CONTAINER_KILLED_ON_REQUEST,
+        ContainerEventType.CONTAINER_LAUNCHED), containerEventTypes);
   }
 
   @Test
@@ -524,6 +601,9 @@ public class TestContainerManager extends BaseContainerManagerTest {
   @Test
   public void testContainerUpgradeSuccessExplicitRollback() throws IOException,
       InterruptedException, YarnException {
+    Listener listener = new Listener();
+    ((NodeManager.DefaultContainerStateListener)containerManager.context.
+        getContainerStateTransitionListener()).addListener(listener);
     String[] pids = testContainerReInitSuccess(false);
 
     // Test that the container can be Restarted after the successful upgrrade.
@@ -575,6 +655,67 @@ public class TestContainerManager extends BaseContainerManagerTest {
 
     Assert.assertNotEquals("The Rolled-back process should be a different pid",
         pids[0], rolledBackPid);
+
+    List<org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+        ContainerState> containerStates =
+        listener.states.get(createContainerId(0));
+    Assert.assertEquals(Arrays.asList(
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.NEW,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.LOCALIZING,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.SCHEDULED,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.RUNNING,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.REINITIALIZING,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.REINITIALIZING_AWAITING_KILL,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.REINITIALIZING_AWAITING_KILL,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.SCHEDULED,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.RUNNING,
+        // This is the successful restart
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.REINITIALIZING_AWAITING_KILL,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.REINITIALIZING_AWAITING_KILL,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.SCHEDULED,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.RUNNING,
+        // This is the rollback
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.REINITIALIZING_AWAITING_KILL,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.REINITIALIZING_AWAITING_KILL,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.SCHEDULED,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.RUNNING), containerStates);
+
+    List<ContainerEventType> containerEventTypes =
+        listener.events.get(createContainerId(0));
+    Assert.assertEquals(Arrays.asList(
+        ContainerEventType.INIT_CONTAINER,
+        ContainerEventType.RESOURCE_LOCALIZED,
+        ContainerEventType.CONTAINER_LAUNCHED,
+        ContainerEventType.REINITIALIZE_CONTAINER,
+        ContainerEventType.RESOURCE_LOCALIZED,
+        ContainerEventType.UPDATE_DIAGNOSTICS_MSG,
+        ContainerEventType.CONTAINER_KILLED_ON_REQUEST,
+        ContainerEventType.CONTAINER_LAUNCHED,
+        ContainerEventType.REINITIALIZE_CONTAINER,
+        ContainerEventType.UPDATE_DIAGNOSTICS_MSG,
+        ContainerEventType.CONTAINER_KILLED_ON_REQUEST,
+        ContainerEventType.CONTAINER_LAUNCHED,
+        ContainerEventType.ROLLBACK_REINIT,
+        ContainerEventType.UPDATE_DIAGNOSTICS_MSG,
+        ContainerEventType.CONTAINER_KILLED_ON_REQUEST,
+        ContainerEventType.CONTAINER_LAUNCHED), containerEventTypes);
   }
 
   @Test
@@ -584,6 +725,9 @@ public class TestContainerManager extends BaseContainerManagerTest {
       return;
     }
     containerManager.start();
+    Listener listener = new Listener();
+    ((NodeManager.DefaultContainerStateListener)containerManager.context.
+        getContainerStateTransitionListener()).addListener(listener);
     // ////// Construct the Container-id
     ContainerId cId = createContainerId(0);
     File oldStartFile = new File(tmpDir, "start_file_o.txt").getAbsoluteFile();
@@ -598,6 +742,32 @@ public class TestContainerManager extends BaseContainerManagerTest {
     // since upgrade was terminated..
     Assert.assertTrue("Process is NOT alive!",
         DefaultContainerExecutor.containerIsAlive(pid));
+
+    List<org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+        ContainerState> containerStates =
+        listener.states.get(createContainerId(0));
+    Assert.assertEquals(Arrays.asList(
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.NEW,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.LOCALIZING,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.SCHEDULED,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.RUNNING,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.REINITIALIZING,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.RUNNING), containerStates);
+
+    List<ContainerEventType> containerEventTypes =
+        listener.events.get(createContainerId(0));
+    Assert.assertEquals(Arrays.asList(
+        ContainerEventType.INIT_CONTAINER,
+        ContainerEventType.RESOURCE_LOCALIZED,
+        ContainerEventType.CONTAINER_LAUNCHED,
+        ContainerEventType.REINITIALIZE_CONTAINER,
+        ContainerEventType.RESOURCE_FAILED), containerEventTypes);
   }
 
   @Test
@@ -632,6 +802,9 @@ public class TestContainerManager extends BaseContainerManagerTest {
       return;
     }
     containerManager.start();
+    Listener listener = new Listener();
+    ((NodeManager.DefaultContainerStateListener)containerManager.context.
+        getContainerStateTransitionListener()).addListener(listener);
     // ////// Construct the Container-id
     ContainerId cId = createContainerId(0);
     File oldStartFile = new File(tmpDir, "start_file_o.txt").getAbsoluteFile();
@@ -666,6 +839,50 @@ public class TestContainerManager extends BaseContainerManagerTest {
 
     Assert.assertNotEquals("The Rolled-back process should be a different pid",
         pid, rolledBackPid);
+
+    List<org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+        ContainerState> containerStates =
+        listener.states.get(createContainerId(0));
+    Assert.assertEquals(Arrays.asList(
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.NEW,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.LOCALIZING,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.SCHEDULED,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.RUNNING,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.REINITIALIZING,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.REINITIALIZING_AWAITING_KILL,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.REINITIALIZING_AWAITING_KILL,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.SCHEDULED,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.RUNNING,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.RUNNING,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.SCHEDULED,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.RUNNING), containerStates);
+
+    List<ContainerEventType> containerEventTypes =
+        listener.events.get(createContainerId(0));
+    Assert.assertEquals(Arrays.asList(
+        ContainerEventType.INIT_CONTAINER,
+        ContainerEventType.RESOURCE_LOCALIZED,
+        ContainerEventType.CONTAINER_LAUNCHED,
+        ContainerEventType.REINITIALIZE_CONTAINER,
+        ContainerEventType.RESOURCE_LOCALIZED,
+        ContainerEventType.UPDATE_DIAGNOSTICS_MSG,
+        ContainerEventType.CONTAINER_KILLED_ON_REQUEST,
+        ContainerEventType.CONTAINER_LAUNCHED,
+        ContainerEventType.UPDATE_DIAGNOSTICS_MSG,
+        ContainerEventType.CONTAINER_EXITED_WITH_FAILURE,
+        ContainerEventType.CONTAINER_LAUNCHED), containerEventTypes);
   }
 
   /**
@@ -1582,16 +1799,12 @@ public class TestContainerManager extends BaseContainerManagerTest {
         containerManager.updateContainer(updateRequest);
     // Check response
     Assert.assertEquals(
-        0, updateResponse.getSuccessfullyUpdatedContainers().size());
-    Assert.assertEquals(2, updateResponse.getFailedRequests().size());
+        1, updateResponse.getSuccessfullyUpdatedContainers().size());
+    Assert.assertEquals(1, updateResponse.getFailedRequests().size());
     for (Map.Entry<ContainerId, SerializedException> entry : updateResponse
         .getFailedRequests().entrySet()) {
       Assert.assertNotNull("Failed message", entry.getValue().getMessage());
-      if (cId0.equals(entry.getKey())) {
-        Assert.assertTrue(entry.getValue().getMessage()
-          .contains("Resource can only be changed when a "
-              + "container is in RUNNING or SCHEDULED state"));
-      } else if (cId7.equals(entry.getKey())) {
+      if (cId7.equals(entry.getKey())) {
         Assert.assertTrue(entry.getValue().getMessage()
             .contains("Container " + cId7.toString()
                 + " is not handled by this NodeManager"));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/df800f6c/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 7c74049..4b380ff 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
@@ -23,6 +23,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 
@@ -47,11 +48,17 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.security.NMTokenIdentifier;
 import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit;
 import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
+import org.apache.hadoop.yarn.server.nodemanager.ContainerStateTransitionListener;
+import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor;
 import org.apache.hadoop.yarn.server.nodemanager.DeletionService;
+import org.apache.hadoop.yarn.server.nodemanager.NodeManager;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.BaseContainerManagerTest;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEvent;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEventType;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerImpl;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerState;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitorImpl;
@@ -76,6 +83,40 @@ public class TestContainerSchedulerQueuing extends BaseContainerManagerTest {
     LOG = LoggerFactory.getLogger(TestContainerSchedulerQueuing.class);
   }
 
+  private static class Listener implements ContainerStateTransitionListener {
+
+    private final Map<ContainerId,
+        List<ContainerState>> states = new HashMap<>();
+    private final Map<ContainerId, List<ContainerEventType>> events =
+        new HashMap<>();
+
+    @Override
+    public void init(Context context) {}
+
+    @Override
+    public void preTransition(ContainerImpl op,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState beforeState,
+        ContainerEvent eventToBeProcessed) {
+      if (!states.containsKey(op.getContainerId())) {
+        states.put(op.getContainerId(), new ArrayList<>());
+        states.get(op.getContainerId()).add(beforeState);
+        events.put(op.getContainerId(), new ArrayList<>());
+      }
+    }
+
+    @Override
+    public void postTransition(ContainerImpl op,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState beforeState,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState afterState,
+        ContainerEvent processedEvent) {
+      states.get(op.getContainerId()).add(afterState);
+      events.get(op.getContainerId()).add(processedEvent.getType());
+    }
+  }
+
   private boolean delayContainers = true;
 
   @Override
@@ -542,6 +583,10 @@ public class TestContainerSchedulerQueuing extends BaseContainerManagerTest {
     containerManager.start();
     containerManager.getContainerScheduler().
         setUsePauseEventForPreemption(true);
+
+    Listener listener = new Listener();
+    ((NodeManager.DefaultContainerStateListener)containerManager.getContext().
+        getContainerStateTransitionListener()).addListener(listener);
     ContainerLaunchContext containerLaunchContext =
         recordFactory.newRecordInstance(ContainerLaunchContext.class);
 
@@ -606,6 +651,39 @@ public class TestContainerSchedulerQueuing extends BaseContainerManagerTest {
     // starts running
     BaseContainerManagerTest.waitForNMContainerState(containerManager,
         createContainerId(0), ContainerState.DONE, 40);
+
+    List<org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+        ContainerState> containerStates =
+        listener.states.get(createContainerId(0));
+    Assert.assertEquals(Arrays.asList(
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.NEW,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.SCHEDULED,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.RUNNING,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.PAUSING,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.PAUSED,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.RESUMING,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.RUNNING,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.EXITED_WITH_SUCCESS,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.DONE), containerStates);
+    List<ContainerEventType> containerEventTypes =
+        listener.events.get(createContainerId(0));
+    Assert.assertEquals(Arrays.asList(ContainerEventType.INIT_CONTAINER,
+        ContainerEventType.CONTAINER_LAUNCHED,
+        ContainerEventType.PAUSE_CONTAINER,
+        ContainerEventType.CONTAINER_PAUSED,
+        ContainerEventType.RESUME_CONTAINER,
+        ContainerEventType.CONTAINER_RESUMED,
+        ContainerEventType.CONTAINER_EXITED_WITH_SUCCESS,
+        ContainerEventType.CONTAINER_RESOURCES_CLEANEDUP), containerEventTypes);
   }
 
   /**
@@ -1068,6 +1146,9 @@ public class TestContainerSchedulerQueuing extends BaseContainerManagerTest {
   @Test
   public void testPromotionOfOpportunisticContainers() throws Exception {
     containerManager.start();
+    Listener listener = new Listener();
+    ((NodeManager.DefaultContainerStateListener)containerManager.getContext().
+        getContainerStateTransitionListener()).addListener(listener);
 
     ContainerLaunchContext containerLaunchContext =
         recordFactory.newRecordInstance(ContainerLaunchContext.class);
@@ -1150,6 +1231,7 @@ public class TestContainerSchedulerQueuing extends BaseContainerManagerTest {
     containerStatuses = containerManager
         .getContainerStatuses(statRequest).getContainerStatuses();
     Assert.assertEquals(1, containerStatuses.size());
+
     for (ContainerStatus status : containerStatuses) {
       if (org.apache.hadoop.yarn.api.records.ContainerState.RUNNING ==
           status.getState()) {
@@ -1160,6 +1242,25 @@ public class TestContainerSchedulerQueuing extends BaseContainerManagerTest {
 
     // Ensure no containers are queued.
     Assert.assertEquals(0, containerScheduler.getNumQueuedContainers());
+
+    List<org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+        ContainerState> containerStates =
+        listener.states.get(createContainerId(1));
+    Assert.assertEquals(Arrays.asList(
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.NEW,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.SCHEDULED,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.SCHEDULED,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.RUNNING), containerStates);
+    List<ContainerEventType> containerEventTypes =
+        listener.events.get(createContainerId(1));
+    Assert.assertEquals(Arrays.asList(
+        ContainerEventType.INIT_CONTAINER,
+        ContainerEventType.UPDATE_CONTAINER_TOKEN,
+        ContainerEventType.CONTAINER_LAUNCHED), containerEventTypes);
   }
 
   @Test


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

Branch: refs/heads/YARN-5734
Commit: 36c474bd91d040911705bf0ff983761d573a743c
Parents: 4653224
Author: Xuan <xg...@apache.org>
Authored: Fri Jul 7 14:16:46 2017 -0700
Committer: Jonathan Hung <jh...@linkedin.com>
Committed: Wed Sep 27 19:52:58 2017 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/hadoop-yarn/bin/yarn        |   3 +
 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, 507 insertions(+), 54 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/36c474bd/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 52138c5..0063596 100755
--- a/hadoop-yarn-project/hadoop-yarn/bin/yarn
+++ b/hadoop-yarn-project/hadoop-yarn/bin/yarn
@@ -142,6 +142,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/36c474bd/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/36c474bd/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/36c474bd/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/36c474bd/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/36c474bd/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/36c474bd/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/36c474bd/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/36c474bd/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/36c474bd/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/36c474bd/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 4d367e0..da395b7 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
@@ -137,11 +137,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/36c474bd/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/36c474bd/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/36c474bd/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 b97e212..1da4e65 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
@@ -199,6 +199,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/36c474bd/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/36c474bd/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/36c474bd/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/36c474bd/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/36c474bd/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


[03/50] [abbrv] hadoop git commit: HDFS-12535. Change the Scope of the Class DFSUtilClient to Private. Contributed by Bharat Viswanadham.

Posted by jh...@apache.org.
HDFS-12535. Change the Scope of the Class DFSUtilClient to Private. Contributed by  Bharat Viswanadham.


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

Branch: refs/heads/YARN-5734
Commit: e9a18b625fe2350f1de20452ecf92976187832ad
Parents: 423ebca
Author: Anu Engineer <ae...@apache.org>
Authored: Sat Sep 23 10:47:17 2017 -0700
Committer: Anu Engineer <ae...@apache.org>
Committed: Sat Sep 23 10:47:17 2017 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java        | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e9a18b62/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
index 3b09f39..24235a1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
@@ -21,6 +21,7 @@ import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Maps;
 import com.google.common.primitives.SignedBytes;
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.crypto.key.KeyProvider;
 import org.apache.hadoop.fs.BlockLocation;
@@ -94,6 +95,7 @@ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_DATA_TRANSF
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX;
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_NAMESERVICES;
 
+@InterfaceAudience.Private
 public class DFSUtilClient {
   public static final byte[] EMPTY_BYTES = {};
   private static final Logger LOG = LoggerFactory.getLogger(


---------------------------------------------------------------------
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-12386] Add fsserver defaults call to WebhdfsFileSystem. (Rushabh Shah via daryn)

Posted by jh...@apache.org.
[HDFS-12386] Add fsserver defaults call to WebhdfsFileSystem. (Rushabh Shah via daryn)


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

Branch: refs/heads/YARN-5734
Commit: 0da29cbeea40cb7839abcd72566b997962829329
Parents: 9df0500
Author: Daryn Sharp <da...@yahoo-inc.com>
Authored: Tue Sep 26 15:15:24 2017 -0500
Committer: Daryn Sharp <da...@yahoo-inc.com>
Committed: Tue Sep 26 15:15:24 2017 -0500

----------------------------------------------------------------------
 .../apache/hadoop/hdfs/web/JsonUtilClient.java  |  38 ++++++
 .../hadoop/hdfs/web/WebHdfsFileSystem.java      |  17 +++
 .../hadoop/hdfs/web/resources/GetOpParam.java   |   3 +-
 .../hdfs/server/namenode/FSNamesystem.java      |   3 +-
 .../web/resources/NamenodeWebHdfsMethods.java   |  21 +++
 .../org/apache/hadoop/hdfs/web/JsonUtil.java    |  20 +++
 .../org/apache/hadoop/hdfs/web/TestWebHDFS.java | 134 +++++++++++++++++++
 7 files changed, 234 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0da29cbe/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
index dcd73bf..53d886d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.ContentSummary.Builder;
 import org.apache.hadoop.fs.FileChecksum;
 import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.MD5MD5CRC32CastagnoliFileChecksum;
 import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
 import org.apache.hadoop.fs.MD5MD5CRC32GzipFileChecksum;
@@ -65,6 +66,8 @@ import java.util.Map;
 
 class JsonUtilClient {
   static final DatanodeInfo[] EMPTY_DATANODE_INFO_ARRAY = {};
+  static final String UNSUPPPORTED_EXCEPTION_STR =
+      UnsupportedOperationException.class.getName();
 
   /** Convert a Json map to a RemoteException. */
   static RemoteException toRemoteException(final Map<?, ?> json) {
@@ -72,6 +75,9 @@ class JsonUtilClient {
         RemoteException.class.getSimpleName());
     final String message = (String)m.get("message");
     final String javaClassName = (String)m.get("javaClassName");
+    if (UNSUPPPORTED_EXCEPTION_STR.equals(javaClassName)) {
+      throw new UnsupportedOperationException(message);
+    }
     return new RemoteException(javaClassName, message);
   }
 
@@ -644,4 +650,36 @@ class JsonUtilClient {
     }
   }
 
+  /*
+   * The parameters which have default value -1 are required fields according
+   * to hdfs.proto.
+   * The default values for optional fields are taken from
+   * hdfs.proto#FsServerDefaultsProto.
+   */
+  public static FsServerDefaults toFsServerDefaults(final Map<?, ?> json) {
+    if (json == null) {
+      return null;
+    }
+    Map<?, ?> m =
+        (Map<?, ?>) json.get(FsServerDefaults.class.getSimpleName());
+    long blockSize =  getLong(m, "blockSize", -1);
+    int bytesPerChecksum = getInt(m, "bytesPerChecksum", -1);
+    int writePacketSize = getInt(m, "writePacketSize", -1);
+    short replication = (short) getInt(m, "replication", -1);
+    int fileBufferSize = getInt(m, "fileBufferSize", -1);
+    boolean encryptDataTransfer = m.containsKey("encryptDataTransfer")
+        ? (Boolean) m.get("encryptDataTransfer")
+        : false;
+    long trashInterval = getLong(m, "trashInterval", 0);
+    DataChecksum.Type type =
+        DataChecksum.Type.valueOf(getInt(m, "checksumType", 1));
+    String keyProviderUri = (String) m.get("keyProviderUri");
+    byte storagepolicyId = m.containsKey("defaultStoragePolicyId")
+        ? ((Number) m.get("defaultStoragePolicyId")).byteValue()
+        : HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
+    return new FsServerDefaults(blockSize, bytesPerChecksum,
+        writePacketSize, replication, fileBufferSize,
+        encryptDataTransfer, trashInterval, type, keyProviderUri,
+        storagepolicyId);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0da29cbe/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
index ee8d5c1..d3a8b23 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
@@ -65,6 +65,7 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FSInputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.GlobalStorageStatistics;
 import org.apache.hadoop.fs.GlobalStorageStatistics.StorageStatisticsProvider;
 import org.apache.hadoop.fs.StorageStatistics;
@@ -1766,6 +1767,22 @@ public class WebHdfsFileSystem extends FileSystem
     new FsPathRunner(op, src).run();
   }
 
+  /*
+   * Caller of this method should handle UnsupportedOperationException in case
+   * when new client is talking to old namenode that don't support
+   * FsServerDefaults call.
+   */
+  @Override
+  public FsServerDefaults getServerDefaults() throws IOException {
+    final HttpOpParam.Op op = GetOpParam.Op.GETSERVERDEFAULTS;
+    return new FsPathResponseRunner<FsServerDefaults>(op, null) {
+      @Override
+      FsServerDefaults decodeResponse(Map<?, ?> json) throws IOException {
+        return JsonUtilClient.toFsServerDefaults(json);
+      }
+    }.run();
+  }
+
   @VisibleForTesting
   InetSocketAddress[] getResolvedNNAddr() {
     return nnAddrs;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0da29cbe/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java
index 6c2c674..4d4d9be 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java
@@ -46,7 +46,8 @@ public class GetOpParam extends HttpOpParam<GetOpParam.Op> {
     NULL(false, HttpURLConnection.HTTP_NOT_IMPLEMENTED),
 
     CHECKACCESS(false, HttpURLConnection.HTTP_OK),
-    LISTSTATUS_BATCH(false, HttpURLConnection.HTTP_OK);
+    LISTSTATUS_BATCH(false, HttpURLConnection.HTTP_OK),
+    GETSERVERDEFAULTS(false, HttpURLConnection.HTTP_OK);
 
     final boolean redirect;
     final int expectedHttpResponseCode;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0da29cbe/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 1534850..4094694 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
@@ -1782,7 +1782,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     return sw.toString();
   }
 
-  FsServerDefaults getServerDefaults() throws StandbyException {
+  @VisibleForTesting
+  public FsServerDefaults getServerDefaults() throws StandbyException {
     checkOperation(OperationCategory.READ);
     return serverDefaults;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0da29cbe/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
index e400847..9e0a1ed 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
@@ -57,6 +57,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.fs.permission.AclStatus;
@@ -115,6 +116,7 @@ public class NamenodeWebHdfsMethods {
   private Principal userPrincipal;
   private String remoteAddr;
 
+  private static volatile String serverDefaultsResponse = null;
   private @Context ServletContext context;
   private @Context HttpServletResponse response;
 
@@ -1121,11 +1123,30 @@ public class NamenodeWebHdfsMethods {
       final String js = JsonUtil.toJsonString(storagePolicy);
       return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
     }
+    case GETSERVERDEFAULTS: {
+      // Since none of the server defaults values are hot reloaded, we can
+      // cache the output of serverDefaults.
+      if (serverDefaultsResponse == null) {
+        FsServerDefaults serverDefaults = np.getServerDefaults();
+        serverDefaultsResponse = JsonUtil.toJsonString(serverDefaults);
+      }
+      return Response.ok(serverDefaultsResponse)
+          .type(MediaType.APPLICATION_JSON).build();
+    }
     default:
       throw new UnsupportedOperationException(op + " is not supported");
     }
   }
 
+  /*
+   * This is used only and only for testing.
+   * Please don't use it otherwise.
+   */
+  @VisibleForTesting
+  public static void resetServerDefaultsResponse() {
+    serverDefaultsResponse = null;
+  }
+
   private static String getTrashRoot(String fullPath,
       Configuration conf) throws IOException {
     FileSystem fs = FileSystem.get(conf != null ? conf : new Configuration());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0da29cbe/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
index 5752948..7fa818a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
@@ -21,6 +21,7 @@ import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FileChecksum;
 import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.XAttr;
@@ -470,4 +471,23 @@ public class JsonUtil {
   public static String toJsonString(BlockStoragePolicy storagePolicy) {
     return toJsonString(BlockStoragePolicy.class, toJsonMap(storagePolicy));
   }
+
+  public static String toJsonString(FsServerDefaults serverDefaults) {
+    return toJsonString(FsServerDefaults.class, toJsonMap(serverDefaults));
+  }
+
+  private static Object toJsonMap(FsServerDefaults serverDefaults) {
+    final Map<String, Object> m = new HashMap<String, Object>();
+    m.put("blockSize", serverDefaults.getBlockSize());
+    m.put("bytesPerChecksum", serverDefaults.getBytesPerChecksum());
+    m.put("writePacketSize", serverDefaults.getWritePacketSize());
+    m.put("replication", serverDefaults.getReplication());
+    m.put("fileBufferSize", serverDefaults.getFileBufferSize());
+    m.put("encryptDataTransfer", serverDefaults.getEncryptDataTransfer());
+    m.put("trashInterval", serverDefaults.getTrashInterval());
+    m.put("checksumType", serverDefaults.getChecksumType().id);
+    m.put("keyProviderUri", serverDefaults.getKeyProviderUri());
+    m.put("defaultStoragePolicyId", serverDefaults.getDefaultStoragePolicyId());
+    return m;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0da29cbe/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java
index de051b3..3ee8ad0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java
@@ -18,6 +18,14 @@
 
 package org.apache.hadoop.hdfs.web;
 
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CHECKSUM_TYPE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_KEY;
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY;
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotEquals;
@@ -53,6 +61,7 @@ import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.StorageType;
@@ -74,7 +83,9 @@ import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper;
 import org.apache.hadoop.hdfs.server.namenode.web.resources.NamenodeWebHdfsMethods;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
@@ -92,11 +103,13 @@ import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.util.DataChecksum;
 import org.apache.log4j.Level;
 import org.codehaus.jettison.json.JSONException;
 import org.codehaus.jettison.json.JSONObject;
 import org.junit.Assert;
 import org.junit.Test;
+import org.mockito.Mockito;
 import org.mockito.internal.util.reflection.Whitebox;
 
 import static org.mockito.Matchers.any;
@@ -1318,4 +1331,125 @@ public class TestWebHDFS {
       }
     }
   }
+
+  /**
+   * Test fsserver defaults response from {@link DistributedFileSystem} and
+   * {@link WebHdfsFileSystem} are the same.
+   * @throws Exception
+   */
+  @Test
+  public void testFsserverDefaults() throws Exception {
+    MiniDFSCluster cluster = null;
+    final Configuration conf = WebHdfsTestUtil.createConf();
+    // Here we override all the default values so that we can verify that it
+    // doesn't pick up the default value.
+    long blockSize = 256*1024*1024;
+    int bytesPerChecksum = 256;
+    int writePacketSize = 128*1024;
+    int replicationFactor = 0;
+    int bufferSize = 1024;
+    boolean encryptDataTransfer = true;
+    long trashInterval = 1;
+    String checksumType = "CRC32";
+    // Setting policy to a special value 7 because BlockManager will
+    // create defaultSuite with policy id 7.
+    byte policyId = (byte) 7;
+
+    conf.setLong(DFS_BLOCK_SIZE_KEY, blockSize);
+    conf.setInt(DFS_BYTES_PER_CHECKSUM_KEY, bytesPerChecksum);
+    conf.setInt(DFS_CLIENT_WRITE_PACKET_SIZE_KEY, writePacketSize);
+    conf.setInt(DFS_REPLICATION_KEY, replicationFactor);
+    conf.setInt(IO_FILE_BUFFER_SIZE_KEY, bufferSize);
+    conf.setBoolean(DFS_ENCRYPT_DATA_TRANSFER_KEY, encryptDataTransfer);
+    conf.setLong(FS_TRASH_INTERVAL_KEY, trashInterval);
+    conf.set(DFS_CHECKSUM_TYPE_KEY, checksumType);
+    FsServerDefaults originalServerDefaults = new FsServerDefaults(blockSize,
+        bytesPerChecksum, writePacketSize, (short)replicationFactor,
+        bufferSize, encryptDataTransfer, trashInterval,
+        DataChecksum.Type.valueOf(checksumType), "", policyId);
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).build();
+      final DistributedFileSystem dfs = cluster.getFileSystem();
+      final WebHdfsFileSystem webfs = WebHdfsTestUtil.getWebHdfsFileSystem(
+          conf, WebHdfsConstants.WEBHDFS_SCHEME);
+      FsServerDefaults dfsServerDefaults = dfs.getServerDefaults();
+      FsServerDefaults webfsServerDefaults = webfs.getServerDefaults();
+      // Verify whether server defaults value that we override is equal to
+      // dfsServerDefaults.
+      compareFsServerDefaults(originalServerDefaults, dfsServerDefaults);
+      // Verify whether dfs serverdefaults is equal to
+      // webhdfsServerDefaults.
+      compareFsServerDefaults(dfsServerDefaults, webfsServerDefaults);
+      webfs.getServerDefaults();
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  private void compareFsServerDefaults(FsServerDefaults serverDefaults1,
+      FsServerDefaults serverDefaults2) throws Exception {
+    Assert.assertEquals("Block size is different",
+        serverDefaults1.getBlockSize(),
+        serverDefaults2.getBlockSize());
+    Assert.assertEquals("Bytes per checksum are different",
+        serverDefaults1.getBytesPerChecksum(),
+        serverDefaults2.getBytesPerChecksum());
+    Assert.assertEquals("Write packet size is different",
+        serverDefaults1.getWritePacketSize(),
+        serverDefaults2.getWritePacketSize());
+    Assert.assertEquals("Default replication is different",
+        serverDefaults1.getReplication(),
+        serverDefaults2.getReplication());
+    Assert.assertEquals("File buffer size are different",
+        serverDefaults1.getFileBufferSize(),
+        serverDefaults2.getFileBufferSize());
+    Assert.assertEquals("Encrypt data transfer key is different",
+        serverDefaults1.getEncryptDataTransfer(),
+        serverDefaults2.getEncryptDataTransfer());
+    Assert.assertEquals("Trash interval is different",
+        serverDefaults1.getTrashInterval(),
+        serverDefaults2.getTrashInterval());
+    Assert.assertEquals("Checksum type is different",
+        serverDefaults1.getChecksumType(),
+        serverDefaults2.getChecksumType());
+    Assert.assertEquals("Key provider uri is different",
+        serverDefaults1.getKeyProviderUri(),
+        serverDefaults2.getKeyProviderUri());
+    Assert.assertEquals("Default storage policy is different",
+        serverDefaults1.getDefaultStoragePolicyId(),
+        serverDefaults2.getDefaultStoragePolicyId());
+  }
+
+  /**
+   * Tests the case when client is upgraded to return {@link FsServerDefaults}
+   * but then namenode is not upgraded.
+   * @throws Exception
+   */
+  @Test
+  public void testFsserverDefaultsBackwardsCompatible() throws Exception {
+    MiniDFSCluster cluster = null;
+    final Configuration conf = WebHdfsTestUtil.createConf();
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).build();
+      final WebHdfsFileSystem webfs = WebHdfsTestUtil.getWebHdfsFileSystem(
+          conf, WebHdfsConstants.WEBHDFS_SCHEME);
+      NamenodeWebHdfsMethods.resetServerDefaultsResponse();
+      FSNamesystem fsnSpy =
+          NameNodeAdapter.spyOnNamesystem(cluster.getNameNode());
+      Mockito.when(fsnSpy.getServerDefaults()).
+          thenThrow(new UnsupportedOperationException());
+      try {
+        webfs.getServerDefaults();
+        Assert.fail("should have thrown UnSupportedOperationException.");
+      } catch (UnsupportedOperationException uoe) {
+       //Expected exception.
+      }
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
 }


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

Branch: refs/heads/YARN-5734
Commit: 46532248718201c5880cbd8a827441079463dd98
Parents: 529e6e6
Author: Xuan <xg...@apache.org>
Authored: Mon Jun 5 16:30:38 2017 -0700
Committer: Jonathan Hung <jh...@linkedin.com>
Committed: Wed Sep 27 19:52:58 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/46532248/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/46532248/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/46532248/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/46532248/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/46532248/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 4f89f7e..4d367e0 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
@@ -137,7 +137,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;
@@ -2617,7 +2617,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/46532248/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/46532248/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/46532248/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 d95465b..b97e212 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
@@ -2459,11 +2459,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/46532248/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/46532248/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/46532248/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/46532248/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/46532248/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


[47/50] [abbrv] hadoop git commit: YARN-7046. Add closing logic to configuration store

Posted by jh...@apache.org.
YARN-7046. Add closing logic to configuration store


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

Branch: refs/heads/YARN-5734
Commit: c7948fe56c46526fa1cf934444d7d193cfe7398f
Parents: 76746cb
Author: Jonathan Hung <jh...@linkedin.com>
Authored: Fri Sep 22 10:18:27 2017 -0700
Committer: Jonathan Hung <jh...@linkedin.com>
Committed: Wed Sep 27 19:52:59 2017 -0700

----------------------------------------------------------------------
 .../scheduler/MutableConfigurationProvider.java |   8 +
 .../scheduler/capacity/CapacityScheduler.java   |   3 +
 .../conf/LeveldbConfigurationStore.java         |  44 ++--
 .../conf/MutableCSConfigurationProvider.java    |   6 +
 .../capacity/conf/YarnConfigurationStore.java   |   6 +
 .../conf/ConfigurationStoreBaseTest.java        |   2 +
 .../conf/TestLeveldbConfigurationStore.java     | 213 +++++++++++++++++++
 7 files changed, 270 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c7948fe5/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 f8e8814..2b9b25a 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
@@ -21,6 +21,8 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.webapp.dao.SchedConfUpdateInfo;
 
+import java.io.IOException;
+
 /**
  * Interface for allowing changing scheduler configurations.
  */
@@ -55,4 +57,10 @@ public interface MutableConfigurationProvider {
    * @throws Exception if confirming mutation fails
    */
   void confirmPendingMutation(boolean isValid) throws Exception;
+
+  /**
+   * Closes the configuration provider, releasing any required resources.
+   * @throws IOException on failure to close
+   */
+  void close() throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c7948fe5/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 16b27c1..de95179 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
@@ -411,6 +411,9 @@ public class CapacityScheduler extends
       writeLock.unlock();
     }
 
+    if (isConfigurationMutable()) {
+      ((MutableConfigurationProvider) csConfProvider).close();
+    }
     super.serviceStop();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c7948fe5/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
index 1b0eb9f..21de7a2 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/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
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.conf;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -72,7 +73,8 @@ public class LeveldbConfigurationStore extends YarnConfigurationStore {
   private long maxLogs;
   private Configuration conf;
   private LogMutation pendingMutation;
-  private static final Version CURRENT_VERSION_INFO = Version
+  @VisibleForTesting
+  protected static final Version CURRENT_VERSION_INFO = Version
       .newInstance(0, 1);
   private Timer compactionTimer;
   private long compactionIntervalMsec;
@@ -82,7 +84,7 @@ public class LeveldbConfigurationStore extends YarnConfigurationStore {
       RMContext rmContext) throws IOException {
     this.conf = config;
     try {
-      this.db = initDatabase(schedConf);
+      initDatabase(schedConf);
       this.maxLogs = config.getLong(
           YarnConfiguration.RM_SCHEDCONF_MAX_LOGS,
           YarnConfiguration.DEFAULT_RM_SCHEDCONF_LEVELDB_MAX_LOGS);
@@ -96,7 +98,7 @@ public class LeveldbConfigurationStore extends YarnConfigurationStore {
     }
   }
 
-  private DB initDatabase(Configuration config) throws Exception {
+  private void initDatabase(Configuration config) throws Exception {
     Path storeRoot = createStorageDir();
     Options options = new Options();
     options.createIfMissing(false);
@@ -108,13 +110,13 @@ public class LeveldbConfigurationStore extends YarnConfigurationStore {
         if (key1Str.equals(key2Str)) {
           return 0;
         } else if (key1Str.equals(VERSION_KEY)) {
-          return -1;
-        } else if (key2Str.equals(VERSION_KEY)) {
           return 1;
-        } else if (key1Str.equals(LOG_KEY)) {
+        } else if (key2Str.equals(VERSION_KEY)) {
           return -1;
-        } else if (key2Str.equals(LOG_KEY)) {
+        } else if (key1Str.equals(LOG_KEY)) {
           return 1;
+        } else if (key2Str.equals(LOG_KEY)) {
+          return -1;
         }
         return key1Str.compareTo(key2Str);
       }
@@ -156,7 +158,6 @@ public class LeveldbConfigurationStore extends YarnConfigurationStore {
         throw e;
       }
     }
-    return db;
   }
 
   private Path createStorageDir() throws IOException {
@@ -176,6 +177,13 @@ public class LeveldbConfigurationStore extends YarnConfigurationStore {
   }
 
   @Override
+  public void close() throws IOException {
+    if (db != null) {
+      db.close();
+    }
+  }
+
+  @Override
   public void logMutation(LogMutation logMutation) throws IOException {
     LinkedList<LogMutation> logs = deserLogMutations(db.get(bytes(LOG_KEY)));
     logs.add(logMutation);
@@ -212,8 +220,12 @@ public class LeveldbConfigurationStore extends YarnConfigurationStore {
       return baos.toByteArray();
     }
   }
+
   private LinkedList<LogMutation> deserLogMutations(byte[] mutations) throws
       IOException {
+    if (mutations == null) {
+      return new LinkedList<>();
+    }
     try (ObjectInput input = new ObjectInputStream(
         new ByteArrayInputStream(mutations))) {
       return (LinkedList<LogMutation>) input.readObject();
@@ -225,13 +237,16 @@ public class LeveldbConfigurationStore extends YarnConfigurationStore {
   @Override
   public synchronized Configuration retrieve() {
     DBIterator itr = db.iterator();
-    itr.seek(bytes(LOG_KEY));
+    itr.seekToFirst();
     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));
+      String key = new String(entry.getKey(), StandardCharsets.UTF_8);
+      String value = new String(entry.getValue(), StandardCharsets.UTF_8);
+      if (key.equals(LOG_KEY) || key.equals(VERSION_KEY)) {
+        break;
+      }
+      config.set(key, value);
     }
     return config;
   }
@@ -268,6 +283,11 @@ public class LeveldbConfigurationStore extends YarnConfigurationStore {
     return version;
   }
 
+  @VisibleForTesting
+  protected LinkedList<LogMutation> getLogs() throws Exception {
+    return deserLogMutations(db.get(bytes(LOG_KEY)));
+  }
+
   @Override
   public void storeVersion() throws Exception {
     String key = VERSION_KEY;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c7948fe5/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 70d1840..ccadf76 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
@@ -92,6 +92,7 @@ public class MutableCSConfigurationProvider implements CSConfigurationProvider,
     }
     try {
       confStore.initialize(config, schedConf, rmContext);
+      confStore.checkVersion();
     } catch (Exception e) {
       throw new IOException(e);
     }
@@ -103,6 +104,11 @@ public class MutableCSConfigurationProvider implements CSConfigurationProvider,
     aclMutationPolicy.init(config, rmContext);
   }
 
+  @Override
+  public void close() throws IOException {
+    confStore.close();
+  }
+
   @VisibleForTesting
   public YarnConfigurationStore getConfStore() {
     return confStore;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c7948fe5/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 1356535..7fb52fc 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
@@ -96,6 +96,12 @@ public abstract class YarnConfigurationStore {
       RMContext rmContext) throws Exception;
 
   /**
+   * Closes the configuration store, releasing any required resources.
+   * @throws IOException on failure to close
+   */
+  public void close() throws IOException {}
+
+  /**
    * Logs the configuration change to backing store.
    * @param logMutation configuration change to be persisted in write ahead log
    * @throws IOException if logging fails

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c7948fe5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/ConfigurationStoreBaseTest.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/ConfigurationStoreBaseTest.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/ConfigurationStoreBaseTest.java
index bbe9570..8f3bc71 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/ConfigurationStoreBaseTest.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/ConfigurationStoreBaseTest.java
@@ -71,6 +71,7 @@ public abstract class ConfigurationStoreBaseTest {
     confStore.confirmMutation(false);
     assertNull("Configuration should not be updated",
         confStore.retrieve().get("keyUpdate2"));
+    confStore.close();
   }
 
   @Test
@@ -86,5 +87,6 @@ public abstract class ConfigurationStoreBaseTest {
     confStore.logMutation(mutation);
     confStore.confirmMutation(true);
     assertNull(confStore.retrieve().get("key"));
+    confStore.close();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c7948fe5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/TestLeveldbConfigurationStore.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/TestLeveldbConfigurationStore.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/TestLeveldbConfigurationStore.java
new file mode 100644
index 0000000..779208a
--- /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/TestLeveldbConfigurationStore.java
@@ -0,0 +1,213 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, 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.fs.FileUtil;
+import org.apache.hadoop.security.UserGroupInformation;
+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.MutableConfScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.MutableConfigurationProvider;
+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.webapp.dao.SchedConfUpdateInfo;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.File;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+/**
+ * Tests {@link LeveldbConfigurationStore}.
+ */
+public class TestLeveldbConfigurationStore extends ConfigurationStoreBaseTest {
+
+  public static final Log LOG =
+      LogFactory.getLog(TestLeveldbConfigurationStore.class);
+  private static final File TEST_DIR = new File(
+      System.getProperty("test.build.data",
+          System.getProperty("java.io.tmpdir")),
+      TestLeveldbConfigurationStore.class.getName());
+
+  private ResourceManager rm;
+
+  @Before
+  public void setUp() throws Exception {
+    super.setUp();
+    FileUtil.fullyDelete(TEST_DIR);
+    conf.set(CapacitySchedulerConfiguration.CS_CONF_PROVIDER,
+        CapacitySchedulerConfiguration.STORE_CS_CONF_PROVIDER);
+    conf.set(YarnConfiguration.SCHEDULER_CONFIGURATION_STORE_CLASS,
+        YarnConfiguration.LEVELDB_CONFIGURATION_STORE);
+    conf.set(YarnConfiguration.RM_SCHEDCONF_STORE_PATH, TEST_DIR.toString());
+  }
+
+  @Test
+  public void testVersioning() throws Exception {
+    confStore.initialize(conf, schedConf, rmContext);
+    assertNull(confStore.getConfStoreVersion());
+    confStore.checkVersion();
+    assertEquals(LeveldbConfigurationStore.CURRENT_VERSION_INFO,
+        confStore.getConfStoreVersion());
+    confStore.close();
+  }
+
+  @Test
+  public void testPersistConfiguration() throws Exception {
+    schedConf.set("key", "val");
+    confStore.initialize(conf, schedConf, rmContext);
+    assertEquals("val", confStore.retrieve().get("key"));
+    confStore.close();
+
+    // Create a new configuration store, and check for old configuration
+    confStore = createConfStore();
+    schedConf.set("key", "badVal");
+    // Should ignore passed-in scheduler configuration.
+    confStore.initialize(conf, schedConf, rmContext);
+    assertEquals("val", confStore.retrieve().get("key"));
+    confStore.close();
+  }
+
+  @Test
+  public void testPersistUpdatedConfiguration() throws Exception {
+    confStore.initialize(conf, schedConf, rmContext);
+    assertNull(confStore.retrieve().get("key"));
+
+    Map<String, String> update = new HashMap<>();
+    update.put("key", "val");
+    YarnConfigurationStore.LogMutation mutation =
+        new YarnConfigurationStore.LogMutation(update, TEST_USER);
+    confStore.logMutation(mutation);
+    confStore.confirmMutation(true);
+    assertEquals("val", confStore.retrieve().get("key"));
+    confStore.close();
+
+    // Create a new configuration store, and check for updated configuration
+    confStore = createConfStore();
+    schedConf.set("key", "badVal");
+    // Should ignore passed-in scheduler configuration.
+    confStore.initialize(conf, schedConf, rmContext);
+    assertEquals("val", confStore.retrieve().get("key"));
+    confStore.close();
+  }
+
+  @Test
+  public void testMaxLogs() throws Exception {
+    conf.setLong(YarnConfiguration.RM_SCHEDCONF_MAX_LOGS, 2);
+    confStore.initialize(conf, schedConf, rmContext);
+    LinkedList<YarnConfigurationStore.LogMutation> logs =
+        ((LeveldbConfigurationStore) confStore).getLogs();
+    assertEquals(0, logs.size());
+
+    Map<String, String> update1 = new HashMap<>();
+    update1.put("key1", "val1");
+    YarnConfigurationStore.LogMutation mutation =
+        new YarnConfigurationStore.LogMutation(update1, TEST_USER);
+    confStore.logMutation(mutation);
+    logs = ((LeveldbConfigurationStore) confStore).getLogs();
+    assertEquals(1, logs.size());
+    assertEquals("val1", logs.get(0).getUpdates().get("key1"));
+    confStore.confirmMutation(true);
+    assertEquals(1, logs.size());
+    assertEquals("val1", logs.get(0).getUpdates().get("key1"));
+
+    Map<String, String> update2 = new HashMap<>();
+    update2.put("key2", "val2");
+    mutation = new YarnConfigurationStore.LogMutation(update2, TEST_USER);
+    confStore.logMutation(mutation);
+    logs = ((LeveldbConfigurationStore) confStore).getLogs();
+    assertEquals(2, logs.size());
+    assertEquals("val1", logs.get(0).getUpdates().get("key1"));
+    assertEquals("val2", logs.get(1).getUpdates().get("key2"));
+    confStore.confirmMutation(true);
+    assertEquals(2, logs.size());
+    assertEquals("val1", logs.get(0).getUpdates().get("key1"));
+    assertEquals("val2", logs.get(1).getUpdates().get("key2"));
+
+    // Next update should purge first update from logs.
+    Map<String, String> update3 = new HashMap<>();
+    update3.put("key3", "val3");
+    mutation = new YarnConfigurationStore.LogMutation(update3, TEST_USER);
+    confStore.logMutation(mutation);
+    logs = ((LeveldbConfigurationStore) confStore).getLogs();
+    assertEquals(2, logs.size());
+    assertEquals("val2", logs.get(0).getUpdates().get("key2"));
+    assertEquals("val3", logs.get(1).getUpdates().get("key3"));
+    confStore.confirmMutation(true);
+    assertEquals(2, logs.size());
+    assertEquals("val2", logs.get(0).getUpdates().get("key2"));
+    assertEquals("val3", logs.get(1).getUpdates().get("key3"));
+    confStore.close();
+  }
+
+  /**
+   * When restarting, RM should read from current state of store, including
+   * any updates from the previous RM instance.
+   * @throws Exception
+   */
+  @Test
+  public void testRestartReadsFromUpdatedStore() throws Exception {
+    ResourceManager rm1 = new MockRM(conf);
+    rm1.start();
+    assertNull(((MutableConfScheduler) rm1.getResourceScheduler())
+        .getConfiguration().get("key"));
+
+    // Update configuration on RM
+    SchedConfUpdateInfo schedConfUpdateInfo = new SchedConfUpdateInfo();
+    schedConfUpdateInfo.getGlobalParams().put("key", "val");
+    MutableConfigurationProvider confProvider = ((MutableConfScheduler)
+        rm1.getResourceScheduler()).getMutableConfProvider();
+    UserGroupInformation user = UserGroupInformation
+        .createUserForTesting(TEST_USER, new String[0]);
+    confProvider.logAndApplyMutation(user, schedConfUpdateInfo);
+    rm1.getResourceScheduler().reinitialize(conf, rm1.getRMContext());
+    assertEquals("val", ((MutableConfScheduler) rm1.getResourceScheduler())
+        .getConfiguration().get("key"));
+    confProvider.confirmPendingMutation(true);
+    assertEquals("val", ((MutableCSConfigurationProvider) confProvider)
+        .getConfStore().retrieve().get("key"));
+    // Next update is not persisted, it should not be recovered
+    schedConfUpdateInfo.getGlobalParams().put("key", "badVal");
+    confProvider.logAndApplyMutation(user, schedConfUpdateInfo);
+    rm1.close();
+
+    // Start RM2 and verifies it starts with updated configuration
+    ResourceManager rm2 = new MockRM(conf);
+    rm2.start();
+    assertEquals("val", ((MutableCSConfigurationProvider) (
+        (CapacityScheduler) rm2.getResourceScheduler())
+        .getMutableConfProvider()).getConfStore().retrieve().get("key"));
+    assertEquals("val", ((MutableConfScheduler) rm2.getResourceScheduler())
+        .getConfiguration().get("key"));
+    rm2.close();
+  }
+
+  @Override
+  public YarnConfigurationStore createConfStore() {
+    return new LeveldbConfigurationStore();
+  }
+}


---------------------------------------------------------------------
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: HADOOP-14904. Fix javadocs issues in Hadoop HDFS. Contributed by Mukul Kumar Singh.

Posted by jh...@apache.org.
 HADOOP-14904. Fix javadocs issues in Hadoop HDFS. Contributed by Mukul Kumar Singh.


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

Branch: refs/heads/YARN-5734
Commit: 40ef9fa171f8aa5209c80acae351bd90afe364ab
Parents: 11af907
Author: Anu Engineer <ae...@apache.org>
Authored: Sat Sep 23 12:16:48 2017 -0700
Committer: Anu Engineer <ae...@apache.org>
Committed: Sat Sep 23 12:16:48 2017 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/server/blockmanagement/BlockIdManager.java     | 6 ++++--
 .../hadoop/hdfs/server/datanode/ShortCircuitRegistry.java      | 1 +
 .../hadoop/hdfs/server/datanode/checker/AbstractFuture.java    | 1 +
 3 files changed, 6 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/40ef9fa1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
index 8463023..321155b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
@@ -24,6 +24,8 @@ import org.apache.hadoop.hdfs.protocol.BlockType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLog;
 
 import java.io.IOException;
 
@@ -31,8 +33,8 @@ import static org.apache.hadoop.hdfs.protocol.BlockType.STRIPED;
 
 /**
  * BlockIdManager allocates the generation stamps and the block ID. The
- * {@see FSNamesystem} is responsible for persisting the allocations in the
- * {@see EditLog}.
+ * {@link FSNamesystem} is responsible for persisting the allocations in the
+ * {@link FSEditLog}.
  */
 public class BlockIdManager {
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40ef9fa1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ShortCircuitRegistry.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ShortCircuitRegistry.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ShortCircuitRegistry.java
index 68ef24e..edb64dd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ShortCircuitRegistry.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ShortCircuitRegistry.java
@@ -44,6 +44,7 @@ import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.SlotId;
 import org.apache.hadoop.io.nativeio.SharedFileDescriptorFactory;
 import org.apache.hadoop.net.unix.DomainSocket;
 import org.apache.hadoop.net.unix.DomainSocketWatcher;
+import org.apache.hadoop.hdfs.shortcircuit.DfsClientShmManager;
 
 import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40ef9fa1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/checker/AbstractFuture.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/checker/AbstractFuture.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/checker/AbstractFuture.java
index 2e0ba18..2719f71 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/checker/AbstractFuture.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/checker/AbstractFuture.java
@@ -40,6 +40,7 @@ import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
+import java.util.concurrent.ThreadPoolExecutor.CallerRunsPolicy;
 import java.util.concurrent.locks.LockSupport;
 import java.util.logging.Level;
 import java.util.logging.Logger;


---------------------------------------------------------------------
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-12339. NFS Gateway on Shutdown Gives Unregistration Failure. Does Not Unregister with rpcbind Portmapper. Contributed by Mukul Kumar Singh.

Posted by jh...@apache.org.
 HDFS-12339. NFS Gateway on Shutdown Gives Unregistration Failure. Does Not Unregister with rpcbind Portmapper. Contributed by Mukul Kumar Singh.


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

Branch: refs/heads/YARN-5734
Commit: ecc85620e83dc051502c3b85df11d2f50b0cd616
Parents: 40ef9fa
Author: Anu Engineer <ae...@apache.org>
Authored: Sat Sep 23 12:27:34 2017 -0700
Committer: Anu Engineer <ae...@apache.org>
Committed: Sat Sep 23 12:27:34 2017 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/mount/MountdBase.java | 14 ++++++++++++--
 .../java/org/apache/hadoop/nfs/nfs3/Nfs3Base.java     | 10 ++++++++--
 .../java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3.java    | 10 ++++++++--
 .../hdfs/nfs/nfs3/PrivilegedNfsGatewayStarter.java    |  7 +++++--
 4 files changed, 33 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ecc85620/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/mount/MountdBase.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/mount/MountdBase.java b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/mount/MountdBase.java
index 8c13b4f..0ff3084 100644
--- a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/mount/MountdBase.java
+++ b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/mount/MountdBase.java
@@ -109,6 +109,17 @@ abstract public class MountdBase {
     }
   }
 
+  public void stop() {
+    if (udpBoundPort > 0) {
+      rpcProgram.unregister(PortmapMapping.TRANSPORT_UDP, udpBoundPort);
+      udpBoundPort = 0;
+    }
+    if (tcpBoundPort > 0) {
+      rpcProgram.unregister(PortmapMapping.TRANSPORT_TCP, tcpBoundPort);
+      tcpBoundPort = 0;
+    }
+  }
+
   /**
    * Priority of the mountd shutdown hook.
    */
@@ -117,8 +128,7 @@ abstract public class MountdBase {
   private class Unregister implements Runnable {
     @Override
     public synchronized void run() {
-      rpcProgram.unregister(PortmapMapping.TRANSPORT_UDP, udpBoundPort);
-      rpcProgram.unregister(PortmapMapping.TRANSPORT_TCP, tcpBoundPort);
+      stop();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ecc85620/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/Nfs3Base.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/Nfs3Base.java b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/Nfs3Base.java
index 00e6d9f..ff83a5f 100644
--- a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/Nfs3Base.java
+++ b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/Nfs3Base.java
@@ -78,6 +78,13 @@ public abstract class Nfs3Base {
     nfsBoundPort = tcpServer.getBoundPort();
   }
 
+  public void stop() {
+    if (nfsBoundPort > 0) {
+      rpcProgram.unregister(PortmapMapping.TRANSPORT_TCP, nfsBoundPort);
+      nfsBoundPort = 0;
+    }
+    rpcProgram.stopDaemons();
+  }
   /**
    * Priority of the nfsd shutdown hook.
    */
@@ -86,8 +93,7 @@ public abstract class Nfs3Base {
   private class NfsShutdownHook implements Runnable {
     @Override
     public synchronized void run() {
-      rpcProgram.unregister(PortmapMapping.TRANSPORT_TCP, nfsBoundPort);
-      rpcProgram.stopDaemons();
+      stop();
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ecc85620/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3.java b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3.java
index ac9abf8..8494493 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3.java
@@ -57,7 +57,7 @@ public class Nfs3 extends Nfs3Base {
     start(register);
   }
   
-  static void startService(String[] args,
+  static Nfs3 startService(String[] args,
       DatagramSocket registrationSocket) throws IOException {
     StringUtils.startupShutdownMessage(Nfs3.class, args, LOG);
     NfsConfiguration conf = new NfsConfiguration();
@@ -67,8 +67,14 @@ public class Nfs3 extends Nfs3Base {
     final Nfs3 nfsServer = new Nfs3(conf, registrationSocket,
         allowInsecurePorts);
     nfsServer.startServiceInternal(true);
+    return nfsServer;
   }
-  
+
+  public void stop() {
+    super.stop();
+    mountd.stop();
+  }
+
   public static void main(String[] args) throws IOException {
     startService(args, null);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ecc85620/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/PrivilegedNfsGatewayStarter.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/PrivilegedNfsGatewayStarter.java b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/PrivilegedNfsGatewayStarter.java
index 9456caf..e495486 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/PrivilegedNfsGatewayStarter.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/PrivilegedNfsGatewayStarter.java
@@ -40,6 +40,7 @@ public class PrivilegedNfsGatewayStarter implements Daemon {
   static final Log LOG = LogFactory.getLog(PrivilegedNfsGatewayStarter.class);
   private String[] args = null;
   private DatagramSocket registrationSocket = null;
+  private Nfs3 nfs3Server = null;
 
   @Override
   public void init(DaemonContext context) throws Exception {
@@ -68,12 +69,14 @@ public class PrivilegedNfsGatewayStarter implements Daemon {
 
   @Override
   public void start() throws Exception {
-    Nfs3.startService(args, registrationSocket);
+    nfs3Server = Nfs3.startService(args, registrationSocket);
   }
 
   @Override
   public void stop() throws Exception {
-    // Nothing to do.
+    if (nfs3Server != null) {
+      nfs3Server.stop();
+    }
   }
 
   @Override


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


[38/50] [abbrv] hadoop git commit: YARN-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/a2109074
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/a2109074
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/a2109074

Branch: refs/heads/YARN-5734
Commit: a210907466e0b46c4224893db3c78b94521251c2
Parents: 36c474b
Author: Xuan <xg...@apache.org>
Authored: Mon Jul 31 11:49:05 2017 -0700
Committer: Jonathan Hung <jh...@linkedin.com>
Committed: Wed Sep 27 19:52:58 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/a2109074/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/a2109074/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/a2109074/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/a2109074/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


[21/50] [abbrv] hadoop git commit: HADOOP-14890 Move up to AWS SDK 1.11.199. Contributed by Steve Loughran.

Posted by jh...@apache.org.
HADOOP-14890 Move up to AWS SDK 1.11.199. Contributed by Steve Loughran.


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

Branch: refs/heads/YARN-5734
Commit: a2b31e355a73c37dada15a18a90a690314be7fd3
Parents: df800f6
Author: Aaron Fabbri <fa...@apache.org>
Authored: Mon Sep 25 17:11:12 2017 -0700
Committer: Aaron Fabbri <fa...@apache.org>
Committed: Mon Sep 25 17:12:01 2017 -0700

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a2b31e35/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 7b342c4..915bd4e 100755
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -135,7 +135,7 @@
     <make-maven-plugin.version>1.0-beta-1</make-maven-plugin.version>
     <native-maven-plugin.version>1.0-alpha-8</native-maven-plugin.version>
     <surefire.fork.timeout>900</surefire.fork.timeout>
-    <aws-java-sdk.version>1.11.134</aws-java-sdk.version>
+    <aws-java-sdk.version>1.11.199</aws-java-sdk.version>
     <hsqldb.version>2.3.4</hsqldb.version>
     <!-- the version of Hadoop declared in the version resources; can be overridden
     so that Hadoop 3.x can declare itself a 2.x artifact. -->


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


[10/50] [abbrv] hadoop git commit: HDFS-12516. Suppress the fsnamesystem lock warning on nn startup. Contributed by Ajay Kumar.

Posted by jh...@apache.org.
HDFS-12516. Suppress the fsnamesystem lock warning on nn startup. 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/d0b2c585
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/d0b2c585
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/d0b2c585

Branch: refs/heads/YARN-5734
Commit: d0b2c5850b523a3888b2fadcfcdf6edbed33f221
Parents: 415e5a1
Author: Anu Engineer <ae...@apache.org>
Authored: Sun Sep 24 21:03:06 2017 -0700
Committer: Anu Engineer <ae...@apache.org>
Committed: Sun Sep 24 21:03:06 2017 -0700

----------------------------------------------------------------------
 .../hdfs/server/namenode/FSNamesystem.java      |  7 ++-
 .../hdfs/server/namenode/FSNamesystemLock.java  | 27 +++++++++--
 .../server/namenode/TestFSNamesystemLock.java   | 47 +++++++++++++++++++-
 3 files changed, 76 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d0b2c585/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 967f26b..217df94 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
@@ -1095,7 +1095,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       if (!success) {
         fsImage.close();
       }
-      writeUnlock("loadFSImage");
+      writeUnlock("loadFSImage", true);
     }
     imageLoadComplete();
   }
@@ -1586,6 +1586,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   public void writeUnlock(String opName) {
     this.fsLock.writeUnlock(opName);
   }
+
+  public void writeUnlock(String opName, boolean suppressWriteLockReport) {
+    this.fsLock.writeUnlock(opName, suppressWriteLockReport);
+  }
+
   @Override
   public boolean hasWriteLock() {
     return this.fsLock.isWriteLockedByCurrentThread();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d0b2c585/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystemLock.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystemLock.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystemLock.java
index 8c60faa..32c7efa 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystemLock.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystemLock.java
@@ -207,13 +207,34 @@ class FSNamesystemLock {
     }
   }
 
+  /**
+   * Unlocks FSNameSystem write lock. This internally calls {@link
+   * FSNamesystemLock#writeUnlock(String, boolean)}
+   */
   public void writeUnlock() {
-    writeUnlock(OP_NAME_OTHER);
+    writeUnlock(OP_NAME_OTHER, false);
   }
 
+  /**
+   * Unlocks FSNameSystem write lock. This internally calls {@link
+   * FSNamesystemLock#writeUnlock(String, boolean)}
+   *
+   * @param opName Operation name.
+   */
   public void writeUnlock(String opName) {
-    final boolean needReport = coarseLock.getWriteHoldCount() == 1 &&
-        coarseLock.isWriteLockedByCurrentThread();
+    writeUnlock(opName, false);
+  }
+
+  /**
+   * Unlocks FSNameSystem write lock.
+   *
+   * @param opName Operation name
+   * @param suppressWriteLockReport When false, event of write lock being held
+   * for long time will be logged in logs and metrics.
+   */
+  public void writeUnlock(String opName, boolean suppressWriteLockReport) {
+    final boolean needReport = !suppressWriteLockReport && coarseLock
+        .getWriteHoldCount() == 1 && coarseLock.isWriteLockedByCurrentThread();
     final long currentTimeNanos = timer.monotonicNowNanos();
     final long writeLockIntervalNanos =
         currentTimeNanos - writeLockHeldTimeStampNanos;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d0b2c585/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystemLock.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystemLock.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystemLock.java
index 7f9746d..94e0b33 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystemLock.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystemLock.java
@@ -38,6 +38,7 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import java.util.regex.Pattern;
+import org.slf4j.LoggerFactory;
 
 import static org.junit.Assert.*;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_FSLOCK_FAIR_KEY;
@@ -347,7 +348,7 @@ public class TestFSNamesystemLock {
 
     fsLock.writeLock();
     timer.advance(1);
-    fsLock.writeUnlock("baz");
+    fsLock.writeUnlock("baz", false);
 
     MetricsRecordBuilder rb = MetricsAsserts.mockMetricsRecordBuilder();
     rates.snapshot(rb, true);
@@ -360,4 +361,48 @@ public class TestFSNamesystemLock {
     assertCounter("FSNWriteLockBazNanosNumOps", 1L, rb);
   }
 
+  /**
+   * Test to suppress FSNameSystem write lock report when it is held for long
+   * time.
+   */
+  @Test(timeout = 45000)
+  public void testFSWriteLockReportSuppressed() throws Exception {
+    final long writeLockReportingThreshold = 1L;
+    final long writeLockSuppressWarningInterval = 10L;
+    Configuration conf = new Configuration();
+    conf.setLong(
+        DFSConfigKeys.DFS_NAMENODE_WRITE_LOCK_REPORTING_THRESHOLD_MS_KEY,
+        writeLockReportingThreshold);
+    conf.setTimeDuration(DFSConfigKeys.DFS_LOCK_SUPPRESS_WARNING_INTERVAL_KEY,
+        writeLockSuppressWarningInterval, TimeUnit.MILLISECONDS);
+
+    final FakeTimer timer = new FakeTimer();
+    final FSNamesystemLock fsnLock = new FSNamesystemLock(conf, null, timer);
+    timer.advance(writeLockSuppressWarningInterval);
+
+    LogCapturer logs = LogCapturer.captureLogs(FSNamesystem.LOG);
+    GenericTestUtils
+        .setLogLevel(LoggerFactory.getLogger(FSNamesystem.class.getName()),
+            org.slf4j.event.Level.INFO);
+
+    // Should trigger the write lock report
+    fsnLock.writeLock();
+    timer.advance(writeLockReportingThreshold + 100);
+    fsnLock.writeUnlock();
+    assertTrue(logs.getOutput().contains(
+        "FSNamesystem write lock held for"));
+
+    logs.clearOutput();
+
+    // Suppress report if the write lock is held for a long time
+    fsnLock.writeLock();
+    timer.advance(writeLockReportingThreshold + 100);
+    fsnLock.writeUnlock("testFSWriteLockReportSuppressed", true);
+    assertFalse(logs.getOutput().contains(GenericTestUtils.getMethodName()));
+    assertFalse(logs.getOutput().contains(
+        "Number of suppressed write-lock reports:"));
+    assertFalse(logs.getOutput().contains(
+        "FSNamesystem write lock held for"));
+  }
+
 }
\ 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


[22/50] [abbrv] hadoop git commit: HADOOP-14893. WritableRpcEngine should use Time.monotonicNow. Contributed by Chetna Chaudhari.

Posted by jh...@apache.org.
HADOOP-14893. WritableRpcEngine should use Time.monotonicNow. Contributed by Chetna Chaudhari.


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

Branch: refs/heads/YARN-5734
Commit: d08b8c801a908b4242e7b21a54f3b1e4072f1eae
Parents: a2b31e3
Author: Akira Ajisaka <aa...@apache.org>
Authored: Tue Sep 26 16:45:39 2017 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Tue Sep 26 16:46:03 2017 +0900

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d08b8c80/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java
index fa0726d..f2b5862 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java
@@ -231,7 +231,7 @@ public class WritableRpcEngine implements RpcEngine {
       throws Throwable {
       long startTime = 0;
       if (LOG.isDebugEnabled()) {
-        startTime = Time.now();
+        startTime = Time.monotonicNow();
       }
 
       // if Tracing is on then start a new span for this rpc.
@@ -251,7 +251,7 @@ public class WritableRpcEngine implements RpcEngine {
         if (traceScope != null) traceScope.close();
       }
       if (LOG.isDebugEnabled()) {
-        long callTime = Time.now() - startTime;
+        long callTime = Time.monotonicNow() - startTime;
         LOG.debug("Call: " + method.getName() + " " + callTime);
       }
       return value.get();


---------------------------------------------------------------------
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-7252. Removing queue then failing over results in exception

Posted by jh...@apache.org.
YARN-7252. Removing queue then failing over results in exception


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

Branch: refs/heads/YARN-5734
Commit: 5c775e61ec64096bcd947956868fd97e96e1725a
Parents: 4bbedd0
Author: Jonathan Hung <jh...@linkedin.com>
Authored: Tue Sep 26 11:41:05 2017 -0700
Committer: Jonathan Hung <jh...@linkedin.com>
Committed: Wed Sep 27 19:52:59 2017 -0700

----------------------------------------------------------------------
 .../capacity/CapacitySchedulerContext.java      |   6 ++
 .../capacity/CapacitySchedulerQueueManager.java |  11 +-
 .../capacity/conf/TestZKConfigurationStore.java | 102 +++++++++++++++++++
 3 files changed, 117 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5c775e61/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerContext.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/CapacitySchedulerContext.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/CapacitySchedulerContext.java
index 9aeaec6..7c918a5 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/CapacitySchedulerContext.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/CapacitySchedulerContext.java
@@ -88,4 +88,10 @@ public interface CapacitySchedulerContext {
    * @return Max Cluster level App priority.
    */
   Priority getMaxClusterLevelAppPriority();
+
+  /**
+   * Returns if configuration is mutable.
+   * @return if configuration is mutable
+   */
+  boolean isConfigurationMutable();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5c775e61/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 1ceb6fb..48c289f 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
@@ -33,6 +33,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ha.HAServiceProtocol;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.QueueState;
@@ -170,8 +171,14 @@ public class CapacitySchedulerQueueManager implements SchedulerQueueManager<
     CSQueue newRoot =  parseQueue(this.csContext, newConf, null,
         CapacitySchedulerConfiguration.ROOT, newQueues, queues, NOOP);
 
-    // Ensure queue hiearchy in the new XML file is proper.
-    validateQueueHierarchy(queues, newQueues);
+    // When failing over, if using configuration store, don't validate queue
+    // hierarchy since queues can be removed without being STOPPED.
+    if (!csContext.isConfigurationMutable() ||
+        csContext.getRMContext().getHAServiceState()
+            != HAServiceProtocol.HAServiceState.STANDBY) {
+      // Ensure queue hiearchy in the new XML file is proper.
+      validateQueueHierarchy(queues, newQueues);
+    }
 
     // Add new queues and delete OldQeueus only after validation.
     updateQueues(queues, newQueues);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5c775e61/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/TestZKConfigurationStore.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/TestZKConfigurationStore.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/TestZKConfigurationStore.java
index 355f741..0cf5e6f 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/TestZKConfigurationStore.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/TestZKConfigurationStore.java
@@ -38,18 +38,21 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.MutableConfSchedu
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.MutableConfigurationProvider;
 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.webapp.dao.QueueConfigInfo;
 import org.apache.hadoop.yarn.webapp.dao.SchedConfUpdateInfo;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
 import java.io.IOException;
+import java.util.Arrays;
 import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.Map;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
 
 /**
  * Tests {@link ZKConfigurationStore}.
@@ -303,6 +306,105 @@ public class TestZKConfigurationStore extends ConfigurationStoreBaseTest {
     rm2.close();
   }
 
+  /**
+   * When failing over, if RM1 stopped and removed a queue that RM2 has in
+   * memory, failing over to RM2 should not throw an exception.
+   * @throws Exception
+   */
+  @Test
+  public void testFailoverAfterRemoveQueue() throws Exception {
+    HAServiceProtocol.StateChangeRequestInfo req =
+        new HAServiceProtocol.StateChangeRequestInfo(
+            HAServiceProtocol.RequestSource.REQUEST_BY_USER);
+
+    Configuration conf1 = createRMHAConf("rm1,rm2", "rm1", 1234);
+    ResourceManager rm1 = new MockRM(conf1);
+    rm1.start();
+    rm1.getRMContext().getRMAdminService().transitionToActive(req);
+    assertEquals("RM with ZKStore didn't start",
+        Service.STATE.STARTED, rm1.getServiceState());
+    assertEquals("RM should be Active",
+        HAServiceProtocol.HAServiceState.ACTIVE,
+        rm1.getRMContext().getRMAdminService().getServiceStatus().getState());
+
+    Configuration conf2 = createRMHAConf("rm1,rm2", "rm2", 5678);
+    ResourceManager rm2 = new MockRM(conf2);
+    rm2.start();
+    assertEquals("RM should be Standby",
+        HAServiceProtocol.HAServiceState.STANDBY,
+        rm2.getRMContext().getRMAdminService().getServiceStatus().getState());
+
+    UserGroupInformation user = UserGroupInformation
+        .createUserForTesting(TEST_USER, new String[0]);
+    MutableConfigurationProvider confProvider = ((MutableConfScheduler)
+        rm1.getResourceScheduler()).getMutableConfProvider();
+    // Add root.a
+    SchedConfUpdateInfo schedConfUpdateInfo = new SchedConfUpdateInfo();
+    Map<String, String> addParams = new HashMap<>();
+    addParams.put("capacity", "100");
+    QueueConfigInfo addInfo = new QueueConfigInfo("root.a", addParams);
+    schedConfUpdateInfo.getAddQueueInfo().add(addInfo);
+    // Stop root.default
+    Map<String, String> stopParams = new HashMap<>();
+    stopParams.put("state", "STOPPED");
+    stopParams.put("capacity", "0");
+    QueueConfigInfo stopInfo = new QueueConfigInfo("root.default", stopParams);
+    schedConfUpdateInfo.getUpdateQueueInfo().add(stopInfo);
+    confProvider.logAndApplyMutation(user, schedConfUpdateInfo);
+    rm1.getResourceScheduler().reinitialize(conf1, rm1.getRMContext());
+    confProvider.confirmPendingMutation(true);
+    assertTrue(Arrays.asList(((MutableConfScheduler) rm1.getResourceScheduler())
+        .getConfiguration().get("yarn.scheduler.capacity.root.queues").split
+            (",")).contains("a"));
+
+    // Remove root.default
+    schedConfUpdateInfo.getUpdateQueueInfo().clear();
+    schedConfUpdateInfo.getAddQueueInfo().clear();
+    schedConfUpdateInfo.getRemoveQueueInfo().add("root.default");
+    confProvider.logAndApplyMutation(user, schedConfUpdateInfo);
+    rm1.getResourceScheduler().reinitialize(conf1, rm1.getRMContext());
+    confProvider.confirmPendingMutation(true);
+    assertEquals("a", ((MutableConfScheduler) rm1.getResourceScheduler())
+        .getConfiguration().get("yarn.scheduler.capacity.root.queues"));
+
+    // Start RM2 and verifies it starts with updated configuration
+    rm2.getRMContext().getRMAdminService().transitionToActive(req);
+    assertEquals("RM with ZKStore didn't start",
+        Service.STATE.STARTED, rm2.getServiceState());
+    assertEquals("RM should be Active",
+        HAServiceProtocol.HAServiceState.ACTIVE,
+        rm2.getRMContext().getRMAdminService().getServiceStatus().getState());
+
+    for (int i = 0; i < ZK_TIMEOUT_MS / 50; i++) {
+      if (HAServiceProtocol.HAServiceState.ACTIVE ==
+          rm1.getRMContext().getRMAdminService().getServiceStatus()
+              .getState()) {
+        Thread.sleep(100);
+      }
+    }
+    assertEquals("RM should have been fenced",
+        HAServiceProtocol.HAServiceState.STANDBY,
+        rm1.getRMContext().getRMAdminService().getServiceStatus().getState());
+    assertEquals("RM should be Active",
+        HAServiceProtocol.HAServiceState.ACTIVE,
+        rm2.getRMContext().getRMAdminService().getServiceStatus().getState());
+
+    assertEquals("a", ((MutableCSConfigurationProvider) (
+        (CapacityScheduler) rm2.getResourceScheduler())
+        .getMutableConfProvider()).getConfStore().retrieve()
+        .get("yarn.scheduler.capacity.root.queues"));
+    assertEquals("a", ((MutableConfScheduler) rm2.getResourceScheduler())
+        .getConfiguration().get("yarn.scheduler.capacity.root.queues"));
+    // Transition to standby will set RM's HA status and then reinitialize in
+    // a separate thread. Despite asserting for STANDBY state, it's
+    // possible for reinitialization to be unfinished. Wait here for it to
+    // finish, otherwise closing rm1 will close zkManager and the unfinished
+    // reinitialization will throw an exception.
+    Thread.sleep(10000);
+    rm1.close();
+    rm2.close();
+  }
+
   @Override
   public YarnConfigurationStore createConfStore() {
     return new ZKConfigurationStore();


---------------------------------------------------------------------
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-6840. Implement zookeeper based store for scheduler configuration updates. (Jonathan Hung via wangda)

Posted by jh...@apache.org.
YARN-6840. Implement zookeeper based store for scheduler configuration updates. (Jonathan Hung via wangda)

Change-Id: I9debea674fe8c7e4109d4ca136965a1ea4c48bcc


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

Branch: refs/heads/YARN-5734
Commit: 76746cb8387b95a8b2e1ec39f0764a43a965ddf3
Parents: a3aec1c
Author: Wangda Tan <wa...@apache.org>
Authored: Mon Sep 18 09:53:42 2017 -0700
Committer: Jonathan Hung <jh...@linkedin.com>
Committed: Wed Sep 27 19:52:58 2017 -0700

----------------------------------------------------------------------
 .../hadoop/yarn/conf/YarnConfiguration.java     |  14 +-
 .../src/main/resources/yarn-default.xml         |  15 +-
 .../server/resourcemanager/AdminService.java    |  18 +-
 .../server/resourcemanager/ResourceManager.java |  24 +-
 .../RMStateVersionIncompatibleException.java    |   2 +-
 .../recovery/ZKRMStateStore.java                |   5 +-
 .../scheduler/MutableConfScheduler.java         |  22 +-
 .../scheduler/MutableConfigurationProvider.java |  36 ++-
 .../scheduler/capacity/CapacityScheduler.java   |  22 +-
 .../conf/InMemoryConfigurationStore.java        |  71 +++--
 .../conf/LeveldbConfigurationStore.java         | 168 +++++-----
 .../conf/MutableCSConfigurationProvider.java    | 148 +++++----
 .../capacity/conf/YarnConfigurationStore.java   | 132 ++++----
 .../capacity/conf/ZKConfigurationStore.java     | 235 ++++++++++++++
 .../resourcemanager/webapp/RMWebServices.java   |  26 +-
 .../conf/ConfigurationStoreBaseTest.java        |  90 ++++++
 .../conf/TestInMemoryConfigurationStore.java    |  30 ++
 .../TestMutableCSConfigurationProvider.java     |  18 +-
 .../conf/TestYarnConfigurationStore.java        |  71 -----
 .../capacity/conf/TestZKConfigurationStore.java | 312 +++++++++++++++++++
 20 files changed, 1037 insertions(+), 422 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/76746cb8/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 612e3d1..e3ce3ecc 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
@@ -678,6 +678,7 @@ public class YarnConfiguration extends Configuration {
       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 ZK_CONFIGURATION_STORE = "zk";
   public static final String DEFAULT_CONFIGURATION_STORE =
       MEMORY_CONFIGURATION_STORE;
   public static final String RM_SCHEDCONF_STORE_PATH = YARN_PREFIX
@@ -689,9 +690,16 @@ public class YarnConfiguration extends Configuration {
   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_SCHEDCONF_MAX_LOGS =
+      YARN_PREFIX + "scheduler.configuration.store.max-logs";
+  public static final long DEFAULT_RM_SCHEDCONF_LEVELDB_MAX_LOGS = 1000;
+  public static final long DEFAULT_RM_SCHEDCONF_ZK_MAX_LOGS = 1000;
+
+  /** Parent znode path under which ZKConfigurationStore will create znodes. */
+  public static final String RM_SCHEDCONF_STORE_ZK_PARENT_PATH = YARN_PREFIX
+      + "scheduler.configuration.zk-store.parent-path";
+  public static final String DEFAULT_RM_SCHEDCONF_STORE_ZK_PARENT_PATH =
+      "/confstore";
 
   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/76746cb8/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 63c6f09..a6521de 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
@@ -3412,11 +3412,20 @@
 
   <property>
     <description>
-      The max number of configuration change log entries kept in LevelDB config
+      The max number of configuration change log entries kept in config
       store, when yarn.scheduler.configuration.store.class is configured to be
-      "leveldb". Default is 1000.
+      "leveldb" or "zk". Default is 1000 for either.
     </description>
-    <name>yarn.scheduler.configuration.leveldb-store.max-logs</name>
+    <name>yarn.scheduler.configuration.store.max-logs</name>
     <value>1000</value>
   </property>
+
+  <property>
+    <description>
+      ZK root node path for configuration store when using zookeeper-based
+      configuration store.
+    </description>
+    <name>yarn.scheduler.configuration.zk-store.parent-path</name>
+    <value>/confstore</value>
+  </property>
 </configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76746cb8/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 fd9e849..6c0a854 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
@@ -387,9 +387,7 @@ 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()) {
+      if (isSchedulerMutable()) {
         throw new IOException("Scheduler configuration is mutable. " +
             operation + " is not allowed in this scenario.");
       }
@@ -413,6 +411,12 @@ public class AdminService extends CompositeService implements
     }
   }
 
+  private boolean isSchedulerMutable() {
+    ResourceScheduler scheduler = rm.getRMContext().getScheduler();
+    return (scheduler instanceof MutableConfScheduler
+        && ((MutableConfScheduler) scheduler).isConfigurationMutable());
+  }
+
   @Override
   public RefreshNodesResponse refreshNodes(RefreshNodesRequest request)
       throws YarnException, StandbyException {
@@ -721,6 +725,14 @@ public class AdminService extends CompositeService implements
   void refreshAll() throws ServiceFailedException {
     try {
       checkAcls("refreshAll");
+      if (isSchedulerMutable()) {
+        try {
+          ((MutableConfScheduler) rm.getRMContext().getScheduler())
+              .getMutableConfProvider().reloadConfigurationFromStore();
+        } catch (Exception e) {
+          throw new IOException("Failed to refresh configuration:", e);
+        }
+      }
       refreshQueues();
       refreshNodes();
       refreshSuperUserGroupsConfiguration();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76746cb8/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 e53a42c..793babe 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
@@ -351,7 +351,7 @@ public class ResourceManager extends CompositeService implements Recoverable {
         conf.getBoolean(YarnConfiguration.CURATOR_LEADER_ELECTOR,
             YarnConfiguration.DEFAULT_CURATOR_LEADER_ELECTOR_ENABLED);
     if (curatorEnabled) {
-      this.zkManager = createAndStartZKManager(conf);
+      this.zkManager = getAndStartZKManager(conf);
       elector = new CuratorBasedElectorService(this);
     } else {
       elector = new ActiveStandbyElectorBasedElectorService(this);
@@ -360,13 +360,16 @@ public class ResourceManager extends CompositeService implements Recoverable {
   }
 
   /**
-   * Create and ZooKeeper Curator manager.
+   * Get ZooKeeper Curator manager, creating and starting if not exists.
    * @param config Configuration for the ZooKeeper curator.
-   * @return New ZooKeeper Curator manager.
+   * @return ZooKeeper Curator manager.
    * @throws IOException If it cannot create the manager.
    */
-  public ZKCuratorManager createAndStartZKManager(Configuration config)
-      throws IOException {
+  public synchronized ZKCuratorManager getAndStartZKManager(Configuration
+      config) throws IOException {
+    if (this.zkManager != null) {
+      return zkManager;
+    }
     ZKCuratorManager manager = new ZKCuratorManager(config);
 
     // Get authentication
@@ -386,15 +389,8 @@ public class ResourceManager extends CompositeService implements Recoverable {
     }
 
     manager.start(authInfos);
-    return manager;
-  }
-
-  /**
-   * Get the ZooKeeper Curator manager.
-   * @return ZooKeeper Curator manager.
-   */
-  public ZKCuratorManager getZKManager() {
-    return this.zkManager;
+    this.zkManager = manager;
+    return zkManager;
   }
 
   public CuratorFramework getCurator() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76746cb8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateVersionIncompatibleException.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/RMStateVersionIncompatibleException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateVersionIncompatibleException.java
index 135868f..d5fce36 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateVersionIncompatibleException.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateVersionIncompatibleException.java
@@ -22,7 +22,7 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
 
 /**
  * This exception is thrown by ResourceManager if it's loading an incompatible
- * version of state from state store on recovery.
+ * version of storage on recovery.
  */
 public class RMStateVersionIncompatibleException extends YarnException {
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76746cb8/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 ac67dcd..5bff77f 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
@@ -327,10 +327,7 @@ public class ZKRMStateStore extends RMStateStore {
     amrmTokenSecretManagerRoot =
         getNodePath(zkRootNodePath, AMRMTOKEN_SECRET_MANAGER_ROOT);
     reservationRoot = getNodePath(zkRootNodePath, RESERVATION_SYSTEM_ROOT);
-    zkManager = resourceManager.getZKManager();
-    if (zkManager == null) {
-      zkManager = resourceManager.createAndStartZKManager(conf);
-    }
+    zkManager = resourceManager.getAndStartZKManager(conf);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76746cb8/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 313bf6a..6f677fb 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
@@ -18,11 +18,6 @@
 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;
 
 /**
  * Interface for a scheduler that supports changing configuration at runtime.
@@ -31,16 +26,6 @@ import java.io.IOException;
 public interface MutableConfScheduler extends ResourceScheduler {
 
   /**
-   * Update the scheduler's configuration.
-   * @param user Caller of this update
-   * @param confUpdate configuration update
-   * @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, YarnException;
-
-  /**
    * Get the scheduler configuration.
    * @return the scheduler configuration
    */
@@ -58,4 +43,11 @@ public interface MutableConfScheduler extends ResourceScheduler {
    * @return whether scheduler configuration is mutable or not.
    */
   boolean isConfigurationMutable();
+
+  /**
+   * Get scheduler's configuration provider, so other classes can directly
+   * call mutation APIs on configuration provider.
+   * @return scheduler's configuration provider
+   */
+  MutableConfigurationProvider getMutableConfProvider();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76746cb8/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 9baf1ad..f8e8814 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,30 +19,40 @@
 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;
-
 /**
  * Interface for allowing changing scheduler configurations.
  */
 public interface MutableConfigurationProvider {
 
   /**
-   * Apply transactions which were not committed.
-   * @throws IOException if recovery fails
+   * Get the acl mutation policy for this configuration provider.
+   * @return The acl mutation policy.
+   */
+  ConfigurationMutationACLPolicy getAclMutationPolicy();
+
+  /**
+   * Called when a new ResourceManager is starting/becomes active. Ensures
+   * configuration is up-to-date.
+   * @throws Exception if configuration could not be refreshed from store
    */
-  void recoverConf() throws IOException;
+  void reloadConfigurationFromStore() throws Exception;
 
   /**
-   * 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
-   * @throws YarnException if reservation system could not be reinitialized
+   * Log user's requested configuration mutation, and applies it in-memory.
+   * @param user User who requested the change
+   * @param confUpdate User's requested configuration change
+   * @throws Exception if logging the mutation fails
    */
-  void mutateConfiguration(UserGroupInformation user, SchedConfUpdateInfo
-      confUpdate) throws IOException, YarnException;
+  void logAndApplyMutation(UserGroupInformation user, SchedConfUpdateInfo
+      confUpdate) throws Exception;
 
+  /**
+   * Confirm last logged mutation.
+   * @param isValid if the last logged mutation is applied to scheduler
+   *                properly.
+   * @throws Exception if confirming mutation fails
+   */
+  void confirmPendingMutation(boolean isValid) throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76746cb8/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 51ee6a7..16b27c1 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
@@ -141,7 +141,6 @@ 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;
@@ -393,9 +392,6 @@ public class CapacityScheduler extends
   @Override
   public void serviceStart() throws Exception {
     startSchedulerThreads();
-    if (this.csConfProvider instanceof MutableConfigurationProvider) {
-      ((MutableConfigurationProvider) csConfProvider).recoverConf();
-    }
     super.serviceStart();
   }
 
@@ -2619,19 +2615,15 @@ public class CapacityScheduler extends
   }
 
   @Override
-  public void updateConfiguration(UserGroupInformation user,
-      SchedConfUpdateInfo confUpdate) throws IOException, YarnException {
-    if (isConfigurationMutable()) {
-      ((MutableConfigurationProvider) csConfProvider).mutateConfiguration(
-          user, confUpdate);
-    } else {
-      throw new UnsupportedOperationException("Configured CS configuration " +
-          "provider does not support updating configuration.");
-    }
+  public boolean isConfigurationMutable() {
+    return csConfProvider instanceof MutableConfigurationProvider;
   }
 
   @Override
-  public boolean isConfigurationMutable() {
-    return csConfProvider instanceof MutableConfigurationProvider;
+  public MutableConfigurationProvider getMutableConfProvider() {
+    if (isConfigurationMutable()) {
+      return (MutableConfigurationProvider) csConfProvider;
+    }
+    return null;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76746cb8/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 c63734d..d69c236 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
@@ -19,8 +19,9 @@
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.conf;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.server.records.Version;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 
-import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 
@@ -28,48 +29,35 @@ 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 {
+public class InMemoryConfigurationStore extends YarnConfigurationStore {
 
   private Configuration schedConf;
-  private LinkedList<LogMutation> pendingMutations;
-  private long pendingId;
+  private LogMutation pendingMutation;
 
   @Override
-  public void initialize(Configuration conf, Configuration schedConf) {
+  public void initialize(Configuration conf, Configuration schedConf,
+      RMContext rmContext) {
     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;
+  public void logMutation(LogMutation logMutation) {
+    pendingMutation = logMutation;
   }
 
   @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()) {
-            if (kv.getValue() == null) {
-              schedConf.unset(kv.getKey());
-            } else {
-              schedConf.set(kv.getKey(), kv.getValue());
-            }
-          }
+  public void confirmMutation(boolean isValid) {
+    if (isValid) {
+      for (Map.Entry<String, String> kv : pendingMutation.getUpdates()
+          .entrySet()) {
+        if (kv.getValue() == null) {
+          schedConf.unset(kv.getKey());
+        } else {
+          schedConf.set(kv.getKey(), kv.getValue());
         }
-        return true;
       }
-      mutation = pendingMutations.poll();
     }
-    return false;
+    pendingMutation = null;
   }
 
   @Override
@@ -78,13 +66,30 @@ public class InMemoryConfigurationStore implements YarnConfigurationStore {
   }
 
   @Override
-  public synchronized List<LogMutation> getPendingMutations() {
-    return new LinkedList<>(pendingMutations);
+  public List<LogMutation> getConfirmedConfHistory(long fromId) {
+    // Unimplemented.
+    return null;
   }
 
   @Override
-  public List<LogMutation> getConfirmedConfHistory(long fromId) {
-    // Unimplemented.
+  public Version getConfStoreVersion() throws Exception {
+    // Does nothing.
     return null;
   }
+
+  @Override
+  public void storeVersion() throws Exception {
+    // Does nothing.
+  }
+
+  @Override
+  public Version getCurrentVersion() {
+    // Does nothing.
+    return null;
+  }
+
+  @Override
+  public void checkVersion() {
+    // Does nothing. (Version is always compatible since it's in memory)
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76746cb8/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
index 1280fab..1b0eb9f 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/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
@@ -26,6 +26,10 @@ 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.apache.hadoop.yarn.proto.YarnServerCommonProtos;
+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.RMContext;
 import org.fusesource.leveldbjni.JniDBFactory;
 import org.fusesource.leveldbjni.internal.NativeDB;
 import org.iq80.leveldb.DB;
@@ -55,58 +59,32 @@ import static org.fusesource.leveldbjni.JniDBFactory.bytes;
 /**
  * A LevelDB implementation of {@link YarnConfigurationStore}.
  */
-public class LeveldbConfigurationStore implements YarnConfigurationStore {
+public class LeveldbConfigurationStore extends 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 static final String LOG_KEY = "log";
+  private static final String VERSION_KEY = "version";
 
   private DB db;
-  // Txnid for the last transaction logged to the store.
-  private long txnId = 0;
-  private long minTxn = 0;
   private long maxLogs;
   private Configuration conf;
-  private LinkedList<LogMutation> pendingMutations = new LinkedList<>();
+  private LogMutation pendingMutation;
+  private static final Version CURRENT_VERSION_INFO = Version
+      .newInstance(0, 1);
   private Timer compactionTimer;
   private long compactionIntervalMsec;
 
   @Override
-  public void initialize(Configuration config, Configuration schedConf)
-      throws IOException {
+  public void initialize(Configuration config, Configuration schedConf,
+      RMContext rmContext) 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()));
-        txnId++;
-      }
-      // 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.RM_SCHEDCONF_MAX_LOGS,
           YarnConfiguration.DEFAULT_RM_SCHEDCONF_LEVELDB_MAX_LOGS);
       this.compactionIntervalMsec = config.getLong(
           YarnConfiguration.RM_SCHEDCONF_LEVELDB_COMPACTION_INTERVAL_SECS,
@@ -127,33 +105,23 @@ public class LeveldbConfigurationStore implements YarnConfigurationStore {
       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);
+        if (key1Str.equals(key2Str)) {
+          return 0;
+        } else if (key1Str.equals(VERSION_KEY)) {
+          return -1;
+        } else if (key2Str.equals(VERSION_KEY)) {
+          return 1;
+        } else if (key1Str.equals(LOG_KEY)) {
+          return -1;
+        } else if (key2Str.equals(LOG_KEY)) {
+          return 1;
         }
-        return key1Txn - key2Txn;
+        return key1Str.compareTo(key2Str);
       }
 
       @Override
       public String name() {
-        return "logComparator";
+        return "keyComparator";
       }
 
       public byte[] findShortestSeparator(byte[] start, byte[] limit) {
@@ -164,6 +132,7 @@ public class LeveldbConfigurationStore implements YarnConfigurationStore {
         return key;
       }
     });
+
     LOG.info("Using conf database at " + storeRoot);
     File dbfile = new File(storeRoot.toString());
     try {
@@ -179,7 +148,6 @@ public class LeveldbConfigurationStore implements YarnConfigurationStore {
           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);
@@ -208,28 +176,22 @@ public class LeveldbConfigurationStore implements YarnConfigurationStore {
   }
 
   @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++;
+  public void logMutation(LogMutation logMutation) throws IOException {
+    LinkedList<LogMutation> logs = deserLogMutations(db.get(bytes(LOG_KEY)));
+    logs.add(logMutation);
+    if (logs.size() > maxLogs) {
+      logs.removeFirst();
     }
-    db.write(logBatch);
-    pendingMutations.add(logMutation);
-    return txnId;
+    db.put(bytes(LOG_KEY), serLogMutations(logs));
+    pendingMutation = logMutation;
   }
 
   @Override
-  public synchronized boolean confirmMutation(long id, boolean isValid)
-      throws IOException {
+  public void confirmMutation(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()) {
+          pendingMutation.getUpdates().entrySet()) {
         if (changes.getValue() == null || changes.getValue().isEmpty()) {
           updateBatch.delete(bytes(changes.getKey()));
         } else {
@@ -237,28 +199,24 @@ public class LeveldbConfigurationStore implements YarnConfigurationStore {
         }
       }
     }
-    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;
+    pendingMutation = null;
   }
 
-  private byte[] serLogMutation(LogMutation mutation) throws IOException {
+  private byte[] serLogMutations(LinkedList<LogMutation> mutations) throws
+      IOException {
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
     try (ObjectOutput oos = new ObjectOutputStream(baos)) {
-      oos.writeObject(mutation);
+      oos.writeObject(mutations);
       oos.flush();
       return baos.toByteArray();
     }
   }
-  private LogMutation deserLogMutation(byte[] mutation) throws IOException {
+  private LinkedList<LogMutation> deserLogMutations(byte[] mutations) throws
+      IOException {
     try (ObjectInput input = new ObjectInputStream(
-        new ByteArrayInputStream(mutation))) {
-      return (LogMutation) input.readObject();
+        new ByteArrayInputStream(mutations))) {
+      return (LinkedList<LogMutation>) input.readObject();
     } catch (ClassNotFoundException e) {
       throw new IOException(e);
     }
@@ -267,7 +225,7 @@ public class LeveldbConfigurationStore implements YarnConfigurationStore {
   @Override
   public synchronized Configuration retrieve() {
     DBIterator itr = db.iterator();
-    itr.seek(bytes(LOG_COMMITTED_TXN));
+    itr.seek(bytes(LOG_KEY));
     Configuration config = new Configuration(false);
     itr.next();
     while (itr.hasNext()) {
@@ -279,11 +237,6 @@ public class LeveldbConfigurationStore implements YarnConfigurationStore {
   }
 
   @Override
-  public List<LogMutation> getPendingMutations() {
-    return new LinkedList<>(pendingMutations);
-  }
-
-  @Override
   public List<LogMutation> getConfirmedConfHistory(long fromId) {
     return null; // unimplemented
   }
@@ -299,6 +252,39 @@ public class LeveldbConfigurationStore implements YarnConfigurationStore {
     }
   }
 
+  // TODO: following is taken from LeveldbRMStateStore
+  @Override
+  public Version getConfStoreVersion() throws Exception {
+    Version version = null;
+    try {
+      byte[] data = db.get(bytes(VERSION_KEY));
+      if (data != null) {
+        version = new VersionPBImpl(YarnServerCommonProtos.VersionProto
+            .parseFrom(data));
+      }
+    } catch (DBException e) {
+      throw new IOException(e);
+    }
+    return version;
+  }
+
+  @Override
+  public void storeVersion() throws Exception {
+    String key = VERSION_KEY;
+    byte[] data = ((VersionPBImpl) CURRENT_VERSION_INFO).getProto()
+        .toByteArray();
+    try {
+      db.put(bytes(key), data);
+    } catch (DBException e) {
+      throw new IOException(e);
+    }
+  }
+
+  @Override
+  public Version getCurrentVersion() {
+    return CURRENT_VERSION_INFO;
+  }
+
   private class CompactionTimerTask extends TimerTask {
     @Override
     public void run() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76746cb8/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 d03b2e2..70d1840 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,20 +18,17 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.conf;
 
+import com.google.common.annotations.VisibleForTesting;
 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;
 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;
 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.webapp.dao.QueueConfigInfo;
@@ -56,6 +53,7 @@ public class MutableCSConfigurationProvider implements CSConfigurationProvider,
       LogFactory.getLog(MutableCSConfigurationProvider.class);
 
   private Configuration schedConf;
+  private Configuration oldConf;
   private YarnConfigurationStore confStore;
   private ConfigurationMutationACLPolicy aclMutationPolicy;
   private RMContext rmContext;
@@ -76,6 +74,9 @@ public class MutableCSConfigurationProvider implements CSConfigurationProvider,
     case YarnConfiguration.LEVELDB_CONFIGURATION_STORE:
       this.confStore = new LeveldbConfigurationStore();
       break;
+    case YarnConfiguration.ZK_CONFIGURATION_STORE:
+      this.confStore = new ZKConfigurationStore();
+      break;
     default:
       this.confStore = YarnConfigurationStoreFactory.getStore(config);
       break;
@@ -89,7 +90,11 @@ public class MutableCSConfigurationProvider implements CSConfigurationProvider,
     for (Map.Entry<String, String> kv : initialSchedConf) {
       schedConf.set(kv.getKey(), kv.getValue());
     }
-    confStore.initialize(config, schedConf);
+    try {
+      confStore.initialize(config, schedConf, rmContext);
+    } catch (Exception e) {
+      throw new IOException(e);
+    }
     // After initializing confStore, the store may already have an existing
     // configuration. Use this one.
     schedConf = confStore.retrieve();
@@ -98,6 +103,11 @@ public class MutableCSConfigurationProvider implements CSConfigurationProvider,
     aclMutationPolicy.init(config, rmContext);
   }
 
+  @VisibleForTesting
+  public YarnConfigurationStore getConfStore() {
+    return confStore;
+  }
+
   @Override
   public CapacitySchedulerConfiguration loadConfiguration(Configuration
       configuration) throws IOException {
@@ -107,16 +117,17 @@ public class MutableCSConfigurationProvider implements CSConfigurationProvider,
   }
 
   @Override
-  public synchronized void mutateConfiguration(UserGroupInformation user,
-      SchedConfUpdateInfo confUpdate) throws IOException, YarnException {
-    if (!aclMutationPolicy.isMutationAllowed(user, confUpdate)) {
-      throw new AccessControlException("User is not admin of all modified" +
-          " queues.");
-    }
-    Configuration oldConf = new Configuration(schedConf);
+  public ConfigurationMutationACLPolicy getAclMutationPolicy() {
+    return aclMutationPolicy;
+  }
+
+  @Override
+  public void logAndApplyMutation(UserGroupInformation user,
+      SchedConfUpdateInfo confUpdate) throws Exception {
+    oldConf = new Configuration(schedConf);
     Map<String, String> kvUpdate = constructKeyValueConfUpdate(confUpdate);
     LogMutation log = new LogMutation(kvUpdate, user.getShortUserName());
-    long id = confStore.logMutation(log);
+    confStore.logMutation(log);
     for (Map.Entry<String, String> kv : kvUpdate.entrySet()) {
       if (kv.getValue() == null) {
         schedConf.unset(kv.getKey());
@@ -124,47 +135,33 @@ public class MutableCSConfigurationProvider implements CSConfigurationProvider,
         schedConf.set(kv.getKey(), kv.getValue());
       }
     }
-    try {
-      rmContext.getRMAdminService().refreshQueues();
-    } catch (IOException | YarnException e) {
+  }
+
+  @Override
+  public void confirmPendingMutation(boolean isValid) throws Exception {
+    confStore.confirmMutation(isValid);
+    if (!isValid) {
       schedConf = oldConf;
-      confStore.confirmMutation(id, false);
-      throw e;
     }
-    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(schedConf, 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");
-    }
+  public void reloadConfigurationFromStore() throws Exception {
+    schedConf = confStore.retrieve();
+  }
+
+  private List<String> getSiblingQueues(String queuePath, Configuration conf) {
+    String parentQueue = queuePath.substring(0, queuePath.lastIndexOf('.'));
+    String childQueuesKey = CapacitySchedulerConfiguration.PREFIX +
+        parentQueue + CapacitySchedulerConfiguration.DOT +
+        CapacitySchedulerConfiguration.QUEUES;
+    return new ArrayList<>(conf.getStringCollection(childQueuesKey));
   }
 
   private Map<String, String> constructKeyValueConfUpdate(
       SchedConfUpdateInfo mutationInfo) throws IOException {
-    CapacityScheduler cs = (CapacityScheduler) rmContext.getScheduler();
     CapacitySchedulerConfiguration proposedConf =
-        new CapacitySchedulerConfiguration(cs.getConfiguration(), false);
+        new CapacitySchedulerConfiguration(schedConf, false);
     Map<String, String> confUpdate = new HashMap<>();
     for (String queueToRemove : mutationInfo.getRemoveQueueInfo()) {
       removeQueue(queueToRemove, proposedConf, confUpdate);
@@ -188,40 +185,35 @@ public class MutableCSConfigurationProvider implements CSConfigurationProvider,
     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) {
+      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);
+        List<String> siblingQueues = getSiblingQueues(queueToRemove,
+            proposedConf);
+        if (!siblingQueues.contains(queueName)) {
+          throw new IOException("Queue " + queueToRemove + " not found");
+        }
+        siblingQueues.remove(queueName);
+        String parentQueuePath = queueToRemove.substring(0, queueToRemove
+            .lastIndexOf('.'));
+        proposedConf.setQueues(parentQueuePath, siblingQueues.toArray(
+            new String[0]));
+        String queuesConfig = CapacitySchedulerConfiguration.PREFIX
+            + parentQueuePath + CapacitySchedulerConfiguration.DOT
+            + CapacitySchedulerConfiguration.QUEUES;
+        if (siblingQueues.size() == 0) {
+          confUpdate.put(queuesConfig, null);
+        } else {
+          confUpdate.put(queuesConfig, Joiner.on(',').join(siblingQueues));
+        }
+        for (Map.Entry<String, String> confRemove : proposedConf.getValByRegex(
+            ".*" + queueToRemove.replaceAll("\\.", "\\.") + "\\..*")
+            .entrySet()) {
+          proposedConf.unset(confRemove.getKey());
+          confUpdate.put(confRemove.getKey(), null);
+        }
       }
     }
   }
@@ -232,13 +224,13 @@ public class MutableCSConfigurationProvider implements CSConfigurationProvider,
     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) {
+      if (queuePath.lastIndexOf('.') == -1) {
         throw new IOException("Can't add invalid queue " + queuePath);
+      } else if (getSiblingQueues(queuePath, proposedConf).contains(
+          queueName)) {
+        throw new IOException("Can't add existing queue " + queuePath);
       }
       String parentQueue = queuePath.substring(0, queuePath.lastIndexOf('.'));
       String[] siblings = proposedConf.getQueues(parentQueue);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76746cb8/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 065c877..1356535 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
@@ -18,7 +18,12 @@
 
 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.yarn.server.records.Version;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateVersionIncompatibleException;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
 
 import java.io.IOException;
@@ -39,36 +44,26 @@ import java.util.Map;
  * {@code getPendingMutations}, and replay/confirm them via
  * {@code confirmMutation} as in the normal case.
  */
-public interface YarnConfigurationStore {
+public abstract class YarnConfigurationStore {
 
+  public static final Log LOG =
+      LogFactory.getLog(YarnConfigurationStore.class);
   /**
    * LogMutation encapsulates the fields needed for configuration mutation
    * audit logging and recovery.
    */
-  class LogMutation implements Serializable {
+  static class LogMutation implements Serializable {
     private Map<String, String> updates;
     private String user;
-    private long id;
 
     /**
-     * Create log mutation prior to logging.
+     * Create log mutation.
      * @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) {
+    LogMutation(Map<String, String> updates, String user) {
       this.updates = updates;
       this.user = user;
-      this.id = id;
     }
 
     /**
@@ -86,75 +81,92 @@ public interface YarnConfigurationStore {
     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.
+   * Initialize the configuration store, with schedConf as the initial
+   * scheduler configuration. If a persisted store already exists, use the
+   * scheduler configuration stored there, and ignore schedConf.
    * @param conf configuration to initialize store with
-   * @param schedConf Initial key-value configuration to persist
+   * @param schedConf Initial key-value scheduler configuration to persist.
+   * @param rmContext RMContext for this configuration store
    * @throws IOException if initialization fails
    */
-  void initialize(Configuration conf, Configuration schedConf)
-      throws IOException;
+  public abstract void initialize(Configuration conf, Configuration schedConf,
+      RMContext rmContext) throws Exception;
 
   /**
-   * Logs the configuration change to backing store. Generates an id associated
-   * with this mutation, sets it in {@code logMutation}, and returns it.
+   * Logs the configuration change to backing store.
    * @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) throws IOException;
+  public abstract void logMutation(LogMutation logMutation) throws Exception;
 
   /**
    * 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
+   * last logged by {@code logMutation} 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
-   * @throws IOException if mutation confirmation fails
+   * @param isValid if true, update persisted configuration with pending
+   *                mutation.
+   * @throws Exception if mutation confirmation fails
    */
-  boolean confirmMutation(long id, boolean isValid) throws IOException;
+  public abstract void confirmMutation(boolean isValid) throws Exception;
 
   /**
    * 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();
+  public abstract Configuration retrieve();
 
   /**
    * 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);
+  public abstract List<LogMutation> getConfirmedConfHistory(long fromId);
+
+  /**
+   * Get schema version of persisted conf store, for detecting compatibility
+   * issues when changing conf store schema.
+   * @return Schema version currently used by the persisted configuration store.
+   * @throws Exception On version fetch failure
+   */
+  protected abstract Version getConfStoreVersion() throws Exception;
+
+  /**
+   * Persist the hard-coded schema version to the conf store.
+   * @throws Exception On storage failure
+   */
+  protected abstract void storeVersion() throws Exception;
+
+  /**
+   * Get the hard-coded schema version, for comparison against the schema
+   * version currently persisted.
+   * @return Current hard-coded schema version
+   */
+  protected abstract Version getCurrentVersion();
+
+  public void checkVersion() throws Exception {
+    // TODO this was taken from RMStateStore. Should probably refactor
+    Version loadedVersion = getConfStoreVersion();
+    LOG.info("Loaded configuration store version info " + loadedVersion);
+    if (loadedVersion != null && loadedVersion.equals(getCurrentVersion())) {
+      return;
+    }
+    // if there is no version info, treat it as CURRENT_VERSION_INFO;
+    if (loadedVersion == null) {
+      loadedVersion = getCurrentVersion();
+    }
+    if (loadedVersion.isCompatibleTo(getCurrentVersion())) {
+      LOG.info("Storing configuration store version info "
+          + getCurrentVersion());
+      storeVersion();
+    } else {
+      throw new RMStateVersionIncompatibleException(
+          "Expecting configuration store version " + getCurrentVersion()
+              + ", but loading version " + loadedVersion);
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76746cb8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/ZKConfigurationStore.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/ZKConfigurationStore.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/ZKConfigurationStore.java
new file mode 100644
index 0000000..a0bba8c
--- /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/ZKConfigurationStore.java
@@ -0,0 +1,235 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, 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 com.google.common.annotations.VisibleForTesting;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.curator.ZKCuratorManager;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.proto.YarnServerCommonProtos;
+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.RMContext;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.data.ACL;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A Zookeeper-based implementation of {@link YarnConfigurationStore}.
+ */
+public class ZKConfigurationStore extends YarnConfigurationStore {
+
+  public static final Log LOG =
+      LogFactory.getLog(ZKConfigurationStore.class);
+
+  private long maxLogs;
+
+  @VisibleForTesting
+  protected static final Version CURRENT_VERSION_INFO = Version
+      .newInstance(0, 1);
+  private Configuration conf;
+  private LogMutation pendingMutation;
+
+  private String znodeParentPath;
+
+  private static final String ZK_VERSION_PATH = "VERSION";
+  private static final String LOGS_PATH = "LOGS";
+  private static final String CONF_STORE_PATH = "CONF_STORE";
+  private static final String FENCING_PATH = "FENCING";
+
+  private String zkVersionPath;
+  private String logsPath;
+  private String confStorePath;
+  private String fencingNodePath;
+
+  @VisibleForTesting
+  protected ZKCuratorManager zkManager;
+  private List<ACL> zkAcl;
+
+  @Override
+  public void initialize(Configuration config, Configuration schedConf,
+      RMContext rmContext) throws Exception {
+    this.conf = config;
+    this.maxLogs = conf.getLong(YarnConfiguration.RM_SCHEDCONF_MAX_LOGS,
+        YarnConfiguration.DEFAULT_RM_SCHEDCONF_ZK_MAX_LOGS);
+    this.znodeParentPath =
+        conf.get(YarnConfiguration.RM_SCHEDCONF_STORE_ZK_PARENT_PATH,
+            YarnConfiguration.DEFAULT_RM_SCHEDCONF_STORE_ZK_PARENT_PATH);
+    this.zkManager = rmContext.getResourceManager().getAndStartZKManager(conf);
+    this.zkAcl = ZKCuratorManager.getZKAcls(conf);
+
+    this.zkVersionPath = getNodePath(znodeParentPath, ZK_VERSION_PATH);
+    this.logsPath = getNodePath(znodeParentPath, LOGS_PATH);
+    this.confStorePath = getNodePath(znodeParentPath, CONF_STORE_PATH);
+    this.fencingNodePath = getNodePath(znodeParentPath, FENCING_PATH);
+
+    zkManager.createRootDirRecursively(znodeParentPath);
+    zkManager.delete(fencingNodePath);
+
+    if (!zkManager.exists(logsPath)) {
+      zkManager.create(logsPath);
+      zkManager.setData(logsPath,
+          serializeObject(new LinkedList<LogMutation>()), -1);
+    }
+
+    if (!zkManager.exists(confStorePath)) {
+      zkManager.create(confStorePath);
+      HashMap<String, String> mapSchedConf = new HashMap<>();
+      for (Map.Entry<String, String> entry : schedConf) {
+        mapSchedConf.put(entry.getKey(), entry.getValue());
+      }
+      zkManager.setData(confStorePath, serializeObject(mapSchedConf), -1);
+    }
+  }
+
+  @VisibleForTesting
+  protected LinkedList<LogMutation> getLogs() throws Exception {
+    return (LinkedList<LogMutation>)
+        deserializeObject(zkManager.getData(logsPath));
+  }
+
+  // TODO: following version-related code is taken from ZKRMStateStore
+  @Override
+  public Version getCurrentVersion() {
+    return CURRENT_VERSION_INFO;
+  }
+
+  @Override
+  public Version getConfStoreVersion() throws Exception {
+    if (zkManager.exists(zkVersionPath)) {
+      byte[] data = zkManager.getData(zkVersionPath);
+      return new VersionPBImpl(YarnServerCommonProtos.VersionProto
+          .parseFrom(data));
+    }
+
+    return null;
+  }
+
+  @Override
+  public synchronized void storeVersion() throws Exception {
+    byte[] data =
+        ((VersionPBImpl) CURRENT_VERSION_INFO).getProto().toByteArray();
+
+    if (zkManager.exists(zkVersionPath)) {
+      zkManager.safeSetData(zkVersionPath, data, -1, zkAcl, fencingNodePath);
+    } else {
+      zkManager.safeCreate(zkVersionPath, data, zkAcl, CreateMode.PERSISTENT,
+          zkAcl, fencingNodePath);
+    }
+  }
+
+  @Override
+  public void logMutation(LogMutation logMutation) throws Exception {
+    byte[] storedLogs = zkManager.getData(logsPath);
+    LinkedList<LogMutation> logs = new LinkedList<>();
+    if (storedLogs != null) {
+      logs = (LinkedList<LogMutation>) deserializeObject(storedLogs);
+    }
+    logs.add(logMutation);
+    if (logs.size() > maxLogs) {
+      logs.remove(logs.removeFirst());
+    }
+    zkManager.safeSetData(logsPath, serializeObject(logs), -1, zkAcl,
+        fencingNodePath);
+    pendingMutation = logMutation;
+  }
+
+  @Override
+  public void confirmMutation(boolean isValid)
+      throws Exception {
+    if (isValid) {
+      Configuration storedConfigs = retrieve();
+      Map<String, String> mapConf = new HashMap<>();
+      for (Map.Entry<String, String> storedConf : storedConfigs) {
+        mapConf.put(storedConf.getKey(), storedConf.getValue());
+      }
+      for (Map.Entry<String, String> confChange :
+          pendingMutation.getUpdates().entrySet()) {
+        if (confChange.getValue() == null || confChange.getValue().isEmpty()) {
+          mapConf.remove(confChange.getKey());
+        } else {
+          mapConf.put(confChange.getKey(), confChange.getValue());
+        }
+      }
+      zkManager.safeSetData(confStorePath, serializeObject(mapConf), -1,
+          zkAcl, fencingNodePath);
+    }
+    pendingMutation = null;
+  }
+
+  @Override
+  public synchronized Configuration retrieve() {
+    byte[] serializedSchedConf;
+    try {
+      serializedSchedConf = zkManager.getData(confStorePath);
+    } catch (Exception e) {
+      LOG.error("Failed to retrieve configuration from zookeeper store", e);
+      return null;
+    }
+    try {
+      Map<String, String> map =
+          (HashMap<String, String>) deserializeObject(serializedSchedConf);
+      Configuration c = new Configuration();
+      for (Map.Entry<String, String> e : map.entrySet()) {
+        c.set(e.getKey(), e.getValue());
+      }
+      return c;
+    } catch (Exception e) {
+      LOG.error("Exception while deserializing scheduler configuration " +
+          "from store", e);
+    }
+    return null;
+  }
+
+  @Override
+  public List<LogMutation> getConfirmedConfHistory(long fromId) {
+    return null; // unimplemented
+  }
+
+  private static String getNodePath(String root, String nodeName) {
+    return ZKCuratorManager.getNodePath(root, nodeName);
+  }
+
+  private static byte[] serializeObject(Object o) throws Exception {
+    try (ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        ObjectOutputStream oos = new ObjectOutputStream(baos);) {
+      oos.writeObject(o);
+      oos.flush();
+      baos.flush();
+      return baos.toByteArray();
+    }
+  }
+
+  private static Object deserializeObject(byte[] bytes) throws Exception {
+    try (ByteArrayInputStream bais = new ByteArrayInputStream(bytes);
+        ObjectInputStream ois = new ObjectInputStream(bais);) {
+      return ois.readObject();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76746cb8/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 1da4e65..d264c10 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
@@ -136,6 +136,7 @@ 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.MutableConfigurationProvider;
 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;
@@ -2464,7 +2465,7 @@ public class RMWebServices extends WebServices implements RMWebServiceProtocol {
   @Produces({ MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
       MediaType.APPLICATION_XML + "; " + JettyUtils.UTF_8 })
   @Consumes({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
-  public Response updateSchedulerConfiguration(SchedConfUpdateInfo
+  public synchronized Response updateSchedulerConfiguration(SchedConfUpdateInfo
       mutationInfo, @Context HttpServletRequest hsr)
       throws AuthorizationException, InterruptedException {
     init();
@@ -2479,17 +2480,32 @@ public class RMWebServices extends WebServices implements RMWebServiceProtocol {
     }
 
     ResourceScheduler scheduler = rm.getResourceScheduler();
-    if (scheduler instanceof MutableConfScheduler) {
+    if (scheduler instanceof MutableConfScheduler && ((MutableConfScheduler)
+        scheduler).isConfigurationMutable()) {
       try {
         callerUGI.doAs(new PrivilegedExceptionAction<Void>() {
           @Override
-          public Void run() throws IOException, YarnException {
-            ((MutableConfScheduler) scheduler).updateConfiguration(callerUGI,
-                mutationInfo);
+          public Void run() throws Exception {
+            MutableConfigurationProvider provider = ((MutableConfScheduler)
+                scheduler).getMutableConfProvider();
+            if (!provider.getAclMutationPolicy().isMutationAllowed(callerUGI,
+                mutationInfo)) {
+              throw new org.apache.hadoop.security.AccessControlException("User"
+                  + " is not admin of all modified queues.");
+            }
+            provider.logAndApplyMutation(callerUGI, mutationInfo);
+            try {
+              rm.getRMContext().getRMAdminService().refreshQueues();
+            } catch (IOException | YarnException e) {
+              provider.confirmPendingMutation(false);
+              throw e;
+            }
+            provider.confirmPendingMutation(true);
             return null;
           }
         });
       } catch (IOException e) {
+        LOG.error("Exception thrown when modifying configuration.", e);
         return Response.status(Status.BAD_REQUEST).entity(e.getMessage())
             .build();
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76746cb8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/ConfigurationStoreBaseTest.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/ConfigurationStoreBaseTest.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/ConfigurationStoreBaseTest.java
new file mode 100644
index 0000000..bbe9570
--- /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/ConfigurationStoreBaseTest.java
@@ -0,0 +1,90 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.conf;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+/**
+ * Base class for {@link YarnConfigurationStore} implementations.
+ */
+public abstract class ConfigurationStoreBaseTest {
+
+  protected YarnConfigurationStore confStore = createConfStore();
+
+  protected abstract YarnConfigurationStore createConfStore();
+
+  protected Configuration conf;
+  protected Configuration schedConf;
+  protected RMContext rmContext;
+
+  protected static final String TEST_USER = "testUser";
+
+  @Before
+  public void setUp() throws Exception {
+    this.conf = new Configuration();
+    this.schedConf = new Configuration(false);
+  }
+
+  @Test
+  public void testConfigurationUpdate() throws Exception {
+    schedConf.set("key1", "val1");
+    confStore.initialize(conf, schedConf, rmContext);
+    assertEquals("val1", confStore.retrieve().get("key1"));
+
+    Map<String, String> update1 = new HashMap<>();
+    update1.put("keyUpdate1", "valUpdate1");
+    YarnConfigurationStore.LogMutation mutation1 =
+        new YarnConfigurationStore.LogMutation(update1, TEST_USER);
+    confStore.logMutation(mutation1);
+    confStore.confirmMutation(true);
+    assertEquals("valUpdate1", confStore.retrieve().get("keyUpdate1"));
+
+    Map<String, String> update2 = new HashMap<>();
+    update2.put("keyUpdate2", "valUpdate2");
+    YarnConfigurationStore.LogMutation mutation2 =
+        new YarnConfigurationStore.LogMutation(update2, TEST_USER);
+    confStore.logMutation(mutation2);
+    confStore.confirmMutation(false);
+    assertNull("Configuration should not be updated",
+        confStore.retrieve().get("keyUpdate2"));
+  }
+
+  @Test
+  public void testNullConfigurationUpdate() throws Exception {
+    schedConf.set("key", "val");
+    confStore.initialize(conf, schedConf, rmContext);
+    assertEquals("val", confStore.retrieve().get("key"));
+
+    Map<String, String> update = new HashMap<>();
+    update.put("key", null);
+    YarnConfigurationStore.LogMutation mutation =
+        new YarnConfigurationStore.LogMutation(update, TEST_USER);
+    confStore.logMutation(mutation);
+    confStore.confirmMutation(true);
+    assertNull(confStore.retrieve().get("key"));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76746cb8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/TestInMemoryConfigurationStore.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/TestInMemoryConfigurationStore.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/TestInMemoryConfigurationStore.java
new file mode 100644
index 0000000..c40d16a
--- /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/TestInMemoryConfigurationStore.java
@@ -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.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.conf;
+
+/**
+ * Tests {@link InMemoryConfigurationStore}.
+ */
+public class TestInMemoryConfigurationStore extends ConfigurationStoreBaseTest {
+
+  @Override
+  protected YarnConfigurationStore createConfStore() {
+    return new InMemoryConfigurationStore();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76746cb8/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 635a184..9b080cd 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,7 +20,6 @@ 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;
@@ -30,14 +29,11 @@ import org.apache.hadoop.yarn.webapp.dao.SchedConfUpdateInfo;
 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.Mockito.doNothing;
-import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -82,25 +78,21 @@ public class TestMutableCSConfigurationProvider {
   }
 
   @Test
-  public void testInMemoryBackedProvider() throws IOException, YarnException {
+  public void testInMemoryBackedProvider() throws Exception {
     Configuration conf = new Configuration();
     confProvider.init(conf);
     assertNull(confProvider.loadConfiguration(conf)
         .get("yarn.scheduler.capacity.root.a.goodKey"));
 
-    doNothing().when(adminService).refreshQueues();
-    confProvider.mutateConfiguration(TEST_USER, goodUpdate);
+    confProvider.logAndApplyMutation(TEST_USER, goodUpdate);
+    confProvider.confirmPendingMutation(true);
     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(adminService).refreshQueues();
-    try {
-      confProvider.mutateConfiguration(TEST_USER, badUpdate);
-    } catch (IOException e) {
-      // Expected exception.
-    }
+    confProvider.logAndApplyMutation(TEST_USER, badUpdate);
+    confProvider.confirmPendingMutation(false);
     assertNull(confProvider.loadConfiguration(conf).get(
         "yarn.scheduler.capacity.root.a.badKey"));
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76746cb8/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
deleted file mode 100644
index 631ce65..0000000
--- 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
+++ /dev/null
@@ -1,71 +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.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.io.IOException;
-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() throws IOException {
-    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


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

Branch: refs/heads/YARN-5734
Commit: 529e6e66839b519b2f8738a254d9d2f9e4b5b1a8
Parents: 794b774
Author: Wangda Tan <wa...@apache.org>
Authored: Mon May 22 13:38:31 2017 -0700
Committer: Jonathan Hung <jh...@linkedin.com>
Committed: Wed Sep 27 19:52:58 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/529e6e66/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 ddb9768..609f689 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
@@ -680,6 +680,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/529e6e66/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 b5adfae..5fcb27c 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
@@ -3379,4 +3379,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/529e6e66/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/529e6e66/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/529e6e66/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/529e6e66/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/529e6e66/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/529e6e66/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 7c5839b..4f89f7e 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
@@ -137,6 +137,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;
@@ -649,6 +650,7 @@ public class CapacityScheduler extends
     preemptionManager.refreshQueues(null, this.getRootQueue());
   }
 
+  @Override
   public CSQueue getQueue(String queueName) {
     if (queueName == null) {
       return null;
@@ -2615,10 +2617,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/529e6e66/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/529e6e66/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/529e6e66/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 215e511..d95465b 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
@@ -142,7 +142,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;
@@ -2484,10 +2483,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;
           }
         });
@@ -2499,129 +2496,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/529e6e66/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/529e6e66/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


[24/50] [abbrv] hadoop git commit: HDFS-12495. TestPendingInvalidateBlock#testPendingDeleteUnknownBlocks fails intermittently. Contributed by Eric Badger.

Posted by jh...@apache.org.
HDFS-12495. TestPendingInvalidateBlock#testPendingDeleteUnknownBlocks fails intermittently. Contributed by Eric Badger.


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

Branch: refs/heads/YARN-5734
Commit: e9b790db9de7c3fbe451bcdc3fd928a27a0d8b4e
Parents: 06e5a7b
Author: Yiqun Lin <yq...@apache.org>
Authored: Tue Sep 26 17:21:42 2017 +0800
Committer: Yiqun Lin <yq...@apache.org>
Committed: Tue Sep 26 17:21:42 2017 +0800

----------------------------------------------------------------------
 .../hdfs/server/blockmanagement/TestPendingInvalidateBlock.java    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e9b790db/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingInvalidateBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingInvalidateBlock.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingInvalidateBlock.java
index 674f883..8f7b5ac 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingInvalidateBlock.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingInvalidateBlock.java
@@ -172,7 +172,7 @@ public class TestPendingInvalidateBlock {
     Assert.assertEquals(0L, cluster.getNamesystem().getPendingDeletionBlocks());
     // restart DataNodes
     for (int i = 0; i < REPLICATION; i++) {
-      cluster.restartDataNode(dnprops[i], true);
+      cluster.restartDataNode(dnprops[i]);
     }
     cluster.waitActive();
 


---------------------------------------------------------------------
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-14220 Enhance S3GuardTool with bucket-info and set-capacity commands, tests. Contributed by Steve Loughran

Posted by jh...@apache.org.
HADOOP-14220 Enhance S3GuardTool with bucket-info and set-capacity commands, tests. Contributed by Steve Loughran


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

Branch: refs/heads/YARN-5734
Commit: 47011d7dd300b0c74bb6cfe25b918c479d718f4f
Parents: cde804b
Author: Aaron Fabbri <fa...@apache.org>
Authored: Mon Sep 25 15:59:38 2017 -0700
Committer: Aaron Fabbri <fa...@apache.org>
Committed: Mon Sep 25 15:59:38 2017 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/fs/s3a/S3AFileSystem.java |  14 +-
 .../java/org/apache/hadoop/fs/s3a/S3AUtils.java |  60 ++-
 .../fs/s3a/s3guard/DynamoDBMetadataStore.java   | 104 +++-
 .../fs/s3a/s3guard/LocalMetadataStore.java      |  25 +-
 .../hadoop/fs/s3a/s3guard/MetadataStore.java    |  17 +
 .../fs/s3a/s3guard/NullMetadataStore.java       |  15 +
 .../apache/hadoop/fs/s3a/s3guard/S3Guard.java   |  17 +-
 .../hadoop/fs/s3a/s3guard/S3GuardTool.java      | 539 +++++++++++++++----
 .../site/markdown/tools/hadoop-aws/s3guard.md   | 160 +++++-
 .../hadoop/fs/s3a/ITestS3AConfiguration.java    |  10 +
 .../fs/s3a/ITestS3GuardListConsistency.java     |   8 +-
 .../hadoop/fs/s3a/ITestS3GuardWriteBack.java    |  37 +-
 .../org/apache/hadoop/fs/s3a/S3ATestUtils.java  |  18 -
 .../s3guard/AbstractS3GuardToolTestBase.java    | 138 ++++-
 .../s3a/s3guard/ITestS3GuardToolDynamoDB.java   | 167 +++++-
 .../fs/s3a/s3guard/ITestS3GuardToolLocal.java   | 142 ++++-
 .../hadoop/fs/s3a/s3guard/TestS3GuardCLI.java   | 121 +++++
 17 files changed, 1405 insertions(+), 187 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/47011d7d/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
index f4709a7..2171957 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
@@ -298,6 +298,10 @@ public class S3AFileSystem extends FileSystem {
       metadataStore = S3Guard.getMetadataStore(this);
       allowAuthoritative = conf.getBoolean(METADATASTORE_AUTHORITATIVE,
           DEFAULT_METADATASTORE_AUTHORITATIVE);
+      if (hasMetadataStore()) {
+        LOG.debug("Using metadata store {}, authoritative={}",
+            getMetadataStore(), allowAuthoritative);
+      }
     } catch (AmazonClientException e) {
       throw translateException("initializing ", new Path(name), e);
     }
@@ -967,7 +971,7 @@ public class S3AFileSystem extends FileSystem {
    * @return the metadata store of this FS instance
    */
   @VisibleForTesting
-  MetadataStore getMetadataStore() {
+  public MetadataStore getMetadataStore() {
     return metadataStore;
   }
 
@@ -2474,9 +2478,11 @@ public class S3AFileSystem extends FileSystem {
     sb.append(", statistics {")
         .append(statistics)
         .append("}");
-    sb.append(", metrics {")
-        .append(instrumentation.dump("{", "=", "} ", true))
-        .append("}");
+    if (instrumentation != null) {
+      sb.append(", metrics {")
+          .append(instrumentation.dump("{", "=", "} ", true))
+          .append("}");
+    }
     sb.append('}');
     return sb.toString();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/47011d7d/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
index 9dd5def..6e6f4b6 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
@@ -736,9 +736,11 @@ public final class S3AUtils {
         // propagate the value, building a new origin field.
         // to track overwrites, the generic key is overwritten even if
         // already matches the new one.
+        String origin = "[" + StringUtils.join(
+            source.getPropertySources(key), ", ") +"]";
         final String generic = FS_S3A_PREFIX + stripped;
-        LOG.debug("Updating {}", generic);
-        dest.set(generic, value, key);
+        LOG.debug("Updating {} from {}", generic, origin);
+        dest.set(generic, value, key + " via " + origin);
       }
     }
     return dest;
@@ -881,4 +883,58 @@ public final class S3AUtils {
     }
   }
 
+  /**
+   * Set a bucket-specific property to a particular value.
+   * If the generic key passed in has an {@code fs.s3a. prefix},
+   * that's stripped off, so that when the the bucket properties are propagated
+   * down to the generic values, that value gets copied down.
+   * @param conf configuration to set
+   * @param bucket bucket name
+   * @param genericKey key; can start with "fs.s3a."
+   * @param value value to set
+   */
+  public static void setBucketOption(Configuration conf, String bucket,
+      String genericKey, String value) {
+    final String baseKey = genericKey.startsWith(FS_S3A_PREFIX) ?
+        genericKey.substring(FS_S3A_PREFIX.length())
+        : genericKey;
+    conf.set(FS_S3A_BUCKET_PREFIX + bucket + '.' + baseKey, value, "S3AUtils");
+  }
+
+  /**
+   * Clear a bucket-specific property.
+   * If the generic key passed in has an {@code fs.s3a. prefix},
+   * that's stripped off, so that when the the bucket properties are propagated
+   * down to the generic values, that value gets copied down.
+   * @param conf configuration to set
+   * @param bucket bucket name
+   * @param genericKey key; can start with "fs.s3a."
+   */
+  public static void clearBucketOption(Configuration conf, String bucket,
+      String genericKey) {
+    final String baseKey = genericKey.startsWith(FS_S3A_PREFIX) ?
+        genericKey.substring(FS_S3A_PREFIX.length())
+        : genericKey;
+    String k = FS_S3A_BUCKET_PREFIX + bucket + '.' + baseKey;
+    LOG.debug("Unset {}", k);
+    conf.unset(k);
+  }
+
+  /**
+   * Get a bucket-specific property.
+   * If the generic key passed in has an {@code fs.s3a. prefix},
+   * that's stripped off.
+   * @param conf configuration to set
+   * @param bucket bucket name
+   * @param genericKey key; can start with "fs.s3a."
+   * @return the bucket option, null if there is none
+   */
+  public static String getBucketOption(Configuration conf, String bucket,
+      String genericKey) {
+    final String baseKey = genericKey.startsWith(FS_S3A_PREFIX) ?
+        genericKey.substring(FS_S3A_PREFIX.length())
+        : genericKey;
+    return conf.get(FS_S3A_BUCKET_PREFIX + bucket + '.' + baseKey);
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/47011d7d/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java
index 1bed03d..458eb83 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java
@@ -29,6 +29,7 @@ import java.util.Date;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.TreeMap;
 import java.util.concurrent.TimeUnit;
 
 import com.amazonaws.AmazonClientException;
@@ -51,6 +52,7 @@ import com.amazonaws.services.dynamodbv2.model.ProvisionedThroughput;
 import com.amazonaws.services.dynamodbv2.model.ProvisionedThroughputDescription;
 import com.amazonaws.services.dynamodbv2.model.ResourceInUseException;
 import com.amazonaws.services.dynamodbv2.model.ResourceNotFoundException;
+import com.amazonaws.services.dynamodbv2.model.TableDescription;
 import com.amazonaws.services.dynamodbv2.model.WriteRequest;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
@@ -184,6 +186,18 @@ public class DynamoDBMetadataStore implements MetadataStore {
    * DynamoDB. Value is {@value} msec. */
   public static final long MIN_RETRY_SLEEP_MSEC = 100;
 
+  @VisibleForTesting
+  static final String DESCRIPTION
+      = "S3Guard metadata store in DynamoDB";
+  @VisibleForTesting
+  static final String READ_CAPACITY = "read-capacity";
+  @VisibleForTesting
+  static final String WRITE_CAPACITY = "write-capacity";
+  @VisibleForTesting
+  static final String STATUS = "status";
+  @VisibleForTesting
+  static final String TABLE = "table";
+
   private static ValueMap deleteTrackingValueMap =
       new ValueMap().withBoolean(":false", false);
 
@@ -788,7 +802,9 @@ public class DynamoDBMetadataStore implements MetadataStore {
     try {
       try {
         LOG.debug("Binding to table {}", tableName);
-        final String status = table.describe().getTableStatus();
+        TableDescription description = table.describe();
+        LOG.debug("Table state: {}", description);
+        final String status = description.getTableStatus();
         switch (status) {
         case "CREATING":
         case "UPDATING":
@@ -824,9 +840,10 @@ public class DynamoDBMetadataStore implements MetadataStore {
 
           createTable(capacity);
         } else {
-          throw new FileNotFoundException("DynamoDB table "
-              + "'" + tableName + "' does not "
-              + "exist in region " + region + "; auto-creation is turned off");
+          throw (FileNotFoundException)new FileNotFoundException(
+              "DynamoDB table '" + tableName + "' does not "
+              + "exist in region " + region + "; auto-creation is turned off")
+              .initCause(rnfe);
         }
       }
 
@@ -1007,4 +1024,83 @@ public class DynamoDBMetadataStore implements MetadataStore {
     Preconditions.checkNotNull(meta.getFileStatus().getPath());
   }
 
+  @Override
+  public Map<String, String> getDiagnostics() throws IOException {
+    Map<String, String> map = new TreeMap<>();
+    if (table != null) {
+      TableDescription desc = getTableDescription(true);
+      map.put("name", desc.getTableName());
+      map.put(STATUS, desc.getTableStatus());
+      map.put("ARN", desc.getTableArn());
+      map.put("size", desc.getTableSizeBytes().toString());
+      map.put(TABLE, desc.toString());
+      ProvisionedThroughputDescription throughput
+          = desc.getProvisionedThroughput();
+      map.put(READ_CAPACITY, throughput.getReadCapacityUnits().toString());
+      map.put(WRITE_CAPACITY, throughput.getWriteCapacityUnits().toString());
+      map.put(TABLE, desc.toString());
+    } else {
+      map.put("name", "DynamoDB Metadata Store");
+      map.put(TABLE, "none");
+      map.put(STATUS, "undefined");
+    }
+    map.put("description", DESCRIPTION);
+    map.put("region", region);
+    if (dataAccessRetryPolicy != null) {
+      map.put("retryPolicy", dataAccessRetryPolicy.toString());
+    }
+    return map;
+  }
+
+  private TableDescription getTableDescription(boolean forceUpdate) {
+    TableDescription desc = table.getDescription();
+    if (desc == null || forceUpdate) {
+      desc = table.describe();
+    }
+    return desc;
+  }
+
+  @Override
+  public void updateParameters(Map<String, String> parameters)
+      throws IOException {
+    Preconditions.checkNotNull(table, "Not initialized");
+    TableDescription desc = getTableDescription(true);
+    ProvisionedThroughputDescription current
+        = desc.getProvisionedThroughput();
+
+    long currentRead = current.getReadCapacityUnits();
+    long newRead = getLongParam(parameters,
+        S3GUARD_DDB_TABLE_CAPACITY_READ_KEY,
+        currentRead);
+    long currentWrite = current.getWriteCapacityUnits();
+    long newWrite = getLongParam(parameters,
+            S3GUARD_DDB_TABLE_CAPACITY_WRITE_KEY,
+            currentWrite);
+
+    ProvisionedThroughput throughput = new ProvisionedThroughput()
+        .withReadCapacityUnits(newRead)
+        .withWriteCapacityUnits(newWrite);
+    if (newRead != currentRead || newWrite != currentWrite) {
+      LOG.info("Current table capacity is read: {}, write: {}",
+          currentRead, currentWrite);
+      LOG.info("Changing capacity of table to read: {}, write: {}",
+          newRead, newWrite);
+      table.updateTable(throughput);
+    } else {
+      LOG.info("Table capacity unchanged at read: {}, write: {}",
+          newRead, newWrite);
+    }
+  }
+
+  private long getLongParam(Map<String, String> parameters,
+      String key,
+      long defVal) {
+    String k = parameters.get(key);
+    if (k != null) {
+      return Long.parseLong(k);
+    } else {
+      return defVal;
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/47011d7d/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LocalMetadataStore.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LocalMetadataStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LocalMetadataStore.java
index 1ef8b0d..9267ab4 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LocalMetadataStore.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LocalMetadataStore.java
@@ -20,6 +20,8 @@ package org.apache.hadoop.fs.s3a.s3guard;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
+
+import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -31,6 +33,7 @@ import org.slf4j.LoggerFactory;
 import java.io.IOException;
 import java.net.URI;
 import java.util.Collection;
+import java.util.HashMap;
 import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.Map;
@@ -100,7 +103,7 @@ public class LocalMetadataStore implements MetadataStore {
   public String toString() {
     final StringBuilder sb = new StringBuilder(
         "LocalMetadataStore{");
-    sb.append(", uriHost='").append(uriHost).append('\'');
+    sb.append("uriHost='").append(uriHost).append('\'');
     sb.append('}');
     return sb.toString();
   }
@@ -153,7 +156,9 @@ public class LocalMetadataStore implements MetadataStore {
         m.setIsEmptyDirectory(isEmptyDirectory(p));
       }
 
-      LOG.debug("get({}) -> {}", path, m == null ? "null" : m.prettyPrint());
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("get({}) -> {}", path, m == null ? "null" : m.prettyPrint());
+      }
       return m;
     }
   }
@@ -424,12 +429,22 @@ public class LocalMetadataStore implements MetadataStore {
     Preconditions.checkArgument(p.isAbsolute(), "Path must be absolute");
     URI uri = p.toUri();
     if (uriHost != null) {
-      Preconditions.checkArgument(!isEmpty(uri.getHost()));
+      Preconditions.checkArgument(StringUtils.isNotEmpty(uri.getHost()));
     }
     return p;
   }
 
-  private static boolean isEmpty(String s) {
-    return (s == null || s.isEmpty());
+  @Override
+  public Map<String, String> getDiagnostics() throws IOException {
+    Map<String, String> map = new HashMap<>();
+    map.put("name", "local://metadata");
+    map.put("uriHost", uriHost);
+    map.put("description", "Local in-VM metadata store for testing");
+    return map;
+  }
+
+  @Override
+  public void updateParameters(Map<String, String> parameters)
+      throws IOException {
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/47011d7d/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStore.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStore.java
index dd8077b..bdab7b7 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStore.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStore.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.fs.s3a.s3guard;
 import java.io.Closeable;
 import java.io.IOException;
 import java.util.Collection;
+import java.util.Map;
 
 import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -218,4 +219,20 @@ public interface MetadataStore extends Closeable {
    * @throws UnsupportedOperationException if not implemented
    */
   void prune(long modTime) throws IOException, UnsupportedOperationException;
+
+  /**
+   * Get any diagnostics information from a store, as a list of (key, value)
+   * tuples for display. Arbitrary values; no guarantee of stability.
+   * These are for debugging only.
+   * @return a map of strings.
+   * @throws IOException if there is an error
+   */
+  Map<String, String> getDiagnostics() throws IOException;
+
+  /**
+   * Tune/update parameters for an existing table.
+   * @param parameters map of params to change.
+   * @throws IOException if there is an error
+   */
+  void updateParameters(Map<String, String> parameters) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/47011d7d/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/NullMetadataStore.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/NullMetadataStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/NullMetadataStore.java
index 08ae89e..e738c61 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/NullMetadataStore.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/NullMetadataStore.java
@@ -24,6 +24,8 @@ import org.apache.hadoop.fs.Path;
 
 import java.io.IOException;
 import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
 
 /**
  * A no-op implementation of MetadataStore.  Clients that use this
@@ -101,4 +103,17 @@ public class NullMetadataStore implements MetadataStore {
   public String toString() {
     return "NullMetadataStore";
   }
+
+  @Override
+  public Map<String, String> getDiagnostics() throws IOException {
+    Map<String, String> map = new HashMap<>();
+    map.put("name", "Null Metadata Store");
+    map.put("description", "This is not a real metadata store");
+    return map;
+  }
+
+  @Override
+  public void updateParameters(Map<String, String> parameters)
+      throws IOException {
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/47011d7d/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3Guard.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3Guard.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3Guard.java
index 7e4aec1..c7c810a 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3Guard.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3Guard.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.fs.s3a.s3guard;
 
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.net.URI;
 import java.util.ArrayList;
@@ -95,6 +96,10 @@ public final class S3Guard {
           msClass.getSimpleName(), fs.getScheme());
       msInstance.initialize(fs);
       return msInstance;
+    } catch (FileNotFoundException e) {
+      // Don't log this exception as it means the table doesn't exist yet;
+      // rely on callers to catch and treat specially
+      throw e;
     } catch (RuntimeException | IOException e) {
       String message = "Failed to instantiate metadata store " +
           conf.get(S3_METADATA_STORE_IMPL)
@@ -109,14 +114,20 @@ public final class S3Guard {
     }
   }
 
-  private static Class<? extends MetadataStore> getMetadataStoreClass(
+  static Class<? extends MetadataStore> getMetadataStoreClass(
       Configuration conf) {
     if (conf == null) {
       return NullMetadataStore.class;
     }
+    if (conf.get(S3_METADATA_STORE_IMPL) != null && LOG.isDebugEnabled()) {
+      LOG.debug("Metastore option source {}",
+          conf.getPropertySources(S3_METADATA_STORE_IMPL));
+    }
 
-    return conf.getClass(S3_METADATA_STORE_IMPL, NullMetadataStore.class,
-            MetadataStore.class);
+    Class<? extends MetadataStore> aClass = conf.getClass(
+        S3_METADATA_STORE_IMPL, NullMetadataStore.class,
+        MetadataStore.class);
+    return aClass;
   }
 
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/47011d7d/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java
index be271ae..4f0e8f7 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java
@@ -36,6 +36,7 @@ import com.google.common.base.Preconditions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.FileStatus;
@@ -43,15 +44,17 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RemoteIterator;
-import org.apache.hadoop.fs.s3a.Constants;
 import org.apache.hadoop.fs.s3a.S3AFileStatus;
 import org.apache.hadoop.fs.s3a.S3AFileSystem;
+import org.apache.hadoop.fs.s3a.S3AUtils;
 import org.apache.hadoop.fs.shell.CommandFormat;
+import org.apache.hadoop.util.ExitUtil;
 import org.apache.hadoop.util.GenericOptionsParser;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 
 import static org.apache.hadoop.fs.s3a.Constants.*;
+import static org.apache.hadoop.service.launcher.LauncherExitCodes.*;
 
 /**
  * CLI to manage S3Guard Metadata Store.
@@ -74,40 +77,46 @@ public abstract class S3GuardTool extends Configured implements Tool {
       "\t" + Init.NAME + " - " + Init.PURPOSE + "\n" +
       "\t" + Destroy.NAME + " - " + Destroy.PURPOSE + "\n" +
       "\t" + Import.NAME + " - " + Import.PURPOSE + "\n" +
+      "\t" + BucketInfo.NAME + " - " + BucketInfo.PURPOSE + "\n" +
       "\t" + Diff.NAME + " - " + Diff.PURPOSE + "\n" +
-      "\t" + Prune.NAME + " - " + Prune.PURPOSE + "\n";
+      "\t" + Prune.NAME + " - " + Prune.PURPOSE + "\n" +
+      "\t" + SetCapacity.NAME + " - " +SetCapacity.PURPOSE + "\n";
   private static final String DATA_IN_S3_IS_PRESERVED
-      = "(all data in S3 is preserved";
+      = "(all data in S3 is preserved)";
 
   abstract public String getUsage();
 
   // Exit codes
-  static final int SUCCESS = 0;
-  static final int INVALID_ARGUMENT = 1;
-  static final int ERROR = 99;
+  static final int SUCCESS = EXIT_SUCCESS;
+  static final int INVALID_ARGUMENT = EXIT_COMMAND_ARGUMENT_ERROR;
+  static final int E_USAGE = EXIT_USAGE;
+  static final int ERROR = EXIT_FAIL;
+  static final int E_BAD_STATE = EXIT_NOT_ACCEPTABLE;
+  static final int E_NOT_FOUND = EXIT_NOT_FOUND;
 
   private S3AFileSystem filesystem;
   private MetadataStore store;
   private final CommandFormat commandFormat;
 
-  private static final String META_FLAG = "meta";
-  private static final String DAYS_FLAG = "days";
-  private static final String HOURS_FLAG = "hours";
-  private static final String MINUTES_FLAG = "minutes";
-  private static final String SECONDS_FLAG = "seconds";
+  public static final String META_FLAG = "meta";
+  public static final String DAYS_FLAG = "days";
+  public static final String HOURS_FLAG = "hours";
+  public static final String MINUTES_FLAG = "minutes";
+  public static final String SECONDS_FLAG = "seconds";
 
-  private static final String REGION_FLAG = "region";
-  private static final String READ_FLAG = "read";
-  private static final String WRITE_FLAG = "write";
+  public static final String REGION_FLAG = "region";
+  public static final String READ_FLAG = "read";
+  public static final String WRITE_FLAG = "write";
 
   /**
    * Constructor a S3Guard tool with HDFS configuration.
    * @param conf Configuration.
+   * @param opts any boolean options to support
    */
-  protected S3GuardTool(Configuration conf) {
+  protected S3GuardTool(Configuration conf, String...opts) {
     super(conf);
 
-    commandFormat = new CommandFormat(0, Integer.MAX_VALUE);
+    commandFormat = new CommandFormat(0, Integer.MAX_VALUE, opts);
     // For metadata store URI
     commandFormat.addOptionWithValue(META_FLAG);
     // DDB region.
@@ -126,10 +135,10 @@ public abstract class S3GuardTool extends Configured implements Tool {
    * {@link Destroy}.
    *
    * @param paths remaining parameters from CLI.
-   * @return false for invalid parameters.
    * @throws IOException on I/O errors.
+   * @throws ExitUtil.ExitException on validation errors
    */
-  boolean parseDynamoDBRegion(List<String> paths) throws IOException {
+  void parseDynamoDBRegion(List<String> paths) throws IOException {
     Configuration conf = getConf();
     String fromCli = getCommandFormat().getOptValue(REGION_FLAG);
     String fromConf = conf.get(S3GUARD_DDB_REGION_KEY);
@@ -137,37 +146,34 @@ public abstract class S3GuardTool extends Configured implements Tool {
 
     if (fromCli != null) {
       if (fromCli.isEmpty()) {
-        System.err.println("No region provided with -" + REGION_FLAG + " flag");
-        return false;
+        throw invalidArgs("No region provided with -" + REGION_FLAG + " flag");
       }
       if (hasS3Path) {
-        System.err.println("Providing both an S3 path and the -" + REGION_FLAG
+        throw invalidArgs("Providing both an S3 path and the"
+            + " -" + REGION_FLAG
             + " flag is not supported. If you need to specify a different "
             + "region than the S3 bucket, configure " + S3GUARD_DDB_REGION_KEY);
-        return false;
       }
       conf.set(S3GUARD_DDB_REGION_KEY, fromCli);
-      return true;
+      return;
     }
 
     if (fromConf != null) {
       if (fromConf.isEmpty()) {
-        System.err.printf("No region provided with config %s, %n",
+        throw invalidArgs("No region provided with config %s",
             S3GUARD_DDB_REGION_KEY);
-        return false;
       }
-      return true;
+      return;
     }
 
     if (hasS3Path) {
       String s3Path = paths.get(0);
       initS3AFileSystem(s3Path);
-      return true;
+      return;
     }
 
-    System.err.println("No region found from -" + REGION_FLAG + " flag, " +
+    throw invalidArgs("No region found from -" + REGION_FLAG + " flag, " +
         "config, or S3 bucket");
-    return false;
   }
 
   /**
@@ -189,7 +195,7 @@ public abstract class S3GuardTool extends Configured implements Tool {
     String metaURI = getCommandFormat().getOptValue(META_FLAG);
     if (metaURI != null && !metaURI.isEmpty()) {
       URI uri = URI.create(metaURI);
-      LOG.info("create metadata store: {}", uri + " scheme: "
+      LOG.info("Create metadata store: {}", uri + " scheme: "
           + uri.getScheme());
       switch (uri.getScheme().toLowerCase(Locale.ENGLISH)) {
       case "local":
@@ -225,35 +231,48 @@ public abstract class S3GuardTool extends Configured implements Tool {
   }
 
   /**
-   * Initialize S3A FileSystem instance.
+   * Create and initialize a new S3A FileSystem instance.
+   * This instance is always created without S3Guard, so allowing
+   * a previously created metastore to be patched in.
+   *
+   * Note: this is a bit convoluted as it needs to also handle the situation
+   * of a per-bucket option in core-site.xml, which isn't easily overridden.
+   * The new config and the setting of the values before any
+   * {@code Configuration.get()} calls are critical.
    *
    * @param path s3a URI
-   * @throws IOException
+   * @throws IOException failure to init filesystem
+   * @throws ExitUtil.ExitException if the FS is not an S3A FS
    */
   void initS3AFileSystem(String path) throws IOException {
-    URI uri;
-    try {
-      uri = new URI(path);
-    } catch (URISyntaxException e) {
-      throw new IOException(e);
-    }
+    URI uri = toUri(path);
     // Make sure that S3AFileSystem does not hold an actual MetadataStore
     // implementation.
-    Configuration conf = getConf();
-    conf.setClass(S3_METADATA_STORE_IMPL, NullMetadataStore.class,
-        MetadataStore.class);
-    FileSystem fs = FileSystem.get(uri, getConf());
+    Configuration conf = new Configuration(getConf());
+    String nullStore = NullMetadataStore.class.getName();
+    conf.set(S3_METADATA_STORE_IMPL, nullStore);
+    String bucket = uri.getHost();
+    S3AUtils.setBucketOption(conf,
+        bucket,
+        S3_METADATA_STORE_IMPL, S3GUARD_METASTORE_NULL);
+    String updatedBucketOption = S3AUtils.getBucketOption(conf, bucket,
+        S3_METADATA_STORE_IMPL);
+    LOG.debug("updated bucket store option {}", updatedBucketOption);
+    Preconditions.checkState(S3GUARD_METASTORE_NULL.equals(updatedBucketOption),
+        "Expected bucket option to be %s but was %s",
+        S3GUARD_METASTORE_NULL, updatedBucketOption);
+
+    FileSystem fs = FileSystem.newInstance(uri, conf);
     if (!(fs instanceof S3AFileSystem)) {
-      throw new IOException(
-          String.format("URI %s is not a S3A file system: %s", uri,
-              fs.getClass().getName()));
+      throw invalidArgs("URI %s is not a S3A file system: %s",
+          uri, fs.getClass().getName());
     }
     filesystem = (S3AFileSystem) fs;
   }
 
   /**
    * Parse CLI arguments and returns the position arguments.
-   * The options are stored in {@link #commandFormat}
+   * The options are stored in {@link #commandFormat}.
    *
    * @param args command line arguments.
    * @return the position arguments from CLI.
@@ -285,11 +304,32 @@ public abstract class S3GuardTool extends Configured implements Tool {
     return commandFormat;
   }
 
+  @Override
+  public final int run(String[] args) throws Exception {
+    return run(args, System.out);
+  }
+
+  /**
+   * Run the tool, capturing the output (if the tool supports that).
+   *
+   * As well as returning an exit code, the implementations can choose to
+   * throw an instance of {@link ExitUtil.ExitException} with their exit
+   * code set to the desired exit value. The exit code of auch an exception
+   * is used for the tool's exit code, and the stack trace only logged at
+   * debug.
+   * @param args argument list
+   * @param out output stream
+   * @return the exit code to return.
+   * @throws Exception on any failure
+   * @throws ExitUtil.ExitException for an alternative clean exit
+   */
+  public abstract int run(String[] args, PrintStream out) throws Exception;
+
   /**
    * Create the metadata store.
    */
   static class Init extends S3GuardTool {
-    private static final String NAME = "init";
+    public static final String NAME = "init";
     public static final String PURPOSE = "initialize metadata repository";
     private static final String USAGE = NAME + " [OPTIONS] [s3a://BUCKET]\n" +
         "\t" + PURPOSE + "\n\n" +
@@ -325,7 +365,7 @@ public abstract class S3GuardTool extends Configured implements Tool {
     }
 
     @Override
-    public int run(String[] args) throws IOException {
+    public int run(String[] args, PrintStream out) throws Exception {
       List<String> paths = parseArgs(args);
 
       String readCap = getCommandFormat().getOptValue(READ_FLAG);
@@ -340,20 +380,92 @@ public abstract class S3GuardTool extends Configured implements Tool {
       }
 
       // Validate parameters.
-      if (!parseDynamoDBRegion(paths)) {
-        System.err.println(USAGE);
-        return INVALID_ARGUMENT;
+      try {
+        parseDynamoDBRegion(paths);
+      } catch (ExitUtil.ExitException e) {
+        errorln(USAGE);
+        throw e;
+      }
+      MetadataStore store = initMetadataStore(true);
+      printStoreDiagnostics(out, store);
+      return SUCCESS;
+    }
+  }
+
+  /**
+   * Change the capacity of the metadata store.
+   */
+  static class SetCapacity extends S3GuardTool {
+    public static final String NAME = "set-capacity";
+    public static final String PURPOSE = "Alter metadata store IO capacity";
+    private static final String USAGE = NAME + " [OPTIONS] [s3a://BUCKET]\n" +
+        "\t" + PURPOSE + "\n\n" +
+        "Common options:\n" +
+        "  -" + META_FLAG + " URL - Metadata repository details " +
+          "(implementation-specific)\n" +
+        "\n" +
+        "Amazon DynamoDB-specific options:\n" +
+        "  -" + READ_FLAG + " UNIT - Provisioned read throughput units\n" +
+        "  -" + WRITE_FLAG + " UNIT - Provisioned write through put units\n" +
+        "\n" +
+        "  URLs for Amazon DynamoDB are of the form dynamodb://TABLE_NAME.\n" +
+        "  Specifying both the -" + REGION_FLAG + " option and an S3A path\n" +
+        "  is not supported.";
+
+    SetCapacity(Configuration conf) {
+      super(conf);
+      // read capacity.
+      getCommandFormat().addOptionWithValue(READ_FLAG);
+      // write capacity.
+      getCommandFormat().addOptionWithValue(WRITE_FLAG);
+    }
+
+    @Override
+    String getName() {
+      return NAME;
+    }
+
+    @Override
+    public String getUsage() {
+      return USAGE;
+    }
+
+    @Override
+    public int run(String[] args, PrintStream out) throws Exception {
+      List<String> paths = parseArgs(args);
+      Map<String, String> options = new HashMap<>();
+
+      String readCap = getCommandFormat().getOptValue(READ_FLAG);
+      if (StringUtils.isNotEmpty(readCap)) {
+        S3GuardTool.println(out, "Read capacity set to %s", readCap);
+        options.put(S3GUARD_DDB_TABLE_CAPACITY_READ_KEY, readCap);
+      }
+      String writeCap = getCommandFormat().getOptValue(WRITE_FLAG);
+      if (StringUtils.isNotEmpty(writeCap)) {
+        S3GuardTool.println(out, "Write capacity set to %s", writeCap);
+        options.put(S3GUARD_DDB_TABLE_CAPACITY_WRITE_KEY, writeCap);
+      }
+
+      // Validate parameters.
+      try {
+        parseDynamoDBRegion(paths);
+      } catch (ExitUtil.ExitException e) {
+        errorln(USAGE);
+        throw e;
       }
-      initMetadataStore(true);
+      MetadataStore store = initMetadataStore(false);
+      store.updateParameters(options);
+      printStoreDiagnostics(out, store);
       return SUCCESS;
     }
   }
 
+
   /**
    * Destroy a metadata store.
    */
   static class Destroy extends S3GuardTool {
-    private static final String NAME = "destroy";
+    public static final String NAME = "destroy";
     public static final String PURPOSE = "destroy Metadata Store data "
         + DATA_IN_S3_IS_PRESERVED;
     private static final String USAGE = NAME + " [OPTIONS] [s3a://BUCKET]\n" +
@@ -383,19 +495,21 @@ public abstract class S3GuardTool extends Configured implements Tool {
       return USAGE;
     }
 
-    public int run(String[] args) throws IOException {
+    public int run(String[] args, PrintStream out) throws Exception {
       List<String> paths = parseArgs(args);
-      if (!parseDynamoDBRegion(paths)) {
-        System.err.println(USAGE);
-        return INVALID_ARGUMENT;
+      try {
+        parseDynamoDBRegion(paths);
+      } catch (ExitUtil.ExitException e) {
+        errorln(USAGE);
+        throw e;
       }
 
       try {
         initMetadataStore(false);
       } catch (FileNotFoundException e) {
         // indication that the table was not found
+        println(out, "Metadata Store does not exist.");
         LOG.debug("Failed to bind to store to be destroyed", e);
-        LOG.info("Metadata Store does not exist.");
         return SUCCESS;
       }
 
@@ -403,7 +517,7 @@ public abstract class S3GuardTool extends Configured implements Tool {
           "Metadata Store is not initialized");
 
       getStore().destroy();
-      LOG.info("Metadata store is deleted.");
+      println(out, "Metadata store is deleted.");
       return SUCCESS;
     }
   }
@@ -412,7 +526,7 @@ public abstract class S3GuardTool extends Configured implements Tool {
    * Import s3 metadata to the metadata store.
    */
   static class Import extends S3GuardTool {
-    private static final String NAME = "import";
+    public static final String NAME = "import";
     public static final String PURPOSE = "import metadata from existing S3 " +
         "data";
     private static final String USAGE = NAME + " [OPTIONS] [s3a://BUCKET]\n" +
@@ -498,21 +612,16 @@ public abstract class S3GuardTool extends Configured implements Tool {
     }
 
     @Override
-    public int run(String[] args) throws IOException {
+    public int run(String[] args, PrintStream out) throws Exception {
       List<String> paths = parseArgs(args);
       if (paths.isEmpty()) {
-        System.err.println(getUsage());
-        return INVALID_ARGUMENT;
+        errorln(getUsage());
+        throw invalidArgs("no arguments");
       }
       String s3Path = paths.get(0);
       initS3AFileSystem(s3Path);
 
-      URI uri;
-      try {
-        uri = new URI(s3Path);
-      } catch (URISyntaxException e) {
-        throw new IOException(e);
-      }
+      URI uri = toUri(s3Path);
       String filePath = uri.getPath();
       if (filePath.isEmpty()) {
         // If they specify a naked S3 URI (e.g. s3a://bucket), we'll consider
@@ -522,7 +631,11 @@ public abstract class S3GuardTool extends Configured implements Tool {
       Path path = new Path(filePath);
       FileStatus status = getFilesystem().getFileStatus(path);
 
-      initMetadataStore(false);
+      try {
+        initMetadataStore(false);
+      } catch (FileNotFoundException e) {
+        throw storeNotFound(e);
+      }
 
       long items = 1;
       if (status.isFile()) {
@@ -532,17 +645,18 @@ public abstract class S3GuardTool extends Configured implements Tool {
         items = importDir(status);
       }
 
-      System.out.printf("Inserted %d items into Metadata Store%n", items);
+      println(out, "Inserted %d items into Metadata Store", items);
 
       return SUCCESS;
     }
+
   }
 
   /**
    * Show diffs between the s3 and metadata store.
    */
   static class Diff extends S3GuardTool {
-    private static final String NAME = "diff";
+    public static final String NAME = "diff";
     public static final String PURPOSE = "report on delta between S3 and " +
         "repository";
     private static final String USAGE = NAME + " [OPTIONS] s3a://BUCKET\n" +
@@ -625,10 +739,10 @@ public abstract class S3GuardTool extends Configured implements Tool {
 
       if (differ(msStatus, s3Status)) {
         if (s3Status != null) {
-          out.printf("%s%s%s%n", S3_PREFIX, SEP, formatFileStatus(s3Status));
+          println(out, "%s%s%s", S3_PREFIX, SEP, formatFileStatus(s3Status));
         }
         if (msStatus != null) {
-          out.printf("%s%s%s%n", MS_PREFIX, SEP, formatFileStatus(msStatus));
+          println(out, "%s%s%s", MS_PREFIX, SEP, formatFileStatus(msStatus));
         }
       }
     }
@@ -705,6 +819,7 @@ public abstract class S3GuardTool extends Configured implements Tool {
       try {
         s3Status = getFilesystem().getFileStatus(qualified);
       } catch (FileNotFoundException e) {
+        /* ignored */
       }
       PathMetadata meta = getStore().get(qualified);
       FileStatus msStatus = (meta != null && !meta.isDeleted()) ?
@@ -717,18 +832,13 @@ public abstract class S3GuardTool extends Configured implements Tool {
       List<String> paths = parseArgs(args);
       if (paths.isEmpty()) {
         out.println(USAGE);
-        return INVALID_ARGUMENT;
+        throw invalidArgs("no arguments");
       }
       String s3Path = paths.get(0);
       initS3AFileSystem(s3Path);
-      initMetadataStore(true);
+      initMetadataStore(false);
 
-      URI uri;
-      try {
-        uri = new URI(s3Path);
-      } catch (URISyntaxException e) {
-        throw new IOException(e);
-      }
+      URI uri = toUri(s3Path);
       Path root;
       if (uri.getPath().isEmpty()) {
         root = new Path("/");
@@ -741,17 +851,13 @@ public abstract class S3GuardTool extends Configured implements Tool {
       return SUCCESS;
     }
 
-    @Override
-    public int run(String[] args) throws IOException {
-      return run(args, System.out);
-    }
   }
 
   /**
    * Prune metadata that has not been modified recently.
    */
   static class Prune extends S3GuardTool {
-    private static final String NAME = "prune";
+    public static final String NAME = "prune";
     public static final String PURPOSE = "truncate older metadata from " +
         "repository "
         + DATA_IN_S3_IS_PRESERVED;;
@@ -803,18 +909,19 @@ public abstract class S3GuardTool extends Configured implements Tool {
       return unit.toMillis(parsed);
     }
 
-    @VisibleForTesting
     public int run(String[] args, PrintStream out) throws
         InterruptedException, IOException {
       List<String> paths = parseArgs(args);
-      if (!parseDynamoDBRegion(paths)) {
-        System.err.println(USAGE);
-        return INVALID_ARGUMENT;
+      try {
+        parseDynamoDBRegion(paths);
+      } catch (ExitUtil.ExitException e) {
+        errorln(USAGE);
+        throw e;
       }
       initMetadataStore(false);
 
       Configuration conf = getConf();
-      long confDelta = conf.getLong(Constants.S3GUARD_CLI_PRUNE_AGE, 0);
+      long confDelta = conf.getLong(S3GUARD_CLI_PRUNE_AGE, 0);
 
       long cliDelta = 0;
       cliDelta += getDeltaComponent(TimeUnit.DAYS, "days");
@@ -823,8 +930,7 @@ public abstract class S3GuardTool extends Configured implements Tool {
       cliDelta += getDeltaComponent(TimeUnit.SECONDS, "seconds");
 
       if (confDelta <= 0 && cliDelta <= 0) {
-        System.err.println(
-            "You must specify a positive age for metadata to prune.");
+        errorln("You must specify a positive age for metadata to prune.");
       }
 
       // A delta provided on the CLI overrides if one is configured
@@ -842,35 +948,235 @@ public abstract class S3GuardTool extends Configured implements Tool {
       return SUCCESS;
     }
 
+  }
+
+  /**
+   * Get info about a bucket and its S3Guard integration status.
+   */
+  static class BucketInfo extends S3GuardTool {
+    public static final String NAME = "bucket-info";
+    public static final String GUARDED_FLAG = "guarded";
+    public static final String UNGUARDED_FLAG = "unguarded";
+    public static final String AUTH_FLAG = "auth";
+    public static final String NONAUTH_FLAG = "nonauth";
+    public static final String ENCRYPTION_FLAG = "encryption";
+
+    public static final String PURPOSE = "provide/check S3Guard information"
+        + " about a specific bucket";
+    private static final String USAGE = NAME + " [OPTIONS] s3a://BUCKET\n"
+        + "\t" + PURPOSE + "\n\n"
+        + "Common options:\n"
+        + "  -" + GUARDED_FLAG + " - Require S3Guard\n"
+        + "  -" + ENCRYPTION_FLAG
+        + " -require {none, sse-s3, sse-kms} - Require encryption policy";
+
+    BucketInfo(Configuration conf) {
+      super(conf, GUARDED_FLAG, UNGUARDED_FLAG, AUTH_FLAG, NONAUTH_FLAG);
+      CommandFormat format = getCommandFormat();
+      format.addOptionWithValue(ENCRYPTION_FLAG);
+    }
+
+    @Override
+    String getName() {
+      return NAME;
+    }
+
     @Override
-    public int run(String[] args) throws InterruptedException, IOException {
-      return run(args, System.out);
+    public String getUsage() {
+      return USAGE;
     }
+
+    public int run(String[] args, PrintStream out)
+        throws InterruptedException, IOException {
+      List<String> paths = parseArgs(args);
+      if (paths.isEmpty()) {
+        errorln(getUsage());
+        throw invalidArgs("No bucket specified");
+      }
+      String s3Path = paths.get(0);
+      S3AFileSystem fs = (S3AFileSystem) FileSystem.newInstance(
+          toUri(s3Path), getConf());
+      setFilesystem(fs);
+      Configuration conf = fs.getConf();
+      URI fsUri = fs.getUri();
+      MetadataStore store = fs.getMetadataStore();
+      println(out, "Filesystem %s", fsUri);
+      println(out, "Location: %s", fs.getBucketLocation());
+      boolean usingS3Guard = !(store instanceof NullMetadataStore);
+      boolean authMode = false;
+      if (usingS3Guard) {
+        out.printf("Filesystem %s is using S3Guard with store %s%n",
+            fsUri, store.toString());
+        printOption(out, "Authoritative S3Guard",
+            METADATASTORE_AUTHORITATIVE, "false");
+        authMode = conf.getBoolean(METADATASTORE_AUTHORITATIVE, false);
+        printStoreDiagnostics(out, store);
+      } else {
+        println(out, "Filesystem %s is not using S3Guard", fsUri);
+      }
+      println(out, "%nS3A Client");
+
+      String endpoint = conf.getTrimmed(ENDPOINT, "");
+      println(out, "\tEndpoint: %s=%s",
+          ENDPOINT,
+          StringUtils.isNotEmpty(endpoint) ? endpoint : "(unset)");
+      String encryption =
+          printOption(out, "\tEncryption", SERVER_SIDE_ENCRYPTION_ALGORITHM,
+              "none");
+      printOption(out, "\tInput seek policy", INPUT_FADVISE, INPUT_FADV_NORMAL);
+
+      CommandFormat commands = getCommandFormat();
+      if (usingS3Guard) {
+        if (commands.getOpt(UNGUARDED_FLAG)) {
+          throw badState("S3Guard is enabled for %s", fsUri);
+        }
+        if (commands.getOpt(AUTH_FLAG) && !authMode) {
+          throw badState("S3Guard is enabled for %s,"
+              + " but not in authoritative mode", fsUri);
+        }
+        if (commands.getOpt(NONAUTH_FLAG) && authMode) {
+          throw badState("S3Guard is enabled in authoritative mode for %s",
+              fsUri);
+        }
+      } else {
+        if (commands.getOpt(GUARDED_FLAG)) {
+          throw badState("S3Guard is not enabled for %s", fsUri);
+        }
+      }
+
+      String desiredEncryption = getCommandFormat()
+          .getOptValue(ENCRYPTION_FLAG);
+      if (StringUtils.isNotEmpty(desiredEncryption)
+          && !desiredEncryption.equalsIgnoreCase(encryption)) {
+        throw badState("Bucket %s: required encryption is %s"
+                    + " but actual encryption is %s",
+                fsUri, desiredEncryption, encryption);
+      }
+
+      out.flush();
+      return SUCCESS;
+    }
+
+    private String printOption(PrintStream out,
+        String description, String key, String defVal) {
+      String t = getFilesystem().getConf().getTrimmed(key, defVal);
+      println(out, "%s: %s=%s", description, key, t);
+      return t;
+    }
+
   }
 
   private static S3GuardTool command;
 
+  /**
+   * Convert a path to a URI, catching any {@code URISyntaxException}
+   * and converting to an invalid args exception.
+   * @param s3Path path to convert to a URI
+   * @return a URI of the path
+   * @throws ExitUtil.ExitException INVALID_ARGUMENT if the URI is invalid
+   */
+  protected static URI toUri(String s3Path) {
+    URI uri;
+    try {
+      uri = new URI(s3Path);
+    } catch (URISyntaxException e) {
+      throw invalidArgs("Not a valid fileystem path: %s", s3Path);
+    }
+    return uri;
+  }
+
   private static void printHelp() {
     if (command == null) {
-      System.err.println("Usage: hadoop " + USAGE);
-      System.err.println("\tperform S3Guard metadata store " +
+      errorln("Usage: hadoop " + USAGE);
+      errorln("\tperform S3Guard metadata store " +
           "administrative commands.");
     } else {
-      System.err.println("Usage: hadoop " + command.getUsage());
+      errorln("Usage: hadoop " + command.getUsage());
     }
+    errorln();
+    errorln(COMMON_USAGE);
+  }
+
+  private static void errorln() {
     System.err.println();
-    System.err.println(COMMON_USAGE);
+  }
+
+  private static void errorln(String x) {
+    System.err.println(x);
+  }
+
+  /**
+   * Print a formatted string followed by a newline to the output stream.
+   * @param out destination
+   * @param format format string
+   * @param args optional arguments
+   */
+  private static void println(PrintStream out, String format, Object... args) {
+    out.println(String.format(format, args));
+  }
+
+  /**
+   * Retrieve and Print store diagnostics.
+   * @param out output stream
+   * @param store store
+   * @throws IOException Failure to retrieve the data.
+   */
+  protected static void printStoreDiagnostics(PrintStream out,
+      MetadataStore store)
+      throws IOException {
+    Map<String, String> diagnostics = store.getDiagnostics();
+    out.println("Metadata Store Diagnostics:");
+    for (Map.Entry<String, String> entry : diagnostics.entrySet()) {
+      println(out, "\t%s=%s", entry.getKey(), entry.getValue());
+    }
+  }
+
+
+  /**
+   * Handle store not found by converting to an exit exception
+   * with specific error code.
+   * @param e exception
+   * @return a new exception to throw
+   */
+  protected static ExitUtil.ExitException storeNotFound(
+      FileNotFoundException e) {
+    return new ExitUtil.ExitException(
+        E_NOT_FOUND, e.toString(), e);
+  }
+
+  /**
+   * Build the exception to raise on invalid arguments.
+   * @param format string format
+   * @param args optional arguments for the string
+   * @return a new exception to throw
+   */
+  protected static ExitUtil.ExitException invalidArgs(
+      String format, Object...args) {
+    return new ExitUtil.ExitException(INVALID_ARGUMENT,
+        String.format(format, args));
+  }
+
+  /**
+   * Build the exception to raise on a bad store/bucket state.
+   * @param format string format
+   * @param args optional arguments for the string
+   * @return a new exception to throw
+   */
+  protected static ExitUtil.ExitException badState(
+      String format, Object...args) {
+    return new ExitUtil.ExitException(E_BAD_STATE,
+        String.format(format, args));
   }
 
   /**
    * Execute the command with the given arguments.
    *
-   * @param args command specific arguments.
    * @param conf Hadoop configuration.
+   * @param args command specific arguments.
    * @return exit code.
    * @throws Exception on I/O errors.
    */
-  public static int run(String[] args, Configuration conf) throws
+  public static int run(Configuration conf, String...args) throws
       Exception {
     /* ToolRunner.run does this too, but we must do it before looking at
     subCommand or instantiating the cmd object below */
@@ -878,9 +1184,10 @@ public abstract class S3GuardTool extends Configured implements Tool {
         .getRemainingArgs();
     if (otherArgs.length == 0) {
       printHelp();
-      return INVALID_ARGUMENT;
+      throw new ExitUtil.ExitException(E_USAGE, "No arguments provided");
     }
     final String subCommand = otherArgs[0];
+    LOG.debug("Executing command {}", subCommand);
     switch (subCommand) {
     case Init.NAME:
       command = new Init(conf);
@@ -891,15 +1198,22 @@ public abstract class S3GuardTool extends Configured implements Tool {
     case Import.NAME:
       command = new Import(conf);
       break;
+    case BucketInfo.NAME:
+      command = new BucketInfo(conf);
+      break;
     case Diff.NAME:
       command = new Diff(conf);
       break;
     case Prune.NAME:
       command = new Prune(conf);
       break;
+    case SetCapacity.NAME:
+      command = new SetCapacity(conf);
+      break;
     default:
       printHelp();
-      return INVALID_ARGUMENT;
+      throw new ExitUtil.ExitException(E_USAGE,
+          "Unknown command " + subCommand);
     }
     return ToolRunner.run(conf, command, otherArgs);
   }
@@ -910,15 +1224,22 @@ public abstract class S3GuardTool extends Configured implements Tool {
    */
   public static void main(String[] args) {
     try {
-      int ret = run(args, new Configuration());
-      System.exit(ret);
+      int ret = run(new Configuration(), args);
+      exit(ret, "");
     } catch (CommandFormat.UnknownOptionException e) {
-      System.err.println(e.getMessage());
+      errorln(e.getMessage());
       printHelp();
-      System.exit(INVALID_ARGUMENT);
+      exit(E_USAGE, e.getMessage());
+    } catch (ExitUtil.ExitException e) {
+      // explicitly raised exit code
+      exit(e.getExitCode(), e.toString());
     } catch (Throwable e) {
       e.printStackTrace(System.err);
-      System.exit(ERROR);
+      exit(ERROR, e.toString());
     }
   }
+
+  protected static void exit(int status, String text) {
+    ExitUtil.terminate(status, text);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/47011d7d/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3guard.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3guard.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3guard.md
index 7957122..893f4ed 100644
--- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3guard.md
+++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3guard.md
@@ -387,7 +387,7 @@ Example 2
 hadoop s3guard init -meta dynamodb://ireland-team -region eu-west-1
 ```
 
-Creates a table "ireland-team" in the same region "s3-eu-west-1.amazonaws.com"
+Creates a table "ireland-team" in the region "eu-west-1.amazonaws.com"
 
 
 ### Import a bucket: `s3guard import`
@@ -421,6 +421,98 @@ Example
 hadoop s3guard diff s3a://ireland-1
 ```
 
+### Display information about a bucket, `s3guard bucket-info`
+
+Prints and optionally checks the s3guard and encryption status of a bucket.
+
+```bash
+hadoop s3guard bucket-info [ -guarded ] [-unguarded] [-auth] [-nonauth] [-encryption ENCRYPTION] s3a://BUCKET
+```
+
+Options
+
+| argument | meaning |
+|-----------|-------------|
+| `-guarded` | Require S3Guard to be enabled |
+| `-unguarded` | Require S3Guard to be disabled |
+| `-auth` | Require the S3Guard mode to be "authoritative" |
+| `-nonauth` | Require the S3Guard mode to be "non-authoritative" |
+| `-encryption <type>` | Require a specific server-side encryption algorithm  |
+
+The server side encryption options are not directly related to S3Guard, but
+it is often convenient to check them at the same time.
+
+Example
+
+```bash
+hadoop s3guard bucket-info -guarded s3a://ireland-1
+```
+
+List the details of bucket `s3a://ireland-1`, mandating that it must have S3Guard enabled
+
+```
+Filesystem s3a://ireland-1
+Location: eu-west-1
+Filesystem s3a://ireland-1 is using S3Guard with store DynamoDBMetadataStore{region=eu-west-1, tableName=ireland-1}
+Authoritative S3Guard: fs.s3a.metadatastore.authoritative=false
+Metadata Store Diagnostics:
+  ARN=arn:aws:dynamodb:eu-west-1:00000000:table/ireland-1
+  description=S3Guard metadata store in DynamoDB
+  name=ireland-1
+  read-capacity=20
+  region=eu-west-1
+  retryPolicy=ExponentialBackoffRetry(maxRetries=9, sleepTime=100 MILLISECONDS)
+  size=12812
+  status=ACTIVE
+  table={AttributeDefinitions: [{AttributeName: child,AttributeType: S},
+    {AttributeName: parent,AttributeType: S}],TableName: ireland-1,
+    KeySchema: [{AttributeName: parent,KeyType: HASH}, {AttributeName: child,KeyType: RANGE}],
+    TableStatus: ACTIVE,
+    CreationDateTime: Fri Aug 25 19:07:25 BST 2017,
+    ProvisionedThroughput: {LastIncreaseDateTime: Tue Aug 29 11:45:18 BST 2017,
+    LastDecreaseDateTime: Wed Aug 30 15:37:51 BST 2017,
+    NumberOfDecreasesToday: 1,
+    ReadCapacityUnits: 20,WriteCapacityUnits: 20},
+    TableSizeBytes: 12812,ItemCount: 91,
+    TableArn: arn:aws:dynamodb:eu-west-1:00000000:table/ireland-1,}
+  write-capacity=20
+
+S3A Client
+  Endpoint: fs.s3a.endpoint=s3-eu-west-1.amazonaws.com
+  Encryption: fs.s3a.server-side-encryption-algorithm=none
+  Input seek policy: fs.s3a.experimental.input.fadvise=normal
+```
+
+This listing includes all the information about the table supplied from
+
+```bash
+hadoop s3guard bucket-info -unguarded -encryption none s3a://landsat-pds
+```
+
+List the S3Guard status of clients of the public `landsat-pds` bucket,
+and verifies that the data is neither tracked with S3Guard nor encrypted.
+
+
+```
+Filesystem s3a://landsat-pdsLocation: us-west-2
+Filesystem s3a://landsat-pds is not using S3Guard
+Endpoint: fs.s3a.endpoints3.amazonaws.com
+Encryption: fs.s3a.server-side-encryption-algorithm=none
+Input seek policy: fs.s3a.experimental.input.fadvise=normal
+```
+
+Note that other clients may have a S3Guard table set up to store metadata
+on this bucket; the checks are all done from the perspective of the configuration
+setttings of the current client.
+
+```bash
+hadoop s3guard bucket-info -guarded -auth s3a://landsat-pds
+```
+
+Require the bucket to be using S3Guard in authoritative mode. This will normally
+fail against this specific bucket.
+
+
 ### Delete a table: `s3guard destroy`
 
 
@@ -449,7 +541,6 @@ hadoop s3guard destroy -meta dynamodb://ireland-team -region eu-west-1
 ```
 
 
-
 ### Clean up a table, `s3guard prune`
 
 Delete all file entries in the MetadataStore table whose object "modification
@@ -460,7 +551,7 @@ hadoop s3guard prune [-days DAYS] [-hours HOURS] [-minutes MINUTES]
     [-seconds SECONDS] [-m URI] ( -region REGION | s3a://BUCKET )
 ```
 
-A time value must be supplied.
+A time value of hours, minutes and/or seconds must be supplied.
 
 1. This does not delete the entries in the bucket itself.
 1. The modification time is effectively the creation time of the objects
@@ -485,6 +576,63 @@ Delete all entries more than 90 minutes old from the table "ireland-team" in
 the region "eu-west-1".
 
 
+### Tune the IO capacity of the DynamoDB Table, `s3guard set-capacity`
+
+Alter the read and/or write capacity of a s3guard table.
+
+```bash
+hadoop s3guard set-capacity [--read UNIT] [--write UNIT] ( -region REGION | s3a://BUCKET )
+```
+
+The `--read` and `--write` units are those of `s3guard init`.
+
+
+Example
+
+```
+hadoop s3guard set-capacity  -read 20 -write 20 s3a://ireland-1
+```
+
+Set the capacity of the table used by bucket `s3a://ireland-1` to 20 read
+and 20 write. (This is a low number, incidentally)
+
+```
+2017-08-30 16:21:26,343 [main] INFO  s3guard.S3GuardTool (S3GuardTool.java:initMetadataStore(229)) - Metadata store DynamoDBMetadataStore{region=eu-west-1, tableName=ireland-1} is initialized.
+2017-08-30 16:21:26,344 [main] INFO  s3guard.DynamoDBMetadataStore (DynamoDBMetadataStore.java:updateParameters(1084)) - Current table capacity is read: 25, write: 25
+2017-08-30 16:21:26,344 [main] INFO  s3guard.DynamoDBMetadataStore (DynamoDBMetadataStore.java:updateParameters(1086)) - Changing capacity of table to read: 20, write: 20
+Metadata Store Diagnostics:
+  ARN=arn:aws:dynamodb:eu-west-1:00000000000:table/ireland-1
+  description=S3Guard metadata store in DynamoDB
+  name=ireland-1
+  read-capacity=25
+  region=eu-west-1
+  retryPolicy=ExponentialBackoffRetry(maxRetries=9, sleepTime=100 MILLISECONDS)
+  size=12812
+  status=UPDATING
+  table={ ... }
+  write-capacity=25
+```
+
+After the update, the table status changes to `UPDATING`; this is a sign that
+the capacity has been changed
+
+Repeating the same command will not change the capacity, as both read and
+write values match that already in use
+
+```
+2017-08-30 16:24:35,337 [main] INFO  s3guard.DynamoDBMetadataStore (DynamoDBMetadataStore.java:updateParameters(1090)) - Table capacity unchanged at read: 20, write: 20
+Metadata Store Diagnostics:
+  ARN=arn:aws:dynamodb:eu-west-1:00000000000:table/ireland-1
+  description=S3Guard metadata store in DynamoDB
+  name=ireland-1
+  read-capacity=20
+  region=eu-west-1
+  retryPolicy=ExponentialBackoffRetry(maxRetries=9, sleepTime=100 MILLISECONDS)
+  size=12812
+  status=ACTIVE
+  table={ ... }
+  write-capacity=20
+```
 
 ## Debugging and Error Handling
 
@@ -607,6 +755,12 @@ or the configuration is preventing S3Guard from finding the table.
 region as the bucket being used.
 1. Create the table if necessary.
 
+### Error `"The level of configured provisioned throughput for the table was exceeded"`
+
+The IO load of clients of the (shared) DynamoDB table was exceeded.
+
+Currently S3Guard doesn't do any throttling and retries here; the way to address
+this is to increase capacity via the AWS console or the `set-capacity` command.
 
 ## Other Topis
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/47011d7d/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java
index b9fe0fd..b38f191 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java
@@ -559,6 +559,16 @@ public class ITestS3AConfiguration {
   }
 
   @Test
+  public void testClearBucketOption() throws Throwable {
+    Configuration config = new Configuration();
+    config.set(USER_AGENT_PREFIX, "base");
+    setBucketOption(config, "bucket", USER_AGENT_PREFIX, "overridden");
+    clearBucketOption(config, "bucket", USER_AGENT_PREFIX);
+    Configuration updated = propagateBucketOptions(config, "c");
+    assertOptionEquals(updated, USER_AGENT_PREFIX, "base");
+  }
+
+  @Test
   public void testBucketConfigurationSkipsUnmodifiable() throws Throwable {
     Configuration config = new Configuration(false);
     String impl = "fs.s3a.impl";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/47011d7d/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardListConsistency.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardListConsistency.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardListConsistency.java
index da7699e..bc03a17 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardListConsistency.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardListConsistency.java
@@ -435,16 +435,16 @@ public class ITestS3GuardListConsistency extends AbstractS3ATestBase {
     // files to list are delaying visibility
     if (!recursive) {
       // in this case only the top level files are listed
+      verifyFileIsListed(listedFiles, baseTestDir, fileNames);
       assertEquals("Unexpected number of files returned by listFiles() call",
           normalFileNum + delayedFileNum, listedFiles.size());
-      verifyFileIsListed(listedFiles, baseTestDir, fileNames);
     } else {
-      assertEquals("Unexpected number of files returned by listFiles() call",
-          filesAndEmptyDirectories,
-          listedFiles.size());
       for (Path dir : testDirs) {
         verifyFileIsListed(listedFiles, dir, fileNames);
       }
+      assertEquals("Unexpected number of files returned by listFiles() call",
+          filesAndEmptyDirectories,
+          listedFiles.size());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/47011d7d/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardWriteBack.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardWriteBack.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardWriteBack.java
index a63b696..c8a54b8 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardWriteBack.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardWriteBack.java
@@ -22,15 +22,17 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
 import org.apache.hadoop.fs.s3a.s3guard.DirListingMetadata;
 import org.junit.Assume;
 import org.junit.Test;
 
-import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.net.URI;
 import java.util.Arrays;
 
+import static org.apache.hadoop.fs.s3a.Constants.*;
+
 /**
  * Test cases that validate S3Guard's behavior for writing things like
  * directory listings back to the MetadataStore.
@@ -66,7 +68,7 @@ public class ITestS3GuardWriteBack extends AbstractS3ATestBase {
     noS3Guard.mkdirs(new Path(directory, "OnS3"));
     // Create a directory on both S3 and metadata store
     Path p = new Path(directory, "OnS3AndMS");
-    assertPathDoesntExist(noWriteBack, p);
+    ContractTestUtils.assertPathDoesNotExist(noWriteBack, "path", p);
     noWriteBack.mkdirs(p);
 
     FileStatus[] fsResults;
@@ -87,7 +89,7 @@ public class ITestS3GuardWriteBack extends AbstractS3ATestBase {
 
     // FS should return both (and will write it back)
     fsResults = yesWriteBack.listStatus(directory);
-    assertEquals("Filesystem enabled S3Guard with write back should have "
+    assertEquals("Filesystem enabled S3Guard with write back should have"
             + " both /OnS3 and /OnS3AndMS: " + Arrays.toString(fsResults),
         2, fsResults.length);
 
@@ -104,7 +106,12 @@ public class ITestS3GuardWriteBack extends AbstractS3ATestBase {
         new Path(directory, "OnS3"));
   }
 
-  /** Create a separate S3AFileSystem instance for testing. */
+  /**
+   * Create a separate S3AFileSystem instance for testing.
+   * There's a bit of complexity as it forces pushes up s3guard options from
+   * the base values to the per-bucket options. This stops explicit bucket
+   * settings in test XML configs from unintentionally breaking tests.
+   */
   private S3AFileSystem createTestFS(URI fsURI, boolean disableS3Guard,
       boolean authoritativeMeta) throws IOException {
     Configuration conf;
@@ -112,12 +119,22 @@ public class ITestS3GuardWriteBack extends AbstractS3ATestBase {
     // Create a FileSystem that is S3-backed only
     conf = createConfiguration();
     S3ATestUtils.disableFilesystemCaching(conf);
+    String host = fsURI.getHost();
     if (disableS3Guard) {
       conf.set(Constants.S3_METADATA_STORE_IMPL,
           Constants.S3GUARD_METASTORE_NULL);
+      S3AUtils.setBucketOption(conf, host,
+          S3_METADATA_STORE_IMPL,
+          S3GUARD_METASTORE_NULL);
     } else {
       S3ATestUtils.maybeEnableS3Guard(conf);
-      conf.setBoolean(Constants.METADATASTORE_AUTHORITATIVE, authoritativeMeta);
+      conf.setBoolean(METADATASTORE_AUTHORITATIVE, authoritativeMeta);
+      S3AUtils.setBucketOption(conf, host,
+          METADATASTORE_AUTHORITATIVE,
+          Boolean.toString(authoritativeMeta));
+      S3AUtils.setBucketOption(conf, host,
+          S3_METADATA_STORE_IMPL,
+          conf.get(S3_METADATA_STORE_IMPL));
     }
     FileSystem fs = FileSystem.get(fsURI, conf);
     return asS3AFS(fs);
@@ -128,14 +145,4 @@ public class ITestS3GuardWriteBack extends AbstractS3ATestBase {
     return (S3AFileSystem)fs;
   }
 
-  private static void assertPathDoesntExist(FileSystem fs, Path p)
-      throws IOException {
-    try {
-      FileStatus s = fs.getFileStatus(p);
-    } catch (FileNotFoundException e) {
-      return;
-    }
-    fail("Path should not exist: " + p);
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/47011d7d/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
index 8dbf90a..b302e72 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
@@ -697,24 +697,6 @@ public final class S3ATestUtils {
   }
 
   /**
-   * Set a bucket specific property to a particular value.
-   * If the generic key passed in has an {@code fs.s3a. prefix},
-   * that's stripped off, so that when the the bucket properties are propagated
-   * down to the generic values, that value gets copied down.
-   * @param conf configuration to set
-   * @param bucket bucket name
-   * @param genericKey key; can start with "fs.s3a."
-   * @param value value to set
-   */
-  public static void setBucketOption(Configuration conf, String bucket,
-      String genericKey, String value) {
-    final String baseKey = genericKey.startsWith(FS_S3A_PREFIX) ?
-        genericKey.substring(FS_S3A_PREFIX.length())
-        : genericKey;
-    conf.set(FS_S3A_BUCKET_PREFIX + bucket + '.' + baseKey, value);
-  }
-
-  /**
    * Assert that a configuration option matches the expected value.
    * @param conf configuration
    * @param key option key

http://git-wip-us.apache.org/repos/asf/hadoop/blob/47011d7d/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java
index ceacdf3..a33c001 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java
@@ -18,9 +18,15 @@
 
 package org.apache.hadoop.fs.s3a.s3guard;
 
+import java.io.ByteArrayOutputStream;
+import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.io.PrintStream;
+import java.util.Collection;
+import java.util.concurrent.Callable;
 import java.util.concurrent.TimeUnit;
 
+import org.junit.Assume;
 import org.junit.Test;
 
 import org.apache.hadoop.conf.Configuration;
@@ -31,8 +37,11 @@ import org.apache.hadoop.fs.s3a.Constants;
 import org.apache.hadoop.fs.s3a.S3AFileStatus;
 import org.apache.hadoop.fs.s3a.S3ATestUtils;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.util.ExitUtil;
+import org.apache.hadoop.util.StringUtils;
 
 import static org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.SUCCESS;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 
 /**
  * Common functionality for S3GuardTool test cases.
@@ -40,6 +49,9 @@ import static org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.SUCCESS;
 public abstract class AbstractS3GuardToolTestBase extends AbstractS3ATestBase {
 
   protected static final String OWNER = "hdfs";
+  protected static final String DYNAMODB_TABLE = "dynamodb://ireland-team";
+  protected static final String S3A_THIS_BUCKET_DOES_NOT_EXIST
+      = "s3a://this-bucket-does-not-exist-00000000000";
 
   private MetadataStore ms;
 
@@ -57,6 +69,51 @@ public abstract class AbstractS3GuardToolTestBase extends AbstractS3ATestBase {
     assertEquals(message, SUCCESS, tool.run(args));
   }
 
+  /**
+   * Run a S3GuardTool command from a varags list.
+   * @param conf configuration
+   * @param args argument list
+   * @return the return code
+   * @throws Exception any exception
+   */
+  protected int run(Configuration conf, String... args)
+      throws Exception {
+    return S3GuardTool.run(conf, args);
+  }
+
+  /**
+   * Run a S3GuardTool command from a varags list and the
+   * configuration returned by {@code getConfiguration()}.
+   * @param args argument list
+   * @return the return code
+   * @throws Exception any exception
+   */
+  protected int run(String... args) throws Exception {
+    return S3GuardTool.run(getConfiguration(), args);
+  }
+
+  /**
+   * Run a S3GuardTool command from a varags list, catch any raised
+   * ExitException and verify the status code matches that expected.
+   * @param status expected status code of the exception
+   * @param args argument list
+   * @throws Exception any exception
+   */
+  protected void runToFailure(int status, String... args)
+      throws Exception {
+    ExitUtil.ExitException ex =
+        intercept(ExitUtil.ExitException.class,
+            new Callable<Integer>() {
+              @Override
+              public Integer call() throws Exception {
+                return run(args);
+              }
+            });
+    if (ex.status != status) {
+      throw ex;
+    }
+  }
+
   protected MetadataStore getMetadataStore() {
     return ms;
   }
@@ -134,16 +191,23 @@ public abstract class AbstractS3GuardToolTestBase extends AbstractS3ATestBase {
       Thread.sleep(TimeUnit.SECONDS.toMillis(2));
       createFile(new Path(parent, "fresh"), true, true);
 
-      assertEquals(2, ms.listChildren(parent).getListing().size());
-      expectSuccess("Prune command did not exit successfully - see output", cmd,
-          args);
-      assertEquals(1, ms.listChildren(parent).getListing().size());
+      assertMetastoreListingCount(parent, "Children count before pruning", 2);
+      exec(cmd, args);
+      assertMetastoreListingCount(parent, "Pruned children count", 1);
     } finally {
       getFileSystem().delete(parent, true);
       ms.prune(Long.MAX_VALUE);
     }
   }
 
+  private void assertMetastoreListingCount(Path parent,
+      String message,
+      int expected) throws IOException {
+    Collection<PathMetadata> listing = ms.listChildren(parent).getListing();
+    assertEquals(message +" [" + StringUtils.join(", ", listing) + "]",
+        expected, listing.size());
+  }
+
   @Test
   public void testPruneCommandCLI() throws Exception {
     String testPath = path("testPruneCommandCLI").toString();
@@ -158,4 +222,70 @@ public abstract class AbstractS3GuardToolTestBase extends AbstractS3ATestBase {
     String testPath = path("testPruneCommandConf").toString();
     testPruneCommand(getConfiguration(), "prune", testPath);
   }
+
+  @Test
+  public void testDestroyNoBucket() throws Throwable {
+    intercept(FileNotFoundException.class,
+        new Callable<Integer>() {
+          @Override
+          public Integer call() throws Exception {
+            return run(S3GuardTool.Destroy.NAME,
+                S3A_THIS_BUCKET_DOES_NOT_EXIST);
+          }
+        });
+  }
+
+  /**
+   * Get the test CSV file; assume() that it is not modified (i.e. we haven't
+   * switched to a new storage infrastructure where the bucket is no longer
+   * read only).
+   * @return test file.
+   */
+  protected String getLandsatCSVFile() {
+    String csvFile = getConfiguration()
+        .getTrimmed(KEY_CSVTEST_FILE, DEFAULT_CSVTEST_FILE);
+    Assume.assumeTrue("CSV test file is not the default",
+        DEFAULT_CSVTEST_FILE.equals(csvFile));
+    return csvFile;
+  }
+
+  /**
+   * Execute a command, returning the buffer if the command actually completes.
+   * If an exception is raised the output is logged instead.
+   * @param cmd command
+   * @param buf buffer to use for tool output (not SLF4J output)
+   * @param args argument list
+   * @throws Exception on any failure
+   */
+  public String exec(S3GuardTool cmd, String...args) throws Exception {
+    ByteArrayOutputStream buf = new ByteArrayOutputStream();
+    try {
+      exec(cmd, buf, args);
+      return buf.toString();
+    } catch (AssertionError e) {
+      throw e;
+    } catch (Exception e) {
+      LOG.error("Command {} failed: \n{}", cmd, buf);
+      throw e;
+    }
+  }
+
+  /**
+   * Execute a command, saving the output into the buffer.
+   * @param cmd command
+   * @param buf buffer to use for tool output (not SLF4J output)
+   * @param args argument list
+   * @throws Exception on any failure
+   */
+  protected void exec(S3GuardTool cmd, ByteArrayOutputStream buf, String...args)
+      throws Exception {
+    LOG.info("exec {}", (Object) args);
+    int r = 0;
+    try(PrintStream out =new PrintStream(buf)) {
+      r = cmd.run(args, out);
+      out.flush();
+    }
+    assertEquals("Command " + cmd + " failed\n"+ buf, 0, r);
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/47011d7d/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolDynamoDB.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolDynamoDB.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolDynamoDB.java
index c13dfc4..c7dffd2 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolDynamoDB.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolDynamoDB.java
@@ -19,19 +19,28 @@
 package org.apache.hadoop.fs.s3a.s3guard;
 
 import java.io.IOException;
+import java.util.Map;
+import java.util.Objects;
 import java.util.Random;
 import java.util.concurrent.Callable;
+import java.util.concurrent.atomic.AtomicInteger;
 
 import com.amazonaws.services.dynamodbv2.document.DynamoDB;
 import com.amazonaws.services.dynamodbv2.document.Table;
 import com.amazonaws.services.dynamodbv2.model.ResourceNotFoundException;
 import org.junit.Test;
 
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.s3a.Constants;
 import org.apache.hadoop.fs.s3a.S3AFileSystem;
+import org.apache.hadoop.fs.s3a.S3AUtils;
 import org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.Destroy;
 import org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.Init;
 import org.apache.hadoop.test.LambdaTestUtils;
 
+import static org.apache.hadoop.fs.s3a.s3guard.DynamoDBMetadataStore.*;
+import static org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.*;
+
 /**
  * Test S3Guard related CLI commands against DynamoDB.
  */
@@ -78,8 +87,76 @@ public class ITestS3GuardToolDynamoDB extends AbstractS3GuardToolTestBase {
         });
   }
 
+  private static class Capacities {
+    private final long read, write;
+
+    Capacities(long read, long write) {
+      this.read = read;
+      this.write = write;
+    }
+
+    public long getRead() {
+      return read;
+    }
+
+    public long getWrite() {
+      return write;
+    }
+
+    String getReadStr() {
+      return Long.toString(read);
+    }
+
+    String getWriteStr() {
+      return Long.toString(write);
+    }
+
+    void checkEquals(String text, Capacities that) throws Exception {
+      if (!this.equals(that)) {
+        throw new Exception(text + " expected = " + this +"; actual = "+ that);
+      }
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      Capacities that = (Capacities) o;
+      return read == that.read && write == that.write;
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(read, write);
+    }
+
+    @Override
+    public String toString() {
+      final StringBuilder sb = new StringBuilder("Capacities{");
+      sb.append("read=").append(read);
+      sb.append(", write=").append(write);
+      sb.append('}');
+      return sb.toString();
+    }
+  }
+
+  private Capacities getCapacities() throws IOException {
+    Map<String, String> diagnostics = getMetadataStore().getDiagnostics();
+    return getCapacities(diagnostics);
+  }
+
+  private Capacities getCapacities(Map<String, String> diagnostics) {
+    return new Capacities(
+        Long.parseLong(diagnostics.get(DynamoDBMetadataStore.READ_CAPACITY)),
+        Long.parseLong(diagnostics.get(DynamoDBMetadataStore.WRITE_CAPACITY)));
+  }
+
   @Test
-  public void testDynamoDBInitDestroyCycle() throws Exception {
+  public void testDynamoDBInitDestroyCycle() throws Throwable {
     String testTableName = "testDynamoDBInitDestroy" + new Random().nextInt();
     String testS3Url = path(testTableName).toString();
     S3AFileSystem fs = getFileSystem();
@@ -99,11 +176,80 @@ public class ITestS3GuardToolDynamoDB extends AbstractS3GuardToolTestBase {
       assertTrue(String.format("%s does not exist", testTableName),
           exist(db, testTableName));
 
+      Configuration conf = fs.getConf();
+      String bucket = fs.getBucket();
+      // force in a new bucket
+      S3AUtils.setBucketOption(conf, bucket, Constants.S3_METADATA_STORE_IMPL,
+          Constants.S3GUARD_METASTORE_DYNAMO);
+      initCmd = new Init(conf);
+      String initOutput = exec(initCmd,
+          "init", "-meta", "dynamodb://" + testTableName, testS3Url);
+      assertTrue("No Dynamo diagnostics in output " + initOutput,
+          initOutput.contains(DESCRIPTION));
+
+      // run a bucket info command and look for
+      // confirmation that it got the output from DDB diags
+      S3GuardTool.BucketInfo infocmd = new S3GuardTool.BucketInfo(conf);
+      String info = exec(infocmd, S3GuardTool.BucketInfo.NAME,
+          "-" + S3GuardTool.BucketInfo.GUARDED_FLAG,
+          testS3Url);
+      assertTrue("No Dynamo diagnostics in output " + info,
+          info.contains(DESCRIPTION));
+
+      // get the current values to set again
+
+      // play with the set-capacity option
+      Capacities original = getCapacities();
+      String fsURI = getFileSystem().getUri().toString();
+      String capacityOut = exec(newSetCapacity(),
+          S3GuardTool.SetCapacity.NAME,
+          fsURI);
+      LOG.info("Set Capacity output=\n{}", capacityOut);
+      capacityOut = exec(newSetCapacity(),
+          S3GuardTool.SetCapacity.NAME,
+          "-" + READ_FLAG, original.getReadStr(),
+          "-" + WRITE_FLAG, original.getWriteStr(),
+          fsURI);
+      LOG.info("Set Capacity output=\n{}", capacityOut);
+
+      // that call does not change the values
+      original.checkEquals("unchanged", getCapacities());
+
+      // now update the value
+      long readCap = original.getRead();
+      long writeCap = original.getWrite();
+      long rc2 = readCap + 1;
+      long wc2 = writeCap + 1;
+      Capacities desired = new Capacities(rc2, wc2);
+      capacityOut = exec(newSetCapacity(),
+          S3GuardTool.SetCapacity.NAME,
+          "-" + READ_FLAG, Long.toString(rc2),
+          "-" + WRITE_FLAG, Long.toString(wc2),
+          fsURI);
+      LOG.info("Set Capacity output=\n{}", capacityOut);
+
+      // to avoid race conditions, spin for the state change
+      AtomicInteger c = new AtomicInteger(0);
+      LambdaTestUtils.eventually(60000,
+          new LambdaTestUtils.VoidCallable() {
+            @Override
+            public void call() throws Exception {
+                c.incrementAndGet();
+                Map<String, String> diags = getMetadataStore().getDiagnostics();
+                Capacities updated = getCapacities(diags);
+                String tableInfo = String.format("[%02d] table state: %s",
+                    c.intValue(), diags.get(STATUS));
+                LOG.info("{}; capacities {}",
+                    tableInfo, updated);
+                desired.checkEquals(tableInfo, updated);
+            }
+          },
+          new LambdaTestUtils.ProportionalRetryInterval(500, 5000));
+
       // Destroy MetadataStore
       Destroy destroyCmd = new Destroy(fs.getConf());
 
-      expectSuccess("Destroy command did not exit successfully - see output",
-          destroyCmd,
+      String destroyed = exec(destroyCmd,
           "destroy", "-meta", "dynamodb://" + testTableName, testS3Url);
       // Verify it does not exist
       assertFalse(String.format("%s still exists", testTableName),
@@ -131,4 +277,19 @@ public class ITestS3GuardToolDynamoDB extends AbstractS3GuardToolTestBase {
       }
     }
   }
+
+  private S3GuardTool newSetCapacity() {
+    S3GuardTool setCapacity = new S3GuardTool.SetCapacity(
+        getFileSystem().getConf());
+    setCapacity.setStore(getMetadataStore());
+    return setCapacity;
+  }
+
+  @Test
+  public void testDestroyUnknownTable() throws Throwable {
+    run(S3GuardTool.Destroy.NAME,
+        "-region", "us-west-2",
+        "-meta", DYNAMODB_TABLE);
+  }
+
 }


---------------------------------------------------------------------
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: HADOOP-14903. Add json-smart explicitly to pom.xml. (rchiang)

Posted by jh...@apache.org.
HADOOP-14903. Add json-smart explicitly to pom.xml. (rchiang)


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

Branch: refs/heads/YARN-5734
Commit: cde804b91f2fed514935d186d89ea7cbbf2701f3
Parents: 0889e5a
Author: Ray Chiang <rc...@apache.org>
Authored: Mon Sep 25 15:30:51 2017 -0700
Committer: Ray Chiang <rc...@apache.org>
Committed: Mon Sep 25 15:30:51 2017 -0700

----------------------------------------------------------------------
 hadoop-project/pom.xml | 9 +++++++++
 1 file changed, 9 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/cde804b9/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 7f657ca..7b342c4 100755
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -1207,6 +1207,15 @@
         </exclusions>
       </dependency>
       <dependency>
+        <!-- HACK.  Transitive dependency for nimbus-jose-jwt.  Needed for
+             packaging.  Please re-check this version when updating
+             nimbus-jose-jwt.  Please read HADOOP-14903 for more details.
+          -->
+        <groupId>net.minidev</groupId>
+        <artifactId>json-smart</artifactId>
+        <version>2.3</version>
+      </dependency>
+      <dependency>
         <groupId>org.skyscreamer</groupId>
         <artifactId>jsonassert</artifactId>
         <version>1.3.0</version>


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


[14/50] [abbrv] hadoop git commit: YARN-7153. Remove duplicated code in AMRMClientAsyncImpl.java. Contributed by Sen Zhao.

Posted by jh...@apache.org.
YARN-7153. Remove duplicated code in AMRMClientAsyncImpl.java. Contributed by Sen Zhao.


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

Branch: refs/heads/YARN-5734
Commit: e928ee583c5a1367e24eab34057f8d8496891452
Parents: 3a10367
Author: Akira Ajisaka <aa...@apache.org>
Authored: Tue Sep 26 02:03:30 2017 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Tue Sep 26 02:03:30 2017 +0900

----------------------------------------------------------------------
 .../hadoop/yarn/client/api/async/impl/AMRMClientAsyncImpl.java | 6 +-----
 1 file changed, 1 insertion(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e928ee58/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/AMRMClientAsyncImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/AMRMClientAsyncImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/AMRMClientAsyncImpl.java
index d12b108..e44f3cd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/AMRMClientAsyncImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/AMRMClientAsyncImpl.java
@@ -142,11 +142,7 @@ extends AMRMClientAsync<T> {
     handlerThread.interrupt();
     super.serviceStop();
   }
-  
-  public void setHeartbeatInterval(int interval) {
-    heartbeatIntervalMs.set(interval);
-  }
-  
+
   public List<? extends Collection<T>> getMatchingRequests(
                                                    Priority priority, 
                                                    String resourceName, 


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


[27/50] [abbrv] hadoop git commit: HDFS-12523. Thread pools in ErasureCodingWorker do not shutdown. (Huafeng Wang via Lei)

Posted by jh...@apache.org.
HDFS-12523. Thread pools in ErasureCodingWorker do not shutdown. (Huafeng Wang via Lei)


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

Branch: refs/heads/YARN-5734
Commit: 1267ff22ce9226b6dd52e3f33cbe3b3094fb0e35
Parents: 9d3e4cc
Author: Lei Xu <le...@apache.org>
Authored: Thu Sep 21 16:10:32 2017 -0700
Committer: Lei Xu <le...@apache.org>
Committed: Tue Sep 26 10:08:16 2017 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/server/datanode/DataNode.java |  6 +++++-
 .../server/datanode/erasurecode/ErasureCodingWorker.java | 11 +++++++++--
 .../datanode/erasurecode/StripedReconstructor.java       |  8 +++-----
 3 files changed, 17 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1267ff22/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index 042a627..6163d93 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@ -1115,7 +1115,7 @@ public class DataNode extends ReconfigurableBase
   /**
    * Shutdown disk balancer.
    */
-  private  void shutdownDiskBalancer() {
+  private void shutdownDiskBalancer() {
     if (this.diskBalancer != null) {
       this.diskBalancer.shutdown();
       this.diskBalancer = null;
@@ -2077,6 +2077,10 @@ public class DataNode extends ReconfigurableBase
       ipcServer.stop();
     }
 
+    if (ecWorker != null) {
+      ecWorker.shutDown();
+    }
+
     if(blockPoolManager != null) {
       try {
         this.blockPoolManager.shutDownAll(bposArray);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1267ff22/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java
index 07d213c..63498bc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java
@@ -27,6 +27,8 @@ import org.apache.hadoop.util.Daemon;
 import org.slf4j.Logger;
 
 import java.util.Collection;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ExecutorCompletionService;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.SynchronousQueue;
 import java.util.concurrent.ThreadPoolExecutor;
@@ -149,7 +151,12 @@ public final class ErasureCodingWorker {
     return conf;
   }
 
-  ThreadPoolExecutor getStripedReadPool() {
-    return stripedReadPool;
+  CompletionService<Void> createReadService() {
+    return new ExecutorCompletionService<>(stripedReadPool);
+  }
+
+  public void shutDown() {
+    stripedReconstructionPool.shutdown();
+    stripedReadPool.shutdown();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1267ff22/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedReconstructor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedReconstructor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedReconstructor.java
index 3202121..bbffcf5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedReconstructor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedReconstructor.java
@@ -39,8 +39,6 @@ import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
 import java.util.BitSet;
 import java.util.concurrent.CompletionService;
-import java.util.concurrent.ExecutorCompletionService;
-import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.atomic.AtomicLong;
 
 /**
@@ -110,7 +108,7 @@ abstract class StripedReconstructor {
   // position in striped internal block
   private long positionInBlock;
   private StripedReader stripedReader;
-  private ThreadPoolExecutor stripedReadPool;
+  private ErasureCodingWorker erasureCodingWorker;
   private final CachingStrategy cachingStrategy;
   private long maxTargetLength = 0L;
   private final BitSet liveBitSet;
@@ -122,7 +120,7 @@ abstract class StripedReconstructor {
 
   StripedReconstructor(ErasureCodingWorker worker,
       StripedReconstructionInfo stripedReconInfo) {
-    this.stripedReadPool = worker.getStripedReadPool();
+    this.erasureCodingWorker = worker;
     this.datanode = worker.getDatanode();
     this.conf = worker.getConf();
     this.ecPolicy = stripedReconInfo.getEcPolicy();
@@ -225,7 +223,7 @@ abstract class StripedReconstructor {
   }
 
   CompletionService<Void> createReadService() {
-    return new ExecutorCompletionService<>(stripedReadPool);
+    return erasureCodingWorker.createReadService();
   }
 
   ExtendedBlock getBlockGroup() {


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


[34/50] [abbrv] hadoop git commit: YARN-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/a7c8a8e3
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/a7c8a8e3
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/a7c8a8e3

Branch: refs/heads/YARN-5734
Commit: a7c8a8e34f0eb884bea47f2f66bac5870c02364e
Parents: e7b4762
Author: Xuan <xg...@apache.org>
Authored: Fri Feb 24 15:58:12 2017 -0800
Committer: Jonathan Hung <jh...@linkedin.com>
Committed: Wed Sep 27 19:52:58 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/a7c8a8e3/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/a7c8a8e3/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/a7c8a8e3/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


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

Branch: refs/heads/YARN-5734
Commit: c85d2ad61751f4000778a42d1a9ea64253d7063d
Parents: a7c8a8e
Author: Jonathan Hung <jh...@linkedin.com>
Authored: Wed Mar 1 16:03:01 2017 -0800
Committer: Jonathan Hung <jh...@linkedin.com>
Committed: Wed Sep 27 19:52:58 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/c85d2ad6/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 f58833c..ddb9768 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
@@ -674,6 +674,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/c85d2ad6/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 8453dc7..b5adfae 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
@@ -3367,4 +3367,16 @@
       </description>
   </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/c85d2ad6/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/c85d2ad6/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 e74cbe1..11f2f6e 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;
@@ -295,10 +296,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/c85d2ad6/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 3821e24..40cb893 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/c85d2ad6/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/c85d2ad6/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/c85d2ad6/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/c85d2ad6/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


[11/50] [abbrv] hadoop git commit: HADOOP-14897. Loosen compatibility guidelines for native dependencies

Posted by jh...@apache.org.
HADOOP-14897. Loosen compatibility guidelines for native dependencies


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

Branch: refs/heads/YARN-5734
Commit: 0807470808156af13e0992285cac070052d86c26
Parents: d0b2c58
Author: Daniel Templeton <te...@apache.org>
Authored: Mon Sep 25 14:12:25 2017 +0200
Committer: Daniel Templeton <te...@apache.org>
Committed: Mon Sep 25 14:12:25 2017 +0200

----------------------------------------------------------------------
 .../hadoop-common/src/site/markdown/Compatibility.md      | 10 +++++++---
 1 file changed, 7 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/08074708/hadoop-common-project/hadoop-common/src/site/markdown/Compatibility.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/Compatibility.md b/hadoop-common-project/hadoop-common/src/site/markdown/Compatibility.md
index 4fa8c02..47fa09a 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/Compatibility.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/Compatibility.md
@@ -191,9 +191,13 @@ dependencies is part of the Hadoop ABI.
 
 The minimum required versions of the native components on which Hadoop depends
 at compile time and/or runtime SHALL be considered
-[Stable](./InterfaceClassification.html#Stable). Changes to the minimum
-required versions MUST NOT increase between minor releases within a major
-version.
+[Evolving](./InterfaceClassification.html#Evolving). Changes to the minimum
+required versions SHOULD NOT increase between minor releases within a major
+version, though updates because of security issues, license issues, or other
+reasons may occur. When the native components on which Hadoop depends must
+be updated between minor releases within a major release, where possible the
+changes SHOULD only change the minor versions of the components without
+changing the major versions.
 
 ### Wire Protocols
 


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


[05/50] [abbrv] hadoop git commit: HADOOP-14894. ReflectionUtils should use Time.monotonicNow to mesaure duration. Contributed by Bharat Viswanadham.

Posted by jh...@apache.org.
HADOOP-14894. ReflectionUtils should use Time.monotonicNow to mesaure duration. Contributed by Bharat Viswanadham.


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

Branch: refs/heads/YARN-5734
Commit: 1c4a798505a2bc8285b709188f6492dbb33a0968
Parents: 2a81af4
Author: Anu Engineer <ae...@apache.org>
Authored: Sat Sep 23 11:56:34 2017 -0700
Committer: Anu Engineer <ae...@apache.org>
Committed: Sat Sep 23 11:56:34 2017 -0700

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c4a7985/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ReflectionUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ReflectionUtils.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ReflectionUtils.java
index f1294e7..ff7357a 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ReflectionUtils.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ReflectionUtils.java
@@ -212,7 +212,7 @@ public class ReflectionUtils {
     boolean dumpStack = false;
     if (log.isInfoEnabled()) {
       synchronized (ReflectionUtils.class) {
-        long now = Time.now();
+        long now = Time.monotonicNow();
         if (now - previousLogTime >= minInterval * 1000) {
           previousLogTime = now;
           dumpStack = true;
@@ -241,7 +241,7 @@ public class ReflectionUtils {
     boolean dumpStack = false;
     if (log.isInfoEnabled()) {
       synchronized (ReflectionUtils.class) {
-        long now = Time.now();
+        long now = Time.monotonicNow();
         if (now - previousLogTime >= minInterval * 1000) {
           previousLogTime = now;
           dumpStack = true;


---------------------------------------------------------------------
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-14822. hadoop-project/pom.xml is executable. Contributed by Ajay Kumar.

Posted by jh...@apache.org.
HADOOP-14822. hadoop-project/pom.xml is executable. 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/dd378775
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/dd378775
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/dd378775

Branch: refs/heads/YARN-5734
Commit: dd378775df5bd79e23c12120535404c4ca0fb386
Parents: 0da29cb
Author: Akira Ajisaka <aa...@apache.org>
Authored: Wed Sep 27 14:25:07 2017 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Wed Sep 27 14:25:07 2017 +0900

----------------------------------------------------------------------
 hadoop-project/pom.xml | 0
 1 file changed, 0 insertions(+), 0 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/dd378775/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
old mode 100755
new mode 100644


---------------------------------------------------------------------
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: HADOOP-14892. MetricsSystemImpl should use Time.monotonicNow for measuring durations. Contributed by Chetna Chaudhari.

Posted by jh...@apache.org.
HADOOP-14892. MetricsSystemImpl should use Time.monotonicNow for measuring durations. Contributed by Chetna Chaudhari.


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

Branch: refs/heads/YARN-5734
Commit: 3a10367a17b99e2c3035ec54048f94f334d93227
Parents: 02e2a9b
Author: Akira Ajisaka <aa...@apache.org>
Authored: Tue Sep 26 01:47:02 2017 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Tue Sep 26 01:47:02 2017 +0900

----------------------------------------------------------------------
 .../org/apache/hadoop/metrics2/impl/MetricsSystemImpl.java   | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a10367a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsSystemImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsSystemImpl.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsSystemImpl.java
index 2248122..ee1672e 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsSystemImpl.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsSystemImpl.java
@@ -415,10 +415,10 @@ public class MetricsSystemImpl extends MetricsSystem implements MetricsSource {
 
   private void snapshotMetrics(MetricsSourceAdapter sa,
                                MetricsBufferBuilder bufferBuilder) {
-    long startTime = Time.now();
+    long startTime = Time.monotonicNow();
     bufferBuilder.add(sa.name(), sa.getMetrics(collector, true));
     collector.clear();
-    snapshotStat.add(Time.now() - startTime);
+    snapshotStat.add(Time.monotonicNow() - startTime);
     LOG.debug("Snapshotted source "+ sa.name());
   }
 
@@ -431,7 +431,7 @@ public class MetricsSystemImpl extends MetricsSystem implements MetricsSource {
   synchronized void publishMetrics(MetricsBuffer buffer, boolean immediate) {
     int dropped = 0;
     for (MetricsSinkAdapter sa : sinks.values()) {
-      long startTime = Time.now();
+      long startTime = Time.monotonicNow();
       boolean result;
       if (immediate) {
         result = sa.putMetricsImmediate(buffer); 
@@ -439,7 +439,7 @@ public class MetricsSystemImpl extends MetricsSystem implements MetricsSource {
         result = sa.putMetrics(buffer, logicalTime);
       }
       dropped += result ? 0 : 1;
-      publishStat.add(Time.now() - startTime);
+      publishStat.add(Time.monotonicNow() - startTime);
     }
     droppedPubAll.incr(dropped);
   }


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


[23/50] [abbrv] hadoop git commit: YARN-65. Reduce RM app memory footprint once app has completed. Contributed by Manikandan R.

Posted by jh...@apache.org.
YARN-65. Reduce RM app memory footprint once app has completed. Contributed by Manikandan R.


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

Branch: refs/heads/YARN-5734
Commit: 06e5a7b5cf141420d3a411088b87acba72e68cad
Parents: d08b8c8
Author: Rohith Sharma K S <ro...@apache.org>
Authored: Tue Sep 26 14:33:48 2017 +0530
Committer: Rohith Sharma K S <ro...@apache.org>
Committed: Tue Sep 26 14:34:09 2017 +0530

----------------------------------------------------------------------
 .../server/resourcemanager/rmapp/RMAppImpl.java |  11 +-
 .../resourcemanager/MockMemoryRMStateStore.java | 130 ++++++++++++
 .../yarn/server/resourcemanager/MockRM.java     |   2 +-
 .../resourcemanager/MockRMMemoryStateStore.java |  32 ---
 .../resourcemanager/TestApplicationCleanup.java |   4 +-
 .../TestContainerResourceUsage.java             |   4 +-
 .../yarn/server/resourcemanager/TestRMHA.java   |   4 +-
 .../server/resourcemanager/TestRMRestart.java   |  47 +++--
 .../TestWorkPreservingRMRestart.java            |  51 +++--
 .../applicationsmanager/TestAMRestart.java      |  13 +-
 .../rmapp/TestRMAppTransitions.java             | 199 +++++++++++++++++--
 .../security/TestRMDelegationTokens.java        |   6 +-
 12 files changed, 414 insertions(+), 89 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/06e5a7b5/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 98192ca..09381f9 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
@@ -1459,7 +1459,7 @@ public class RMAppImpl implements RMApp, Recoverable {
       app.rmContext.getSystemMetricsPublisher()
           .appFinished(app, finalState, app.finishTime);
       // set the memory free
-      app.submissionContext.getAMContainerSpec().setTokensConf(null);
+      app.clearUnusedFields();
     };
   }
 
@@ -2021,4 +2021,13 @@ public class RMAppImpl implements RMApp, Recoverable {
   public void setApplicationPriority(Priority applicationPriority) {
     this.applicationPriority = applicationPriority;
   }
+
+  /**
+     * Clear Unused fields to free memory.
+     * @param app
+     */
+  private void clearUnusedFields() {
+    this.submissionContext.setAMContainerSpec(null);
+    this.submissionContext.setLogAggregationContext(null);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06e5a7b5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockMemoryRMStateStore.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/MockMemoryRMStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockMemoryRMStateStore.java
new file mode 100644
index 0000000..698f1c6
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockMemoryRMStateStore.java
@@ -0,0 +1,130 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.server.resourcemanager;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.MemoryRMStateStore;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.ApplicationStateData;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Test helper for MemoryRMStateStore will make sure the event.
+ */
+public class MockMemoryRMStateStore extends MemoryRMStateStore {
+
+  private Map<ApplicationId, ApplicationSubmissionContext> appSubCtxtCopy =
+      new HashMap<ApplicationId, ApplicationSubmissionContext>();
+
+  @SuppressWarnings("rawtypes")
+  @Override
+  protected EventHandler getRMStateStoreEventHandler() {
+    return rmStateStoreEventHandler;
+  }
+
+  @Override
+  public synchronized RMState loadState() throws Exception {
+
+    RMState cloneState = super.loadState();
+
+    for(Entry<ApplicationId, ApplicationStateData> state :
+        cloneState.getApplicationState().entrySet()) {
+      ApplicationStateData oldStateData = state.getValue();
+      oldStateData.setApplicationSubmissionContext(
+          this.appSubCtxtCopy.get(state.getKey()));
+      cloneState.getApplicationState().put(state.getKey(), oldStateData);
+    }
+    return cloneState;
+  }
+
+  @Override
+  public synchronized void storeApplicationStateInternal(
+      ApplicationId appId, ApplicationStateData appState)
+      throws Exception {
+    // Clone Application Submission Context
+    this.cloneAppSubmissionContext(appState);
+    super.storeApplicationStateInternal(appId, appState);
+  }
+
+  @Override
+  public synchronized void updateApplicationStateInternal(
+      ApplicationId appId, ApplicationStateData appState)
+      throws Exception {
+    // Clone Application Submission Context
+    this.cloneAppSubmissionContext(appState);
+    super.updateApplicationStateInternal(appId, appState);
+  }
+
+  /**
+   * Clone Application Submission Context and Store in Map for
+   * later use.
+   *
+   * @param appState
+   */
+  private void cloneAppSubmissionContext(ApplicationStateData appState) {
+    ApplicationSubmissionContext oldAppSubCtxt =
+        appState.getApplicationSubmissionContext();
+    ApplicationSubmissionContext context =
+        ApplicationSubmissionContext.newInstance(
+            oldAppSubCtxt.getApplicationId(),
+            oldAppSubCtxt.getApplicationName(),
+            oldAppSubCtxt.getQueue(),
+            oldAppSubCtxt.getPriority(),
+            oldAppSubCtxt.getAMContainerSpec(),
+            oldAppSubCtxt.getUnmanagedAM(),
+            oldAppSubCtxt.getCancelTokensWhenComplete(),
+            oldAppSubCtxt.getMaxAppAttempts(),
+            oldAppSubCtxt.getResource()
+            );
+    context.setAttemptFailuresValidityInterval(
+        oldAppSubCtxt.getAttemptFailuresValidityInterval());
+    context.setKeepContainersAcrossApplicationAttempts(
+        oldAppSubCtxt.getKeepContainersAcrossApplicationAttempts());
+    context.setAMContainerResourceRequests(
+        oldAppSubCtxt.getAMContainerResourceRequests());
+    context.setLogAggregationContext(oldAppSubCtxt.getLogAggregationContext());
+    context.setApplicationType(oldAppSubCtxt.getApplicationType());
+    this.appSubCtxtCopy.put(oldAppSubCtxt.getApplicationId(), context);
+  }
+
+  /**
+   * Traverse each app state and replace cloned app sub context
+   * into the state.
+   *
+   * @param actualState
+   * @return actualState
+   */
+  @VisibleForTesting
+  public RMState reloadStateWithClonedAppSubCtxt(RMState actualState) {
+    for(Entry<ApplicationId, ApplicationStateData> state :
+        actualState.getApplicationState().entrySet()) {
+      ApplicationStateData oldStateData = state.getValue();
+      oldStateData.setApplicationSubmissionContext(
+          this.appSubCtxtCopy.get(state.getKey()));
+      actualState.getApplicationState().put(state.getKey(),
+          oldStateData);
+    }
+    return actualState;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06e5a7b5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
index b772e80..973f487 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
@@ -164,7 +164,7 @@ public class MockRM extends ResourceManager {
     } else {
       Class storeClass = getRMContext().getStateStore().getClass();
       if (storeClass.equals(MemoryRMStateStore.class)) {
-        MockRMMemoryStateStore mockStateStore = new MockRMMemoryStateStore();
+        MockMemoryRMStateStore mockStateStore = new MockMemoryRMStateStore();
         mockStateStore.init(conf);
         setRMStateStore(mockStateStore);
       } else if (storeClass.equals(NullRMStateStore.class)) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06e5a7b5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRMMemoryStateStore.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/MockRMMemoryStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRMMemoryStateStore.java
deleted file mode 100644
index d88ee1e..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRMMemoryStateStore.java
+++ /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.
- */
-package org.apache.hadoop.yarn.server.resourcemanager;
-
-import org.apache.hadoop.yarn.event.EventHandler;
-import org.apache.hadoop.yarn.server.resourcemanager.recovery.MemoryRMStateStore;
-
-/**
- * Test helper for MemoryRMStateStore will make sure the event.
- */
-public class MockRMMemoryStateStore extends MemoryRMStateStore {
-  @SuppressWarnings("rawtypes")
-  @Override
-  protected EventHandler getRMStateStoreEventHandler() {
-    return rmStateStoreEventHandler;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06e5a7b5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationCleanup.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/TestApplicationCleanup.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationCleanup.java
index ebca7a3..c12ae33 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationCleanup.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationCleanup.java
@@ -293,6 +293,8 @@ public class TestApplicationCleanup {
     // start RM
     MockRM rm1 = new MockRM(conf);
     rm1.start();
+    MockMemoryRMStateStore memStore =
+        (MockMemoryRMStateStore) rm1.getRMStateStore();
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
     nm1.registerNode();
@@ -304,7 +306,7 @@ public class TestApplicationCleanup {
     rm1.waitForState(app0.getApplicationId(), RMAppState.FAILED);
 
     // start new RM
-    MockRM rm2 = new MockRM(conf, rm1.getRMStateStore());
+    MockRM rm2 = new MockRM(conf, memStore);
     rm2.start();
     
     // nm1 register to rm2, and do a heartbeat

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06e5a7b5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestContainerResourceUsage.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/TestContainerResourceUsage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestContainerResourceUsage.java
index 9ed4978..3508ab4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestContainerResourceUsage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestContainerResourceUsage.java
@@ -143,6 +143,8 @@ public class TestContainerResourceUsage {
     conf.set(YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName());
     MockRM rm0 = new MockRM(conf);
     rm0.start();
+    MockMemoryRMStateStore memStore =
+        (MockMemoryRMStateStore) rm0.getRMStateStore();
     MockNM nm =
         new MockNM("127.0.0.1:1234", 65536, rm0.getResourceTrackerService());
     nm.registerNode();
@@ -229,7 +231,7 @@ public class TestContainerResourceUsage {
         vcoreSeconds, metricsBefore.getVcoreSeconds());
 
     // create new RM to represent RM restart. Load up the state store.
-    MockRM rm1 = new MockRM(conf, rm0.getRMStateStore());
+    MockRM rm1 = new MockRM(conf, memStore);
     rm1.start();
     RMApp app0After =
         rm1.getRMContext().getRMApps().get(app0.getApplicationId());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06e5a7b5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHA.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/TestRMHA.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHA.java
index b5293a5..20e9ff4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHA.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHA.java
@@ -415,7 +415,7 @@ public class TestRMHA {
     configuration.setBoolean(YarnConfiguration.AUTO_FAILOVER_ENABLED, false);
     Configuration conf = new YarnConfiguration(configuration);
 
-    MemoryRMStateStore memStore = new MockRMMemoryStateStore() {
+    MemoryRMStateStore memStore = new MockMemoryRMStateStore() {
       int count = 0;
 
       @Override
@@ -465,7 +465,7 @@ public class TestRMHA {
     configuration.setBoolean(YarnConfiguration.AUTO_FAILOVER_ENABLED, false);
     Configuration conf = new YarnConfiguration(configuration);
 
-    MemoryRMStateStore memStore = new MockRMMemoryStateStore() {
+    MemoryRMStateStore memStore = new MockMemoryRMStateStore() {
       @Override
       public void updateApplicationState(ApplicationStateData appState) {
         notifyStoreOperationFailed(new StoreFencedException());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06e5a7b5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.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/TestRMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
index 0346f4f..f0f51f3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
@@ -199,7 +199,8 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
 
     // PHASE 1: create RM and get state
     MockRM rm1 = createMockRM(conf);
-    MemoryRMStateStore memStore = (MemoryRMStateStore) rm1.getRMStateStore();
+    MockMemoryRMStateStore memStore =
+        (MockMemoryRMStateStore) rm1.getRMStateStore();
     Map<ApplicationId, ApplicationStateData> rmAppState =
         memStore.getState().getApplicationState();
 
@@ -679,7 +680,7 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
   @Test (timeout = 60000)
   public void testRMRestartWaitForPreviousSucceededAttempt() throws Exception {
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
-    MemoryRMStateStore memStore = new MockRMMemoryStateStore() {
+    MemoryRMStateStore memStore = new MockMemoryRMStateStore() {
       int count = 0;
 
       @Override
@@ -734,7 +735,8 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1);
     // create RM
     MockRM rm1 = createMockRM(conf);
-    MemoryRMStateStore memStore = (MemoryRMStateStore) rm1.getRMStateStore();
+    MockMemoryRMStateStore memStore =
+        (MockMemoryRMStateStore) rm1.getRMStateStore();
     Map<ApplicationId, ApplicationStateData> rmAppState =
         memStore.getState().getApplicationState();
     // start RM
@@ -780,7 +782,8 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
       YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS);
     // create RM
     MockRM rm1 = createMockRM(conf);
-    MemoryRMStateStore memStore = (MemoryRMStateStore) rm1.getRMStateStore();
+    MockMemoryRMStateStore memStore =
+        (MockMemoryRMStateStore) rm1.getRMStateStore();
     Map<ApplicationId, ApplicationStateData> rmAppState =
         memStore.getState().getApplicationState();
     // start RM
@@ -824,18 +827,18 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
 
   @Test (timeout = 60000)
   public void testRMRestartKilledAppWithNoAttempts() throws Exception {
-    MemoryRMStateStore memStore = new MockRMMemoryStateStore() {
+    MockMemoryRMStateStore memStore = new MockMemoryRMStateStore() {
       @Override
       public synchronized void storeApplicationAttemptStateInternal(
-          ApplicationAttemptId attemptId,
-          ApplicationAttemptStateData attemptStateData) throws Exception {
+          ApplicationAttemptId appAttemptId,
+          ApplicationAttemptStateData attemptState) throws Exception {
         // ignore attempt saving request.
       }
 
       @Override
       public synchronized void updateApplicationAttemptStateInternal(
-          ApplicationAttemptId attemptId,
-          ApplicationAttemptStateData attemptStateData) throws Exception {
+          ApplicationAttemptId appAttemptId,
+          ApplicationAttemptStateData attemptState) throws Exception {
         // ignore attempt saving request.
       }
     };
@@ -868,7 +871,8 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
       YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS);
     // PHASE 1: create RM and get state
     MockRM rm1 = createMockRM(conf);
-    MemoryRMStateStore memStore = (MemoryRMStateStore) rm1.getRMStateStore();
+    MockMemoryRMStateStore memStore =
+        (MockMemoryRMStateStore) rm1.getRMStateStore();
     Map<ApplicationId, ApplicationStateData> rmAppState =
         memStore.getState().getApplicationState();
 
@@ -926,6 +930,9 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
         new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
     nm1.registerNode();
 
+    MockMemoryRMStateStore memStore =
+        (MockMemoryRMStateStore) rm1.getRMStateStore();
+
     // a succeeded app.
     RMApp app0 = rm1.submitApp(200, "name", "user", null,
       false, "default", 1, null, "myType");
@@ -953,7 +960,7 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
     .appCreated(any(RMApp.class), anyLong());
     // restart rm
 
-    MockRM rm2 = new MockRM(conf, rm1.getRMStateStore()) {
+    MockRM rm2 = new MockRM(conf, memStore) {
       @Override
       protected RMAppManager createRMAppManager() {
         return spy(super.createRMAppManager());
@@ -1625,7 +1632,8 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
     // start RM
     MockRM rm1 = createMockRM(conf);
     rm1.start();
-    MemoryRMStateStore memStore = (MemoryRMStateStore) rm1.getRMStateStore();
+    MockMemoryRMStateStore memStore =
+        (MockMemoryRMStateStore) rm1.getRMStateStore();
     RMState rmState = memStore.getState();
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
@@ -1664,7 +1672,7 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
   // This is to test RM does not get hang on shutdown.
   @Test (timeout = 10000)
   public void testRMShutdown() throws Exception {
-    MemoryRMStateStore memStore = new MockRMMemoryStateStore() {
+    MemoryRMStateStore memStore = new MockMemoryRMStateStore() {
       @Override
       public synchronized void checkVersion()
           throws Exception {
@@ -1743,7 +1751,8 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
       }
     };
     rm1.start();
-    MemoryRMStateStore memStore = (MemoryRMStateStore) rm1.getRMStateStore();
+    MockMemoryRMStateStore memStore =
+        (MockMemoryRMStateStore) rm1.getRMStateStore();
     RMApp app1 = null;
     try {
        app1 = rm1.submitApp(200, "name", "user",
@@ -1767,7 +1776,7 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
 
   @Test (timeout = 20000)
   public void testAppRecoveredInOrderOnRMRestart() throws Exception {
-    MemoryRMStateStore memStore = new MockRMMemoryStateStore();
+    MemoryRMStateStore memStore = new MemoryRMStateStore();
     memStore.init(conf);
 
     for (int i = 10; i > 0; i--) {
@@ -2405,6 +2414,8 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
     MockRM rm1 = new MockRM(conf);
     rm1.start();
     CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
+    MockMemoryRMStateStore memStore =
+        (MockMemoryRMStateStore) rm1.getRMStateStore();
 
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
@@ -2441,7 +2452,7 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
     MockRM rm2 = null;
     // start RM2
     try {
-      rm2 = new MockRM(conf, rm1.getRMStateStore());
+      rm2 = new MockRM(conf, memStore);
       rm2.start();
       Assert.assertTrue("RM start successfully", true);
     } catch (Exception e) {
@@ -2542,6 +2553,8 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
       }
     };
     rm1.start();
+    MockMemoryRMStateStore memStore =
+        (MockMemoryRMStateStore) rm1.getRMStateStore();
 
     // add node label "x" and set node to label mapping
     Set<String> clusterNodeLabels = new HashSet<String>();
@@ -2568,7 +2581,7 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
     conf.setBoolean(YarnConfiguration.NODE_LABELS_ENABLED, false);
     MockRM rm2 = new MockRM(
         TestUtils.getConfigurationWithDefaultQueueLabels(conf),
-        rm1.getRMStateStore()) {
+        memStore) {
       @Override
       protected RMNodeLabelsManager createNodeLabelManager() {
         RMNodeLabelsManager mgr = new RMNodeLabelsManager();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06e5a7b5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.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/TestWorkPreservingRMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java
index a13cae7..c1cb4c1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java
@@ -572,6 +572,8 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     conf.set(YarnConfiguration.YARN_ADMIN_ACL, "");
     rm1 = new MockRM(conf);
     rm1.start();
+    MockMemoryRMStateStore memStore =
+        (MockMemoryRMStateStore) rm1.getRMStateStore();
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 8192, rm1.getResourceTrackerService());
     nm1.registerNode();
@@ -583,7 +585,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[]{QUEUE_DOESNT_EXIST});
     final String noQueue = CapacitySchedulerConfiguration.ROOT + "." + QUEUE_DOESNT_EXIST;
     csConf.setCapacity(noQueue, 100);
-    rm2 = new MockRM(csConf, rm1.getRMStateStore());
+    rm2 = new MockRM(csConf, memStore);
 
     rm2.start();
     UserGroupInformation user2 = UserGroupInformation.createRemoteUser("user2");
@@ -721,11 +723,15 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
 
   private void verifyAppRecoveryWithWrongQueueConfig(
       CapacitySchedulerConfiguration csConf, RMApp app, String diagnostics,
-      MemoryRMStateStore memStore, RMState state) throws Exception {
+      MockMemoryRMStateStore memStore, RMState state) throws Exception {
     // Restart RM with fail-fast as false. App should be killed.
     csConf.setBoolean(YarnConfiguration.RM_FAIL_FAST, false);
     rm2 = new MockRM(csConf, memStore);
     rm2.start();
+
+    MockMemoryRMStateStore memStore2 =
+        (MockMemoryRMStateStore) rm2.getRMStateStore();
+
     // Wait for app to be killed.
     rm2.waitForState(app.getApplicationId(), RMAppState.KILLED);
     ApplicationReport report = rm2.getApplicationReport(app.getApplicationId());
@@ -734,24 +740,27 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     assertEquals(report.getYarnApplicationState(), YarnApplicationState.KILLED);
     assertEquals(report.getDiagnostics(), diagnostics);
 
+    //Reload previous state with cloned app sub context object
+    RMState newState = memStore2.reloadStateWithClonedAppSubCtxt(state);
+
     // Remove updated app info(app being KILLED) from state store and reinstate
     // state store to previous state i.e. which indicates app is RUNNING.
     // This is to simulate app recovery with fail fast config as true.
     for(Map.Entry<ApplicationId, ApplicationStateData> entry :
-        state.getApplicationState().entrySet()) {
+        newState.getApplicationState().entrySet()) {
       ApplicationStateData appState = mock(ApplicationStateData.class);
       ApplicationSubmissionContext ctxt =
           mock(ApplicationSubmissionContext.class);
       when(appState.getApplicationSubmissionContext()).thenReturn(ctxt);
       when(ctxt.getApplicationId()).thenReturn(entry.getKey());
-      memStore.removeApplicationStateInternal(appState);
-      memStore.storeApplicationStateInternal(
+      memStore2.removeApplicationStateInternal(appState);
+      memStore2.storeApplicationStateInternal(
           entry.getKey(), entry.getValue());
     }
 
     // Now restart RM with fail-fast as true. QueueException should be thrown.
     csConf.setBoolean(YarnConfiguration.RM_FAIL_FAST, true);
-    MockRM rm = new MockRM(csConf, memStore);
+    MockRM rm = new MockRM(csConf, memStore2);
     try {
       rm.start();
       Assert.fail("QueueException must have been thrown");
@@ -781,6 +790,9 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     setupQueueConfiguration(csConf);
     rm1 = new MockRM(csConf);
     rm1.start();
+
+    MockMemoryRMStateStore memStore =
+        (MockMemoryRMStateStore) rm1.getRMStateStore();
     MockNM nm =
         new MockNM("127.1.1.1:4321", 8192, rm1.getResourceTrackerService());
     nm.registerNode();
@@ -801,7 +813,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     String diags = "Application killed on recovery as it was submitted to " +
         "queue QueueB which is no longer a leaf queue after restart.";
     verifyAppRecoveryWithWrongQueueConfig(csConf, app, diags,
-        (MemoryRMStateStore) rm1.getRMStateStore(), state);
+        memStore, state);
   }
 
   //Test behavior of an app if queue is removed during recovery. Test case does
@@ -826,6 +838,8 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     setupQueueConfiguration(csConf);
     rm1 = new MockRM(csConf);
     rm1.start();
+    MockMemoryRMStateStore memStore =
+        (MockMemoryRMStateStore) rm1.getRMStateStore();
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 8192, rm1.getResourceTrackerService());
     MockNM nm2 =
@@ -853,7 +867,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     rm1.clearQueueMetrics(app2);
 
     // Take a copy of state store so that it can be reset to this state.
-    RMState state = rm1.getRMStateStore().loadState();
+    RMState state = memStore.loadState();
 
     // Set new configuration with QueueB removed.
     csConf = new CapacitySchedulerConfiguration(conf);
@@ -862,7 +876,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     String diags = "Application killed on recovery as it was submitted to " +
         "queue QueueB which no longer exists after restart.";
     verifyAppRecoveryWithWrongQueueConfig(csConf, app2, diags,
-        (MemoryRMStateStore) rm1.getRMStateStore(), state);
+        memStore, state);
   }
 
   private void checkParentQueue(ParentQueue parentQueue, int numContainers,
@@ -931,6 +945,8 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
   public void testContainersNotRecoveredForCompletedApps() throws Exception {
     rm1 = new MockRM(conf);
     rm1.start();
+    MockMemoryRMStateStore memStore =
+        (MockMemoryRMStateStore) rm1.getRMStateStore();
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 8192, rm1.getResourceTrackerService());
     nm1.registerNode();
@@ -938,7 +954,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
     MockRM.finishAMAndVerifyAppState(app1, rm1, nm1, am1);
 
-    rm2 = new MockRM(conf, rm1.getRMStateStore());
+    rm2 = new MockRM(conf, memStore);
     rm2.start();
     nm1.setResourceTrackerService(rm2.getResourceTrackerService());
     NMContainerStatus runningContainer =
@@ -1212,6 +1228,9 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     // start RM
     rm1 = new MockRM(conf);
     rm1.start();
+
+    MockMemoryRMStateStore memStore =
+        (MockMemoryRMStateStore) rm1.getRMStateStore();
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
     nm1.registerNode();
@@ -1230,7 +1249,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
 
 
     // start new RM
-    rm2 = new MockRM(conf, rm1.getRMStateStore());
+    rm2 = new MockRM(conf, memStore);
     rm2.start();
 
     am0.setAMRMProtocol(rm2.getApplicationMasterService(), rm2.getRMContext());
@@ -1370,7 +1389,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
   // RM should start correctly.
   @Test (timeout = 20000)
   public void testAppStateSavedButAttemptStateNotSaved() throws Exception {
-    MemoryRMStateStore memStore = new MemoryRMStateStore() {
+    MockMemoryRMStateStore memStore = new MockMemoryRMStateStore() {
       @Override public synchronized void updateApplicationAttemptStateInternal(
           ApplicationAttemptId appAttemptId,
           ApplicationAttemptStateData attemptState) {
@@ -1414,6 +1433,8 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     // start RM
     rm1 = new MockRM(conf);
     rm1.start();
+    MockMemoryRMStateStore memStore =
+        (MockMemoryRMStateStore) rm1.getRMStateStore();
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
     nm1.registerNode();
@@ -1438,8 +1459,10 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     }
 
     // start new RM
-    rm2 = new MockRM(conf, rm1.getRMStateStore());
+    rm2 = new MockRM(conf, memStore);
     rm2.start();
+    MockMemoryRMStateStore memStore2 =
+        (MockMemoryRMStateStore) rm2.getRMStateStore();
     rm2.waitForState(app0.getApplicationId(), RMAppState.ACCEPTED);
     rm2.waitForState(am0.getApplicationAttemptId(), RMAppAttemptState.LAUNCHED);
 
@@ -1488,7 +1511,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
         recoveredApp.getFinalApplicationStatus());
 
     // Restart RM once more to check UAM is not re-run
-    MockRM rm3 = new MockRM(conf, rm1.getRMStateStore());
+    MockRM rm3 = new MockRM(conf, memStore2);
     rm3.start();
     recoveredApp = rm3.getRMContext().getRMApps().get(app0.getApplicationId());
     Assert.assertEquals(RMAppState.FINISHED, recoveredApp.getState());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06e5a7b5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.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/applicationsmanager/TestAMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java
index 528afac..9d0d879 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus;
 import org.apache.hadoop.yarn.server.resourcemanager.MockAM;
 import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
+import org.apache.hadoop.yarn.server.resourcemanager.MockMemoryRMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.MemoryRMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.ApplicationStateData;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
@@ -702,8 +703,11 @@ public class TestAMRestart {
     // explicitly set max-am-retry count as 2.
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
     MockRM rm1 = new MockRM(conf);
-    MemoryRMStateStore memStore = (MemoryRMStateStore) rm1.getRMStateStore();
     rm1.start();
+
+    MockMemoryRMStateStore memStore =
+        (MockMemoryRMStateStore) rm1.getRMStateStore();
+
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 8000, rm1.getResourceTrackerService());
     nm1.registerNode();
@@ -735,7 +739,6 @@ public class TestAMRestart {
     RMAppImpl app1 = (RMAppImpl)rm1.submitApp(200, 10000, false);
     app1.setSystemClock(clock);
     MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
-
     // Fail attempt1 normally
     nm1.nodeHeartbeat(am1.getApplicationAttemptId(),
       1, ContainerState.COMPLETE);
@@ -771,8 +774,12 @@ public class TestAMRestart {
     @SuppressWarnings("resource")
     MockRM rm2 = new MockRM(conf, memStore);
     rm2.start();
+
+    MockMemoryRMStateStore memStore1 =
+        (MockMemoryRMStateStore) rm2.getRMStateStore();
     ApplicationStateData app1State =
-        memStore.getState().getApplicationState().get(app1.getApplicationId());
+        memStore1.getState().getApplicationState().
+        get(app1.getApplicationId());
     Assert.assertEquals(1, app1State.getFirstAttemptId());
 
     // re-register the NM

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06e5a7b5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java
index 7c54b60..344f8bb7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.rmapp;
 
+import static org.junit.Assert.assertEquals;
 import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anyLong;
 import static org.mockito.Matchers.eq;
@@ -28,17 +29,23 @@ import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 
+import java.io.File;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Map;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.UnsupportedFileSystemException;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.security.Credentials;
@@ -46,14 +53,22 @@ import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
 import org.apache.hadoop.yarn.MockApps;
+import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.hadoop.yarn.api.records.ContainerRetryContext;
+import org.apache.hadoop.yarn.api.records.ContainerRetryPolicy;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.api.records.LocalResourceType;
+import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
+import org.apache.hadoop.yarn.api.records.LogAggregationContext;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.api.records.URL;
 import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationSubmissionContextPBImpl;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.DrainDispatcher;
@@ -251,7 +266,113 @@ public class TestRMAppTransitions {
     rmDispatcher.start();
   }
 
-  protected RMApp createNewTestApp(ApplicationSubmissionContext submissionContext) {
+  private ByteBuffer getTokens() throws IOException {
+    Credentials ts = new Credentials();
+    DataOutputBuffer dob = new DataOutputBuffer();
+    ts.writeTokenStorageToStream(dob);
+    ByteBuffer securityTokens =
+        ByteBuffer.wrap(dob.getData(), 0, dob.getLength());
+    return securityTokens;
+  }
+
+  private ByteBuffer getTokensConf() throws IOException {
+
+    DataOutputBuffer dob = new DataOutputBuffer();
+    Configuration appConf = new Configuration(false);
+    appConf.clear();
+    appConf.set("dfs.nameservices", "mycluster1,mycluster2");
+    appConf.set("dfs.namenode.rpc-address.mycluster2.nn1",
+        "123.0.0.1");
+    appConf.set("dfs.namenode.rpc-address.mycluster3.nn2",
+        "123.0.0.2");
+    appConf.write(dob);
+    ByteBuffer tokenConf =
+        ByteBuffer.wrap(dob.getData(), 0, dob.getLength());
+    return tokenConf;
+  }
+
+  private Map<String, LocalResource> getLocalResources()
+      throws UnsupportedFileSystemException {
+    FileContext localFS = FileContext.getLocalFSFileContext();
+    File tmpDir = new File("target");
+    File scriptFile = new File(tmpDir, "scriptFile.sh");
+    URL resourceURL =
+        URL.fromPath(localFS
+            .makeQualified(new Path(scriptFile.getAbsolutePath())));
+    LocalResource localRes =
+        Records.newRecord(LocalResource.class);
+    localRes.setResource(resourceURL);
+    localRes.setSize(-1);
+    localRes.setVisibility(LocalResourceVisibility.APPLICATION);
+    localRes.setType(LocalResourceType.FILE);
+    localRes.setTimestamp(scriptFile.lastModified());
+    String destinationFile = "dest_file";
+    Map<String, LocalResource> localResources =
+        new HashMap<String, LocalResource>();
+    localResources.put(destinationFile, localRes);
+    return localResources;
+  }
+
+  private Map<String, String> getEnvironment() {
+    Map<String, String> userSetEnv = new HashMap<String, String>();
+    userSetEnv.put(Environment.CONTAINER_ID.name(), "user_set_container_id");
+    userSetEnv.put(Environment.NM_HOST.name(), "user_set_NM_HOST");
+    userSetEnv.put(Environment.NM_PORT.name(), "user_set_NM_PORT");
+    userSetEnv.put(Environment.NM_HTTP_PORT.name(), "user_set_NM_HTTP_PORT");
+    userSetEnv.put(Environment.LOCAL_DIRS.name(), "user_set_LOCAL_DIR");
+    userSetEnv.put(Environment.USER.key(), "user_set_" +
+        Environment.USER.key());
+    userSetEnv.put(Environment.LOGNAME.name(), "user_set_LOGNAME");
+    userSetEnv.put(Environment.PWD.name(), "user_set_PWD");
+    userSetEnv.put(Environment.HOME.name(), "user_set_HOME");
+    return userSetEnv;
+  }
+
+  private ContainerRetryContext getContainerRetryContext() {
+    ContainerRetryContext containerRetryContext = ContainerRetryContext
+        .newInstance(
+            ContainerRetryPolicy.RETRY_ON_SPECIFIC_ERROR_CODES,
+            new HashSet<>(Arrays.asList(Integer.valueOf(111))), 0, 0);
+    return containerRetryContext;
+  }
+
+  private Map<String, ByteBuffer> getServiceData() {
+    Map<String, ByteBuffer> serviceData = new HashMap<String, ByteBuffer>();
+    String serviceName = "non_exist_auxService";
+    serviceData.put(serviceName, ByteBuffer.wrap(serviceName.getBytes()));
+    return serviceData;
+  }
+
+  private ContainerLaunchContext prepareContainerLaunchContext()
+      throws IOException {
+
+    ContainerLaunchContext clc =
+        Records.newRecord(ContainerLaunchContext.class);
+    clc.setCommands(Arrays.asList("/bin/sleep 5"));
+    if (UserGroupInformation.isSecurityEnabled()) {
+      clc.setTokens(getTokens());
+      clc.setTokensConf(getTokensConf());
+    }
+    clc.setLocalResources(getLocalResources());
+    clc.setEnvironment(getEnvironment());
+    clc.setContainerRetryContext(getContainerRetryContext());
+    clc.setServiceData(getServiceData());
+    return clc;
+  }
+
+  private LogAggregationContext getLogAggregationContext() {
+    LogAggregationContext logAggregationContext =
+        LogAggregationContext.newInstance(
+          "includePattern", "excludePattern",
+          "rolledLogsIncludePattern",
+          "rolledLogsExcludePattern",
+          "policyClass",
+          "policyParameters");
+    return logAggregationContext;
+  }
+
+  protected RMApp createNewTestApp(ApplicationSubmissionContext
+      submissionContext) throws IOException {
     ApplicationId applicationId = MockApps.newAppID(appId++);
     String user = MockApps.newUserName();
     String name = MockApps.newAppName();
@@ -270,7 +391,9 @@ public class TestRMAppTransitions {
     // but applicationId is still set for safety
     submissionContext.setApplicationId(applicationId);
     submissionContext.setPriority(Priority.newInstance(0));
-    submissionContext.setAMContainerSpec(mock(ContainerLaunchContext.class));
+    submissionContext.setAMContainerSpec(prepareContainerLaunchContext());
+    submissionContext.setLogAggregationContext(getLogAggregationContext());
+
     RMApp application = new RMAppImpl(applicationId, rmContext, conf, name,
         user, queue, submissionContext, scheduler, masterService,
         System.currentTimeMillis(), "YARN", null,
@@ -405,6 +528,7 @@ public class TestRMAppTransitions {
     // verify sendATSCreateEvent() is get called during
     // AddApplicationToSchedulerTransition.
     verify(publisher).appCreated(eq(application), anyLong());
+    verifyRMAppFieldsForNonFinalTransitions(application);
     return application;
   }
 
@@ -422,6 +546,7 @@ public class TestRMAppTransitions {
     application.handle(event);
     assertStartTimeSet(application);
     assertAppState(RMAppState.SUBMITTED, application);
+    verifyRMAppFieldsForNonFinalTransitions(application);
     return application;
   }
 
@@ -530,6 +655,7 @@ public class TestRMAppTransitions {
     assertFailed(application,
         ".*Unmanaged application.*Failing the application.*");
     assertAppFinalStateSaved(application);
+    verifyRMAppFieldsForFinalTransitions(application);
   }
   
   @Test
@@ -539,6 +665,7 @@ public class TestRMAppTransitions {
     RMApp application = testCreateAppFinished(null, diagMsg);
     Assert.assertTrue("Finished application missing diagnostics",
         application.getDiagnostics().indexOf(diagMsg) != -1);
+    verifyRMAppFieldsForFinalTransitions(application);
   }
 
   @Test (timeout = 30000)
@@ -546,15 +673,7 @@ public class TestRMAppTransitions {
     LOG.info("--- START: testAppRecoverPath ---");
     ApplicationSubmissionContext sub =
         Records.newRecord(ApplicationSubmissionContext.class);
-    ContainerLaunchContext clc =
-        Records.newRecord(ContainerLaunchContext.class);
-    Credentials credentials = new Credentials();
-    DataOutputBuffer dob = new DataOutputBuffer();
-    credentials.writeTokenStorageToStream(dob);
-    ByteBuffer securityTokens =
-        ByteBuffer.wrap(dob.getData(), 0, dob.getLength());
-    clc.setTokens(securityTokens);
-    sub.setAMContainerSpec(clc);
+    sub.setAMContainerSpec(prepareContainerLaunchContext());
     testCreateAppSubmittedRecovery(sub);
   }
 
@@ -577,6 +696,7 @@ public class TestRMAppTransitions {
     assertAppFinalStateNotSaved(application);
     verifyApplicationFinished(RMAppState.KILLED);
     verifyAppRemovedSchedulerEvent(RMAppState.KILLED);
+    verifyRMAppFieldsForFinalTransitions(application);
   }
 
   @Test
@@ -594,6 +714,7 @@ public class TestRMAppTransitions {
     assertFailed(application, rejectedText);
     assertAppFinalStateSaved(application);
     verifyApplicationFinished(RMAppState.FAILED);
+    verifyRMAppFieldsForFinalTransitions(application);
   }
 
   @Test (timeout = 30000)
@@ -611,6 +732,7 @@ public class TestRMAppTransitions {
     assertFailed(application, rejectedText);
     assertAppFinalStateSaved(application);
     verifyApplicationFinished(RMAppState.FAILED);
+    verifyRMAppFieldsForFinalTransitions(application);
     rmContext.getStateStore().removeApplication(application);
   }
 
@@ -633,6 +755,7 @@ public class TestRMAppTransitions {
     assertKilled(application);
     verifyApplicationFinished(RMAppState.KILLED);
     verifyAppRemovedSchedulerEvent(RMAppState.KILLED);
+    verifyRMAppFieldsForFinalTransitions(application);
   }
 
   @Test (timeout = 30000)
@@ -650,6 +773,7 @@ public class TestRMAppTransitions {
     assertFailed(application, rejectedText);
     assertAppFinalStateSaved(application);
     verifyApplicationFinished(RMAppState.FAILED);
+    verifyRMAppFieldsForFinalTransitions(application);
   }
 
   @Test (timeout = 30000)
@@ -684,6 +808,7 @@ public class TestRMAppTransitions {
     assertFailed(application, rejectedText);
     assertAppFinalStateSaved(application);
     verifyApplicationFinished(RMAppState.FAILED);
+    verifyRMAppFieldsForFinalTransitions(application);
   }
 
   @Test
@@ -706,6 +831,7 @@ public class TestRMAppTransitions {
     assertAppFinalStateSaved(application);
     verifyApplicationFinished(RMAppState.KILLED);
     verifyAppRemovedSchedulerEvent(RMAppState.KILLED);
+    verifyRMAppFieldsForFinalTransitions(application);
   }
 
   @Test
@@ -769,8 +895,9 @@ public class TestRMAppTransitions {
     assertAppFinalStateSaved(application);
     verifyApplicationFinished(RMAppState.KILLED);
     verifyAppRemovedSchedulerEvent(RMAppState.KILLED);
+    verifyRMAppFieldsForFinalTransitions(application);
   }
-  
+
   @Test
   public void testAppAcceptedAttemptKilled() throws IOException,
       InterruptedException {
@@ -816,6 +943,7 @@ public class TestRMAppTransitions {
     assertKilled(application);
     verifyApplicationFinished(RMAppState.KILLED);
     verifyAppRemovedSchedulerEvent(RMAppState.KILLED);
+    verifyRMAppFieldsForFinalTransitions(application);
   }
 
   @Test
@@ -873,6 +1001,7 @@ public class TestRMAppTransitions {
     assertFailed(application, ".*Failing the application.*");
     assertAppFinalStateSaved(application);
     verifyApplicationFinished(RMAppState.FAILED);
+    verifyRMAppFieldsForFinalTransitions(application);
   }
 
   @Test
@@ -914,6 +1043,7 @@ public class TestRMAppTransitions {
     assertFinalAppStatus(FinalApplicationStatus.FAILED, application);
     Assert.assertTrue("Finished app missing diagnostics", application
       .getDiagnostics().indexOf(diagMsg) != -1);
+    verifyRMAppFieldsForFinalTransitions(application);
   }
 
   @Test
@@ -933,6 +1063,7 @@ public class TestRMAppTransitions {
     Assert.assertEquals("application diagnostics is not correct",
         "", diag.toString());
     verifyApplicationFinished(RMAppState.FINISHED);
+    verifyRMAppFieldsForFinalTransitions(application);
   }
 
   @Test (timeout = 30000)
@@ -962,6 +1093,7 @@ public class TestRMAppTransitions {
 
     assertTimesAtFinish(application);
     assertAppState(RMAppState.FAILED, application);
+    verifyRMAppFieldsForFinalTransitions(application);
   }
 
   @Test (timeout = 30000)
@@ -1016,6 +1148,7 @@ public class TestRMAppTransitions {
 
     assertTimesAtFinish(application);
     assertAppState(RMAppState.KILLED, application);
+    verifyRMAppFieldsForFinalTransitions(application);
   }
   
   @Test(timeout = 30000)
@@ -1061,11 +1194,12 @@ public class TestRMAppTransitions {
     RMAppState finalState = appState.getState();
     Assert.assertEquals("Application is not in finalState.", finalState,
         application.getState());
+    verifyRMAppFieldsForFinalTransitions(application);
   }
   
   public void createRMStateForApplications(
       Map<ApplicationId, ApplicationStateData> applicationState,
-      RMAppState rmAppState) {
+      RMAppState rmAppState) throws IOException {
     RMApp app = createNewTestApp(null);
     ApplicationStateData appState =
         ApplicationStateData.newInstance(app.getSubmitTime(), app.getStartTime(),
@@ -1075,7 +1209,7 @@ public class TestRMAppTransitions {
   }
   
   @Test
-  public void testGetAppReport() {
+  public void testGetAppReport() throws IOException {
     RMApp app = createNewTestApp(null);
     assertAppState(RMAppState.NEW, app);
     ApplicationReport report = app.createAndGetApplicationReport(null, true);
@@ -1109,4 +1243,41 @@ public class TestRMAppTransitions {
       Assert.assertEquals(finalState, appRemovedEvent.getFinalState());
     }
   }
+
+  private void verifyRMAppFieldsForNonFinalTransitions(RMApp application)
+      throws IOException {
+    assertEquals(Arrays.asList("/bin/sleep 5"),
+        application.getApplicationSubmissionContext().
+        getAMContainerSpec().getCommands());
+    assertEquals(getLocalResources(),
+        application.getApplicationSubmissionContext().
+        getAMContainerSpec().getLocalResources());
+    if(UserGroupInformation.isSecurityEnabled()) {
+      assertEquals(getTokens(),
+          application.getApplicationSubmissionContext().
+          getAMContainerSpec().getTokens());
+      assertEquals(getTokensConf(),
+          application.getApplicationSubmissionContext().
+          getAMContainerSpec().getTokensConf());
+    }
+    assertEquals(getEnvironment(),
+        application.getApplicationSubmissionContext().
+        getAMContainerSpec().getEnvironment());
+    assertEquals(getContainerRetryContext(),
+        application.getApplicationSubmissionContext().
+        getAMContainerSpec().getContainerRetryContext());
+    assertEquals(getServiceData(),
+        application.getApplicationSubmissionContext().
+        getAMContainerSpec().getServiceData());
+    assertEquals(getLogAggregationContext(),
+        application.getApplicationSubmissionContext().
+        getLogAggregationContext());
+  }
+
+  private void verifyRMAppFieldsForFinalTransitions(RMApp application) {
+    assertEquals(null, application.getApplicationSubmissionContext().
+        getAMContainerSpec());
+    assertEquals(null, application.getApplicationSubmissionContext().
+        getLogAggregationContext());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/06e5a7b5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestRMDelegationTokens.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestRMDelegationTokens.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestRMDelegationTokens.java
index 640293c..2c52377 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestRMDelegationTokens.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestRMDelegationTokens.java
@@ -38,7 +38,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenResponse;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
-import org.apache.hadoop.yarn.server.resourcemanager.MockRMMemoryStateStore;
+import org.apache.hadoop.yarn.server.resourcemanager.MockMemoryRMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.RMSecretManagerService;
 import org.apache.hadoop.yarn.server.resourcemanager.TestRMRestart.TestSecurityMockRM;
@@ -78,7 +78,7 @@ public class TestRMDelegationTokens {
     UserGroupInformation.getLoginUser()
         .setAuthenticationMethod(AuthenticationMethod.KERBEROS);
 
-    MemoryRMStateStore memStore = new MockRMMemoryStateStore();
+    MemoryRMStateStore memStore = new MockMemoryRMStateStore();
     memStore.init(conf);
     RMState rmState = memStore.getState();
 
@@ -132,7 +132,7 @@ public class TestRMDelegationTokens {
   // Test all expired keys are removed from state-store.
   @Test(timeout = 15000)
   public void testRemoveExpiredMasterKeyInRMStateStore() throws Exception {
-    MemoryRMStateStore memStore = new MockRMMemoryStateStore();
+    MemoryRMStateStore memStore = new MockMemoryRMStateStore();
     memStore.init(testConf);
     RMState rmState = memStore.getState();
 


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