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/21 01:07:04 UTC

[01/50] [abbrv] hadoop git commit: HDFS-12323. NameNode terminates after full GC thinking QJM unresponsive if full GC is much longer than timeout. Contributed by Erik Krogen. [Forced Update!]

Repository: hadoop
Updated Branches:
  refs/heads/YARN-5734 9726e1fc2 -> 034e6f4f8 (forced update)


HDFS-12323. NameNode terminates after full GC thinking QJM unresponsive if full GC is much longer than timeout. Contributed by Erik Krogen.

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

Branch: refs/heads/YARN-5734
Commit: 90894c7262df0243e795b675f3ac9f7b322ccd11
Parents: b9b607d
Author: Erik Krogen <ek...@linkedin.com>
Authored: Thu Sep 14 15:53:33 2017 -0700
Committer: Konstantin V Shvachko <sh...@apache.org>
Committed: Fri Sep 15 13:56:27 2017 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/qjournal/client/QuorumCall.java | 65 ++++++++++++++++----
 .../hdfs/qjournal/client/TestQuorumCall.java    | 31 +++++++++-
 2 files changed, 82 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/90894c72/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumCall.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumCall.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumCall.java
index dc32318..dee74e6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumCall.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumCall.java
@@ -24,7 +24,7 @@ import java.util.concurrent.TimeUnit;
 
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.util.StopWatch;
-import org.apache.hadoop.util.Time;
+import org.apache.hadoop.util.Timer;
 
 import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
@@ -35,6 +35,7 @@ import com.google.common.util.concurrent.ListenableFuture;
 import com.google.protobuf.Message;
 import com.google.protobuf.TextFormat;
 
+
 /**
  * Represents a set of calls for which a quorum of results is needed.
  * @param <KEY> a key used to identify each of the outgoing calls
@@ -60,11 +61,12 @@ class QuorumCall<KEY, RESULT> {
    * fraction of the configured timeout for any call.
    */
   private static final float WAIT_PROGRESS_WARN_THRESHOLD = 0.7f;
-  private final StopWatch quorumStopWatch = new StopWatch();
+  private final StopWatch quorumStopWatch;
+  private final Timer timer;
   
   static <KEY, RESULT> QuorumCall<KEY, RESULT> create(
-      Map<KEY, ? extends ListenableFuture<RESULT>> calls) {
-    final QuorumCall<KEY, RESULT> qr = new QuorumCall<KEY, RESULT>();
+      Map<KEY, ? extends ListenableFuture<RESULT>> calls, Timer timer) {
+    final QuorumCall<KEY, RESULT> qr = new QuorumCall<KEY, RESULT>(timer);
     for (final Entry<KEY, ? extends ListenableFuture<RESULT>> e : calls.entrySet()) {
       Preconditions.checkArgument(e.getValue() != null,
           "null future for key: " + e.getKey());
@@ -82,18 +84,53 @@ class QuorumCall<KEY, RESULT> {
     }
     return qr;
   }
-  
+
+  static <KEY, RESULT> QuorumCall<KEY, RESULT> create(
+      Map<KEY, ? extends ListenableFuture<RESULT>> calls) {
+    return create(calls, new Timer());
+  }
+
+  /**
+   * Not intended for outside use.
+   */
   private QuorumCall() {
+    this(new Timer());
+  }
+
+  private QuorumCall(Timer timer) {
     // Only instantiated from factory method above
+    this.timer = timer;
+    this.quorumStopWatch = new StopWatch(timer);
   }
 
+  /**
+   * Used in conjunction with {@link #getQuorumTimeoutIncreaseMillis(long, int)}
+   * to check for pauses.
+   */
   private void restartQuorumStopWatch() {
     quorumStopWatch.reset().start();
   }
 
-  private boolean shouldIncreaseQuorumTimeout(long offset, int millis) {
+  /**
+   * Check for a pause (e.g. GC) since the last time
+   * {@link #restartQuorumStopWatch()} was called. If detected, return the
+   * length of the pause; else, -1.
+   * @param offset Offset the elapsed time by this amount; use if some amount
+   *               of pause was expected
+   * @param millis Total length of timeout in milliseconds
+   * @return Length of pause, if detected, else -1
+   */
+  private long getQuorumTimeoutIncreaseMillis(long offset, int millis) {
     long elapsed = quorumStopWatch.now(TimeUnit.MILLISECONDS);
-    return elapsed + offset > (millis * WAIT_PROGRESS_INFO_THRESHOLD);
+    long pauseTime = elapsed + offset;
+    if (pauseTime > (millis * WAIT_PROGRESS_INFO_THRESHOLD)) {
+      QuorumJournalManager.LOG.info("Pause detected while waiting for " +
+          "QuorumCall response; increasing timeout threshold by pause time " +
+          "of " + pauseTime + " ms.");
+      return pauseTime;
+    } else {
+      return -1;
+    }
   }
 
   
@@ -119,7 +156,7 @@ class QuorumCall<KEY, RESULT> {
       int minResponses, int minSuccesses, int maxExceptions,
       int millis, String operationName)
       throws InterruptedException, TimeoutException {
-    long st = Time.monotonicNow();
+    long st = timer.monotonicNow();
     long nextLogTime = st + (long)(millis * WAIT_PROGRESS_INFO_THRESHOLD);
     long et = st + millis;
     while (true) {
@@ -128,7 +165,7 @@ class QuorumCall<KEY, RESULT> {
       if (minResponses > 0 && countResponses() >= minResponses) return;
       if (minSuccesses > 0 && countSuccesses() >= minSuccesses) return;
       if (maxExceptions >= 0 && countExceptions() > maxExceptions) return;
-      long now = Time.monotonicNow();
+      long now = timer.monotonicNow();
       
       if (now > nextLogTime) {
         long waited = now - st;
@@ -154,8 +191,9 @@ class QuorumCall<KEY, RESULT> {
       long rem = et - now;
       if (rem <= 0) {
         // Increase timeout if a full GC occurred after restarting stopWatch
-        if (shouldIncreaseQuorumTimeout(0, millis)) {
-          et = et + millis;
+        long timeoutIncrease = getQuorumTimeoutIncreaseMillis(0, millis);
+        if (timeoutIncrease > 0) {
+          et += timeoutIncrease;
         } else {
           throw new TimeoutException();
         }
@@ -165,8 +203,9 @@ class QuorumCall<KEY, RESULT> {
       rem = Math.max(rem, 1);
       wait(rem);
       // Increase timeout if a full GC occurred after restarting stopWatch
-      if (shouldIncreaseQuorumTimeout(-rem, millis)) {
-        et = et + millis;
+      long timeoutIncrease = getQuorumTimeoutIncreaseMillis(-rem, millis);
+      if (timeoutIncrease > 0) {
+        et += timeoutIncrease;
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/90894c72/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumCall.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumCall.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumCall.java
index 506497e..97cf2f3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumCall.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQuorumCall.java
@@ -23,7 +23,7 @@ import java.util.Map;
 import java.util.TreeMap;
 import java.util.concurrent.TimeoutException;
 
-import org.apache.hadoop.hdfs.qjournal.client.QuorumCall;
+import org.apache.hadoop.util.FakeTimer;
 import org.junit.Test;
 
 import com.google.common.base.Joiner;
@@ -83,4 +83,33 @@ public class TestQuorumCall {
     }
   }
 
+  @Test(timeout=10000)
+  public void testQuorumSucceedsWithLongPause() throws Exception {
+    final Map<String, SettableFuture<String>> futures = ImmutableMap.of(
+        "f1", SettableFuture.<String>create());
+
+    FakeTimer timer = new FakeTimer() {
+      private int callCount = 0;
+      @Override
+      public long monotonicNowNanos() {
+        callCount++;
+        if (callCount == 1) {
+          long old = super.monotonicNowNanos();
+          advance(1000000);
+          return old;
+        } else if (callCount == 10) {
+          futures.get("f1").set("first future");
+          return super.monotonicNowNanos();
+        } else {
+          return super.monotonicNowNanos();
+        }
+      }
+    };
+
+    QuorumCall<String, String> q = QuorumCall.create(futures, timer);
+    assertEquals(0, q.countResponses());
+
+    q.waitFor(1, 0, 0, 3000, "test"); // wait for 1 response
+  }
+
 }


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


[02/50] [abbrv] hadoop git commit: YARN-7174. Add retry logic in LogsCLI when fetch running application logs. Contributed by Xuan Gong.

Posted by jh...@apache.org.
YARN-7174. Add retry logic in LogsCLI when fetch running application logs. Contributed by Xuan Gong.


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

Branch: refs/heads/YARN-5734
Commit: 1a84c24b0cf6674fa755403971fa57d8e412b320
Parents: 90894c7
Author: Junping Du <ju...@apache.org>
Authored: Fri Sep 15 15:33:24 2017 -0700
Committer: Junping Du <ju...@apache.org>
Committed: Fri Sep 15 15:33:24 2017 -0700

----------------------------------------------------------------------
 .../apache/hadoop/yarn/client/cli/LogsCLI.java  | 175 +++++++++++++++-
 .../hadoop/yarn/client/cli/TestLogsCLI.java     | 205 +++++++++++++------
 2 files changed, 309 insertions(+), 71 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1a84c24b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java
index 1a3db26..9a8ba4a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java
@@ -22,6 +22,9 @@ import java.io.File;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.PrintStream;
+import java.net.ConnectException;
+import java.net.SocketException;
+import java.net.SocketTimeoutException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -75,9 +78,11 @@ import org.codehaus.jettison.json.JSONObject;
 import com.google.common.annotations.VisibleForTesting;
 import com.sun.jersey.api.client.Client;
 import com.sun.jersey.api.client.ClientHandlerException;
+import com.sun.jersey.api.client.ClientRequest;
 import com.sun.jersey.api.client.ClientResponse;
 import com.sun.jersey.api.client.UniformInterfaceException;
 import com.sun.jersey.api.client.WebResource;
+import com.sun.jersey.api.client.filter.ClientFilter;
 
 @Public
 @Evolving
@@ -98,14 +103,27 @@ public class LogsCLI extends Configured implements Tool {
       = "show_container_log_info";
   private static final String OUT_OPTION = "out";
   private static final String SIZE_OPTION = "size";
+  private static final String CLIENT_MAX_RETRY_OPTION = "client_max_retries";
+  private static final String CLIENT_RETRY_INTERVAL_OPTION
+      = "client_retry_interval_ms";
   public static final String HELP_CMD = "help";
+
   private PrintStream outStream = System.out;
   private YarnClient yarnClient = null;
+  private Client webServiceClient = null;
+
+  private static final int DEFAULT_MAX_RETRIES = 30;
+  private static final long DEFAULT_RETRY_INTERVAL = 1000;
+
+  @Private
+  @VisibleForTesting
+  ClientConnectionRetry connectionRetry;
 
   @Override
   public int run(String[] args) throws Exception {
     try {
       yarnClient = createYarnClient();
+      webServiceClient = Client.create();
       return runCommand(args);
     } finally {
       if (yarnClient != null) {
@@ -140,6 +158,8 @@ public class LogsCLI extends Configured implements Tool {
     List<String> amContainersList = new ArrayList<String>();
     String localDir = null;
     long bytes = Long.MAX_VALUE;
+    int maxRetries = DEFAULT_MAX_RETRIES;
+    long retryInterval = DEFAULT_RETRY_INTERVAL;
     try {
       CommandLine commandLine = parser.parse(opts, args, false);
       appIdStr = commandLine.getOptionValue(APPLICATION_ID_OPTION);
@@ -171,6 +191,14 @@ public class LogsCLI extends Configured implements Tool {
       if (commandLine.hasOption(SIZE_OPTION)) {
         bytes = Long.parseLong(commandLine.getOptionValue(SIZE_OPTION));
       }
+      if (commandLine.hasOption(CLIENT_MAX_RETRY_OPTION)) {
+        maxRetries = Integer.parseInt(commandLine.getOptionValue(
+            CLIENT_MAX_RETRY_OPTION));
+      }
+      if (commandLine.hasOption(CLIENT_RETRY_INTERVAL_OPTION)) {
+        retryInterval = Long.parseLong(commandLine.getOptionValue(
+            CLIENT_RETRY_INTERVAL_OPTION));
+      }
     } catch (ParseException e) {
       System.err.println("options parsing failed: " + e.getMessage());
       printHelpMessage(printOpts);
@@ -232,6 +260,11 @@ public class LogsCLI extends Configured implements Tool {
       }
     }
 
+    // Set up Retry WebService Client
+    connectionRetry = new ClientConnectionRetry(maxRetries, retryInterval);
+    ClientJerseyRetryFilter retryFilter = new ClientJerseyRetryFilter();
+    webServiceClient.addFilter(retryFilter);
+
     LogCLIHelpers logCliHelper = new LogCLIHelpers();
     logCliHelper.setConf(getConf());
 
@@ -342,7 +375,6 @@ public class LogsCLI extends Configured implements Tool {
   protected List<JSONObject> getAMContainerInfoForRMWebService(
       Configuration conf, String appId) throws ClientHandlerException,
       UniformInterfaceException, JSONException {
-    Client webServiceClient = Client.create();
     String webAppAddress = WebAppUtils.getRMWebAppURLWithScheme(conf);
 
     WebResource webResource = webServiceClient.resource(webAppAddress);
@@ -364,7 +396,6 @@ public class LogsCLI extends Configured implements Tool {
   private List<JSONObject> getAMContainerInfoForAHSWebService(
       Configuration conf, String appId) throws ClientHandlerException,
       UniformInterfaceException, JSONException {
-    Client webServiceClient = Client.create();
     String webAppAddress =
         WebAppUtils.getHttpSchemePrefix(conf)
             + WebAppUtils.getAHSWebAppURLWithoutScheme(conf);
@@ -417,7 +448,6 @@ public class LogsCLI extends Configured implements Tool {
       throws IOException {
     List<Pair<ContainerLogFileInfo, String>> logFileInfos
         = new ArrayList<>();
-    Client webServiceClient = Client.create();
     try {
       WebResource webResource = webServiceClient
           .resource(WebAppUtils.getHttpSchemePrefix(conf) + nodeHttpAddress);
@@ -490,7 +520,6 @@ public class LogsCLI extends Configured implements Tool {
         lastModificationTime);
   }
 
-
   @Private
   @VisibleForTesting
   public int printContainerLogsFromRunningApplication(Configuration conf,
@@ -521,7 +550,6 @@ public class LogsCLI extends Configured implements Tool {
       ContainerLogsRequest newOptions = new ContainerLogsRequest(request);
       newOptions.setLogTypes(matchedFiles);
 
-      Client webServiceClient = Client.create();
       boolean foundAnyLogs = false;
       byte[] buffer = new byte[65536];
       for (String logFile : newOptions.getLogTypes()) {
@@ -797,6 +825,10 @@ public class LogsCLI extends Configured implements Tool {
     }
   }
 
+  /**
+   * Create Command Options.
+   * @return the command options
+   */
   private Options createCommandOpts() {
     Options opts = new Options();
     opts.addOption(HELP_CMD, false, "Displays help for all commands.");
@@ -859,6 +891,13 @@ public class LogsCLI extends Configured implements Tool {
     opts.addOption(SIZE_OPTION, true, "Prints the log file's first 'n' bytes "
         + "or the last 'n' bytes. Use negative values as bytes to read from "
         + "the end and positive values as bytes to read from the beginning.");
+    opts.addOption(CLIENT_MAX_RETRY_OPTION, true, "Set max retry number for a"
+        + " retry client to get the container logs for the running "
+        + "applications. Use a negative value to make retry forever. "
+        + "The default value is 30.");
+    opts.addOption(CLIENT_RETRY_INTERVAL_OPTION, true,
+        "Work with --client_max_retries to create a retry client. "
+        + "The default value is 1000.");
     opts.getOption(APPLICATION_ID_OPTION).setArgName("Application ID");
     opts.getOption(CONTAINER_ID_OPTION).setArgName("Container ID");
     opts.getOption(NODE_ADDRESS_OPTION).setArgName("Node Address");
@@ -866,9 +905,17 @@ public class LogsCLI extends Configured implements Tool {
     opts.getOption(AM_CONTAINER_OPTION).setArgName("AM Containers");
     opts.getOption(OUT_OPTION).setArgName("Local Directory");
     opts.getOption(SIZE_OPTION).setArgName("size");
+    opts.getOption(CLIENT_MAX_RETRY_OPTION).setArgName("Max Retries");
+    opts.getOption(CLIENT_RETRY_INTERVAL_OPTION)
+        .setArgName("Retry Interval");
     return opts;
   }
 
+  /**
+   * Create Print options for helper message.
+   * @param commandOpts the options
+   * @return the print options
+   */
   private Options createPrintOpts(Options commandOpts) {
     Options printOpts = new Options();
     printOpts.addOption(commandOpts.getOption(HELP_CMD));
@@ -884,6 +931,8 @@ public class LogsCLI extends Configured implements Tool {
     printOpts.addOption(commandOpts.getOption(SIZE_OPTION));
     printOpts.addOption(commandOpts.getOption(
         PER_CONTAINER_LOG_FILES_REGEX_OPTION));
+    printOpts.addOption(commandOpts.getOption(CLIENT_MAX_RETRY_OPTION));
+    printOpts.addOption(commandOpts.getOption(CLIENT_RETRY_INTERVAL_OPTION));
     return printOpts;
   }
 
@@ -1286,4 +1335,120 @@ public class LogsCLI extends Configured implements Tool {
     return nodeInfo.has("nodeHTTPAddress") ?
         nodeInfo.getString("nodeHTTPAddress") : null;
   }
+
+  // Class to handle retry
+  static class ClientConnectionRetry {
+
+    // maxRetries < 0 means keep trying
+    @Private
+    @VisibleForTesting
+    public int maxRetries;
+
+    @Private
+    @VisibleForTesting
+    public long retryInterval;
+
+    // Indicates if retries happened last time. Only tests should read it.
+    // In unit tests, retryOn() calls should _not_ be concurrent.
+    private boolean retried = false;
+
+    @Private
+    @VisibleForTesting
+    boolean getRetired() {
+      return retried;
+    }
+
+    // Constructor with default retry settings
+    public ClientConnectionRetry(int inputMaxRetries,
+        long inputRetryInterval) {
+      this.maxRetries = inputMaxRetries;
+      this.retryInterval = inputRetryInterval;
+    }
+
+    public Object retryOn(ClientRetryOp op)
+        throws RuntimeException, IOException {
+      int leftRetries = maxRetries;
+      retried = false;
+
+      // keep trying
+      while (true) {
+        try {
+          // try perform the op, if fail, keep retrying
+          return op.run();
+        } catch (IOException | RuntimeException e) {
+          // break if there's no retries left
+          if (leftRetries == 0) {
+            break;
+          }
+          if (op.shouldRetryOn(e)) {
+            logException(e, leftRetries);
+          } else {
+            throw e;
+          }
+        }
+        if (leftRetries > 0) {
+          leftRetries--;
+        }
+        retried = true;
+        try {
+          // sleep for the given time interval
+          Thread.sleep(retryInterval);
+        } catch (InterruptedException ie) {
+          System.out.println("Client retry sleep interrupted! ");
+        }
+      }
+      throw new RuntimeException("Connection retries limit exceeded.");
+    };
+
+    private void logException(Exception e, int leftRetries) {
+      if (leftRetries > 0) {
+        System.out.println("Exception caught by ClientConnectionRetry,"
+              + " will try " + leftRetries + " more time(s).\nMessage: "
+              + e.getMessage());
+      } else {
+        // note that maxRetries may be -1 at the very beginning
+        System.out.println("ConnectionException caught by ClientConnectionRetry,"
+            + " will keep retrying.\nMessage: "
+            + e.getMessage());
+      }
+    }
+  }
+
+  private class ClientJerseyRetryFilter extends ClientFilter {
+    @Override
+    public ClientResponse handle(final ClientRequest cr)
+        throws ClientHandlerException {
+      // Set up the retry operation
+      ClientRetryOp jerseyRetryOp = new ClientRetryOp() {
+        @Override
+        public Object run() {
+          // Try pass the request, if fail, keep retrying
+          return getNext().handle(cr);
+        }
+
+        @Override
+        public boolean shouldRetryOn(Exception e) {
+          // Only retry on connection exceptions
+          return (e instanceof ClientHandlerException)
+              && (e.getCause() instanceof ConnectException ||
+                  e.getCause() instanceof SocketTimeoutException ||
+                  e.getCause() instanceof SocketException);
+        }
+      };
+      try {
+        return (ClientResponse) connectionRetry.retryOn(jerseyRetryOp);
+      } catch (IOException e) {
+        throw new ClientHandlerException("Jersey retry failed!\nMessage: "
+              + e.getMessage());
+      }
+    }
+  }
+
+  // Abstract class for an operation that should be retried by client
+  private static abstract class ClientRetryOp {
+    // The operation that should be retried
+    public abstract Object run() throws IOException;
+    // The method to indicate if we should retry given the incoming exception
+    public abstract boolean shouldRetryOn(Exception e);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1a84c24b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java
index d58732e..fed7488 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java
@@ -196,7 +196,7 @@ public class TestLogsCLI {
         "Unable to get ApplicationState"));
   }
 
-  @Test(timeout = 5000l)
+  @Test (timeout = 10000)
   public void testHelpMessage() throws Exception {
     Configuration conf = new YarnConfiguration();
     YarnClient mockYarnClient = createMockYarnClient(
@@ -207,79 +207,102 @@ public class TestLogsCLI {
 
     int exitCode = dumper.run(new String[]{});
     assertTrue(exitCode == -1);
+
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
     PrintWriter pw = new PrintWriter(baos);
     pw.println("Retrieve logs for YARN applications.");
     pw.println("usage: yarn logs -applicationId <application ID> [OPTIONS]");
     pw.println();
     pw.println("general options are:");
-    pw.println(" -am <AM Containers>                     Prints the AM Container logs for");
-    pw.println("                                         this application. Specify");
-    pw.println("                                         comma-separated value to get logs");
-    pw.println("                                         for related AM Container. For");
-    pw.println("                                         example, If we specify -am 1,2,");
-    pw.println("                                         we will get the logs for the");
-    pw.println("                                         first AM Container as well as the");
-    pw.println("                                         second AM Container. To get logs");
-    pw.println("                                         for all AM Containers, use -am");
-    pw.println("                                         ALL. To get logs for the latest");
-    pw.println("                                         AM Container, use -am -1. By");
-    pw.println("                                         default, it will print all");
-    pw.println("                                         available logs. Work with");
-    pw.println("                                         -log_files to get only specific");
-    pw.println("                                         logs.");
-    pw.println(" -appOwner <Application Owner>           AppOwner (assumed to be current");
-    pw.println("                                         user if not specified)");
-    pw.println(" -containerId <Container ID>             ContainerId. By default, it will");
-    pw.println("                                         print all available logs. Work");
-    pw.println("                                         with -log_files to get only");
-    pw.println("                                         specific logs. If specified, the");
-    pw.println("                                         applicationId can be omitted");
-    pw.println(" -help                                   Displays help for all commands.");
-    pw.println(" -list_nodes                             Show the list of nodes that");
-    pw.println("                                         successfully aggregated logs.");
-    pw.println("                                         This option can only be used with");
-    pw.println("                                         finished applications.");
-    pw.println(" -log_files <Log File Name>              Specify comma-separated value to");
-    pw.println("                                         get exact matched log files. Use");
-    pw.println("                                         \"ALL\" or \"*\" to fetch all the log");
-    pw.println("                                         files for the container.");
-    pw.println(" -log_files_pattern <Log File Pattern>   Specify comma-separated value to");
-    pw.println("                                         get matched log files by using");
-    pw.println("                                         java regex. Use \".*\" to fetch all");
-    pw.println("                                         the log files for the container.");
-    pw.println(" -nodeAddress <Node Address>             NodeAddress in the format");
-    pw.println("                                         nodename:port");
-    pw.println(" -out <Local Directory>                  Local directory for storing");
-    pw.println("                                         individual container logs. The");
-    pw.println("                                         container logs will be stored");
-    pw.println("                                         based on the node the container");
-    pw.println("                                         ran on.");
-    pw.println(" -show_application_log_info              Show the containerIds which");
-    pw.println("                                         belong to the specific");
-    pw.println("                                         Application. You can combine this");
-    pw.println("                                         with --nodeAddress to get");
-    pw.println("                                         containerIds for all the");
-    pw.println("                                         containers on the specific");
-    pw.println("                                         NodeManager.");
-    pw.println(" -show_container_log_info                Show the container log metadata,");
-    pw.println("                                         including log-file names, the");
-    pw.println("                                         size of the log files. You can");
-    pw.println("                                         combine this with --containerId");
-    pw.println("                                         to get log metadata for the");
-    pw.println("                                         specific container, or with");
-    pw.println("                                         --nodeAddress to get log metadata");
-    pw.println("                                         for all the containers on the");
-    pw.println("                                         specific NodeManager.");
-    pw.println(" -size <size>                            Prints the log file's first 'n'");
-    pw.println("                                         bytes or the last 'n' bytes. Use");
-    pw.println("                                         negative values as bytes to read");
-    pw.println("                                         from the end and positive values");
-    pw.println("                                         as bytes to read from the");
-    pw.println("                                         beginning.");
+    pw.println(" -am <AM Containers>                          Prints the AM Container logs");
+    pw.println("                                              for this application.");
+    pw.println("                                              Specify comma-separated");
+    pw.println("                                              value to get logs for");
+    pw.println("                                              related AM Container. For");
+    pw.println("                                              example, If we specify -am");
+    pw.println("                                              1,2, we will get the logs");
+    pw.println("                                              for the first AM Container");
+    pw.println("                                              as well as the second AM");
+    pw.println("                                              Container. To get logs for");
+    pw.println("                                              all AM Containers, use -am");
+    pw.println("                                              ALL. To get logs for the");
+    pw.println("                                              latest AM Container, use -am");
+    pw.println("                                              -1. By default, it will");
+    pw.println("                                              print all available logs.");
+    pw.println("                                              Work with -log_files to get");
+    pw.println("                                              only specific logs.");
+    pw.println(" -appOwner <Application Owner>                AppOwner (assumed to be");
+    pw.println("                                              current user if not");
+    pw.println("                                              specified)");
+    pw.println(" -client_max_retries <Max Retries>            Set max retry number for a");
+    pw.println("                                              retry client to get the");
+    pw.println("                                              container logs for the");
+    pw.println("                                              running applications. Use a");
+    pw.println("                                              negative value to make retry");
+    pw.println("                                              forever. The default value");
+    pw.println("                                              is 30.");
+    pw.println(" -client_retry_interval_ms <Retry Interval>   Work with");
+    pw.println("                                              --client_max_retries to");
+    pw.println("                                              create a retry client. The");
+    pw.println("                                              default value is 1000.");
+    pw.println(" -containerId <Container ID>                  ContainerId. By default, it");
+    pw.println("                                              will print all available");
+    pw.println("                                              logs. Work with -log_files");
+    pw.println("                                              to get only specific logs.");
+    pw.println("                                              If specified, the");
+    pw.println("                                              applicationId can be omitted");
+    pw.println(" -help                                        Displays help for all");
+    pw.println("                                              commands.");
+    pw.println(" -list_nodes                                  Show the list of nodes that");
+    pw.println("                                              successfully aggregated");
+    pw.println("                                              logs. This option can only");
+    pw.println("                                              be used with finished");
+    pw.println("                                              applications.");
+    pw.println(" -log_files <Log File Name>                   Specify comma-separated");
+    pw.println("                                              value to get exact matched");
+    pw.println("                                              log files. Use \"ALL\" or \"*\"");
+    pw.println("                                              to fetch all the log files");
+    pw.println("                                              for the container.");
+    pw.println(" -log_files_pattern <Log File Pattern>        Specify comma-separated");
+    pw.println("                                              value to get matched log");
+    pw.println("                                              files by using java regex.");
+    pw.println("                                              Use \".*\" to fetch all the");
+    pw.println("                                              log files for the container.");
+    pw.println(" -nodeAddress <Node Address>                  NodeAddress in the format");
+    pw.println("                                              nodename:port");
+    pw.println(" -out <Local Directory>                       Local directory for storing");
+    pw.println("                                              individual container logs.");
+    pw.println("                                              The container logs will be");
+    pw.println("                                              stored based on the node the");
+    pw.println("                                              container ran on.");
+    pw.println(" -show_application_log_info                   Show the containerIds which");
+    pw.println("                                              belong to the specific");
+    pw.println("                                              Application. You can combine");
+    pw.println("                                              this with --nodeAddress to");
+    pw.println("                                              get containerIds for all the");
+    pw.println("                                              containers on the specific");
+    pw.println("                                              NodeManager.");
+    pw.println(" -show_container_log_info                     Show the container log");
+    pw.println("                                              metadata, including log-file");
+    pw.println("                                              names, the size of the log");
+    pw.println("                                              files. You can combine this");
+    pw.println("                                              with --containerId to get");
+    pw.println("                                              log metadata for the");
+    pw.println("                                              specific container, or with");
+    pw.println("                                              --nodeAddress to get log");
+    pw.println("                                              metadata for all the");
+    pw.println("                                              containers on the specific");
+    pw.println("                                              NodeManager.");
+    pw.println(" -size <size>                                 Prints the log file's first");
+    pw.println("                                              'n' bytes or the last 'n'");
+    pw.println("                                              bytes. Use negative values");
+    pw.println("                                              as bytes to read from the");
+    pw.println("                                              end and positive values as");
+    pw.println("                                              bytes to read from the");
+    pw.println("                                              beginning.");
     pw.close();
     String appReportStr = baos.toString("UTF-8");
-    Assert.assertEquals(appReportStr, sysOutStream.toString());
+    Assert.assertTrue(sysOutStream.toString().contains(appReportStr));
   }
 
   @Test (timeout = 15000)
@@ -609,6 +632,56 @@ public class TestLogsCLI {
     fs.delete(new Path(rootLogDir), true);
   }
 
+  @Test
+  public void testCheckRetryCount() throws Exception {
+    UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+
+    NodeId nodeId = NodeId.newInstance("localhost", 1234);
+    ApplicationId appId = ApplicationId.newInstance(0, 1);
+    ApplicationAttemptId appAttemptId = ApplicationAttemptId
+        .newInstance(appId, 1);
+
+    // Create a mock ApplicationAttempt Report
+    ApplicationAttemptReport mockAttemptReport = mock(
+        ApplicationAttemptReport.class);
+    doReturn(appAttemptId).when(mockAttemptReport).getApplicationAttemptId();
+    List<ApplicationAttemptReport> attemptReports = Arrays.asList(
+        mockAttemptReport);
+
+    // Create one mock containerReport
+    ContainerId containerId1 = ContainerId.newContainerId(appAttemptId, 1);
+    ContainerReport mockContainerReport1 = mock(ContainerReport.class);
+    doReturn(containerId1).when(mockContainerReport1).getContainerId();
+    doReturn(nodeId).when(mockContainerReport1).getAssignedNode();
+    doReturn("http://localhost:2345").when(mockContainerReport1)
+        .getNodeHttpAddress();
+    doReturn(ContainerState.RUNNING).when(mockContainerReport1)
+        .getContainerState();
+    List<ContainerReport> containerReports = Arrays.asList(
+        mockContainerReport1);
+    // Mock the YarnClient, and it would report the previous created
+    // mockAttemptReport and previous two created mockContainerReports
+    YarnClient mockYarnClient = createMockYarnClient(
+        YarnApplicationState.RUNNING, ugi.getShortUserName(), true,
+        attemptReports, containerReports);
+    doReturn(mockContainerReport1).when(mockYarnClient).getContainerReport(
+        any(ContainerId.class));
+    LogsCLI cli = new LogsCLIForTest(mockYarnClient);
+    cli.setConf(new YarnConfiguration());
+    try {
+      cli.run(new String[] {"-containerId",
+          containerId1.toString(), "-client_max_retries", "5"});
+      Assert.fail("Exception expected! "
+          + "NodeManager should be off to run this test. ");
+    } catch (RuntimeException ce) {
+      Assert.assertTrue(
+          "Handler exception for reason other than retry: " + ce.getMessage(),
+          ce.getMessage().contains("Connection retries limit exceeded"));
+      Assert.assertTrue("Retry filter didn't perform any retries! ", cli
+           .connectionRetry.getRetired());
+    }
+  }
+
   @Test (timeout = 5000)
   public void testGetRunningContainerLogs() throws Exception {
     UserGroupInformation ugi = UserGroupInformation.getCurrentUser();


---------------------------------------------------------------------
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: Revert "HADOOP-14771. hadoop-client does not include hadoop-yarn-client. (Ajay Kumar via Haibo Chen)" HADOOP-14879 Build failure due to failing hadoop-client-check-invariants This reverts commit 1ee25278c891e95ba2ab142e

Posted by jh...@apache.org.
Revert "HADOOP-14771. hadoop-client does not include hadoop-yarn-client. (Ajay Kumar via Haibo Chen)"
HADOOP-14879 Build failure due to failing hadoop-client-check-invariants
This reverts commit 1ee25278c891e95ba2ab142e5b78aebd752ea163.


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

Branch: refs/heads/YARN-5734
Commit: aa6e8d2dff533c3d0c86776567c860548723c21c
Parents: dba7a7d
Author: Steve Loughran <st...@apache.org>
Authored: Tue Sep 19 11:53:11 2017 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Tue Sep 19 11:53:11 2017 +0100

----------------------------------------------------------------------
 hadoop-client-modules/hadoop-client/pom.xml | 31 ------------------------
 1 file changed, 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa6e8d2d/hadoop-client-modules/hadoop-client/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-client-modules/hadoop-client/pom.xml b/hadoop-client-modules/hadoop-client/pom.xml
index 6500ebf..bed3f5c 100644
--- a/hadoop-client-modules/hadoop-client/pom.xml
+++ b/hadoop-client-modules/hadoop-client/pom.xml
@@ -179,37 +179,6 @@
 
     <dependency>
       <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-yarn-client</artifactId>
-      <scope>compile</scope>
-      <exclusions>
-        <!--Excluding hadoop-yarn-api & hadoop-annotations as they are already
-        included as direct dependencies. Guava,commons-cli and log4j are
-        transitive dependencies -->
-        <exclusion>
-          <groupId>org.apache.hadoop</groupId>
-          <artifactId>hadoop-yarn-api</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>org.apache.hadoop</groupId>
-          <artifactId>hadoop-annotations</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>com.google.guava</groupId>
-          <artifactId>guava</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>commons-cli</groupId>
-          <artifactId>commons-cli</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>log4j</groupId>
-          <artifactId>log4j</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-mapreduce-client-core</artifactId>
       <scope>compile</scope>
       <exclusions>


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

Branch: refs/heads/YARN-5734
Commit: 0de6349a2435c367507c9a631dc4e4c536cf4c66
Parents: 8d9ba97
Author: Jonathan Hung <jh...@linkedin.com>
Authored: Wed Mar 1 16:03:01 2017 -0800
Committer: Jonathan Hung <jh...@linkedin.com>
Committed: Wed Sep 20 17:40:02 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/0de6349a/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 114453f..2f77316 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/0de6349a/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 0440458..f05e005 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
@@ -3348,4 +3348,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/0de6349a/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/0de6349a/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/0de6349a/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/0de6349a/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/0de6349a/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/0de6349a/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/0de6349a/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


[06/50] [abbrv] hadoop git commit: HADOOP-13714. Tighten up our compatibility guidelines for Hadoop 3

Posted by jh...@apache.org.
HADOOP-13714. Tighten up our compatibility guidelines for Hadoop 3


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

Branch: refs/heads/YARN-5734
Commit: 7618fa9194b40454405f11a25bec4e2d79506912
Parents: 38c14ef
Author: Daniel Templeton <te...@apache.org>
Authored: Sat Sep 16 09:20:33 2017 +0200
Committer: Daniel Templeton <te...@apache.org>
Committed: Sat Sep 16 09:20:33 2017 +0200

----------------------------------------------------------------------
 .../src/site/markdown/Compatibility.md          | 645 +++++++++++++++----
 .../site/markdown/InterfaceClassification.md    | 227 ++++---
 2 files changed, 675 insertions(+), 197 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7618fa91/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 05b18b5..4fa8c02 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/Compatibility.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/Compatibility.md
@@ -20,109 +20,276 @@ Apache Hadoop Compatibility
 Purpose
 -------
 
-This document captures the compatibility goals of the Apache Hadoop project. The different types of compatibility between Hadoop releases that affects Hadoop developers, downstream projects, and end-users are enumerated. For each type of compatibility we:
+This document captures the compatibility goals of the Apache Hadoop project.
+The different types of compatibility between Hadoop releases that affect
+Hadoop developers, downstream projects, and end-users are enumerated. For each
+type of compatibility this document will:
 
 * describe the impact on downstream projects or end-users
 * where applicable, call out the policy adopted by the Hadoop developers when incompatible changes are permitted.
 
+All Hadoop interfaces are classified according to the intended audience and
+stability in order to maintain compatibility with previous releases. See the
+[Hadoop Interface Taxonomy](./InterfaceClassification.html) for details
+about the classifications.
+
+### Target Audience
+
+This document is intended for consumption by the Hadoop developer community.
+This document describes the lens through which changes to the Hadoop project
+should be viewed. In order for end users and third party developers to have
+confidence about cross-release compatibility, the developer community must
+ensure that development efforts adhere to these policies. It is the
+responsibility of the project committers to validate that all changes either
+maintain compatibility or are explicitly marked as incompatible.
+
+Within a component Hadoop developers are free to use Private and Limited Private
+APIs, but when using components from a different module Hadoop developers
+should follow the same guidelines as third-party developers: do not
+use Private or Limited Private (unless explicitly allowed) interfaces and
+prefer instead Stable interfaces to Evolving or Unstable interfaces where
+possible. Where not possible, the preferred solution is to expand the audience
+of the API rather than introducing or perpetuating an exception to these
+compatibility guidelines. When working within a Maven module Hadoop developers
+should observe where possible the same level of restraint with regard to
+using components located in other Maven modules.
+
+Above all, Hadoop developers must be mindful of the impact of their changes.
+Stable interfaces must not change between major releases. Evolving interfaces
+must not change between minor releases. New classes and components must be
+labeled appropriately for audience and stability. See the
+[Hadoop Interface Taxonomy](./InterfaceClassification.html) for details about
+when the various labels are appropriate. As a general rule, all new interfaces
+and APIs should have the most limited labels (e.g. Private Unstable) that will
+not inhibit the intent of the interface or API.
+
+### Notational Conventions
+
+The key words "MUST" "MUST NOT", "REQUIRED", "SHALL", "SHALL NOT", "SHOULD",
+"SHOULD NOT", "RECOMMENDED", "MAY", and "OPTIONAL" are to be interpreted as
+described in [RFC 2119](http://tools.ietf.org/html/rfc2119).
+
+Deprecation
+-----------
+
+The Java API provides a @Deprecated annotation to mark an API element as
+flagged for removal. The standard meaning of the annotation is that the
+API element should not be used and may be removed in a later version.
+
+In all cases removing an element from an API is an incompatible
+change. In the case of [Stable](./InterfaceClassification.html#Stable) APIs,
+the change cannot be made between minor releases within the same major
+version. In addition, to allow consumers of the API time to adapt to the change,
+the API element to be removed should be marked as deprecated for a full major
+release before it is removed. For example, if a method is marked as deprecated
+in Hadoop 2.8, it cannot be removed until Hadoop 4.0.
+
+### Policy
+
+[Stable](./InterfaceClassification.html#Stable) API elements MUST NOT be removed
+until they have been marked as deprecated (through the @Deprecated annotation or
+other appropriate documentation) for a full major release. In the case that an
+API element was introduced as deprecated (to indicate that it is a temporary
+measure that is intended to be removed) the API element MAY be removed in the
+following major release. When modifying a
+[Stable](./InterfaceClassification.html#Stable) API, developers SHOULD prefer
+introducing a new method or endpoint and deprecating the existing one to making
+incompatible changes to the method or endpoint.
+
 Compatibility types
 -------------------
 
 ### Java API
 
-Hadoop interfaces and classes are annotated to describe the intended audience and stability in order to maintain compatibility with previous releases. See [Hadoop Interface Classification](./InterfaceClassification.html) for details.
-
-* InterfaceAudience: captures the intended audience, possible values are Public (for end users and external projects), LimitedPrivate (for other Hadoop components, and closely related projects like YARN, MapReduce, HBase etc.), and Private (for intra component use).
-* InterfaceStability: describes what types of interface changes are permitted. Possible values are Stable, Evolving, Unstable, and Deprecated.
+Developers SHOULD annotate all Hadoop interfaces and classes with the
+@InterfaceAudience and @InterfaceStability annotations to describe the
+intended audience and stability. Annotations may be at the package, class, or
+member variable or method level. Member variable and method annotations SHALL
+override class annotations, and class annotations SHALL override package
+annotations. A package, class, or member variable or method that is not
+annotated SHALL be interpreted as implicitly
+[Private](./InterfaceClassification.html#Private) and
+[Unstable](./InterfaceClassification.html#Unstable).
+
+* @InterfaceAudience captures the intended audience. Possible values are
+[Public](./InterfaceClassification.html#Public) (for end users and external
+projects), Limited[Private](./InterfaceClassification.html#Private) (for other
+Hadoop components, and closely related projects like YARN, MapReduce, HBase
+etc.), and [Private](./InterfaceClassification.html#Private)
+(for intra component use).
+* @InterfaceStability describes what types of interface changes are permitted. Possible values are [Stable](./InterfaceClassification.html#Stable), [Evolving](./InterfaceClassification.html#Evolving), and [Unstable](./InterfaceClassification.html#Unstable).
+* @Deprecated notes that the package, class, or member variable or method could potentially be removed in the future and should not be used.
 
 #### Use Cases
 
-* Public-Stable API compatibility is required to ensure end-user programs and downstream projects continue to work without modification.
-* LimitedPrivate-Stable API compatibility is required to allow upgrade of individual components across minor releases.
-* Private-Stable API compatibility is required for rolling upgrades.
+* [Public](./InterfaceClassification.html#Public)-[Stable](./InterfaceClassification.html#Stable) API compatibility is required to ensure end-user programs and downstream projects continue to work without modification.
+* [Public](./InterfaceClassification.html#Public)-[Evolving](./InterfaceClassification.html#Evolving) API compatibility is useful to make functionality available for consumption before it is fully baked.
+* Limited Private-[Stable](./InterfaceClassification.html#Stable) API compatibility is required to allow upgrade of individual components across minor releases.
+* [Private](./InterfaceClassification.html#Private)-[Stable](./InterfaceClassification.html#Stable) API compatibility is required for rolling upgrades.
+* [Private](./InterfaceClassification.html#Private)-[Unstable](./InterfaceClassification.html#Unstable) API compatibility allows internal components to evolve rapidly without concern for downstream consumers, and is how most interfaces should be labeled.
 
 #### Policy
 
-* Public-Stable APIs must be deprecated for at least one major release prior to their removal in a major release.
-* LimitedPrivate-Stable APIs can change across major releases, but not within a major release.
-* Private-Stable APIs can change across major releases, but not within a major release.
-* Classes not annotated are implicitly "Private". Class members not annotated inherit the annotations of the enclosing class.
-* Note: APIs generated from the proto files need to be compatible for rolling-upgrades. See the section on wire-compatibility for more details. The compatibility policies for APIs and wire-communication need to go hand-in-hand to address this.
-
-### Semantic compatibility
-
-Apache Hadoop strives to ensure that the behavior of APIs remains consistent over versions, though changes for correctness may result in changes in behavior. Tests and javadocs specify the API's behavior. The community is in the process of specifying some APIs more rigorously, and enhancing test suites to verify compliance with the specification, effectively creating a formal specification for the subset of behaviors that can be easily tested.
+The compatibility policy SHALL be determined by the relevant package, class, or
+member variable or method annotations.
+
+Note: APIs generated from the proto files MUST be compatible for rolling
+upgrades. See the section on wire protocol compatibility for more details. The
+compatibility policies for APIs and wire protocols must therefore go hand
+in hand.
+
+#### Semantic compatibility
+
+Apache Hadoop strives to ensure that the behavior of APIs remains consistent
+over versions, though changes for correctness may result in changes in
+behavior. API behavior SHALL be specified by the JavaDoc API documentation
+where present and complete. When JavaDoc API documentation is not available,
+behavior SHALL be specified by the behavior expected by the related unit tests.
+In cases with no JavaDoc API documentation or unit test coverage, the expected
+behavior is presumed to be obvious and SHOULD be assumed to be the minimum
+functionality implied by the interface naming. The community is in the process
+of specifying some APIs more rigorously and enhancing test suites to verify
+compliance with the specification, effectively creating a formal specification
+for the subset of behaviors that can be easily tested.
+
+The behavior of any API MAY be changed to fix incorrect behavior according to
+the stability of the API, with such a change to be accompanied by updating
+existing documentation and tests and/or adding new documentation or tests.
+
+#### Java Binary compatibility for end-user applications i.e. Apache Hadoop ABI
+
+Apache Hadoop revisions SHOUD retain binary compatability such that end-user
+applications continue to work without any modifications. Minor Apache Hadoop
+revisions within the same major revision MUST retain compatibility such that
+existing MapReduce applications (e.g. end-user applications and projects such
+as Apache Pig, Apache Hive, et al), existing YARN applications (e.g.
+end-user applications and projects such as Apache Spark, Apache Tez et al),
+and applications that accesses HDFS directly (e.g. end-user applications and
+projects such as Apache HBase, Apache Flume, et al) work unmodified and without
+recompilation when used with any Apache Hadoop cluster within the same major
+release as the original build target.
+
+For MapReduce applications in particular, i.e. applications using the
+org.apache.hadoop.mapred and/or org.apache.hadoop.mapreduce APIs, the developer
+community SHALL support binary compatibility across major releases. The
+MapReduce APIs SHALL be supported compatibly across major releases. See
+[Compatibility for MapReduce applications between hadoop-1.x and hadoop-2.x](../../hadoop-mapreduce-client/hadoop-mapreduce-client-core/MapReduce_Compatibility_Hadoop1_Hadoop2.html) for more details.
+
+Some applications may be affected by changes to disk layouts or other internal
+changes. See the sections that follow for policies on how incompatible
+changes to non-API interfaces are handled.
+
+### Native Dependencies
+
+Hadoop includes several native components, including compression, the
+container executor binary, and various native integrations. These native
+components introduce a set of native dependencies for Hadoop, both at compile
+time and at runtime, such as cmake, gcc, zlib, etc. This set of native
+dependencies is part of the Hadoop ABI.
 
 #### Policy
 
-The behavior of API may be changed to fix incorrect behavior, such a change to be accompanied by updating existing buggy tests or adding tests in cases there were none prior to the change.
+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.
 
-### Wire compatibility
+### Wire Protocols
 
-Wire compatibility concerns data being transmitted over the wire between Hadoop processes. Hadoop uses Protocol Buffers for most RPC communication. Preserving compatibility requires prohibiting modification as described below. Non-RPC communication should be considered as well, for example using HTTP to transfer an HDFS image as part of snapshotting or transferring MapTask output. The potential communications can be categorized as follows:
+Wire compatibility concerns data being transmitted "over the wire" between
+Hadoop processes. Hadoop uses
+[Protocol Buffers](https://developers.google.com/protocol-buffers/) for most
+RPC communication. Preserving compatibility requires prohibiting modification
+as described below. Non-RPC communication should be considered as well, for
+example using HTTP to transfer an HDFS image as part of snapshotting or
+transferring MapReduce map task output. The communications can be categorized as
+follows:
 
 * Client-Server: communication between Hadoop clients and servers (e.g., the HDFS client to NameNode protocol, or the YARN client to ResourceManager protocol).
-* Client-Server (Admin): It is worth distinguishing a subset of the Client-Server protocols used solely by administrative commands (e.g., the HAAdmin protocol) as these protocols only impact administrators who can tolerate changes that end users (which use general Client-Server protocols) can not.
+* Client-Server (Admin): It is worth distinguishing a subset of the Client-Server protocols used solely by administrative commands (e.g., the HAAdmin protocol) as these protocols only impact administrators who can tolerate changes that end users (which use general Client-Server protocols) cannot.
 * Server-Server: communication between servers (e.g., the protocol between the DataNode and NameNode, or NodeManager and ResourceManager)
 
-#### Use Cases
+#### Protocol Dependencies
 
-* Client-Server compatibility is required to allow users to continue using the old clients even after upgrading the server (cluster) to a later version (or vice versa). For example, a Hadoop 2.1.0 client talking to a Hadoop 2.3.0 cluster.
-* Client-Server compatibility is also required to allow users to upgrade the client before upgrading the server (cluster). For example, a Hadoop 2.4.0 client talking to a Hadoop 2.3.0 cluster. This allows deployment of client-side bug fixes ahead of full cluster upgrades. Note that new cluster features invoked by new client APIs or shell commands will not be usable. YARN applications that attempt to use new APIs (including new fields in data structures) that have not yet been deployed to the cluster can expect link exceptions.
-* Client-Server compatibility is also required to allow upgrading individual components without upgrading others. For example, upgrade HDFS from version 2.1.0 to 2.2.0 without upgrading MapReduce.
-* Server-Server compatibility is required to allow mixed versions within an active cluster so the cluster may be upgraded without downtime in a rolling fashion.
+The components of Apache Hadoop may have dependencies that include their own
+protocols, such as Zookeeper, S3, Kerberos, etc. These protocol dependencies
+SHALL be treated as internal protocols and governed by the same policy.
 
-#### Policy
+#### Transports
+
+In addition to compatibility of the protocols themselves, maintaining
+cross-version communications requires that the transports supported also be
+stable. The most likely source of transport changes stems from secure
+transports, such as SSL. Upgrading a service from SSLv2 to SSLv3 may break
+existing SSLv2 clients. The minimum supported major version of any transports
+MUST not increase across minor releases within a major version.
 
-* Both Client-Server and Server-Server compatibility is preserved within a major release. (Different policies for different categories are yet to be considered.)
-* Compatibility can be broken only at a major release, though breaking compatibility even at major releases has grave consequences and should be discussed in the Hadoop community.
-* Hadoop protocols are defined in .proto (ProtocolBuffers) files. Client-Server protocols and Server-Server protocol .proto files are marked as stable. When a .proto file is marked as stable it means that changes should be made in a compatible fashion as described below:
-    * The following changes are compatible and are allowed at any time:
-        * Add an optional field, with the expectation that the code deals with the field missing due to communication with an older version of the code.
-        * Add a new rpc/method to the service
-        * Add a new optional request to a Message
-        * Rename a field
-        * Rename a .proto file
-        * Change .proto annotations that effect code generation (e.g. name of java package)
-    * The following changes are incompatible but can be considered only at a major release
-        * Change the rpc/method name
-        * Change the rpc/method parameter type or return type
-        * Remove an rpc/method
-        * Change the service name
-        * Change the name of a Message
-        * Modify a field type in an incompatible way (as defined recursively)
-        * Change an optional field to required
-        * Add or delete a required field
-        * Delete an optional field as long as the optional field has reasonable defaults to allow deletions
-    * The following changes are incompatible and hence never allowed
-        * Change a field id
-        * Reuse an old field that was previously deleted.
-        * Field numbers are cheap and changing and reusing is not a good idea.
-
-### Java Binary compatibility for end-user applications i.e. Apache Hadoop ABI
-
-As Apache Hadoop revisions are upgraded end-users reasonably expect that their applications should continue to work without any modifications. This is fulfilled as a result of supporting API compatibility, Semantic compatibility and Wire compatibility.
-
-However, Apache Hadoop is a very complex, distributed system and services a very wide variety of use-cases. In particular, Apache Hadoop MapReduce is a very, very wide API; in the sense that end-users may make wide-ranging assumptions such as layout of the local disk when their map/reduce tasks are executing, environment variables for their tasks etc. In such cases, it becomes very hard to fully specify, and support, absolute compatibility.
-
-#### Use cases
-
-* Existing MapReduce applications, including jars of existing packaged end-user applications and projects such as Apache Pig, Apache Hive, Cascading etc. should work unmodified when pointed to an upgraded Apache Hadoop cluster within a major release.
-* Existing YARN applications, including jars of existing packaged end-user applications and projects such as Apache Tez etc. should work unmodified when pointed to an upgraded Apache Hadoop cluster within a major release.
-* Existing applications which transfer data in/out of HDFS, including jars of existing packaged end-user applications and frameworks such as Apache Flume, should work unmodified when pointed to an upgraded Apache Hadoop cluster within a major release.
+Service ports are considered as part of the transport mechanism. Fixed
+service port numbers MUST be kept consistent to prevent breaking clients.
 
 #### Policy
 
-* Existing MapReduce, YARN & HDFS applications and frameworks should work unmodified within a major release i.e. Apache Hadoop ABI is supported.
-* A very minor fraction of applications maybe affected by changes to disk layouts etc., the developer community will strive to minimize these changes and will not make them within a minor version. In more egregious cases, we will consider strongly reverting these breaking changes and invalidating offending releases if necessary.
-* In particular for MapReduce applications, the developer community will try our best to support providing binary compatibility across major releases e.g. applications using org.apache.hadoop.mapred.
-* APIs are supported compatibly across hadoop-1.x and hadoop-2.x. See [Compatibility for MapReduce applications between hadoop-1.x and hadoop-2.x](../../hadoop-mapreduce-client/hadoop-mapreduce-client-core/MapReduce_Compatibility_Hadoop1_Hadoop2.html) for more details.
+Hadoop wire protocols are defined in .proto (ProtocolBuffers) files.
+Client-Server and Server-Server protocols SHALL be classified according to the
+audience and stability classifications noted in their .proto files. In cases
+where no classifications are present, the protocols SHOULD be assumed to be
+[Private](./InterfaceClassification.html#Private) and
+[Stable](./InterfaceClassification.html#Stable).
+
+The following changes to a .proto file SHALL be considered compatible:
+
+* Add an optional field, with the expectation that the code deals with the field missing due to communication with an older version of the code
+* Add a new rpc/method to the service
+* Add a new optional request to a Message
+* Rename a field
+* Rename a .proto file
+* Change .proto annotations that effect code generation (e.g. name of java package)
+
+The following changes to a .proto file SHALL be considered incompatible:
+
+* Change an rpc/method name
+* Change an rpc/method parameter type or return type
+* Remove an rpc/method
+* Change the service name
+* Change the name of a Message
+* Modify a field type in an incompatible way (as defined recursively)
+* Change an optional field to required
+* Add or delete a required field
+* Delete an optional field as long as the optional field has reasonable defaults to allow deletions
+
+The following changes to a .proto file SHALL be considered incompatible:
+
+* Change a field id
+* Reuse an old field that was previously deleted.
+
+Hadoop wire protocols that are not defined via .proto files SHOULD be considered
+to be [Private](./InterfaceClassification.html#Private) and
+[Stable](./InterfaceClassification.html#Stable).
+
+In addition to the limitations imposed by being
+[Stable](./InterfaceClassification.html#Stable), Hadoop's wire protocols
+MUST also be forward compatible across minor releases within a major version
+according to the following:
+
+* Client-Server compatibility MUST be maintained so as to allow users to continue using older clients even after upgrading the server (cluster) to a later version (or vice versa). For example, a Hadoop 2.1.0 client talking to a Hadoop 2.3.0 cluster.
+* Client-Server compatibility MUST be maintained so as to allow users to upgrade the client before upgrading the server (cluster). For example, a Hadoop 2.4.0 client talking to a Hadoop 2.3.0 cluster. This allows deployment of client-side bug fixes ahead of full cluster upgrades. Note that new cluster features invoked by new client APIs or shell commands will not be usable. YARN applications that attempt to use new APIs (including new fields in data structures) that have not yet been deployed to the cluster can expect link exceptions.
+* Client-Server compatibility MUST be maintained so as to allow upgrading individual components without upgrading others. For example, upgrade HDFS from version 2.1.0 to 2.2.0 without upgrading MapReduce.
+* Server-Server compatibility MUST be maintained so as to allow mixed versions within an active cluster so the cluster may be upgraded without downtime in a rolling fashion.
+
+New transport mechanisms MUST only be introduced with minor or major version
+changes. Existing transport mechanisms MUST continue to be supported across
+minor versions within a major version. Service port numbers MUST remain
+consistent across minor version numbers within a major version.
 
 ### REST APIs
 
-REST API compatibility corresponds to both the requests (URLs) and responses to each request (content, which may contain other URLs). Hadoop REST APIs are specifically meant for stable use by clients across releases, even major ones. The following are the exposed REST APIs:
+REST API compatibility applies to the REST endpoints (URLs) and response data
+format. Hadoop REST APIs are specifically meant for stable use by clients across
+releases, even major ones. The following is a non-exhaustive list of the
+exposed REST APIs:
 
-* [WebHDFS](../hadoop-hdfs/WebHDFS.html) - Stable
+* [WebHDFS](../hadoop-hdfs/WebHDFS.html)
 * [ResourceManager](../../hadoop-yarn/hadoop-yarn-site/ResourceManagerRest.html)
 * [NodeManager](../../hadoop-yarn/hadoop-yarn-site/NodeManagerRest.html)
 * [MR Application Master](../../hadoop-mapreduce-client/hadoop-mapreduce-client-core/MapredAppMasterRest.html)
@@ -130,134 +297,390 @@ REST API compatibility corresponds to both the requests (URLs) and responses to
 * [Timeline Server v1 REST API](../../hadoop-yarn/hadoop-yarn-site/TimelineServer.html)
 * [Timeline Service v2 REST API](../../hadoop-yarn/hadoop-yarn-site/TimelineServiceV2.html)
 
+Each API has an API-specific version number. Any incompatible changes MUST
+increment the API version number.
+
+#### Policy
+
+The Hadoop REST APIs SHALL be considered
+[Public](./InterfaceClassification.html#Public) and
+[Evolving](./InterfaceClassification.html#Evolving). With respect to API version
+numbers, the Hadoop REST APIs SHALL be considered
+[Public](./InterfaceClassification.html#Public) and
+[Stable](./InterfaceClassification.html#Stable), i.e. no incompatible changes
+are allowed to within an API version number.
+
+### Log Output
+
+The Hadoop daemons and CLIs produce log output via Log4j that is intended to
+aid administrators and developers in understanding and troubleshooting cluster
+behavior. Log messages are intended for human consumption, though automation
+use cases are also supported.
+
 #### Policy
 
-The APIs annotated stable in the text above preserve compatibility across at least one major release, and maybe deprecated by a newer version of the REST API in a major release.
+All log output SHALL be considered
+[Public](./InterfaceClassification.html#Public) and
+[Evolving](./InterfaceClassification.html#Evolving).
+
+### Audit Log Output
+
+Several components have audit logging systems that record system information in
+a machine readable format. Incompatible changes to that data format may break
+existing automation utilities. For the audit log, an incompatible change is any
+change that changes the format such that existing parsers no longer can parse
+the logs.
+
+#### Policy
+
+All audit log output SHALL be considered
+[Public](./InterfaceClassification.html#Public) and
+[Stable](./InterfaceClassification.html#Stable). Any change to the
+data format SHALL be considered an incompatible change.
 
 ### Metrics/JMX
 
-While the Metrics API compatibility is governed by Java API compatibility, the actual metrics exposed by Hadoop need to be compatible for users to be able to automate using them (scripts etc.). Adding additional metrics is compatible. Modifying (e.g. changing the unit or measurement) or removing existing metrics breaks compatibility. Similarly, changes to JMX MBean object names also break compatibility.
+While the Metrics API compatibility is governed by Java API compatibility, the
+Metrics data format exposed by Hadoop MUST be maintained as compatible for
+consumers of the data, e.g. for automation tasks.
 
 #### Policy
 
-Metrics should preserve compatibility within the major release.
+The data format exposed via Metrics SHALL be considered
+[Public](./InterfaceClassification.html#Public) and
+[Stable](./InterfaceClassification.html#Stable).
 
 ### File formats & Metadata
 
-User and system level data (including metadata) is stored in files of different formats. Changes to the metadata or the file formats used to store data/metadata can lead to incompatibilities between versions.
+User and system level data (including metadata) is stored in files of various
+formats. Changes to the metadata or the file formats used to store
+data/metadata can lead to incompatibilities between versions. Each class of file
+format is addressed below.
 
 #### User-level file formats
 
-Changes to formats that end-users use to store their data can prevent them from accessing the data in later releases, and hence it is highly important to keep those file-formats compatible. One can always add a "new" format improving upon an existing format. Examples of these formats include har, war, SequenceFileFormat etc.
+Changes to formats that end users use to store their data can prevent them from
+accessing the data in later releases, and hence are important to be compatible.
+Examples of these formats include har, war, SequenceFileFormat, etc.
 
 ##### Policy
 
-* Non-forward-compatible user-file format changes are restricted to major releases. When user-file formats change, new releases are expected to read existing formats, but may write data in formats incompatible with prior releases. Also, the community shall prefer to create a new format that programs must opt in to instead of making incompatible changes to existing formats.
+User-level file formats SHALL be considered
+[Public](./InterfaceClassification.html#Public) and
+[Stable](./InterfaceClassification.html#Stable). User-lever file
+format changes SHOULD be made forward compatible across major releases and MUST
+be made forward compatible within a major release. The developer community
+SHOULD prefer the creation of a new derivative file format to making
+incompatible changes to an existing file format. Such new file formats MUST be
+created as opt-in, meaning that users must be able to continue using the
+existing compatible format until and unless they explicitly opt in to using
+the new file format.
 
-#### System-internal file formats
+#### System-internal data schemas
 
-Hadoop internal data is also stored in files and again changing these formats can lead to incompatibilities. While such changes are not as devastating as the user-level file formats, a policy on when the compatibility can be broken is important.
+Hadoop internal data may also be stored in files or other data stores. Changing
+the schemas of these data stores can lead to incompatibilities.
 
 ##### MapReduce
 
 MapReduce uses formats like I-File to store MapReduce-specific data.
 
-##### Policy
+###### Policy
 
-MapReduce-internal formats like IFile maintain compatibility within a major release. Changes to these formats can cause in-flight jobs to fail and hence we should ensure newer clients can fetch shuffle-data from old servers in a compatible manner.
+All MapReduce-internal file formats, such as I-File format or the job history
+server's jhist file format, SHALL be considered
+[Private](./InterfaceClassification.html#Private) and
+[Stable](./InterfaceClassification.html#Stable).
 
 ##### HDFS Metadata
 
-HDFS persists metadata (the image and edit logs) in a particular format. Incompatible changes to either the format or the metadata prevent subsequent releases from reading older metadata. Such incompatible changes might require an HDFS "upgrade" to convert the metadata to make it accessible. Some changes can require more than one such "upgrades".
+HDFS persists metadata (the image and edit logs) in a private file format.
+Incompatible changes to either the format or the metadata prevent subsequent
+releases from reading older metadata. Incompatible changes MUST include a
+process by which existing metadata may be upgraded. Changes SHALL be
+allowed to require more than one upgrade. Incompatible changes MUST result in
+the metadata version number being incremented.
 
-Depending on the degree of incompatibility in the changes, the following potential scenarios can arise:
+Depending on the degree of incompatibility in the changes, the following
+potential scenarios can arise:
 
 * Automatic: The image upgrades automatically, no need for an explicit "upgrade".
 * Direct: The image is upgradable, but might require one explicit release "upgrade".
 * Indirect: The image is upgradable, but might require upgrading to intermediate release(s) first.
 * Not upgradeable: The image is not upgradeable.
 
-##### Policy
-
-* A release upgrade must allow a cluster to roll-back to the older version and its older disk format. The rollback needs to restore the original data, but not required to restore the updated data.
-* HDFS metadata changes must be upgradeable via any of the upgrade paths - automatic, direct or indirect.
-* More detailed policies based on the kind of upgrade are yet to be considered.
+HDFS data nodes store data in a private directory structure. The schema of that
+directory structure must remain stable to retain compatibility.
+
+###### Policy
+
+The HDFS metadata format SHALL be considered
+[Private](./InterfaceClassification.html#Private) and
+[Evolving](./InterfaceClassification.html#Evolving). Incompatible
+changes MUST include a process by which existing metada may be upgraded. The
+upgrade process MUST allow the cluster metadata to be rolled back to the older
+version and its older disk format. The rollback MUST restore the original data
+but is not REQUIRED to restore the updated data. Any incompatible change
+to the format MUST result in the major version number of the schema being
+incremented.
+
+The data node directory format SHALL be considered
+[Private](./InterfaceClassification.html#Private) and
+[Evolving](./InterfaceClassification.html#Evolving).
+
+##### AWS S3A Guard Metadata
+
+For each operation in the Hadoop S3 client (s3a) that reads or modifies
+file metadata, a shadow copy of that file metadata is stored in a separate
+metadata store, which offers HDFS-like consistency for the metadata, and may
+also provide faster lookups for things like file status or directory listings.
+S3A guard tables are created with a version marker which indicates
+compatibility.
+
+###### Policy
+
+The S3A guard metadata schema SHALL be considered
+[Private](./InterfaceClassification.html#Private) and
+[Unstable](./InterfaceClassification.html#Unstable). Any incompatible change
+to the schema MUST result in the version number of the schema being incremented.
+
+##### YARN Resource Manager State Store
+
+The YARN resource manager stores information about the cluster state in an
+external state store for use in fail over and recovery. If the schema used for
+the state store data does not remain compatible, the resource manager will not
+be able to recover its state and will fail to start. The state store data
+schema includes a version number that indicates compatibility.
+
+###### Policy
+
+The YARN resource manager state store data schema SHALL be considered
+[Private](./InterfaceClassification.html#Private) and
+[Evolving](./InterfaceClassification.html#Evolving). Any incompatible change
+to the schema MUST result in the major version number of the schema being
+incremented. Any compatible change to the schema MUST result in the minor
+version number being incremented.
+
+##### YARN Node Manager State Store
+
+The YARN node manager stores information about the node state in an
+external state store for use in recovery. If the schema used for the state
+store data does not remain compatible, the node manager will not
+be able to recover its state and will fail to start. The state store data
+schema includes a version number that indicates compatibility.
+
+###### Policy
+
+The YARN node manager state store data schema SHALL be considered
+[Private](./InterfaceClassification.html#Private) and
+[Evolving](./InterfaceClassification.html#Evolving). Any incompatible change
+to the schema MUST result in the major version number of the schema being
+incremented. Any compatible change to the schema MUST result in the minor
+version number being incremented.
+
+##### YARN Federation State Store
+
+The YARN resource manager federation service stores information about the
+federated clusters, running applications, and routing policies in an
+external state store for use in replication and recovery. If the schema used
+for the state store data does not remain compatible, the federation service
+will fail to initialize. The state store data schema includes a version number
+that indicates compatibility.
+
+###### Policy
+
+The YARN federation service state store data schema SHALL be considered
+[Private](./InterfaceClassification.html#Private) and
+[Evolving](./InterfaceClassification.html#Evolving). Any incompatible change
+to the schema MUST result in the major version number of the schema being
+incremented. Any compatible change to the schema MUST result in the minor
+version number being incremented.
 
 ### Command Line Interface (CLI)
 
-The Hadoop command line programs may be used either directly via the system shell or via shell scripts. Changing the path of a command, removing or renaming command line options, the order of arguments, or the command return code and output break compatibility and may adversely affect users.
+The Hadoop command line programs may be used either directly via the system
+shell or via shell scripts. The CLIs include both the user-facing commands, such
+as the hdfs command or the yarn command, and the admin-facing commands, such as
+the scripts used to start and stop daemons.  Changing the path of a command,
+removing or renaming command line options, the order of arguments, or the
+command return codes and output break compatibility and adversely affect users.
 
 #### Policy
 
-CLI commands are to be deprecated (warning when used) for one major release before they are removed or incompatibly modified in a subsequent major release.
+All Hadoop CLI paths, usage, and output SHALL be considered
+[Public](./InterfaceClassification.html#Public) and
+[Stable](./InterfaceClassification.html#Stable).
+Note that the CLI output SHALL be considered distinct from the log output
+generated by the Hadoop CLIs. The latter SHALL be governed by the policy on log
+output. Note also that for CLI output, all changes SHALL be considered
+incompatible changes.
 
 ### Web UI
 
-Web UI, particularly the content and layout of web pages, changes could potentially interfere with attempts to screen scrape the web pages for information.
+Web UI, particularly the content and layout of web pages, changes could
+potentially interfere with attempts to screen scrape the web pages for
+information. The Hadoop Web UI pages, however, are not meant to be scraped, e.g.
+for automation purposes. Users are expected to use REST APIs to programmatically
+access cluster information.
 
 #### Policy
 
-Web pages are not meant to be scraped and hence incompatible changes to them are allowed at any time. Users are expected to use REST APIs to get any information.
+The Hadoop Web UI SHALL be considered
+[Public](./InterfaceClassification.html#Public) and
+[Unstable](./InterfaceClassification.html#Unstable).
 
 ### Hadoop Configuration Files
 
-Users use (1) Hadoop-defined properties to configure and provide hints to Hadoop and (2) custom properties to pass information to jobs. Hence, compatibility of config properties is two-fold:
+Users use Hadoop-defined properties to configure and provide hints to Hadoop and
+custom properties to pass information to jobs. Users are encouraged to avoid
+using custom configuration property names that conflict with the namespace of
+Hadoop-defined properties and should avoid using any prefixes used by Hadoop,
+e.g. hadoop, io, ipc, fs, net, file, ftp, s3, kfs, ha, file, dfs, mapred,
+mapreduce, and yarn.
+
+#### Policy
+
+Hadoop-defined properties (names and meanings) SHALL be considered
+[Public](./InterfaceClassification.html#Public) and
+[Stable](./InterfaceClassification.html#Stable). The units implied by a
+Hadoop-defined property MUST NOT change, even
+across major versions. Default values of Hadoop-defined properties SHALL be
+considered [Public](./InterfaceClassification.html#Public) and
+[Evolving](./InterfaceClassification.html#Evolving).
+
+### Log4j Configuration Files
 
-* Modifying key-names, units of values, and default values of Hadoop-defined properties.
-* Custom configuration property keys should not conflict with the namespace of Hadoop-defined properties. Typically, users should avoid using prefixes used by Hadoop: hadoop, io, ipc, fs, net, file, ftp, s3, kfs, ha, file, dfs, mapred, mapreduce, yarn.
+The log output produced by Hadoop daemons and CLIs is governed by a set of
+configuration files. These files control the minimum level of log message that
+will be output by the various components of Hadoop, as well as where and how
+those messages are stored.
 
 #### Policy
 
-* Hadoop-defined properties are to be deprecated at least for one major release before being removed. Modifying units for existing properties is not allowed.
-* The default values of Hadoop-defined properties can be changed across minor/major releases, but will remain the same across point releases within a minor release.
-* Currently, there is NO explicit policy regarding when new prefixes can be added/removed, and the list of prefixes to be avoided for custom configuration properties. However, as noted above, users should avoid using prefixes used by Hadoop: hadoop, io, ipc, fs, net, file, ftp, s3, kfs, ha, file, dfs, mapred, mapreduce, yarn.
+All Log4j configurations SHALL be considered
+[Public](./InterfaceClassification.html#Public) and
+[Evolving](./InterfaceClassification.html#Evolving).
 
 ### Directory Structure
 
-Source code, artifacts (source and tests), user logs, configuration files, output and job history are all stored on disk either local file system or HDFS. Changing the directory structure of these user-accessible files break compatibility, even in cases where the original path is preserved via symbolic links (if, for example, the path is accessed by a servlet that is configured to not follow symbolic links).
+Source code, artifacts (source and tests), user logs, configuration files,
+output, and job history are all stored on disk either local file system or HDFS.
+Changing the directory structure of these user-accessible files can break
+compatibility, even in cases where the original path is preserved via symbolic
+links (such as when the path is accessed by a servlet that is configured to
+not follow symbolic links).
 
 #### Policy
 
-* The layout of source code and build artifacts can change anytime, particularly so across major versions. Within a major version, the developers will attempt (no guarantees) to preserve the directory structure; however, individual files can be added/moved/deleted. The best way to ensure patches stay in sync with the code is to get them committed to the Apache source tree.
-* The directory structure of configuration files, user logs, and job history will be preserved across minor and point releases within a major release.
+The layout of source code and build artifacts SHALL be considered
+[Private](./InterfaceClassification.html#Private) and
+[Unstable](./InterfaceClassification.html#Unstable). Within a major version,
+the developer community SHOULD preserve the
+overall directory structure, though individual files MAY be added, moved, or
+deleted with no warning.
+
+The directory structure of configuration files, user logs, and job history SHALL
+be considered [Public](./InterfaceClassification.html#Public) and
+[Evolving](./InterfaceClassification.html#Evolving).
 
 ### Java Classpath
 
-User applications built against Hadoop might add all Hadoop jars (including Hadoop's library dependencies) to the application's classpath. Adding new dependencies or updating the version of existing dependencies may interfere with those in applications' classpaths.
+Hadoop provides several client artifacts that applications use to interact
+with the system. These artifacts typically have their own dependencies on
+common libraries. In the cases where these dependencies are exposed to
+end user applications or downstream consumers (i.e. not
+[shaded](https://stackoverflow.com/questions/13620281/what-is-the-maven-shade-plugin-used-for-and-why-would-you-want-to-relocate-java))
+changes to these dependencies can be disruptive. Developers are strongly
+encouraged to avoid exposing dependencies to clients by using techniques
+such as
+[shading](https://stackoverflow.com/questions/13620281/what-is-the-maven-shade-plugin-used-for-and-why-would-you-want-to-relocate-java).
+
+With regard to dependencies, adding a dependency is an incompatible change,
+whereas removing a dependency is a compatible change.
+
+Some user applications built against Hadoop may add all Hadoop JAR files
+(including Hadoop's library dependencies) to the application's classpath.
+Adding new dependencies or updating the versions of existing dependencies may
+interfere with those in applications' classpaths and hence their correct
+operation. Users are therefore discouraged from adopting this practice.
 
 #### Policy
 
-Currently, there is NO policy on when Hadoop's dependencies can change.
+The set of dependencies exposed by the Hadoop client artifacts SHALL be
+considered [Public](./InterfaceClassification.html#Public) and
+[Stable](./InterfaceClassification.html#Stable). Any dependencies that are not
+exposed to clients (either because they are shaded or only exist in non-client
+artifacts) SHALL be considered [Private](./InterfaceClassification.html#Private)
+and [Unstable](./InterfaceClassification.html#Unstable)
 
 ### Environment variables
 
-Users and related projects often utilize the exported environment variables (eg HADOOP\_CONF\_DIR), therefore removing or renaming environment variables is an incompatible change.
+Users and related projects often utilize the environment variables exported by
+Hadoop (e.g. HADOOP\_CONF\_DIR). Removing or renaming environment variables can
+therefore impact end user applications.
 
 #### Policy
 
-Currently, there is NO policy on when the environment variables can change. Developers try to limit changes to major releases.
+The environment variables consumed by Hadoop and the environment variables made
+accessible to applications through YARN SHALL be considered
+[Public](./InterfaceClassification.html#Public) and
+[Evolving](./InterfaceClassification.html#Evolving).
+The developer community SHOULD limit changes to major releases.
 
 ### Build artifacts
 
-Hadoop uses maven for project management and changing the artifacts can affect existing user workflows.
+Hadoop uses Maven for project management. Changes to the contents of
+generated artifacts can impact existing user applications.
 
 #### Policy
 
-* Test artifacts: The test jars generated are strictly for internal use and are not expected to be used outside of Hadoop, similar to APIs annotated @Private, @Unstable.
-* Built artifacts: The hadoop-client artifact (maven groupId:artifactId) stays compatible within a major release, while the other artifacts can change in incompatible ways.
+The contents of Hadoop test artifacts SHALL be considered
+[Private](./InterfaceClassification.html#Private) and
+[Unstable](./InterfaceClassification.html#Unstable). Test artifacts include
+all JAR files generated from test source code and all JAR files that include
+"tests" in the file name.
+
+The Hadoop client artifacts SHALL be considered
+[Public](./InterfaceClassification.html#Public) and
+[Stable](./InterfaceClassification.html#Stable). Client artifacts are the
+following:
+
+* hadoop-client
+* hadoop-client-api
+* hadoop-client-minicluster
+* hadoop-client-runtime
+* hadoop-hdfs-client
+* hadoop-hdfs-native-client
+* hadoop-mapreduce-client-app
+* hadoop-mapreduce-client-common
+* hadoop-mapreduce-client-core
+* hadoop-mapreduce-client-hs
+* hadoop-mapreduce-client-hs-plugins
+* hadoop-mapreduce-client-jobclient
+* hadoop-mapreduce-client-nativetask
+* hadoop-mapreduce-client-shuffle
+* hadoop-yarn-client
+
+All other build artifacts SHALL be considered
+[Private](./InterfaceClassification.html#Private) and
+[Unstable](./InterfaceClassification.html#Unstable).
 
 ### Hardware/Software Requirements
 
-To keep up with the latest advances in hardware, operating systems, JVMs, and other software, new Hadoop releases or some of their features might require higher versions of the same. For a specific environment, upgrading Hadoop might require upgrading other dependent software components.
+To keep up with the latest advances in hardware, operating systems, JVMs, and
+other software, new Hadoop releases may include features that require
+newer hardware, operating systems releases, or JVM versions than previous
+Hadoop releases. For a specific environment, upgrading Hadoop might require
+upgrading other dependent software components.
 
 #### Policies
 
 * Hardware
     * Architecture: The community has no plans to restrict Hadoop to specific architectures, but can have family-specific optimizations.
-    * Minimum resources: While there are no guarantees on the minimum resources required by Hadoop daemons, the community attempts to not increase requirements within a minor release.
-* Operating Systems: The community will attempt to maintain the same OS requirements (OS kernel versions) within a minor release. Currently GNU/Linux and Microsoft Windows are the OSes officially supported by the community while Apache Hadoop is known to work reasonably well on other OSes such as Apple MacOSX, Solaris etc.
-* The JVM requirements will not change across point releases within the same minor release except if the JVM version under question becomes unsupported. Minor/major releases might require later versions of JVM for some/all of the supported operating systems.
-* Other software: The community tries to maintain the minimum versions of additional software required by Hadoop. For example, ssh, kerberos etc.
+    * Minimum resources: While there are no guarantees on the minimum resources required by Hadoop daemons, the developer community SHOULD avoid increasing requirements within a minor release.
+* Operating Systems: The community SHOULD maintain the same minimum OS requirements (OS kernel versions) within a minor release. Currently GNU/Linux and Microsoft Windows are the OSes officially supported by the community, while Apache Hadoop is known to work reasonably well on other OSes such as Apple MacOSX, Solaris, etc.
+* The JVM requirements SHALL NOT change across minor releases within the same major release unless the JVM version in question becomes unsupported. The JVM version requirement MAY be different for different operating systems or even operating system releases.
+* File systems supported by Hadoop, e.g. through the HDFS FileSystem API, SHOULD not become unsupported between minor releases within a major version unless a migration path to an alternate client implementation is available.
 
 References
 ----------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7618fa91/hadoop-common-project/hadoop-common/src/site/markdown/InterfaceClassification.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/InterfaceClassification.md b/hadoop-common-project/hadoop-common/src/site/markdown/InterfaceClassification.md
index c7309ab..451f9be 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/InterfaceClassification.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/InterfaceClassification.md
@@ -66,54 +66,103 @@ Hadoop uses the following kinds of audience in order of increasing/wider visibil
 
 #### Private
 
-The interface is for internal use within the project (such as HDFS or MapReduce)
-and should not be used by applications or by other projects. It is subject to
-change at anytime without notice. Most interfaces of a project are Private (also
-referred to as project-private).
+A Private interface is for internal use within the project (such as HDFS or
+MapReduce) and should not be used by applications or by other projects. Most
+interfaces of a project are Private (also referred to as project-private).
+Unless an interface is intentionally exposed for external consumption, it should
+be marked Private.
 
 #### Limited-Private
 
-The interface is used by a specified set of projects or systems (typically
-closely related projects). Other projects or systems should not use the
-interface. Changes to the interface will be communicated/negotiated with the
+A Limited-Private interface is used by a specified set of projects or systems
+(typically closely related projects). Other projects or systems should not use
+the interface. Changes to the interface will be communicated/negotiated with the
 specified projects. For example, in the Hadoop project, some interfaces are
 LimitedPrivate{HDFS, MapReduce} in that they are private to the HDFS and
 MapReduce projects.
 
 #### Public
 
-The interface is for general use by any application.
+A Public interface is for general use by any application.
+
+### Change Compatibility
+
+Changes to an API fall into two broad categories: compatible and incompatible.
+A compatible change is a change that meets the following criteria:
+
+* no existing capabilities are removed,
+* no existing capabilities are modified in a way that prevents their use by clients that were constructed to use the interface prior to the change, and
+* no capabilities are added that require changes to clients that were constructed to use the interface prior to the change.
+
+Any change that does not meet these three criteria is an incompatible change.
+Stated simply a compatible change will not break existing clients.  These
+examples are compatible changes:
+
+* adding a method to a Java class,
+* adding an optional parameter to a RESTful web service, or
+* adding a tag to an XML document.
+* making the audience annotation of an interface more broad (e.g. from Private to Public) or the change compatibility annotation more restrictive (e.g. from Evolving to Stable)
+
+These examples are incompatible changes:
+
+* removing a method from a Java class,
+* adding a method to a Java interface,
+* adding a required parameter to a RESTful web service, or
+* renaming a field in a JSON document.
+* making the audience annotation of an interface less broad (e.g. from Public to Limited Private) or the change compatibility annotation more restrictive (e.g. from Evolving to Unstable)
 
 ### Stability
 
-Stability denotes how stable an interface is, as in when incompatible changes to
-the interface are allowed. Hadoop APIs have the following levels of stability.
+Stability denotes how stable an interface is and when compatible and
+incompatible changes to the interface are allowed. Hadoop APIs have the
+following levels of stability.
 
 #### Stable
 
-Can evolve while retaining compatibility for minor release boundaries; in other
-words, incompatible changes to APIs marked as Stable are allowed only at major
-releases (i.e. at m.0).
+A Stable interface is exposed as a preferred means of communication. A Stable
+interface is expected not to change incompatibly within a major release and
+hence serves as a safe development target. A Stable interface may evolve
+compatibly between minor releases.
+
+Incompatible changes allowed: major (X.0.0)
+Compatible changes allowed: maintenance (x.Y.0)
 
 #### Evolving
 
-Evolving, but incompatible changes are allowed at minor releases (i.e. m .x)
+An Evolving interface is typically exposed so that users or external code can
+make use of a feature before it has stabilized. The expectation that an
+interface should "eventually" stabilize and be promoted to Stable, however,
+is not a requirement for the interface to be labeled as Evolving.
+
+Incompatible changes are allowed for Evolving interface only at minor releases.
+
+Incompatible changes allowed: minor (x.Y.0)
+Compatible changes allowed: maintenance (x.y.Z)
 
 #### Unstable
 
-Incompatible changes to Unstable APIs are allowed at any time. This usually makes
-sense for only private interfaces.
+An Unstable interface is one for which no compatibility guarantees are made. An
+Unstable interface is not necessarily unstable. An unstable interface is
+typically exposed because a user or external code needs to access an interface
+that is not intended for consumption. The interface is exposed as an Unstable
+interface to state clearly that even though the interface is exposed, it is not
+the preferred access path, and no compatibility guarantees are made for it.
 
-However one may call this out for a supposedly public interface to highlight
-that it should not be used as an interface; for public interfaces, labeling it
-as Not-an-interface is probably more appropriate than "Unstable".
+Unless there is a reason to offer a compatibility guarantee on an interface,
+whether it is exposed or not, it should be labeled as Unstable.  Private
+interfaces also should be Unstable in most cases.
 
-Examples of publicly visible interfaces that are unstable
-(i.e. not-an-interface): GUI, CLIs whose output format will change.
+Incompatible changes to Unstable interfaces are allowed at any time.
+
+Incompatible changes allowed: maintenance (x.y.Z)
+Compatible changes allowed: maintenance (x.y.Z)
 
 #### Deprecated
 
-APIs that could potentially be removed in the future and should not be used.
+A Deprecated interface could potentially be removed in the future and should
+not be used.  Even so, a Deprecated interface will continue to function until
+it is removed.  When a Deprecated interface can be removed depends on whether
+it is also Stable, Evolving, or Unstable.
 
 How are the Classifications Recorded?
 -------------------------------------
@@ -121,95 +170,101 @@ How are the Classifications Recorded?
 How will the classification be recorded for Hadoop APIs?
 
 * Each interface or class will have the audience and stability recorded using
-  annotations in org.apache.hadoop.classification package.
+  annotations in the org.apache.hadoop.classification package.
 
-* The javadoc generated by the maven target javadoc:javadoc lists only the public API.
+* The javadoc generated by the maven target javadoc:javadoc lists only the
+  public API.
 
 * One can derive the audience of java classes and java interfaces by the
   audience of the package in which they are contained. Hence it is useful to
   declare the audience of each java package as public or private (along with the
   private audience variations).
 
+How will the classification be recorded for other interfaces, such as CLIs?
+
+* See the [Hadoop Compatibility](Compatibility.html) page for details.
+
 FAQ
 ---
 
 * Why aren’t the java scopes (private, package private and public) good enough?
     * Java’s scoping is not very complete. One is often forced to make a class
-      public in order for other internal components to use it. It does not have
-      friends or sub-package-private like C++.
-
-* But I can easily access a private implementation interface if it is Java public.
-  Where is the protection and control?
-    * The purpose of this is not providing absolute access control. Its purpose
-      is to communicate to users and developers. One can access private
-      implementation functions in libc; however if they change the internal
-      implementation details, your application will break and you will have
-      little sympathy from the folks who are supplying libc. If you use a
-      non-public interface you understand the risks.
-
-* Why bother declaring the stability of a private interface?
-  Aren’t private interfaces always unstable?
-    * Private interfaces are not always unstable. In the cases where they are
-      stable they capture internal properties of the system and can communicate
+      public in order for other internal components to use it. It also does not
+      have friends or sub-package-private like C++.
+
+* But I can easily access a Private interface if it is Java public. Where is the
+  protection and control?
+    * The purpose of this classification scheme is not providing absolute
+      access control. Its purpose is to communicate to users and developers.
+      One can access private implementation functions in libc; however if
+      they change the internal implementation details, the application will
+      break and one will receive little sympathy from the folks who are
+      supplying libc. When using a non-public interface, the risks are
+      understood.
+
+* Why bother declaring the stability of a Private interface? Aren’t Private
+  interfaces always Unstable?
+    * Private interfaces are not always Unstable. In the cases where they are
+      Stable they capture internal properties of the system and can communicate
       these properties to its internal users and to developers of the interface.
-        * e.g. In HDFS, NN-DN protocol is private but stable and can help
-          implement rolling upgrades. It communicates that this interface should
-          not be changed in incompatible ways even though it is private.
-        * e.g. In HDFS, FSImage stability provides more flexible rollback.
-
-* What is the harm in applications using a private interface that is stable? How
-  is it different than a public stable interface?
-    * While a private interface marked as stable is targeted to change only at
+        * e.g. In HDFS, NN-DN protocol is Private but Stable and can help
+          implement rolling upgrades. The stability annotation communicates that
+          this interface should not be changed in incompatible ways even though
+          it is Private.
+        * e.g. In HDFS, FSImage the Stabile designation provides more flexible
+          rollback.
+
+* What is the harm in applications using a Private interface that is Stable?
+  How is it different from a Public Stable interface?
+    * While a Private interface marked as Stable is targeted to change only at
       major releases, it may break at other times if the providers of that
-      interface are willing to change the internal users of that
-      interface. Further, a public stable interface is less likely to break even
+      interface also are willing to change the internal consumers of that
+      interface. Further, a Public Stable interface is less likely to break even
       at major releases (even though it is allowed to break compatibility)
-      because the impact of the change is larger. If you use a private interface
+      because the impact of the change is larger. If you use a Private interface
       (regardless of its stability) you run the risk of incompatibility.
 
-* Why bother with Limited-private? Isn’t it giving special treatment to some projects?
-  That is not fair.
-    * First, most interfaces should be public or private; actually let us state
-      it even stronger: make it private unless you really want to expose it to
-      public for general use.
-    * Limited-private is for interfaces that are not intended for general
+* Why bother with Limited-Private? Isn’t it giving special treatment to some
+  projects? That is not fair.
+    * Most interfaces should be Public or Private. An interface should be
+      Private unless it is explicitly intended for general use.
+    * Limited-Private is for interfaces that are not intended for general
       use. They are exposed to related projects that need special hooks. Such a
-      classification has a cost to both the supplier and consumer of the limited
+      classification has a cost to both the supplier and consumer of the
       interface. Both will have to work together if ever there is a need to
       break the interface in the future; for example the supplier and the
       consumers will have to work together to get coordinated releases of their
-      respective projects. This should not be taken lightly – if you can get
-      away with private then do so; if the interface is really for general use
-      for all applications then do so. But remember that making an interface
-      public has huge responsibility. Sometimes Limited-private is just right.
-    * A good example of a limited-private interface is BlockLocations, This is a
-      fairly low-level interface that we are willing to expose to MR and perhaps
-      HBase. We are likely to change it down the road and at that time we will
-      coordinate release effort with the MR team.
-      While MR and HDFS are always released in sync today, they may
-      change down the road.
-    * If you have a limited-private interface with many projects listed then you
-      are fooling yourself. It is practically public.
-    * It might be worth declaring a special audience classification called
-      Hadoop-Private for the Hadoop family.
-
-* Lets treat all private interfaces as Hadoop-private. What is the harm in
-  projects in the Hadoop family have access to private classes?
-    * Do we want MR accessing class files that are implementation details inside
-      HDFS. There used to be many such layer violations in the code that we have
-      been cleaning up over the last few years. We don’t want such layer
-      violations to creep back in by no separating between the major components
-      like HDFS and MR.
-
-* Aren't all public interfaces stable?
-    * One may mark a public interface as evolving in its early days. Here one is
+      respective projects. This contract should not be taken lightly–use
+      Private if possible; if the interface is really for general use
+      for all applications then use Public. Always remember that making an
+      interface Public comes with large burden of responsibility. Sometimes
+      Limited-Private is just right.
+    * A good example of a Limited-Private interface is BlockLocations. This
+      interface is a fairly low-level interface that is exposed to MapReduce
+      and HBase. The interface is likely to change down the road, and at that
+      time the release effort will have to be coordinated with the
+      MapReduce development team. While MapReduce and HDFS are always released
+      in sync today, that policy may change down the road.
+    * If you have a Limited-Private interface with many projects listed then
+      the interface is probably a good candidate to be made Public.
+
+* Let's treat all Private interfaces as Limited-Private for all of Hadoop. What
+  is the harm if projects in the Hadoop family have access to private classes?
+    * There used to be many cases in the code where one project depended on the
+      internal implementation details of another. A significant effort went
+      into cleaning up those issues. Opening up all interfaces as
+      Limited-Private for all of Hadoop would open the door to reintroducing
+      such coupling issues.
+
+* Aren't all Public interfaces Stable?
+    * One may mark a Public interface as Evolving in its early days. Here one is
       promising to make an effort to make compatible changes but may need to
       break it at minor releases.
-    * One example of a public interface that is unstable is where one is
+    * One example of a Public interface that is Unstable is where one is
       providing an implementation of a standards-body based interface that is
       still under development. For example, many companies, in an attempt to be
       first to market, have provided implementations of a new NFS protocol even
       when the protocol was not fully completed by IETF. The implementor cannot
-      evolve the interface in a fashion that causes least distruption because
+      evolve the interface in a fashion that causes least disruption because
       the stability is controlled by the standards body. Hence it is appropriate
-      to label the interface as unstable.
+      to label the interface as Unstable.


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


[37/50] [abbrv] hadoop git commit: HDFS-12447. Rename AddECPolicyResponse to AddErasureCodingPolicyResponse. Contributed by SammiChen.

Posted by jh...@apache.org.
HDFS-12447. Rename AddECPolicyResponse to AddErasureCodingPolicyResponse. Contributed by SammiChen.


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

Branch: refs/heads/YARN-5734
Commit: a12f09ba3c4a3aa4c4558090c5e1b7bcaebe3b94
Parents: ce943eb
Author: Andrew Wang <wa...@apache.org>
Authored: Wed Sep 20 11:51:17 2017 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Wed Sep 20 11:51:17 2017 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/DFSClient.java  | 10 +--
 .../hadoop/hdfs/DistributedFileSystem.java      |  4 +-
 .../apache/hadoop/hdfs/client/HdfsAdmin.java    |  4 +-
 .../hdfs/protocol/AddECPolicyResponse.java      | 68 --------------------
 .../AddErasureCodingPolicyResponse.java         | 68 ++++++++++++++++++++
 .../hadoop/hdfs/protocol/ClientProtocol.java    |  2 +-
 .../ClientNamenodeProtocolTranslatorPB.java     | 11 ++--
 .../hadoop/hdfs/protocolPB/PBHelperClient.java  | 22 ++++---
 .../src/main/proto/erasurecoding.proto          |  2 +-
 .../src/main/proto/hdfs.proto                   |  2 +-
 ...tNamenodeProtocolServerSideTranslatorPB.java | 13 ++--
 .../hdfs/server/namenode/FSNamesystem.java      | 15 +++--
 .../hdfs/server/namenode/NameNodeRpcServer.java |  9 +--
 .../org/apache/hadoop/hdfs/tools/ECAdmin.java   |  7 +-
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     |  4 +-
 .../hadoop/hdfs/TestErasureCodingPolicies.java  |  7 +-
 .../hadoop/hdfs/protocolPB/TestPBHelper.java    | 25 +++----
 .../server/namenode/TestFSEditLogLoader.java    |  4 +-
 .../hdfs/server/namenode/TestFSImage.java       |  5 +-
 19 files changed, 147 insertions(+), 135 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a12f09ba/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index 7e8e95b..8d51a9c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -102,7 +102,7 @@ import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
 import org.apache.hadoop.hdfs.client.impl.LeaseRenewer;
 import org.apache.hadoop.hdfs.net.Peer;
 import org.apache.hadoop.hdfs.protocol.AclException;
-import org.apache.hadoop.hdfs.protocol.AddECPolicyResponse;
+import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
@@ -2807,13 +2807,14 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     }
   }
 
-  public AddECPolicyResponse[] addErasureCodingPolicies(
+  public AddErasureCodingPolicyResponse[] addErasureCodingPolicies(
       ErasureCodingPolicy[] policies) throws IOException {
     checkOpen();
     try (TraceScope ignored = tracer.newScope("addErasureCodingPolicies")) {
       return namenode.addErasureCodingPolicies(policies);
     } catch (RemoteException re) {
-      throw re.unwrapRemoteException(AccessControlException.class);
+      throw re.unwrapRemoteException(AccessControlException.class,
+          SafeModeException.class);
     }
   }
 
@@ -2823,7 +2824,8 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     try (TraceScope ignored = tracer.newScope("removeErasureCodingPolicy")) {
       namenode.removeErasureCodingPolicy(ecPolicyName);
     } catch (RemoteException re) {
-      throw re.unwrapRemoteException(AccessControlException.class);
+      throw re.unwrapRemoteException(AccessControlException.class,
+          SafeModeException.class);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a12f09ba/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
index f6331cf..c9f4490 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
@@ -64,7 +64,7 @@ import org.apache.hadoop.hdfs.DFSOpsCountStatistics.OpType;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
 import org.apache.hadoop.hdfs.client.impl.CorruptFileBlockIterator;
-import org.apache.hadoop.hdfs.protocol.AddECPolicyResponse;
+import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
@@ -2650,7 +2650,7 @@ public class DistributedFileSystem extends FileSystem {
    * @return Return the response list of adding operations.
    * @throws IOException
    */
-  public AddECPolicyResponse[] addErasureCodingPolicies(
+  public AddErasureCodingPolicyResponse[] addErasureCodingPolicies(
       ErasureCodingPolicy[] policies)  throws IOException {
     return dfs.addErasureCodingPolicies(policies);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a12f09ba/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java
index bfc6010..8dc5810 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java
@@ -35,7 +35,7 @@ import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSInotifyEventInputStream;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
-import org.apache.hadoop.hdfs.protocol.AddECPolicyResponse;
+import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
@@ -562,7 +562,7 @@ public class HdfsAdmin {
    * @return Return the response list of adding operations.
    * @throws IOException
    */
-  public AddECPolicyResponse[] addErasureCodingPolicies(
+  public AddErasureCodingPolicyResponse[] addErasureCodingPolicies(
       ErasureCodingPolicy[] policies)  throws IOException {
     return dfs.addErasureCodingPolicies(policies);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a12f09ba/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/AddECPolicyResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/AddECPolicyResponse.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/AddECPolicyResponse.java
deleted file mode 100644
index 4e55680..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/AddECPolicyResponse.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.protocol;
-
-import org.apache.hadoop.HadoopIllegalArgumentException;
-
-/**
- * A response of add an ErasureCoding policy.
- */
-public class AddECPolicyResponse {
-  private boolean succeed;
-  private ErasureCodingPolicy policy;
-  private String errorMsg;
-
-  public AddECPolicyResponse(ErasureCodingPolicy policy) {
-    this.policy = policy;
-    this.succeed = true;
-  }
-
-  public AddECPolicyResponse(ErasureCodingPolicy policy,
-      String errorMsg) {
-    this.policy = policy;
-    this.errorMsg = errorMsg;
-    this.succeed = false;
-  }
-
-  public AddECPolicyResponse(ErasureCodingPolicy policy,
-      HadoopIllegalArgumentException e) {
-    this(policy, e.getMessage());
-  }
-
-  public boolean isSucceed() {
-    return succeed;
-  }
-
-  public ErasureCodingPolicy getPolicy() {
-    return policy;
-  }
-
-  public String getErrorMsg() {
-    return errorMsg;
-  }
-
-  @Override
-  public String toString() {
-    if (isSucceed()) {
-      return "Add ErasureCodingPolicy " + getPolicy().getName() + " succeed.";
-    } else {
-      return "Add ErasureCodingPolicy " + getPolicy().getName() + " failed and "
-          + "error message is " + getErrorMsg();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a12f09ba/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/AddErasureCodingPolicyResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/AddErasureCodingPolicyResponse.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/AddErasureCodingPolicyResponse.java
new file mode 100644
index 0000000..2e8d081
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/AddErasureCodingPolicyResponse.java
@@ -0,0 +1,68 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.protocol;
+
+import org.apache.hadoop.HadoopIllegalArgumentException;
+
+/**
+ * A response of add an ErasureCoding policy.
+ */
+public class AddErasureCodingPolicyResponse {
+  private boolean succeed;
+  private ErasureCodingPolicy policy;
+  private String errorMsg;
+
+  public AddErasureCodingPolicyResponse(ErasureCodingPolicy policy) {
+    this.policy = policy;
+    this.succeed = true;
+  }
+
+  public AddErasureCodingPolicyResponse(ErasureCodingPolicy policy,
+                                        String errorMsg) {
+    this.policy = policy;
+    this.errorMsg = errorMsg;
+    this.succeed = false;
+  }
+
+  public AddErasureCodingPolicyResponse(ErasureCodingPolicy policy,
+                                        HadoopIllegalArgumentException e) {
+    this(policy, e.getMessage());
+  }
+
+  public boolean isSucceed() {
+    return succeed;
+  }
+
+  public ErasureCodingPolicy getPolicy() {
+    return policy;
+  }
+
+  public String getErrorMsg() {
+    return errorMsg;
+  }
+
+  @Override
+  public String toString() {
+    if (isSucceed()) {
+      return "Add ErasureCodingPolicy " + getPolicy().getName() + " succeed.";
+    } else {
+      return "Add ErasureCodingPolicy " + getPolicy().getName() + " failed and "
+          + "error message is " + getErrorMsg();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a12f09ba/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
index 8d5503f..25f63bd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
@@ -1584,7 +1584,7 @@ public interface ClientProtocol {
    * @throws IOException
    */
   @AtMostOnce
-  AddECPolicyResponse[] addErasureCodingPolicies(
+  AddErasureCodingPolicyResponse[] addErasureCodingPolicies(
       ErasureCodingPolicy[] policies) throws IOException;
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a12f09ba/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
index 209eee7..0b443c6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
@@ -50,7 +50,7 @@ import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.AddBlockFlag;
 import org.apache.hadoop.hdfs.inotify.EventBatchList;
-import org.apache.hadoop.hdfs.protocol.AddECPolicyResponse;
+import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
@@ -1718,7 +1718,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
 
   @Override
-  public AddECPolicyResponse[] addErasureCodingPolicies(
+  public AddErasureCodingPolicyResponse[] addErasureCodingPolicies(
       ErasureCodingPolicy[] policies) throws IOException {
     List<ErasureCodingPolicyProto> protos = Arrays.stream(policies)
         .map(PBHelperClient::convertErasureCodingPolicy)
@@ -1729,9 +1729,10 @@ public class ClientNamenodeProtocolTranslatorPB implements
     try {
       AddErasureCodingPoliciesResponseProto rep = rpcProxy
           .addErasureCodingPolicies(null, req);
-      AddECPolicyResponse[] responses = rep.getResponsesList().stream()
-          .map(PBHelperClient::convertAddECPolicyResponse)
-          .toArray(AddECPolicyResponse[]::new);
+      AddErasureCodingPolicyResponse[] responses =
+          rep.getResponsesList().stream()
+              .map(PBHelperClient::convertAddErasureCodingPolicyResponse)
+              .toArray(AddErasureCodingPolicyResponse[]::new);
       return responses;
     }  catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a12f09ba/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
index d92d91e..db97031 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
@@ -58,7 +58,7 @@ import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.inotify.Event;
 import org.apache.hadoop.hdfs.inotify.EventBatch;
 import org.apache.hadoop.hdfs.inotify.EventBatchList;
-import org.apache.hadoop.hdfs.protocol.AddECPolicyResponse;
+import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.BlockType;
@@ -137,7 +137,7 @@ import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ReencryptionS
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ZoneReencryptionStatusProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.AccessModeProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.AddECPolicyResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.AddErasureCodingPolicyResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockStoragePolicyProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTypeProto;
@@ -2981,10 +2981,11 @@ public class PBHelperClient {
     return builder.build();
   }
 
-  public static AddECPolicyResponseProto convertAddECPolicyResponse(
-      AddECPolicyResponse response) {
-    AddECPolicyResponseProto.Builder builder =
-        AddECPolicyResponseProto.newBuilder()
+  public static AddErasureCodingPolicyResponseProto
+      convertAddErasureCodingPolicyResponse(
+          AddErasureCodingPolicyResponse response) {
+    AddErasureCodingPolicyResponseProto.Builder builder =
+        AddErasureCodingPolicyResponseProto.newBuilder()
         .setPolicy(convertErasureCodingPolicy(response.getPolicy()))
         .setSucceed(response.isSucceed());
     if (!response.isSucceed()) {
@@ -2993,13 +2994,14 @@ public class PBHelperClient {
     return builder.build();
   }
 
-  public static AddECPolicyResponse convertAddECPolicyResponse(
-      AddECPolicyResponseProto proto) {
+  public static AddErasureCodingPolicyResponse
+      convertAddErasureCodingPolicyResponse(
+          AddErasureCodingPolicyResponseProto proto) {
     ErasureCodingPolicy policy = convertErasureCodingPolicy(proto.getPolicy());
     if (proto.getSucceed()) {
-      return new AddECPolicyResponse(policy);
+      return new AddErasureCodingPolicyResponse(policy);
     } else {
-      return new AddECPolicyResponse(policy, proto.getErrorMsg());
+      return new AddErasureCodingPolicyResponse(policy, proto.getErrorMsg());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a12f09ba/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/erasurecoding.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/erasurecoding.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/erasurecoding.proto
index 9f80350..5f5f66e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/erasurecoding.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/erasurecoding.proto
@@ -58,7 +58,7 @@ message AddErasureCodingPoliciesRequestProto {
 }
 
 message AddErasureCodingPoliciesResponseProto {
-  repeated AddECPolicyResponseProto responses = 1;
+  repeated AddErasureCodingPolicyResponseProto responses = 1;
 }
 
 message RemoveErasureCodingPolicyRequestProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a12f09ba/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
index ddb5566..7769f68 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
@@ -390,7 +390,7 @@ message ErasureCodingPolicyProto {
   optional ErasureCodingPolicyState state = 5 [default = ENABLED];
 }
 
-message AddECPolicyResponseProto {
+message AddErasureCodingPolicyResponseProto {
   required ErasureCodingPolicyProto policy = 1;
   required bool succeed = 2;
   optional string errorMsg = 3;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a12f09ba/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
index a79e75f..0527b3b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
@@ -36,7 +36,7 @@ import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.fs.QuotaUsage;
-import org.apache.hadoop.hdfs.protocol.AddECPolicyResponse;
+import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
@@ -1721,15 +1721,16 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       ErasureCodingPolicy[] policies = request.getEcPoliciesList().stream()
           .map(PBHelperClient::convertErasureCodingPolicy)
           .toArray(ErasureCodingPolicy[]::new);
-      AddECPolicyResponse[] result = server
+      AddErasureCodingPolicyResponse[] result = server
           .addErasureCodingPolicies(policies);
 
-      List<HdfsProtos.AddECPolicyResponseProto> responseProtos = Arrays
-          .stream(result).map(PBHelperClient::convertAddECPolicyResponse)
-          .collect(Collectors.toList());
+      List<HdfsProtos.AddErasureCodingPolicyResponseProto> responseProtos =
+          Arrays.stream(result)
+              .map(PBHelperClient::convertAddErasureCodingPolicyResponse)
+              .collect(Collectors.toList());
       AddErasureCodingPoliciesResponseProto response =
           AddErasureCodingPoliciesResponseProto.newBuilder()
-            .addAllResponses(responseProtos).build();
+              .addAllResponses(responseProtos).build();
       return response;
     } catch (IOException e) {
       throw new ServiceException(e);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a12f09ba/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 029557f..967f26b 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
@@ -180,7 +180,7 @@ import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.UnknownCryptoProtocolVersionException;
-import org.apache.hadoop.hdfs.protocol.AddECPolicyResponse;
+import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse;
 import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockType;
@@ -7193,12 +7193,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    *                      rebuilding
    * @return The according result of add operation.
    */
-  AddECPolicyResponse[] addErasureCodingPolicies(ErasureCodingPolicy[] policies,
-      final boolean logRetryCache) throws IOException {
+  AddErasureCodingPolicyResponse[] addErasureCodingPolicies(
+      ErasureCodingPolicy[] policies, final boolean logRetryCache)
+      throws IOException {
     final String operationName = "addErasureCodingPolicies";
     String addECPolicyName = "";
     checkOperation(OperationCategory.WRITE);
-    List<AddECPolicyResponse> responses = new ArrayList<>();
+    List<AddErasureCodingPolicyResponse> responses = new ArrayList<>();
     boolean success = false;
     writeLock();
     try {
@@ -7210,13 +7211,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
               FSDirErasureCodingOp.addErasureCodingPolicy(this, policy,
                   logRetryCache);
           addECPolicyName = newPolicy.getName();
-          responses.add(new AddECPolicyResponse(newPolicy));
+          responses.add(new AddErasureCodingPolicyResponse(newPolicy));
         } catch (HadoopIllegalArgumentException e) {
-          responses.add(new AddECPolicyResponse(policy, e));
+          responses.add(new AddErasureCodingPolicyResponse(policy, e));
         }
       }
       success = true;
-      return responses.toArray(new AddECPolicyResponse[0]);
+      return responses.toArray(new AddErasureCodingPolicyResponse[0]);
     } finally {
       writeUnlock(operationName);
       if (success) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a12f09ba/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 d62c0f2..c07b521 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
@@ -85,7 +85,7 @@ import org.apache.hadoop.hdfs.HDFSPolicyProvider;
 import org.apache.hadoop.hdfs.inotify.EventBatch;
 import org.apache.hadoop.hdfs.inotify.EventBatchList;
 import org.apache.hadoop.hdfs.protocol.AclException;
-import org.apache.hadoop.hdfs.protocol.AddECPolicyResponse;
+import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse;
 import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
@@ -2333,17 +2333,18 @@ public class NameNodeRpcServer implements NamenodeProtocols {
   }
 
   @Override
-  public AddECPolicyResponse[] addErasureCodingPolicies(
+  public AddErasureCodingPolicyResponse[] addErasureCodingPolicies(
       ErasureCodingPolicy[] policies) throws IOException {
     checkNNStartup();
     namesystem.checkSuperuserPrivilege();
     final CacheEntryWithPayload cacheEntry =
         RetryCache.waitForCompletion(retryCache, null);
     if (cacheEntry != null && cacheEntry.isSuccess()) {
-      return (AddECPolicyResponse[]) cacheEntry.getPayload();
+      return (AddErasureCodingPolicyResponse[]) cacheEntry.getPayload();
     }
     boolean success = false;
-    AddECPolicyResponse[] responses = new AddECPolicyResponse[0];
+    AddErasureCodingPolicyResponse[] responses =
+        new AddErasureCodingPolicyResponse[0];
     try {
       responses =
           namesystem.addErasureCodingPolicies(policies, cacheEntry != null);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a12f09ba/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/ECAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/ECAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/ECAdmin.java
index 0b4e0c2..652f78a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/ECAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/ECAdmin.java
@@ -21,7 +21,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
-import org.apache.hadoop.hdfs.protocol.AddECPolicyResponse;
+import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.util.ECPolicyLoader;
 import org.apache.hadoop.io.erasurecode.ErasureCodeConstants;
@@ -175,9 +175,10 @@ public class ECAdmin extends Configured implements Tool {
         List<ErasureCodingPolicy> policies =
             new ECPolicyLoader().loadPolicy(filePath);
         if (policies.size() > 0) {
-          AddECPolicyResponse[] responses = dfs.addErasureCodingPolicies(
+          AddErasureCodingPolicyResponse[] responses =
+              dfs.addErasureCodingPolicies(
             policies.toArray(new ErasureCodingPolicy[policies.size()]));
-          for (AddECPolicyResponse response : responses) {
+          for (AddErasureCodingPolicyResponse response : responses) {
             System.out.println(response);
           }
         } else {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a12f09ba/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
index c6fe1a2..17db352 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
@@ -108,7 +108,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.MiniDFSCluster.NameNodeInfo;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
-import org.apache.hadoop.hdfs.protocol.AddECPolicyResponse;
+import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
@@ -1472,7 +1472,7 @@ public class DFSTestUtil {
     ErasureCodingPolicy newPolicy1 =
         new ErasureCodingPolicy(ErasureCodeConstants.RS_3_2_SCHEMA, 8 * 1024);
     ErasureCodingPolicy[] policyArray = new ErasureCodingPolicy[] {newPolicy1};
-    AddECPolicyResponse[] responses =
+    AddErasureCodingPolicyResponse[] responses =
         filesystem.addErasureCodingPolicies(policyArray);
     newPolicy1 = responses[0].getPolicy();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a12f09ba/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
index 4f2040b..2c2b05e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
@@ -23,7 +23,7 @@ import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.protocol.AddECPolicyResponse;
+import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@@ -686,7 +686,7 @@ public class TestErasureCodingPolicies {
     ErasureCodingPolicy newPolicy =
         new ErasureCodingPolicy(toAddSchema, 128 * 1024);
     ErasureCodingPolicy[] policyArray = new ErasureCodingPolicy[]{newPolicy};
-    AddECPolicyResponse[] responses =
+    AddErasureCodingPolicyResponse[] responses =
         fs.addErasureCodingPolicies(policyArray);
     assertEquals(1, responses.length);
     assertFalse(responses[0].isSucceed());
@@ -839,7 +839,8 @@ public class TestErasureCodingPolicies {
         new ErasureCodingPolicy(ErasureCodeConstants.RS_3_2_SCHEMA, 8 * 1024);
     ErasureCodingPolicy[] policyArray =
         new ErasureCodingPolicy[] {newPolicy1};
-    AddECPolicyResponse[] responses = fs.addErasureCodingPolicies(policyArray);
+    AddErasureCodingPolicyResponse[] responses =
+        fs.addErasureCodingPolicies(policyArray);
     assertEquals(1, responses.length);
     assertTrue(responses[0].isSucceed());
     newPolicy1 = responses[0].getPolicy();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a12f09ba/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java
index 3559825..da56c15 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java
@@ -19,7 +19,7 @@ package org.apache.hadoop.hdfs.protocolPB;
 
 
 import com.google.protobuf.UninitializedMessageException;
-import org.apache.hadoop.hdfs.protocol.AddECPolicyResponse;
+import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse;
 import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
 import org.apache.hadoop.hdfs.server.protocol.SlowDiskReports;
 
@@ -913,14 +913,15 @@ public class TestPBHelper {
     // Check conversion of the built-in policies.
     for (ErasureCodingPolicy policy :
         SystemErasureCodingPolicies.getPolicies()) {
-      AddECPolicyResponse response = new AddECPolicyResponse(policy);
-      HdfsProtos.AddECPolicyResponseProto proto = PBHelperClient
-          .convertAddECPolicyResponse(response);
+      AddErasureCodingPolicyResponse response =
+          new AddErasureCodingPolicyResponse(policy);
+      HdfsProtos.AddErasureCodingPolicyResponseProto proto = PBHelperClient
+          .convertAddErasureCodingPolicyResponse(response);
       // Optional fields should not be set.
       assertFalse("Unnecessary field is set.", proto.hasErrorMsg());
       // Convert proto back to an object and check for equality.
-      AddECPolicyResponse convertedResponse = PBHelperClient
-          .convertAddECPolicyResponse(proto);
+      AddErasureCodingPolicyResponse convertedResponse = PBHelperClient
+          .convertAddErasureCodingPolicyResponse(proto);
       assertEquals("Converted policy not equal", response.getPolicy(),
           convertedResponse.getPolicy());
       assertEquals("Converted policy not equal", response.isSucceed(),
@@ -929,13 +930,13 @@ public class TestPBHelper {
 
     ErasureCodingPolicy policy = SystemErasureCodingPolicies
         .getPolicies().get(0);
-    AddECPolicyResponse response =
-        new AddECPolicyResponse(policy, "failed");
-    HdfsProtos.AddECPolicyResponseProto proto = PBHelperClient
-        .convertAddECPolicyResponse(response);
+    AddErasureCodingPolicyResponse response =
+        new AddErasureCodingPolicyResponse(policy, "failed");
+    HdfsProtos.AddErasureCodingPolicyResponseProto proto = PBHelperClient
+        .convertAddErasureCodingPolicyResponse(response);
     // Convert proto back to an object and check for equality.
-    AddECPolicyResponse convertedResponse = PBHelperClient
-        .convertAddECPolicyResponse(proto);
+    AddErasureCodingPolicyResponse convertedResponse = PBHelperClient
+        .convertAddErasureCodingPolicyResponse(proto);
     assertEquals("Converted policy not equal", response.getPolicy(),
         convertedResponse.getPolicy());
     assertEquals("Converted policy not equal", response.getErrorMsg(),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a12f09ba/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
index ec80bff..14d2e26 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
@@ -47,7 +47,7 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.StripedFileTestUtil;
-import org.apache.hadoop.hdfs.protocol.AddECPolicyResponse;
+import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicyState;
@@ -735,7 +735,7 @@ public class TestFSEditLogLoader {
       ErasureCodingPolicy newPolicy =
           new ErasureCodingPolicy(schema, cellSize, (byte) 0);
       ErasureCodingPolicy[] policyArray = new ErasureCodingPolicy[]{newPolicy};
-      AddECPolicyResponse[] responses =
+      AddErasureCodingPolicyResponse[] responses =
           fs.addErasureCodingPolicies(policyArray);
       assertEquals(1, responses.length);
       assertTrue(responses[0].isSucceed());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a12f09ba/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
index c9d3255..d316c36 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
@@ -35,7 +35,7 @@ import java.io.IOException;
 import java.util.EnumSet;
 
 import org.apache.hadoop.hdfs.StripedFileTestUtil;
-import org.apache.hadoop.hdfs.protocol.AddECPolicyResponse;
+import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicyState;
 import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
@@ -847,7 +847,8 @@ public class TestFSImage {
       ErasureCodingPolicy newPolicy =
           new ErasureCodingPolicy(newSchema, 2 * 1024, (byte) 254);
       ErasureCodingPolicy[] policies = new ErasureCodingPolicy[]{newPolicy};
-      AddECPolicyResponse[] ret = fs.addErasureCodingPolicies(policies);
+      AddErasureCodingPolicyResponse[] ret =
+          fs.addErasureCodingPolicies(policies);
       assertEquals(1, ret.length);
       assertEquals(true, ret[0].isSucceed());
       newPolicy = ret[0].getPolicy();


---------------------------------------------------------------------
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-10701. TestDFSStripedOutputStreamWithFailure#testBlockTokenExpired occasionally fails. Contributed by SammiChen.

Posted by jh...@apache.org.
HDFS-10701. TestDFSStripedOutputStreamWithFailure#testBlockTokenExpired occasionally fails. Contributed by SammiChen.


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

Branch: refs/heads/YARN-5734
Commit: ef8cd5dc565f901b4954befe784675e130e84c3c
Parents: 1a84c24
Author: Andrew Wang <wa...@apache.org>
Authored: Fri Sep 15 16:20:36 2017 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Fri Sep 15 16:20:36 2017 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java     | 6 ++----
 1 file changed, 2 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef8cd5dc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
index ea889e3..57da439 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java
@@ -260,8 +260,6 @@ public class TestDFSStripedOutputStreamWithFailure {
 
   @Test(timeout=240000)
   public void testBlockTokenExpired() throws Exception {
-    // TODO: this is very flaky, re-enable it later. See HDFS-12417.
-    assumeTrue("Test has been temporarily disabled. See HDFS-12417.", false);
     final int length = dataBlocks * (blockSize - cellSize);
     final HdfsConfiguration conf = newHdfsConfiguration();
 
@@ -494,8 +492,8 @@ public class TestDFSStripedOutputStreamWithFailure {
       final BlockManager bm = nn.getNamesystem().getBlockManager();
       final BlockTokenSecretManager sm = bm.getBlockTokenSecretManager();
 
-      // set a short token lifetime (1 second)
-      SecurityTestUtil.setBlockTokenLifetime(sm, 1000L);
+      // set a short token lifetime (6 second)
+      SecurityTestUtil.setBlockTokenLifetime(sm, 6000L);
     }
 
     final AtomicInteger pos = new AtomicInteger();


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

Branch: refs/heads/YARN-5734
Commit: fbcc60ce7d75812fd5957e8fe5b17abf0421a613
Parents: 0de6349
Author: Wangda Tan <wa...@apache.org>
Authored: Mon Apr 3 10:12:01 2017 -0700
Committer: Jonathan Hung <jh...@linkedin.com>
Committed: Wed Sep 20 17:40:53 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/fbcc60ce/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/fbcc60ce/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/fbcc60ce/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/fbcc60ce/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/fbcc60ce/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/fbcc60ce/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/fbcc60ce/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/fbcc60ce/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/fbcc60ce/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/fbcc60ce/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


[17/50] [abbrv] hadoop git commit: MAPREDUCE-6958. Shuffle audit logger should log size of shuffle transfer. Contributed by Jason Lowe

Posted by jh...@apache.org.
MAPREDUCE-6958. Shuffle audit logger should log size of shuffle transfer. 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/b3d61304
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/b3d61304
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/b3d61304

Branch: refs/heads/YARN-5734
Commit: b3d61304f2fa4a99526f7a60ccaac9f262083079
Parents: 1ee2527
Author: Jason Lowe <jl...@apache.org>
Authored: Mon Sep 18 17:04:43 2017 -0500
Committer: Jason Lowe <jl...@apache.org>
Committed: Mon Sep 18 17:04:43 2017 -0500

----------------------------------------------------------------------
 .../org/apache/hadoop/mapred/ShuffleHandler.java  | 18 +++++++++++-------
 1 file changed, 11 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b3d61304/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java
index 863da7e..06a3e42 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java
@@ -992,13 +992,6 @@ public class ShuffleHandler extends AuxiliaryService {
         return;
       }
 
-      // this audit log is disabled by default,
-      // to turn it on please enable this audit log
-      // on log4j.properties by uncommenting the setting
-      if (AUDITLOG.isDebugEnabled()) {
-        AUDITLOG.debug("shuffle for " + jobQ.get(0) + " mappers: " + mapIds +
-                         " reducer " + reduceQ.get(0));
-      }
       int reduceId;
       String jobId;
       try {
@@ -1183,6 +1176,17 @@ public class ShuffleHandler extends AuxiliaryService {
 
       // Now set the response headers.
       setResponseHeaders(response, keepAliveParam, contentLength);
+
+      // this audit log is disabled by default,
+      // to turn it on please enable this audit log
+      // on log4j.properties by uncommenting the setting
+      if (AUDITLOG.isDebugEnabled()) {
+        StringBuilder sb = new StringBuilder("shuffle for ").append(jobId);
+        sb.append(" mappers: ").append(mapIds);
+        sb.append(" reducer ").append(reduce);
+        sb.append(" length ").append(contentLength);
+        AUDITLOG.debug(sb.toString());
+      }
     }
 
     protected void setResponseHeaders(HttpResponse response,


---------------------------------------------------------------------
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: MAPREDUCE-6954. Disable erasure coding for files that are uploaded to the MR staging area (pbacsko via rkanter)

Posted by jh...@apache.org.
MAPREDUCE-6954. Disable erasure coding for files that are uploaded to the MR staging area (pbacsko via rkanter)


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

Branch: refs/heads/YARN-5734
Commit: 0adc0471d0c06f66a31060f270dcb50a7b4ffafa
Parents: 5f49668
Author: Robert Kanter <rk...@apache.org>
Authored: Mon Sep 18 10:40:06 2017 -0700
Committer: Robert Kanter <rk...@apache.org>
Committed: Mon Sep 18 10:40:06 2017 -0700

----------------------------------------------------------------------
 .../hadoop-mapreduce-client-core/pom.xml        |  4 ++
 .../hadoop/mapreduce/JobResourceUploader.java   | 17 ++++++++
 .../apache/hadoop/mapreduce/MRJobConfig.java    |  5 +++
 .../src/main/resources/mapred-default.xml       |  9 ++++
 .../mapreduce/TestJobResourceUploader.java      | 46 ++++++++++++++++++++
 5 files changed, 81 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0adc0471/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml
index c34f7bd..ce5fdc8 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml
@@ -44,6 +44,10 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-hdfs-client</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-hdfs</artifactId>
       <scope>test</scope>
     </dependency>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0adc0471/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobResourceUploader.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobResourceUploader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobResourceUploader.java
index f1cad57..d9bf988 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobResourceUploader.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobResourceUploader.java
@@ -36,6 +36,8 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
 import org.apache.hadoop.mapreduce.filecache.ClientDistributedCacheManager;
 import org.apache.hadoop.mapreduce.filecache.DistributedCache;
 
@@ -94,6 +96,11 @@ class JobResourceUploader {
         new FsPermission(JobSubmissionFiles.JOB_DIR_PERMISSION);
     mkdirs(jtFs, submitJobDir, mapredSysPerms);
 
+    if (!conf.getBoolean(MRJobConfig.MR_AM_STAGING_DIR_ERASURECODING_ENABLED,
+        MRJobConfig.DEFAULT_MR_AM_STAGING_ERASURECODING_ENABLED)) {
+      disableErasureCodingForPath(jtFs, submitJobDir);
+    }
+
     Collection<String> files = conf.getStringCollection("tmpfiles");
     Collection<String> libjars = conf.getStringCollection("tmpjars");
     Collection<String> archives = conf.getStringCollection("tmparchives");
@@ -575,4 +582,14 @@ class JobResourceUploader {
     }
     return finalPath;
   }
+
+  private void disableErasureCodingForPath(FileSystem fs, Path path)
+      throws IOException {
+    if (jtFs instanceof DistributedFileSystem) {
+      LOG.info("Disabling Erasure Coding for path: " + path);
+      DistributedFileSystem dfs = (DistributedFileSystem) jtFs;
+      dfs.setErasureCodingPolicy(path,
+          SystemErasureCodingPolicies.getReplicationPolicy().getName());
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0adc0471/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 2023ba3..86abb42 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
@@ -1037,4 +1037,9 @@ public interface MRJobConfig {
   String FINISH_JOB_WHEN_REDUCERS_DONE =
       "mapreduce.job.finish-when-all-reducers-done";
   boolean DEFAULT_FINISH_JOB_WHEN_REDUCERS_DONE = true;
+
+  String MR_AM_STAGING_DIR_ERASURECODING_ENABLED =
+      MR_AM_STAGING_DIR + "erasurecoding.enabled";
+
+  boolean DEFAULT_MR_AM_STAGING_ERASURECODING_ENABLED = false;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0adc0471/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
index ee9b906..6b6faf2 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
@@ -1261,6 +1261,15 @@
 </property>
 
 <property>
+  <name>yarn.app.mapreduce.am.staging-dir.erasurecoding.enabled</name>
+  <value>false</value>
+  <description>Whether Erasure Coding should be enabled for
+  files that are copied to the MR staging area. This is a job-level
+  setting.
+  </description>
+</property>
+
+<property>
   <name>mapreduce.am.max-attempts</name>
   <value>2</value>
   <description>The maximum number of application attempts. It is a

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0adc0471/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestJobResourceUploader.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestJobResourceUploader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestJobResourceUploader.java
index 20b7b7d..d0d7a34 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestJobResourceUploader.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestJobResourceUploader.java
@@ -20,6 +20,11 @@ package org.apache.hadoop.mapreduce;
 
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.times;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.eq;
 
 import java.io.IOException;
 import java.net.URI;
@@ -36,9 +41,12 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
 import org.apache.hadoop.mapred.JobConf;
 import org.junit.Assert;
 import org.junit.Test;
+import org.mockito.verification.VerificationMode;
 
 /**
  * A class for unit testing JobResourceUploader.
@@ -357,6 +365,40 @@ public class TestJobResourceUploader {
         expectedArchivesWithFrags, expectedJobJar);
   }
 
+  @Test
+  public void testErasureCodingDefault() throws IOException {
+    testErasureCodingSetting(true);
+  }
+
+  @Test
+  public void testErasureCodingDisabled() throws IOException {
+    testErasureCodingSetting(false);
+  }
+
+  private void testErasureCodingSetting(boolean defaultBehavior)
+      throws IOException {
+    JobConf jConf = new JobConf();
+    // don't set to false if EC remains disabled to check default setting
+    if (!defaultBehavior) {
+      jConf.setBoolean(MRJobConfig.MR_AM_STAGING_DIR_ERASURECODING_ENABLED,
+          true);
+    }
+
+    DistributedFileSystem fs = mock(DistributedFileSystem.class);
+    Path path = new Path("/");
+    when(fs.makeQualified(any(Path.class))).thenReturn(path);
+    JobResourceUploader uploader = new StubedUploader(fs, true);
+    Job job = Job.getInstance(jConf);
+
+    uploader.uploadResources(job, new Path("/test"));
+
+    String replicationPolicyName = SystemErasureCodingPolicies
+        .getReplicationPolicy().getName();
+    VerificationMode mode = defaultBehavior ? times(1) : never();
+    verify(fs, mode).setErasureCodingPolicy(eq(path),
+        eq(replicationPolicyName));
+  }
+
   private void runTmpResourcePathTest(JobResourceUploader uploader,
       ResourceConf rConf, JobConf jConf, String[] expectedFiles,
       String[] expectedArchives, String expectedJobJar) throws IOException {
@@ -698,6 +740,10 @@ public class TestJobResourceUploader {
       super(FileSystem.getLocal(conf), useWildcard);
     }
 
+    StubedUploader(FileSystem fs, boolean useWildcard) throws IOException {
+      super(fs, useWildcard);
+    }
+
     @Override
     FileStatus getFileStatus(Map<URI, FileStatus> statCache, Configuration job,
         Path p) throws IOException {


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


[08/50] [abbrv] hadoop git commit: YARN-7172. ResourceCalculator.fitsIn() should not take a cluster resource parameter. (Sen Zhao via wangda)

Posted by jh...@apache.org.
YARN-7172. ResourceCalculator.fitsIn() should not take a cluster resource parameter. (Sen Zhao via wangda)

Change-Id: Icc3670c9381ce7591ca69ec12da5aa52d3612d34


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

Branch: refs/heads/YARN-5734
Commit: e81596d06d226f1cfa44b2390ce3095ed4dee621
Parents: 8d7cc22
Author: Wangda Tan <wa...@apache.org>
Authored: Sun Sep 17 21:20:43 2017 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Sun Sep 17 21:20:43 2017 -0700

----------------------------------------------------------------------
 .../resource/DefaultResourceCalculator.java     |  3 +-
 .../resource/DominantResourceCalculator.java    |  2 +-
 .../yarn/util/resource/ResourceCalculator.java  |  3 +-
 .../hadoop/yarn/util/resource/Resources.java    |  4 +--
 .../util/resource/TestResourceCalculator.java   | 24 +++++++-------
 .../server/resourcemanager/RMServerUtils.java   |  3 +-
 .../CapacitySchedulerPreemptionUtils.java       |  4 +--
 ...QueuePriorityContainerCandidateSelector.java |  5 ++-
 .../ReservedContainerCandidatesSelector.java    | 34 +++++++++-----------
 .../scheduler/capacity/AbstractCSQueue.java     |  2 +-
 .../allocator/RegularContainerAllocator.java    |  8 ++---
 .../scheduler/common/fica/FiCaSchedulerApp.java | 21 +++++-------
 .../scheduler/capacity/TestReservations.java    | 20 +++++-------
 13 files changed, 55 insertions(+), 78 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e81596d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DefaultResourceCalculator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DefaultResourceCalculator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DefaultResourceCalculator.java
index bdf60bd..7f155e7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DefaultResourceCalculator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DefaultResourceCalculator.java
@@ -123,8 +123,7 @@ public class DefaultResourceCalculator extends ResourceCalculator {
   }
 
   @Override
-  public boolean fitsIn(Resource cluster,
-      Resource smaller, Resource bigger) {
+  public boolean fitsIn(Resource smaller, Resource bigger) {
     return smaller.getMemorySize() <= bigger.getMemorySize();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e81596d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java
index d64f03e..ca828a5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/DominantResourceCalculator.java
@@ -538,7 +538,7 @@ public class DominantResourceCalculator extends ResourceCalculator {
   }
 
   @Override
-  public boolean fitsIn(Resource cluster, Resource smaller, Resource bigger) {
+  public boolean fitsIn(Resource smaller, Resource bigger) {
     int maxLength = ResourceUtils.getNumberOfKnownResourceTypes();
     for (int i = 0; i < maxLength; i++) {
       ResourceInformation sResourceInformation = smaller

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e81596d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceCalculator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceCalculator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceCalculator.java
index 398dac5..d59560f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceCalculator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceCalculator.java
@@ -225,8 +225,7 @@ public abstract class ResourceCalculator {
   /**
    * Check if a smaller resource can be contained by bigger resource.
    */
-  public abstract boolean fitsIn(Resource cluster,
-      Resource smaller, Resource bigger);
+  public abstract boolean fitsIn(Resource smaller, Resource bigger);
 
   /**
    * Check if resource has any major resource types (which are all NodeManagers

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e81596d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java
index 325bce4..793aebf 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/resource/Resources.java
@@ -437,9 +437,9 @@ public class Resources {
     return true;
   }
 
-  public static boolean fitsIn(ResourceCalculator rc, Resource cluster,
+  public static boolean fitsIn(ResourceCalculator rc,
       Resource smaller, Resource bigger) {
-    return rc.fitsIn(cluster, smaller, bigger);
+    return rc.fitsIn(smaller, bigger);
   }
   
   public static Resource componentwiseMin(Resource lhs, Resource rhs) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e81596d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/resource/TestResourceCalculator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/resource/TestResourceCalculator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/resource/TestResourceCalculator.java
index 5b4155c..5f3ed19 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/resource/TestResourceCalculator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/resource/TestResourceCalculator.java
@@ -24,14 +24,13 @@ import java.util.Collection;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
 
 @RunWith(Parameterized.class)
 public class TestResourceCalculator {
@@ -64,29 +63,28 @@ public class TestResourceCalculator {
   
   @Test(timeout = 10000)
   public void testFitsIn() {
-    Resource cluster = Resource.newInstance(1024, 1);
 
     if (resourceCalculator instanceof DefaultResourceCalculator) {
-      assertTrue(resourceCalculator.fitsIn(cluster,
+      Assert.assertTrue(resourceCalculator.fitsIn(
           Resource.newInstance(1, 2), Resource.newInstance(2, 1)));
-      assertTrue(resourceCalculator.fitsIn(cluster,
+      Assert.assertTrue(resourceCalculator.fitsIn(
           Resource.newInstance(1, 2), Resource.newInstance(2, 2)));
-      assertTrue(resourceCalculator.fitsIn(cluster,
+      Assert.assertTrue(resourceCalculator.fitsIn(
           Resource.newInstance(1, 2), Resource.newInstance(1, 2)));
-      assertTrue(resourceCalculator.fitsIn(cluster,
+      Assert.assertTrue(resourceCalculator.fitsIn(
           Resource.newInstance(1, 2), Resource.newInstance(1, 1)));
-      assertFalse(resourceCalculator.fitsIn(cluster,
+      Assert.assertFalse(resourceCalculator.fitsIn(
           Resource.newInstance(2, 1), Resource.newInstance(1, 2)));
     } else if (resourceCalculator instanceof DominantResourceCalculator) {
-      assertFalse(resourceCalculator.fitsIn(cluster,
+      Assert.assertFalse(resourceCalculator.fitsIn(
           Resource.newInstance(1, 2), Resource.newInstance(2, 1)));
-      assertTrue(resourceCalculator.fitsIn(cluster,
+      Assert.assertTrue(resourceCalculator.fitsIn(
           Resource.newInstance(1, 2), Resource.newInstance(2, 2)));
-      assertTrue(resourceCalculator.fitsIn(cluster,
+      Assert.assertTrue(resourceCalculator.fitsIn(
           Resource.newInstance(1, 2), Resource.newInstance(1, 2)));
-      assertFalse(resourceCalculator.fitsIn(cluster,
+      Assert.assertFalse(resourceCalculator.fitsIn(
           Resource.newInstance(1, 2), Resource.newInstance(1, 1)));
-      assertFalse(resourceCalculator.fitsIn(cluster,
+      Assert.assertFalse(resourceCalculator.fitsIn(
           Resource.newInstance(2, 1), Resource.newInstance(1, 2)));
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e81596d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServerUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServerUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServerUtils.java
index 2aae3a5..4934243 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServerUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServerUtils.java
@@ -300,8 +300,7 @@ public class RMServerUtils {
     // Target resource of the increase request is more than NM can offer
     ResourceScheduler scheduler = rmContext.getScheduler();
     RMNode rmNode = request.getSchedulerNode().getRMNode();
-    if (!Resources.fitsIn(scheduler.getResourceCalculator(),
-        scheduler.getClusterResource(), targetResource,
+    if (!Resources.fitsIn(scheduler.getResourceCalculator(), targetResource,
         rmNode.getTotalCapability())) {
       String msg = "Target resource=" + targetResource + " of containerId="
           + containerId + " is more than node's total resource="

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e81596d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/CapacitySchedulerPreemptionUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/CapacitySchedulerPreemptionUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/CapacitySchedulerPreemptionUtils.java
index 0ae3ef0..f097e9c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/CapacitySchedulerPreemptionUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/CapacitySchedulerPreemptionUtils.java
@@ -156,8 +156,8 @@ public class CapacitySchedulerPreemptionUtils {
     if (null != toObtainByPartition
         && Resources.greaterThan(rc, clusterResource, toObtainByPartition,
             Resources.none())
-        && Resources.fitsIn(rc, clusterResource,
-            rmContainer.getAllocatedResource(), totalPreemptionAllowed)
+        && Resources.fitsIn(rc, rmContainer.getAllocatedResource(),
+            totalPreemptionAllowed)
         && !Resources.isAnyMajorResourceZero(rc, toObtainByPartition)) {
       Resources.subtractFrom(toObtainByPartition,
           rmContainer.getAllocatedResource());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e81596d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/QueuePriorityContainerCandidateSelector.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/QueuePriorityContainerCandidateSelector.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/QueuePriorityContainerCandidateSelector.java
index c730a2d..7b7404c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/QueuePriorityContainerCandidateSelector.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/QueuePriorityContainerCandidateSelector.java
@@ -229,8 +229,7 @@ public class QueuePriorityContainerCandidateSelector
 
     // If we already can allocate the reserved container after preemption,
     // skip following steps
-    if (Resources.fitsIn(rc, clusterResource, lacking,
-        Resources.none())) {
+    if (Resources.fitsIn(rc, lacking, Resources.none())) {
       return true;
     }
 
@@ -270,7 +269,7 @@ public class QueuePriorityContainerCandidateSelector
       }
 
       // Lacking <= 0 means we can allocate the reserved container
-      if (Resources.fitsIn(rc, clusterResource, lacking, Resources.none())) {
+      if (Resources.fitsIn(rc, lacking, Resources.none())) {
         return true;
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e81596d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ReservedContainerCandidatesSelector.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ReservedContainerCandidatesSelector.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ReservedContainerCandidatesSelector.java
index de23d0a..ff100d9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ReservedContainerCandidatesSelector.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ReservedContainerCandidatesSelector.java
@@ -87,8 +87,8 @@ public class ReservedContainerCandidatesSelector
 
     // Get list of nodes for preemption, ordered by preemption cost
     List<NodeForPreemption> nodesForPreemption = getNodesForPreemption(
-        clusterResource, queueToPreemptableResourceByPartition,
-        selectedCandidates, totalPreemptedResourceAllowed);
+        queueToPreemptableResourceByPartition, selectedCandidates,
+        totalPreemptedResourceAllowed);
 
     for (NodeForPreemption nfp : nodesForPreemption) {
       RMContainer reservedContainer = nfp.schedulerNode.getReservedContainer();
@@ -97,9 +97,8 @@ public class ReservedContainerCandidatesSelector
       }
 
       NodeForPreemption preemptionResult = getPreemptionCandidatesOnNode(
-          nfp.schedulerNode, clusterResource,
-          queueToPreemptableResourceByPartition, selectedCandidates,
-          totalPreemptedResourceAllowed, false);
+          nfp.schedulerNode, queueToPreemptableResourceByPartition,
+          selectedCandidates, totalPreemptedResourceAllowed, false);
       if (null != preemptionResult) {
         for (RMContainer c : preemptionResult.selectedContainers) {
           ApplicationAttemptId appId = c.getApplicationAttemptId();
@@ -135,8 +134,7 @@ public class ReservedContainerCandidatesSelector
     return preemptable;
   }
 
-  private boolean tryToPreemptFromQueue(Resource cluster, String queueName,
-      String partitionName,
+  private boolean tryToPreemptFromQueue(String queueName, String partitionName,
       Map<String, Map<String, Resource>> queueToPreemptableResourceByPartition,
       Resource required, Resource totalPreemptionAllowed, boolean readOnly) {
     Resource preemptable = getPreemptableResource(queueName, partitionName,
@@ -145,11 +143,11 @@ public class ReservedContainerCandidatesSelector
       return false;
     }
 
-    if (!Resources.fitsIn(rc, cluster, required, preemptable)) {
+    if (!Resources.fitsIn(rc, required, preemptable)) {
       return false;
     }
 
-    if (!Resources.fitsIn(rc, cluster, required, totalPreemptionAllowed)) {
+    if (!Resources.fitsIn(rc, required, totalPreemptionAllowed)) {
       return false;
     }
 
@@ -165,7 +163,6 @@ public class ReservedContainerCandidatesSelector
   /**
    * Try to check if we can preempt resources for reserved container in given node
    * @param node
-   * @param cluster
    * @param queueToPreemptableResourceByPartition it's a map of
    *                 <queueName, <partition, preemptable-resource>>
    * @param readOnly do we want to modify preemptable resource after we selected
@@ -174,7 +171,7 @@ public class ReservedContainerCandidatesSelector
    * to satisfy reserved resource
    */
   private NodeForPreemption getPreemptionCandidatesOnNode(
-      FiCaSchedulerNode node, Resource cluster,
+      FiCaSchedulerNode node,
       Map<String, Map<String, Resource>> queueToPreemptableResourceByPartition,
       Map<ApplicationAttemptId, Set<RMContainer>> selectedCandidates,
       Resource totalPreemptionAllowed, boolean readOnly) {
@@ -204,8 +201,7 @@ public class ReservedContainerCandidatesSelector
     String partition = node.getPartition();
 
     // Avoid preempt any container if required <= available + killable
-    if (Resources.fitsIn(rc, cluster, reservedContainer.getReservedResource(),
-        cur)) {
+    if (Resources.fitsIn(rc, reservedContainer.getReservedResource(), cur)) {
       return null;
     }
 
@@ -232,9 +228,9 @@ public class ReservedContainerCandidatesSelector
 
       // Can we preempt container c?
       // Check if we have quota to preempt this container
-      boolean canPreempt = tryToPreemptFromQueue(cluster, containerQueueName,
-          partition, queueToPreemptableResourceByPartition,
-          c.getAllocatedResource(), totalPreemptionAllowed, readOnly);
+      boolean canPreempt = tryToPreemptFromQueue(containerQueueName, partition,
+          queueToPreemptableResourceByPartition, c.getAllocatedResource(),
+          totalPreemptionAllowed, readOnly);
 
       // If we can, add to selected container, and change resource accordingly.
       if (canPreempt) {
@@ -246,7 +242,7 @@ public class ReservedContainerCandidatesSelector
           Resources.addTo(totalSelected, c.getAllocatedResource());
         }
         Resources.addTo(cur, c.getAllocatedResource());
-        if (Resources.fitsIn(rc, cluster,
+        if (Resources.fitsIn(rc,
             reservedContainer.getReservedResource(), cur)) {
           canAllocateReservedContainer = true;
           break;
@@ -282,7 +278,7 @@ public class ReservedContainerCandidatesSelector
     return nfp;
   }
 
-  private List<NodeForPreemption> getNodesForPreemption(Resource cluster,
+  private List<NodeForPreemption> getNodesForPreemption(
       Map<String, Map<String, Resource>> queueToPreemptableResourceByPartition,
       Map<ApplicationAttemptId, Set<RMContainer>> selectedCandidates,
       Resource totalPreemptionAllowed) {
@@ -292,7 +288,7 @@ public class ReservedContainerCandidatesSelector
     for (FiCaSchedulerNode node : preemptionContext.getScheduler()
         .getAllNodes()) {
       if (node.getReservedContainer() != null) {
-        NodeForPreemption nfp = getPreemptionCandidatesOnNode(node, cluster,
+        NodeForPreemption nfp = getPreemptionCandidatesOnNode(node,
             queueToPreemptableResourceByPartition, selectedCandidates,
             totalPreemptionAllowed, true);
         if (null != nfp) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e81596d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java
index d7c452a..250f4e6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java
@@ -909,7 +909,7 @@ public abstract class AbstractCSQueue implements CSQueue {
           maxResourceLimit = labelManager.getResourceByLabel(
               schedulerContainer.getNodePartition(), cluster);
         }
-        if (!Resources.fitsIn(resourceCalculator, cluster,
+        if (!Resources.fitsIn(resourceCalculator,
             Resources.add(queueUsage.getUsed(partition), netAllocated),
             maxResourceLimit)) {
           if (LOG.isDebugEnabled()) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e81596d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java
index f753d31..72dfbdd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java
@@ -548,10 +548,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
         toKillContainers.add(killableContainer);
         Resources.addTo(availableAndKillable,
                         killableContainer.getAllocatedResource());
-        if (Resources.fitsIn(rc,
-                             clusterResource,
-                             capability,
-                             availableAndKillable)) {
+        if (Resources.fitsIn(rc, capability, availableAndKillable)) {
           // Stop if we find enough spaces
           availableContainers = 1;
           break;
@@ -579,8 +576,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator {
             // under the limit.
             resourceNeedToUnReserve = capability;
           }
-          unreservedContainer =
-              application.findNodeToUnreserve(clusterResource, node,
+          unreservedContainer = application.findNodeToUnreserve(node,
                   schedulerKey, resourceNeedToUnReserve);
           // When (minimum-unreserved-resource > 0 OR we cannot allocate
           // new/reserved

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e81596d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
index 17bb104..a12c5ec 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
@@ -314,7 +314,6 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
   }
 
   private boolean commonCheckContainerAllocation(
-      Resource cluster,
       ContainerAllocationProposal<FiCaSchedulerApp, FiCaSchedulerNode> allocation,
       SchedulerContainer<FiCaSchedulerApp, FiCaSchedulerNode> schedulerContainer) {
     // Make sure node is not reserved by anyone else
@@ -355,8 +354,7 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
         }
       }
     }
-    if (!Resources.fitsIn(rc, cluster,
-        allocation.getAllocatedOrReservedResource(),
+    if (!Resources.fitsIn(rc, allocation.getAllocatedOrReservedResource(),
         availableResource)) {
       if (LOG.isDebugEnabled()) {
         LOG.debug("Node doesn't have enough available resource, asked="
@@ -419,8 +417,7 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
 
           // Common part of check container allocation regardless if it is a
           // increase container or regular container
-          commonCheckContainerAllocation(cluster, allocation,
-              schedulerContainer);
+          commonCheckContainerAllocation(allocation, schedulerContainer);
         } else {
           // Container reserved first time will be NEW, after the container
           // accepted & confirmed, it will become RESERVED state
@@ -721,9 +718,8 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
   }
 
   @VisibleForTesting
-  public NodeId getNodeIdToUnreserve(
-      SchedulerRequestKey schedulerKey, Resource resourceNeedUnreserve,
-      ResourceCalculator rc, Resource clusterResource) {
+  public NodeId getNodeIdToUnreserve(SchedulerRequestKey schedulerKey,
+      Resource resourceNeedUnreserve, ResourceCalculator resourceCalculator) {
     // first go around make this algorithm simple and just grab first
     // reservation that has enough resources
     Map<NodeId, RMContainer> reservedContainers = this.reservedContainers.get(
@@ -738,7 +734,7 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
 
         // make sure we unreserve one with at least the same amount of
         // resources, otherwise could affect capacity limits
-        if (Resources.fitsIn(rc, clusterResource, resourceNeedUnreserve,
+        if (Resources.fitsIn(resourceCalculator, resourceNeedUnreserve,
             reservedResource)) {
           if (LOG.isDebugEnabled()) {
             LOG.debug(
@@ -806,14 +802,13 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
   }
 
   @VisibleForTesting
-  public RMContainer findNodeToUnreserve(Resource clusterResource,
-      FiCaSchedulerNode node, SchedulerRequestKey schedulerKey,
-      Resource minimumUnreservedResource) {
+  public RMContainer findNodeToUnreserve(FiCaSchedulerNode node,
+      SchedulerRequestKey schedulerKey, Resource minimumUnreservedResource) {
     try {
       readLock.lock();
       // need to unreserve some other container first
       NodeId idToUnreserve = getNodeIdToUnreserve(schedulerKey,
-          minimumUnreservedResource, rc, clusterResource);
+          minimumUnreservedResource, rc);
       if (idToUnreserve == null) {
         if (LOG.isDebugEnabled()) {
           LOG.debug("checked to see if could unreserve for app but nothing "

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e81596d0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestReservations.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/TestReservations.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/TestReservations.java
index 5e6548b..32f022f 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/TestReservations.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/TestReservations.java
@@ -896,8 +896,7 @@ public class TestReservations {
     String host_1 = "host_1";
     FiCaSchedulerNode node_1 = TestUtils.getMockNode(host_1, DEFAULT_RACK, 0,
         8 * GB);
-    
-    Resource clusterResource = Resources.createResource(2 * 8 * GB);
+
 
     // Setup resource-requests
     Priority p = TestUtils.createMockPriority(5);
@@ -933,28 +932,27 @@ public class TestReservations {
         node_0.getNodeID(), "user", rmContext);
 
     // no reserved containers
-    NodeId unreserveId =
-        app_0.getNodeIdToUnreserve(priorityMap, capability,
-            cs.getResourceCalculator(), clusterResource);
+    NodeId unreserveId = app_0.getNodeIdToUnreserve(priorityMap, capability,
+            cs.getResourceCalculator());
     assertEquals(null, unreserveId);
 
     // no reserved containers - reserve then unreserve
     app_0.reserve(node_0, priorityMap, rmContainer_1, container_1);
     app_0.unreserve(priorityMap, node_0, rmContainer_1);
     unreserveId = app_0.getNodeIdToUnreserve(priorityMap, capability,
-        cs.getResourceCalculator(), clusterResource);
+        cs.getResourceCalculator());
     assertEquals(null, unreserveId);
 
     // no container large enough is reserved
     app_0.reserve(node_0, priorityMap, rmContainer_1, container_1);
     unreserveId = app_0.getNodeIdToUnreserve(priorityMap, capability,
-        cs.getResourceCalculator(), clusterResource);
+        cs.getResourceCalculator());
     assertEquals(null, unreserveId);
 
     // reserve one that is now large enough
     app_0.reserve(node_1, priorityMap, rmContainer, container);
     unreserveId = app_0.getNodeIdToUnreserve(priorityMap, capability,
-        cs.getResourceCalculator(), clusterResource);
+        cs.getResourceCalculator());
     assertEquals(node_1.getNodeID(), unreserveId);
   }
 
@@ -1001,16 +999,14 @@ public class TestReservations {
         node_1.getNodeID(), "user", rmContext);
 
     // nothing reserved
-    RMContainer toUnreserveContainer =
-        app_0.findNodeToUnreserve(csContext.getClusterResource(), node_1,
+    RMContainer toUnreserveContainer = app_0.findNodeToUnreserve(node_1,
             priorityMap, capability);
     assertTrue(toUnreserveContainer == null);
 
     // reserved but scheduler doesn't know about that node.
     app_0.reserve(node_1, priorityMap, rmContainer, container);
     node_1.reserveResource(app_0, priorityMap, rmContainer);
-    toUnreserveContainer =
-        app_0.findNodeToUnreserve(csContext.getClusterResource(), node_1,
+    toUnreserveContainer = app_0.findNodeToUnreserve(node_1,
             priorityMap, capability);
     assertTrue(toUnreserveContainer == null);
   }


---------------------------------------------------------------------
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: HDFS-12437. Fix test setup in TestLeaseRecoveryStriped.

Posted by jh...@apache.org.
HDFS-12437. Fix test setup in TestLeaseRecoveryStriped.


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

Branch: refs/heads/YARN-5734
Commit: 12d9d7bc509bca82b8f40301e3dc5ca764be45eb
Parents: 51edaac
Author: Andrew Wang <wa...@apache.org>
Authored: Tue Sep 19 16:42:20 2017 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Tue Sep 19 16:42:20 2017 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/TestLeaseRecoveryStriped.java   | 156 ++++++++++++++-----
 1 file changed, 113 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/12d9d7bc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecoveryStriped.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecoveryStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecoveryStriped.java
index 2846dbf..36ac8b3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecoveryStriped.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecoveryStriped.java
@@ -19,8 +19,7 @@ package org.apache.hadoop.hdfs;
 
 import com.google.common.base.Preconditions;
 import com.google.common.base.Supplier;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import org.apache.commons.lang.builder.ToStringBuilder;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -28,6 +27,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.apache.hadoop.hdfs.server.datanode.BlockRecoveryWorker;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 import org.apache.hadoop.io.IOUtils;
@@ -40,34 +40,41 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 import org.mockito.internal.util.reflection.Whitebox;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.HashSet;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Random;
+import java.util.Set;
 import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.TimeoutException;
 
 public class TestLeaseRecoveryStriped {
-  public static final Log LOG = LogFactory
-      .getLog(TestLeaseRecoveryStriped.class);
+  public static final Logger LOG = LoggerFactory
+      .getLogger(TestLeaseRecoveryStriped.class);
 
   private final ErasureCodingPolicy ecPolicy =
       StripedFileTestUtil.getDefaultECPolicy();
   private final int dataBlocks = ecPolicy.getNumDataUnits();
   private final int parityBlocks = ecPolicy.getNumParityUnits();
   private final int cellSize = ecPolicy.getCellSize();
-  private final int stripSize = dataBlocks * cellSize;
-  private final int stripesPerBlock = 15;
+  private final int stripeSize = dataBlocks * cellSize;
+  private final int stripesPerBlock = 4;
   private final int blockSize = cellSize * stripesPerBlock;
   private final int blockGroupSize = blockSize * dataBlocks;
   private static final int bytesPerChecksum = 512;
 
   static {
     GenericTestUtils.setLogLevel(DataNode.LOG, Level.ALL);
+    GenericTestUtils.setLogLevel(DFSStripedOutputStream.LOG, Level.DEBUG);
+    GenericTestUtils.setLogLevel(BlockRecoveryWorker.LOG, Level.DEBUG);
+    GenericTestUtils.setLogLevel(DataStreamer.LOG, Level.DEBUG);
   }
 
   static private final String fakeUsername = "fakeUser1";
@@ -83,7 +90,7 @@ public class TestLeaseRecoveryStriped {
   public void setup() throws IOException {
     conf = new HdfsConfiguration();
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
-    conf.setLong(HdfsClientConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY, 6000L);
+    conf.setLong(HdfsClientConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY, 60000L);
     conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_CONSIDERLOAD_KEY,
         false);
     conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
@@ -104,78 +111,118 @@ public class TestLeaseRecoveryStriped {
     }
   }
 
-  private int[][][] getBlockLengthsSuite() {
+  private static class BlockLengths {
+    private final int[] blockLengths;
+    private final long safeLength;
+
+    BlockLengths(ErasureCodingPolicy policy, int[] blockLengths) {
+      this.blockLengths = blockLengths;
+      long[] longArray = Arrays.stream(blockLengths).asLongStream().toArray();
+      this.safeLength = StripedBlockUtil.getSafeLength(policy, longArray);
+    }
+
+    @Override
+    public String toString() {
+      return new ToStringBuilder(this)
+          .append("blockLengths", getBlockLengths())
+          .append("safeLength", getSafeLength())
+          .toString();
+    }
+
+    /**
+     * Length of each block in a block group.
+     */
+    public int[] getBlockLengths() {
+      return blockLengths;
+    }
+
+    /**
+     * Safe length, calculated by the block lengths.
+     */
+    public long getSafeLength() {
+      return safeLength;
+    }
+  }
+
+  private BlockLengths[] getBlockLengthsSuite() {
     final int groups = 4;
-    final int minNumCell = 3;
-    final int maxNumCell = 11;
+    final int minNumCell = 1;
+    final int maxNumCell = stripesPerBlock;
     final int minNumDelta = -4;
     final int maxNumDelta = 2;
-    int delta = 0;
-    int[][][] blkLenSuite = new int[groups][][];
+    BlockLengths[] suite = new BlockLengths[groups];
     Random random = ThreadLocalRandom.current();
-    for (int i = 0; i < blkLenSuite.length; i++) {
-      if (i == blkLenSuite.length - 1) {
-        delta = bytesPerChecksum;
-      }
-      int[][] suite = new int[2][];
-      int[] lens = new int[dataBlocks + parityBlocks];
-      long[] lenInLong = new long[lens.length];
-      for (int j = 0; j < lens.length; j++) {
+    for (int i = 0; i < groups; i++) {
+      int[] blockLengths = new int[dataBlocks + parityBlocks];
+      for (int j = 0; j < blockLengths.length; j++) {
+        // Choose a random number of cells for the block
         int numCell = random.nextInt(maxNumCell - minNumCell + 1) + minNumCell;
-        int numDelta = j < dataBlocks ?
-            random.nextInt(maxNumDelta - minNumDelta + 1) + minNumDelta : 0;
-        lens[j] = cellSize * numCell + delta * numDelta;
-        lenInLong[j] = lens[j];
+        // For data blocks, jitter the length a bit
+        int numDelta = 0;
+        if (i == groups - 1 && j < dataBlocks) {
+          numDelta = random.nextInt(maxNumDelta - minNumDelta + 1) +
+              minNumDelta;
+        }
+        blockLengths[j] = (cellSize * numCell) + (bytesPerChecksum * numDelta);
       }
-      suite[0] = lens;
-      suite[1] = new int[]{
-          (int) StripedBlockUtil.getSafeLength(ecPolicy, lenInLong)};
-      blkLenSuite[i] = suite;
+      suite[i] = new BlockLengths(ecPolicy, blockLengths);
     }
-    return blkLenSuite;
+    return suite;
   }
 
-  private final int[][][] blockLengthsSuite = getBlockLengthsSuite();
+  private final BlockLengths[] blockLengthsSuite = getBlockLengthsSuite();
 
   @Test
   public void testLeaseRecovery() throws Exception {
+    LOG.info("blockLengthsSuite: " +
+        Arrays.toString(blockLengthsSuite));
     for (int i = 0; i < blockLengthsSuite.length; i++) {
-      int[] blockLengths = blockLengthsSuite[i][0];
-      int safeLength = blockLengthsSuite[i][1][0];
+      BlockLengths blockLengths = blockLengthsSuite[i];
       try {
-        runTest(blockLengths, safeLength);
+        runTest(blockLengths.getBlockLengths(), blockLengths.getSafeLength());
       } catch (Throwable e) {
         String msg = "failed testCase at i=" + i + ", blockLengths="
-            + Arrays.toString(blockLengths) + "\n"
+            + blockLengths + "\n"
             + StringUtils.stringifyException(e);
         Assert.fail(msg);
       }
     }
   }
 
-  private void runTest(int[] blockLengths, int safeLength) throws Exception {
+  private void runTest(int[] blockLengths, long safeLength) throws Exception {
     writePartialBlocks(blockLengths);
     recoverLease();
 
     List<Long> oldGS = new ArrayList<>();
     oldGS.add(1001L);
-    StripedFileTestUtil.checkData(dfs, p, safeLength,
+    StripedFileTestUtil.checkData(dfs, p, (int)safeLength,
         new ArrayList<DatanodeInfo>(), oldGS, blockGroupSize);
     // After recovery, storages are reported by primary DN. we should verify
     // storages reported by blockReport.
     cluster.restartNameNode(true);
     cluster.waitFirstBRCompleted(0, 10000);
-    StripedFileTestUtil.checkData(dfs, p, safeLength,
+    StripedFileTestUtil.checkData(dfs, p, (int)safeLength,
         new ArrayList<DatanodeInfo>(), oldGS, blockGroupSize);
   }
 
+  /**
+   * Write a file with blocks of different lengths.
+   *
+   * This method depends on completing before the DFS socket timeout.
+   * Otherwise, the client will mark timed-out streamers as failed, and the
+   * write will fail if there are too many failed streamers.
+   *
+   * @param blockLengths lengths of blocks to write
+   * @throws Exception
+   */
   private void writePartialBlocks(int[] blockLengths) throws Exception {
     final FSDataOutputStream out = dfs.create(p);
     final DFSStripedOutputStream stripedOut = (DFSStripedOutputStream) out
         .getWrappedStream();
-    int length = (stripesPerBlock - 1) * stripSize;
+    int length = (stripesPerBlock - 1) * stripeSize;
     int[] posToKill = getPosToKill(blockLengths);
     int checkingPos = nextCheckingPos(posToKill, 0);
+    Set<Integer> stoppedStreamerIndexes = new HashSet<>();
     try {
       for (int pos = 0; pos < length; pos++) {
         out.write(StripedFileTestUtil.getByte(pos));
@@ -183,15 +230,31 @@ public class TestLeaseRecoveryStriped {
           for (int index : getIndexToStop(posToKill, pos)) {
             out.flush();
             stripedOut.enqueueAllCurrentPackets();
+            LOG.info("Stopping block stream idx {} at file offset {} block " +
+                    "length {}", index, pos, blockLengths[index]);
             StripedDataStreamer s = stripedOut.getStripedDataStreamer(index);
             waitStreamerAllAcked(s);
             waitByteSent(s, blockLengths[index]);
             stopBlockStream(s);
+            stoppedStreamerIndexes.add(index);
           }
           checkingPos = nextCheckingPos(posToKill, pos);
         }
       }
     } finally {
+      // Flush everything
+      out.flush();
+      stripedOut.enqueueAllCurrentPackets();
+      // Wait for streamers that weren't killed above to be written out
+      for (int i=0; i< blockLengths.length; i++) {
+        if (stoppedStreamerIndexes.contains(i)) {
+          continue;
+        }
+        StripedDataStreamer s = stripedOut.getStripedDataStreamer(i);
+        LOG.info("Waiting for block stream idx {} to reach length {}", i,
+            blockLengths[i]);
+        waitStreamerAllAcked(s);
+      }
       DFSTestUtil.abortStream(stripedOut);
     }
   }
@@ -210,7 +273,7 @@ public class TestLeaseRecoveryStriped {
     int[] posToKill = new int[dataBlocks + parityBlocks];
     for (int i = 0; i < dataBlocks; i++) {
       int numStripe = (blockLengths[i] - 1) / cellSize;
-      posToKill[i] = numStripe * stripSize + i * cellSize
+      posToKill[i] = numStripe * stripeSize + i * cellSize
           + blockLengths[i] % cellSize;
       if (blockLengths[i] % cellSize == 0) {
         posToKill[i] += cellSize;
@@ -220,7 +283,7 @@ public class TestLeaseRecoveryStriped {
         + parityBlocks; i++) {
       Preconditions.checkArgument(blockLengths[i] % cellSize == 0);
       int numStripe = (blockLengths[i]) / cellSize;
-      posToKill[i] = numStripe * stripSize;
+      posToKill[i] = numStripe * stripeSize;
     }
     return posToKill;
   }
@@ -243,13 +306,20 @@ public class TestLeaseRecoveryStriped {
         public Boolean get() {
           return s.bytesSent >= byteSent;
         }
-      }, 100, 3000);
+      }, 100, 30000);
     } catch (TimeoutException e) {
       throw new IOException("Timeout waiting for streamer " + s + ". Sent="
           + s.bytesSent + ", expected=" + byteSent);
     }
   }
 
+  /**
+   * Stop the block stream without immediately inducing a hard failure.
+   * Packets can continue to be queued until the streamer hits a socket timeout.
+   *
+   * @param s
+   * @throws Exception
+   */
   private void stopBlockStream(StripedDataStreamer s) throws Exception {
     IOUtils.NullOutputStream nullOutputStream = new IOUtils.NullOutputStream();
     Whitebox.setInternalState(s, "blockStream",
@@ -257,8 +327,8 @@ public class TestLeaseRecoveryStriped {
   }
 
   private void recoverLease() throws Exception {
-    final DistributedFileSystem dfs2 = (DistributedFileSystem) getFSAsAnotherUser(
-        conf);
+    final DistributedFileSystem dfs2 =
+        (DistributedFileSystem) getFSAsAnotherUser(conf);
     try {
       GenericTestUtils.waitFor(new Supplier<Boolean>() {
         @Override


---------------------------------------------------------------------
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: MAPREDUCE-6947. Moving logging APIs over to slf4j in hadoop-mapreduce-examples. Contributed by Gergery Novák.

Posted by jh...@apache.org.
MAPREDUCE-6947. Moving logging APIs over to slf4j in hadoop-mapreduce-examples. Contributed by Gergery Novák.


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

Branch: refs/heads/YARN-5734
Commit: 2018538fdba1a95a6556187569e872fce7f9e1c3
Parents: 56ef527
Author: Akira Ajisaka <aa...@apache.org>
Authored: Tue Sep 19 11:05:54 2017 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Tue Sep 19 11:05:54 2017 +0900

----------------------------------------------------------------------
 .../java/org/apache/hadoop/examples/BaileyBorweinPlouffe.java | 7 ++++---
 .../main/java/org/apache/hadoop/examples/DBCountPageView.java | 7 ++++---
 .../java/org/apache/hadoop/examples/dancing/DancingLinks.java | 7 +++----
 .../src/main/java/org/apache/hadoop/examples/pi/DistSum.java  | 6 +++---
 .../java/org/apache/hadoop/examples/terasort/TeraGen.java     | 6 +++---
 .../org/apache/hadoop/examples/terasort/TeraOutputFormat.java | 7 ++++---
 .../org/apache/hadoop/examples/terasort/TeraScheduler.java    | 7 ++++---
 .../java/org/apache/hadoop/examples/terasort/TeraSort.java    | 6 +++---
 .../org/apache/hadoop/examples/terasort/TestTeraSort.java     | 6 +++---
 9 files changed, 31 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2018538f/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/BaileyBorweinPlouffe.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/BaileyBorweinPlouffe.java b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/BaileyBorweinPlouffe.java
index 7e98d7d..da4ec79 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/BaileyBorweinPlouffe.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/BaileyBorweinPlouffe.java
@@ -29,8 +29,6 @@ import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.FileSystem;
@@ -51,6 +49,8 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Charsets;
 
@@ -83,7 +83,8 @@ public class BaileyBorweinPlouffe extends Configured implements Tool {
   private static final String DIGIT_SIZE_PROPERTY = NAME + ".digit.size";
   private static final String DIGIT_PARTS_PROPERTY = NAME + ".digit.parts";
 
-  private static final Log LOG = LogFactory.getLog(BaileyBorweinPlouffe.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(BaileyBorweinPlouffe.class);
 
   /** Mapper class computing digits of Pi. */
   public static class BbpMapper extends

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2018538f/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/DBCountPageView.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/DBCountPageView.java b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/DBCountPageView.java
index 8dec39d..7b73820 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/DBCountPageView.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/DBCountPageView.java
@@ -29,8 +29,6 @@ import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.Random;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.io.LongWritable;
@@ -49,6 +47,8 @@ import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 import org.hsqldb.server.Server;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * This is a demonstrative program, which uses DBInputFormat for reading
@@ -77,7 +77,8 @@ import org.hsqldb.server.Server;
  */
 public class DBCountPageView extends Configured implements Tool {
 
-  private static final Log LOG = LogFactory.getLog(DBCountPageView.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(DBCountPageView.class);
   
   private Connection connection;
   private boolean initialized = false;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2018538f/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/dancing/DancingLinks.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/dancing/DancingLinks.java b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/dancing/DancingLinks.java
index 94d2c83..537b4d4 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/dancing/DancingLinks.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/dancing/DancingLinks.java
@@ -19,8 +19,8 @@ package org.apache.hadoop.examples.dancing;
 
 import java.util.*;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * A generic solver for tile laying problems using Knuth's dancing link
@@ -35,8 +35,7 @@ import org.apache.commons.logging.LogFactory;
  * The type parameter ColumnName is the class of application's column names.
  */
 public class DancingLinks<ColumnName> {
-  private static final Log LOG = 
-    LogFactory.getLog(DancingLinks.class.getName());
+  private static final Logger LOG = LoggerFactory.getLogger(DancingLinks.class);
   
   /**
    * A cell in the table with up/down and left/right links that form doubly

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2018538f/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/pi/DistSum.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/pi/DistSum.java b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/pi/DistSum.java
index 99f7c24..ffe63fe 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/pi/DistSum.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/pi/DistSum.java
@@ -28,8 +28,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.concurrent.Callable;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.examples.pi.math.Summation;
@@ -55,6 +53,8 @@ import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * The main class for computing sums using map/reduce jobs.
@@ -66,7 +66,7 @@ import org.apache.hadoop.util.ToolRunner;
  * a mix-type job may be executed on either side.
  */
 public final class DistSum extends Configured implements Tool {
-  private static final Log LOG = LogFactory.getLog(DistSum.class);
+  private static final Logger LOG = LoggerFactory.getLogger(DistSum.class);
 
   private static final String NAME = DistSum.class.getSimpleName();
   private static final String N_PARTS = "mapreduce.pi." + NAME + ".nParts";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2018538f/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraGen.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraGen.java b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraGen.java
index 53bbdc4..ef20c4c 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraGen.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraGen.java
@@ -25,8 +25,6 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.zip.Checksum;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.Path;
@@ -49,6 +47,8 @@ import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.hadoop.util.PureJavaCrc32;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Generate the official GraySort input data set.
@@ -66,7 +66,7 @@ import org.apache.hadoop.util.ToolRunner;
  * <b>bin/hadoop jar hadoop-*-examples.jar teragen 10000000000 in-dir</b>
  */
 public class TeraGen extends Configured implements Tool {
-  private static final Log LOG = LogFactory.getLog(TeraGen.class);
+  private static final Logger LOG = LoggerFactory.getLogger(TeraGen.class);
 
   public enum Counters {CHECKSUM}
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2018538f/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraOutputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraOutputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraOutputFormat.java
index 96580b1..e0ce36c 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraOutputFormat.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraOutputFormat.java
@@ -21,8 +21,6 @@ package org.apache.hadoop.examples.terasort;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
@@ -38,12 +36,15 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.hadoop.mapreduce.security.TokenCache;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * An output format that writes the key and value appended together.
  */
 public class TeraOutputFormat extends FileOutputFormat<Text,Text> {
-  private static final Log LOG = LogFactory.getLog(TeraOutputFormat.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TeraOutputFormat.class);
   private OutputCommitter committer = null;
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2018538f/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraScheduler.java b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraScheduler.java
index 3e12a3d..7a9e44b 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraScheduler.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraScheduler.java
@@ -21,17 +21,18 @@ package org.apache.hadoop.examples.terasort;
 import java.io.*;
 import java.util.*;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.lib.input.FileSplit;
 import org.apache.hadoop.mapreduce.server.tasktracker.TTConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Charsets;
 
 class TeraScheduler {
-  private static final Log LOG = LogFactory.getLog(TeraScheduler.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TeraScheduler.class);
   private Split[] splits;
   private List<Host> hosts = new ArrayList<Host>();
   private int slotsPerHost;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2018538f/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraSort.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraSort.java b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraSort.java
index 040d13f..8b698e3 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraSort.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraSort.java
@@ -23,8 +23,6 @@ import java.io.IOException;
 import java.io.PrintStream;
 import java.net.URI;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
@@ -38,6 +36,8 @@ import org.apache.hadoop.mapreduce.Partitioner;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Generates the sampled split points, launches the job, and waits for it to
@@ -47,7 +47,7 @@ import org.apache.hadoop.util.ToolRunner;
  * <b>bin/hadoop jar hadoop-*-examples.jar terasort in-dir out-dir</b>
  */
 public class TeraSort extends Configured implements Tool {
-  private static final Log LOG = LogFactory.getLog(TeraSort.class);
+  private static final Logger LOG = LoggerFactory.getLogger(TeraSort.class);
 
   /**
    * A partitioner that splits text keys into roughly equal partitions

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2018538f/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/test/java/org/apache/hadoop/examples/terasort/TestTeraSort.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/test/java/org/apache/hadoop/examples/terasort/TestTeraSort.java b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/test/java/org/apache/hadoop/examples/terasort/TestTeraSort.java
index b835a3b..b301659 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/test/java/org/apache/hadoop/examples/terasort/TestTeraSort.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/test/java/org/apache/hadoop/examples/terasort/TestTeraSort.java
@@ -20,8 +20,6 @@ package org.apache.hadoop.examples.terasort;
 import java.io.File;
 import java.io.IOException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.FileAlreadyExistsException;
@@ -29,12 +27,14 @@ import org.apache.hadoop.mapred.HadoopTestCase;
 import org.apache.hadoop.util.ToolRunner;
 import org.junit.After;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 
 public class TestTeraSort extends HadoopTestCase {
-  private static Log LOG = LogFactory.getLog(TestTeraSort.class);
+  private static final Logger LOG = LoggerFactory.getLogger(TestTeraSort.class);
   
   public TestTeraSort()
       throws IOException {


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


[10/50] [abbrv] hadoop git commit: YARN-7192. Add a pluggable StateMachine Listener that is notified of NM Container State changes. Contributed by Arun Suresh

Posted by jh...@apache.org.
YARN-7192. Add a pluggable StateMachine Listener that is notified of NM Container State changes. Contributed by Arun Suresh


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

Branch: refs/heads/YARN-5734
Commit: a4f9c7c9247801dd37beec6fc195622af1b884ad
Parents: 0f9af24
Author: Jason Lowe <jl...@apache.org>
Authored: Mon Sep 18 10:16:09 2017 -0500
Committer: Jason Lowe <jl...@apache.org>
Committed: Mon Sep 18 10:16:09 2017 -0500

----------------------------------------------------------------------
 .../hadoop/yarn/conf/YarnConfiguration.java     |  6 +-
 .../state/MultiStateTransitionListener.java     | 61 ++++++++++++++++++
 .../hadoop/yarn/state/StateMachineFactory.java  | 40 ++++++++++++
 .../yarn/state/StateTransitionListener.java     | 50 ++++++++++++++
 .../src/main/resources/yarn-default.xml         |  6 ++
 .../ContainerStateTransitionListener.java       | 48 ++++++++++++++
 .../hadoop/yarn/server/nodemanager/Context.java |  2 +
 .../yarn/server/nodemanager/NodeManager.java    | 48 +++++++++++++-
 .../container/ContainerImpl.java                |  3 +-
 .../server/nodemanager/TestNodeManager.java     | 68 ++++++++++++++++++++
 .../amrmproxy/BaseAMRMProxyTest.java            |  8 +++
 .../container/TestContainer.java                | 53 +++++++++++++++
 12 files changed, 389 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4f9c7c9/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 48910b3..114453f 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
@@ -968,9 +968,13 @@ public class YarnConfiguration extends Configuration {
     NM_PREFIX + "bind-host";
 
   /** who will execute(launch) the containers.*/
-  public static final String NM_CONTAINER_EXECUTOR = 
+  public static final String NM_CONTAINER_EXECUTOR =
     NM_PREFIX + "container-executor.class";
 
+  /** List of container state transition listeners.*/
+  public static final String NM_CONTAINER_STATE_TRANSITION_LISTENERS =
+      NM_PREFIX + "container-state-transition-listener.classes";
+
   /**  
    * Adjustment to make to the container os scheduling priority.
    * The valid values for this could vary depending on the platform.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4f9c7c9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/MultiStateTransitionListener.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/MultiStateTransitionListener.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/MultiStateTransitionListener.java
new file mode 100644
index 0000000..1a28fc5
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/MultiStateTransitionListener.java
@@ -0,0 +1,61 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.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.state;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * A {@link StateTransitionListener} that dispatches the pre and post
+ * state transitions to multiple registered listeners.
+ * NOTE: The registered listeners are called in a for loop. Clients should
+ *       know that a listener configured earlier might prevent a later listener
+ *       from being called, if for instance it throws an un-caught Exception.
+ */
+public abstract class MultiStateTransitionListener
+    <OPERAND, EVENT, STATE extends Enum<STATE>> implements
+    StateTransitionListener<OPERAND, EVENT, STATE> {
+
+  private final List<StateTransitionListener<OPERAND, EVENT, STATE>> listeners =
+      new ArrayList<>();
+
+  /**
+   * Add a listener to the list of listeners.
+   * @param listener A listener.
+   */
+  public void addListener(StateTransitionListener<OPERAND, EVENT, STATE>
+      listener) {
+    listeners.add(listener);
+  }
+
+  @Override
+  public void preTransition(OPERAND op, STATE beforeState,
+      EVENT eventToBeProcessed) {
+    for (StateTransitionListener<OPERAND, EVENT, STATE> listener : listeners) {
+      listener.preTransition(op, beforeState, eventToBeProcessed);
+    }
+  }
+
+  @Override
+  public void postTransition(OPERAND op, STATE beforeState, STATE afterState,
+      EVENT processedEvent) {
+    for (StateTransitionListener<OPERAND, EVENT, STATE> listener : listeners) {
+      listener.postTransition(op, beforeState, afterState, processedEvent);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4f9c7c9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/StateMachineFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/StateMachineFactory.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/StateMachineFactory.java
index 5b76ce8..4bb005c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/StateMachineFactory.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/StateMachineFactory.java
@@ -391,6 +391,21 @@ final public class StateMachineFactory
     }
   }
 
+  /**
+   * A StateMachine that accepts a transition listener.
+   * @param operand the object upon which the returned
+   *                {@link StateMachine} will operate.
+   * @param initialState the state in which the returned
+   *                {@link StateMachine} will start.
+   * @param listener An implementation of a {@link StateTransitionListener}.
+   * @return A (@link StateMachine}.
+   */
+  public StateMachine<STATE, EVENTTYPE, EVENT>
+        make(OPERAND operand, STATE initialState,
+             StateTransitionListener<OPERAND, EVENT, STATE> listener) {
+    return new InternalStateMachine(operand, initialState, listener);
+  }
+
   /* 
    * @return a {@link StateMachine} that starts in 
    *         {@code initialState} and whose {@link Transition} s are
@@ -424,14 +439,36 @@ final public class StateMachineFactory
     return new InternalStateMachine(operand, defaultInitialState);
   }
 
+  private static class NoopStateTransitionListener
+      implements StateTransitionListener {
+    @Override
+    public void preTransition(Object op, Enum beforeState,
+        Object eventToBeProcessed) { }
+
+    @Override
+    public void postTransition(Object op, Enum beforeState, Enum afterState,
+        Object processedEvent) { }
+  }
+
+  private static final NoopStateTransitionListener NOOP_LISTENER =
+      new NoopStateTransitionListener();
+
   private class InternalStateMachine
         implements StateMachine<STATE, EVENTTYPE, EVENT> {
     private final OPERAND operand;
     private STATE currentState;
+    private final StateTransitionListener<OPERAND, EVENT, STATE> listener;
 
     InternalStateMachine(OPERAND operand, STATE initialState) {
+      this(operand, initialState, null);
+    }
+
+    InternalStateMachine(OPERAND operand, STATE initialState,
+        StateTransitionListener<OPERAND, EVENT, STATE> transitionListener) {
       this.operand = operand;
       this.currentState = initialState;
+      this.listener =
+          (transitionListener == null) ? NOOP_LISTENER : transitionListener;
       if (!optimized) {
         maybeMakeStateMachineTable();
       }
@@ -445,8 +482,11 @@ final public class StateMachineFactory
     @Override
     public synchronized STATE doTransition(EVENTTYPE eventType, EVENT event)
          throws InvalidStateTransitionException  {
+      listener.preTransition(operand, currentState, event);
+      STATE oldState = currentState;
       currentState = StateMachineFactory.this.doTransition
           (operand, currentState, eventType, event);
+      listener.postTransition(operand, oldState, currentState, event);
       return currentState;
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4f9c7c9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/StateTransitionListener.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/StateTransitionListener.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/StateTransitionListener.java
new file mode 100644
index 0000000..657c193
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/state/StateTransitionListener.java
@@ -0,0 +1,50 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.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.state;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * A State Transition Listener.
+ * It exposes a pre and post transition hook called before and
+ * after the transition.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public interface StateTransitionListener
+    <OPERAND, EVENT, STATE extends Enum<STATE>> {
+
+  /**
+   * Pre Transition Hook. This will be called before transition.
+   * @param op Operand.
+   * @param beforeState State before transition.
+   * @param eventToBeProcessed Incoming Event.
+   */
+  void preTransition(OPERAND op, STATE beforeState, EVENT eventToBeProcessed);
+
+  /**
+   * Post Transition Hook. This will be called after the transition.
+   * @param op Operand.
+   * @param beforeState State before transition.
+   * @param afterState State after transition.
+   * @param processedEvent Processed Event.
+   */
+  void postTransition(OPERAND op, STATE beforeState, STATE afterState,
+      EVENT processedEvent);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4f9c7c9/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 6444da9..0440458 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
@@ -1004,6 +1004,12 @@
   </property>
 
   <property>
+    <description>Comma separated List of container state transition listeners.</description>
+    <name>yarn.nodemanager.container-state-transition-listener.classes</name>
+    <value></value>
+  </property>
+
+  <property>
     <description>Number of threads container manager uses.</description>
     <name>yarn.nodemanager.container-manager.thread-count</name>
     <value>20</value>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4f9c7c9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerStateTransitionListener.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/ContainerStateTransitionListener.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerStateTransitionListener.java
new file mode 100644
index 0000000..24cdb1f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerStateTransitionListener.java
@@ -0,0 +1,48 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <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;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEvent;
+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.state.StateTransitionListener;
+
+/**
+ * Interface to be used by external cluster operators to implement a
+ * State Transition listener that is notified before and after a container
+ * state transition.
+ * NOTE: The pre and post transition callbacks will be made in the synchronized
+ *       block as the call to the instrumented transition - Serially, in the
+ *       order: preTransition, transition and postTransition. The implementor
+ *       must ensure that the callbacks return in a timely manner to avoid
+ *       blocking the state-machine.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public interface ContainerStateTransitionListener extends
+    StateTransitionListener<ContainerImpl, ContainerEvent, ContainerState> {
+
+  /**
+   * Init method which will be invoked by the Node Manager to inject the
+   * NM {@link Context}.
+   * @param context NM Context.
+   */
+  void init(Context context);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4f9c7c9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/Context.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/Context.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/Context.java
index 00bd0ef..a2d00a4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/Context.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/Context.java
@@ -120,4 +120,6 @@ public interface Context {
   NMTimelinePublisher getNMTimelinePublisher();
 
   ContainerExecutor getContainerExecutor();
+
+  ContainerStateTransitionListener getContainerStateTransitionListener();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4f9c7c9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
index 3e919c5..a97b3f2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
@@ -20,12 +20,18 @@ package org.apache.hadoop.yarn.server.nodemanager;
 
 import java.io.IOException;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.ConcurrentSkipListMap;
 import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEvent;
+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.state.MultiStateTransitionListener;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -130,6 +136,17 @@ public class NodeManager extends CompositeService
   private boolean rmWorkPreservingRestartEnabled;
   private boolean shouldExitOnShutdownEvent = false;
 
+  /**
+   * Default Container State transition listener.
+   */
+  public static class DefaultContainerStateListener extends
+      MultiStateTransitionListener
+          <ContainerImpl, ContainerEvent, ContainerState>
+      implements ContainerStateTransitionListener {
+    @Override
+    public void init(Context context) {}
+  }
+
   public NodeManager() {
     super(NodeManager.class.getName());
   }
@@ -219,8 +236,22 @@ public class NodeManager extends CompositeService
       NMTokenSecretManagerInNM nmTokenSecretManager,
       NMStateStoreService stateStore, boolean isDistSchedulerEnabled,
       Configuration conf) {
-    return new NMContext(containerTokenSecretManager, nmTokenSecretManager,
-        dirsHandler, aclsManager, stateStore, isDistSchedulerEnabled, conf);
+    List<ContainerStateTransitionListener> listeners =
+        conf.getInstances(
+            YarnConfiguration.NM_CONTAINER_STATE_TRANSITION_LISTENERS,
+        ContainerStateTransitionListener.class);
+    NMContext nmContext = new NMContext(containerTokenSecretManager,
+        nmTokenSecretManager, dirsHandler, aclsManager, stateStore,
+        isDistSchedulerEnabled, conf);
+    DefaultContainerStateListener defaultListener =
+        new DefaultContainerStateListener();
+    nmContext.setContainerStateTransitionListener(defaultListener);
+    defaultListener.init(nmContext);
+    for (ContainerStateTransitionListener listener : listeners) {
+      listener.init(nmContext);
+      defaultListener.addListener(listener);
+    }
+    return nmContext;
   }
 
   protected void doSecureLogin() throws IOException {
@@ -563,6 +594,8 @@ public class NodeManager extends CompositeService
 
     private NMTimelinePublisher nmTimelinePublisher;
 
+    private ContainerStateTransitionListener containerStateTransitionListener;
+
     public NMContext(NMContainerTokenSecretManager containerTokenSecretManager,
         NMTokenSecretManagerInNM nmTokenSecretManager,
         LocalDirsHandlerService dirsHandler, ApplicationACLsManager aclsManager,
@@ -752,6 +785,17 @@ public class NodeManager extends CompositeService
     public void setContainerExecutor(ContainerExecutor executor) {
       this.executor = executor;
     }
+
+    @Override
+    public ContainerStateTransitionListener
+        getContainerStateTransitionListener() {
+      return this.containerStateTransitionListener;
+    }
+
+    public void setContainerStateTransitionListener(
+        ContainerStateTransitionListener transitionListener) {
+      this.containerStateTransitionListener = transitionListener;
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4f9c7c9/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 9b9c47f..df107a7 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
@@ -240,7 +240,8 @@ public class ContainerImpl implements Container {
     this.containerRetryContext = configureRetryContext(
         conf, launchContext, this.containerId);
     this.remainingRetryAttempts = this.containerRetryContext.getMaxRetries();
-    stateMachine = stateMachineFactory.make(this);
+    stateMachine = stateMachineFactory.make(this, ContainerState.NEW,
+        context.getContainerStateTransitionListener());
     this.context = context;
     this.resourceSet = new ResourceSet();
     this.resourceMappings = new ResourceMappings();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4f9c7c9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManager.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/TestNodeManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManager.java
index 2d390ac..9279711 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManager.java
@@ -25,6 +25,9 @@ import java.io.IOException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEvent;
+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.nodelabels.NodeLabelsProvider;
 import org.junit.Assert;
 import org.junit.Test;
@@ -57,6 +60,71 @@ public class TestNodeManager {
     }
   }
 
+  private static int initCalls = 0;
+  private static int preCalls = 0;
+  private static int postCalls = 0;
+
+  private static class DummyCSTListener1
+      implements ContainerStateTransitionListener {
+    @Override
+    public void init(Context context) {
+      initCalls++;
+    }
+
+    @Override
+    public void preTransition(ContainerImpl op, ContainerState beforeState,
+        ContainerEvent eventToBeProcessed) {
+      preCalls++;
+    }
+
+    @Override
+    public void postTransition(ContainerImpl op, ContainerState beforeState,
+        ContainerState afterState, ContainerEvent processedEvent) {
+      postCalls++;
+    }
+  }
+
+  private static class DummyCSTListener2
+      implements ContainerStateTransitionListener {
+    @Override
+    public void init(Context context) {
+      initCalls++;
+    }
+
+    @Override
+    public void preTransition(ContainerImpl op, ContainerState beforeState,
+        ContainerEvent eventToBeProcessed) {
+      preCalls++;
+    }
+
+    @Override
+    public void postTransition(ContainerImpl op, ContainerState beforeState,
+        ContainerState afterState, ContainerEvent processedEvent) {
+      postCalls++;
+    }
+  }
+
+  @Test
+  public void testListenerInitialization() throws Exception{
+    NodeManager nodeManager = new NodeManager();
+    Configuration conf = new Configuration();
+    conf.set(YarnConfiguration.NM_CONTAINER_STATE_TRANSITION_LISTENERS,
+        DummyCSTListener1.class.getName() + ","
+            + DummyCSTListener2.class.getName());
+    initCalls = 0;
+    preCalls = 0;
+    postCalls = 0;
+    NodeManager.NMContext nmContext =
+        nodeManager.createNMContext(null, null, null, false, conf);
+    Assert.assertEquals(2, initCalls);
+    nmContext.getContainerStateTransitionListener().preTransition(
+        null, null, null);
+    nmContext.getContainerStateTransitionListener().postTransition(
+        null, null, null, null);
+    Assert.assertEquals(2, preCalls);
+    Assert.assertEquals(2, postCalls);
+  }
+
   @Test
   public void testCreationOfNodeLabelsProviderService()
       throws InterruptedException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4f9c7c9/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/BaseAMRMProxyTest.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/amrmproxy/BaseAMRMProxyTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/BaseAMRMProxyTest.java
index 7c8551e..0838f1e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/BaseAMRMProxyTest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/BaseAMRMProxyTest.java
@@ -33,6 +33,8 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.yarn.server.nodemanager.ContainerStateTransitionListener;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -765,5 +767,11 @@ public abstract class BaseAMRMProxyTest {
     public ContainerExecutor getContainerExecutor() {
       return null;
     }
+
+    @Override
+    public ContainerStateTransitionListener
+        getContainerStateTransitionListener() {
+      return null;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a4f9c7c9/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 8909088..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
@@ -71,7 +71,9 @@ import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
 import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
 import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.ExitCode;
+import org.apache.hadoop.yarn.server.nodemanager.ContainerStateTransitionListener;
 import org.apache.hadoop.yarn.server.nodemanager.Context;
+import org.apache.hadoop.yarn.server.nodemanager.NodeManager;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.AuxServicesEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.AuxServicesEventType;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
@@ -287,6 +289,29 @@ public class TestContainer {
       assertEquals(ContainerState.DONE, wc.c.getContainerState());
       assertEquals(completed + 1, metrics.getCompletedContainers());
       assertEquals(running, metrics.getRunningContainers());
+
+      ContainerEventType e1 = wc.initStateToEvent.get(ContainerState.NEW);
+      ContainerState s2 = wc.eventToFinalState.get(e1);
+      ContainerEventType e2 = wc.initStateToEvent.get(s2);
+      ContainerState s3 = wc.eventToFinalState.get(e2);
+      ContainerEventType e3 = wc.initStateToEvent.get(s3);
+      ContainerState s4 = wc.eventToFinalState.get(e3);
+      ContainerEventType e4 = wc.initStateToEvent.get(s4);
+      ContainerState s5 = wc.eventToFinalState.get(e4);
+      ContainerEventType e5 = wc.initStateToEvent.get(s5);
+      ContainerState s6 = wc.eventToFinalState.get(e5);
+
+      Assert.assertEquals(ContainerState.LOCALIZING, s2);
+      Assert.assertEquals(ContainerState.SCHEDULED, s3);
+      Assert.assertEquals(ContainerState.RUNNING, s4);
+      Assert.assertEquals(ContainerState.EXITED_WITH_SUCCESS, s5);
+      Assert.assertEquals(ContainerState.DONE, s6);
+
+      Assert.assertEquals(ContainerEventType.INIT_CONTAINER, e1);
+      Assert.assertEquals(ContainerEventType.RESOURCE_LOCALIZED, e2);
+      Assert.assertEquals(ContainerEventType.CONTAINER_LAUNCHED, e3);
+      Assert.assertEquals(ContainerEventType.CONTAINER_EXITED_WITH_SUCCESS, e4);
+      Assert.assertEquals(ContainerEventType.CONTAINER_RESOURCES_CLEANEDUP, e5);
     }
     finally {
       if (wc != null) {
@@ -401,6 +426,10 @@ public class TestContainer {
       Assert.assertTrue(
           containerMetrics.finishTime.value() > containerMetrics.startTime
               .value());
+      Assert.assertEquals(ContainerEventType.KILL_CONTAINER,
+          wc.initStateToEvent.get(ContainerState.NEW));
+      Assert.assertEquals(ContainerState.DONE,
+          wc.eventToFinalState.get(ContainerEventType.KILL_CONTAINER));
     } finally {
       if (wc != null) {
         wc.finished();
@@ -942,6 +971,10 @@ public class TestContainer {
     final Map<String, LocalResource> localResources;
     final Map<String, ByteBuffer> serviceData;
     final Context context = mock(Context.class);
+    private final Map<ContainerState, ContainerEventType> initStateToEvent =
+        new HashMap<>();
+    private final Map<ContainerEventType, ContainerState> eventToFinalState =
+        new HashMap<>();
 
     WrappedContainer(int appId, long timestamp, int id, String user)
         throws IOException {
@@ -1048,7 +1081,27 @@ public class TestContainer {
       }
       when(ctxt.getServiceData()).thenReturn(serviceData);
       when(ctxt.getContainerRetryContext()).thenReturn(containerRetryContext);
+      ContainerStateTransitionListener listener =
+          new ContainerStateTransitionListener() {
+        @Override
+        public void init(Context cntxt) {}
+
+        @Override
+        public void preTransition(ContainerImpl op, ContainerState beforeState,
+            ContainerEvent eventToBeProcessed) {
+          initStateToEvent.put(beforeState, eventToBeProcessed.getType());
+        }
 
+        @Override
+        public void postTransition(ContainerImpl op, ContainerState beforeState,
+            ContainerState afterState, ContainerEvent processedEvent) {
+          eventToFinalState.put(processedEvent.getType(), afterState);
+        }
+      };
+      NodeManager.DefaultContainerStateListener multi =
+          new NodeManager.DefaultContainerStateListener();
+      multi.addListener(listener);
+      when(context.getContainerStateTransitionListener()).thenReturn(multi);
       c = new ContainerImpl(conf, dispatcher, ctxt, null, metrics, identifier,
           context);
       dispatcher.register(ContainerEventType.class,


---------------------------------------------------------------------
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-12444. Reduce runtime of TestWriteReadStripedFile. Contributed by Huafeng Wang and Andrew Wang.

Posted by jh...@apache.org.
HDFS-12444. Reduce runtime of TestWriteReadStripedFile. Contributed by Huafeng Wang and Andrew Wang.


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

Branch: refs/heads/YARN-5734
Commit: 59830ca772dfb5dcc8b3e5281ca482dea5a5fa3e
Parents: 7bbeacb
Author: Andrew Wang <wa...@apache.org>
Authored: Tue Sep 19 13:44:42 2017 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Tue Sep 19 13:44:42 2017 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hdfs/StripedFileTestUtil.java | 13 +++++++----
 .../hadoop/hdfs/TestWriteReadStripedFile.java   | 24 ++++++++++++--------
 .../hdfs/TestWriteStripedFileWithFailure.java   |  3 ++-
 3 files changed, 25 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/59830ca7/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java
index 1489e48..c771d21 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java
@@ -79,10 +79,15 @@ public class StripedFileTestUtil {
     assertEquals("File length should be the same", fileLength, status.getLen());
   }
 
-  static void verifyPread(FileSystem fs, Path srcPath,  int fileLength,
-      byte[] expected, byte[] buf) throws IOException {
-    final ErasureCodingPolicy ecPolicy =
-        ((DistributedFileSystem)fs).getErasureCodingPolicy(srcPath);
+  static void verifyPread(DistributedFileSystem fs, Path srcPath,
+      int fileLength, byte[] expected, byte[] buf) throws IOException {
+    final ErasureCodingPolicy ecPolicy = fs.getErasureCodingPolicy(srcPath);
+    verifyPread(fs, srcPath, fileLength, expected, buf, ecPolicy);
+  }
+
+  static void verifyPread(FileSystem fs, Path srcPath, int fileLength,
+      byte[] expected, byte[] buf, ErasureCodingPolicy ecPolicy)
+      throws IOException {
     try (FSDataInputStream in = fs.open(srcPath)) {
       int[] startOffsets = {0, 1, ecPolicy.getCellSize() - 102,
           ecPolicy.getCellSize(), ecPolicy.getCellSize() + 102,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/59830ca7/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java
index f27c978..805bcea 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.web.WebHdfsConstants;
@@ -47,12 +48,13 @@ import java.util.Random;
 public class TestWriteReadStripedFile {
   public static final Log LOG = LogFactory.getLog(TestWriteReadStripedFile.class);
   private final ErasureCodingPolicy ecPolicy =
-      StripedFileTestUtil.getDefaultECPolicy();
+      SystemErasureCodingPolicies.getByID(
+          SystemErasureCodingPolicies.RS_3_2_POLICY_ID);
   private final int cellSize = ecPolicy.getCellSize();
   private final short dataBlocks = (short) ecPolicy.getNumDataUnits();
   private final short parityBlocks = (short) ecPolicy.getNumParityUnits();
   private final int numDNs = dataBlocks + parityBlocks;
-  private final int stripesPerBlock = 4;
+  private final int stripesPerBlock = 2;
   private final int blockSize = stripesPerBlock * cellSize;
   private final int blockGroupSize = blockSize * dataBlocks;
 
@@ -78,11 +80,10 @@ public class TestWriteReadStripedFile {
         false);
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
     fs = cluster.getFileSystem();
-    fs.enableErasureCodingPolicy(
-        StripedFileTestUtil.getDefaultECPolicy().getName());
+    fs.enableErasureCodingPolicy(ecPolicy.getName());
     fs.mkdirs(new Path("/ec"));
     cluster.getFileSystem().getClient().setErasureCodingPolicy("/ec",
-        StripedFileTestUtil.getDefaultECPolicy().getName());
+        ecPolicy.getName());
   }
 
   @After
@@ -225,7 +226,8 @@ public class TestWriteReadStripedFile {
 
     byte[] smallBuf = new byte[1024];
     byte[] largeBuf = new byte[fileLength + 100];
-    StripedFileTestUtil.verifyPread(fs, srcPath, fileLength, expected, largeBuf);
+    StripedFileTestUtil.verifyPread(fs, srcPath, fileLength, expected,
+        largeBuf);
 
     StripedFileTestUtil.verifyStatefulRead(fs, srcPath, fileLength, expected,
         largeBuf);
@@ -268,13 +270,15 @@ public class TestWriteReadStripedFile {
 
     byte[] smallBuf = new byte[1024];
     byte[] largeBuf = new byte[fileLength + 100];
-    // TODO: HDFS-8797
-    //StripedFileTestUtil.verifyPread(fs, srcPath, fileLength, expected, largeBuf);
+    StripedFileTestUtil
+        .verifyPread(fs, srcPath, fileLength, expected, largeBuf, ecPolicy);
 
-    StripedFileTestUtil.verifyStatefulRead(fs, srcPath, fileLength, expected, largeBuf);
+    StripedFileTestUtil
+        .verifyStatefulRead(fs, srcPath, fileLength, expected, largeBuf);
     StripedFileTestUtil.verifySeek(fs, srcPath, fileLength, ecPolicy,
         blockGroupSize);
-    StripedFileTestUtil.verifyStatefulRead(fs, srcPath, fileLength, expected, smallBuf);
+    StripedFileTestUtil
+        .verifyStatefulRead(fs, srcPath, fileLength, expected, smallBuf);
     // webhdfs doesn't support bytebuffer read
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/59830ca7/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java
index 03e9e10..c859b71 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java
@@ -160,7 +160,8 @@ public class TestWriteStripedFileWithFailure {
         blockSize * dataBlocks);
     StripedFileTestUtil.verifyStatefulRead(fs, srcPath, fileLength, expected,
         smallBuf);
-    StripedFileTestUtil.verifyPread(fs, srcPath, fileLength, expected, largeBuf);
+    StripedFileTestUtil.verifyPread((DistributedFileSystem)fs, srcPath,
+        fileLength, expected, largeBuf);
 
     // delete the file
     fs.delete(srcPath, true);


---------------------------------------------------------------------
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: HDFS-12472. Add JUNIT timeout to TestBlockStatsMXBean. Contributed by Bharat Viswanadham.

Posted by jh...@apache.org.
HDFS-12472. Add JUNIT timeout to TestBlockStatsMXBean. 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/8d7cc22a
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/8d7cc22a
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/8d7cc22a

Branch: refs/heads/YARN-5734
Commit: 8d7cc22ac286302960c7939bc53574cbfeab1846
Parents: 7618fa9
Author: Arpit Agarwal <ar...@apache.org>
Authored: Sat Sep 16 10:09:27 2017 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Sat Sep 16 10:09:27 2017 -0700

----------------------------------------------------------------------
 .../hdfs/server/blockmanagement/TestBlockStatsMXBean.java       | 5 +++++
 1 file changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d7cc22a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockStatsMXBean.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockStatsMXBean.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockStatsMXBean.java
index bcf38d6..64364cb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockStatsMXBean.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockStatsMXBean.java
@@ -41,8 +41,10 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.junit.After;
 import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
 import org.eclipse.jetty.util.ajax.JSON;
+import org.junit.rules.Timeout;
 
 /**
  * Class for testing {@link BlockStatsMXBean} implementation
@@ -51,6 +53,9 @@ public class TestBlockStatsMXBean {
 
   private MiniDFSCluster cluster;
 
+  @Rule
+  public Timeout globalTimeout = new Timeout(300000);
+
   @Before
   public void setup() throws IOException {
     HdfsConfiguration conf = new HdfsConfiguration();


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

Branch: refs/heads/YARN-5734
Commit: e462f10e65ada949bac159a8a7317b346e2a36e5
Parents: 6684c9a
Author: Xuan <xg...@apache.org>
Authored: Mon Jul 31 11:49:05 2017 -0700
Committer: Jonathan Hung <jh...@linkedin.com>
Committed: Wed Sep 20 17:40:53 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/e462f10e/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/e462f10e/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/e462f10e/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/e462f10e/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


[33/50] [abbrv] hadoop git commit: YARN-7186. Fix finicky TestContainerManager tests. Contributed by Arun Suresh.

Posted by jh...@apache.org.
YARN-7186. Fix finicky TestContainerManager tests. Contributed by Arun Suresh.


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

Branch: refs/heads/YARN-5734
Commit: 647b7527a9cdf4717e7dcbbb660e5812b67a17f1
Parents: 12d9d7b
Author: Junping Du <ju...@apache.org>
Authored: Tue Sep 19 18:31:15 2017 -0700
Committer: Junping Du <ju...@apache.org>
Committed: Tue Sep 19 18:31:15 2017 -0700

----------------------------------------------------------------------
 .../containermanager/TestContainerManager.java  | 128 -------------------
 .../TestContainerSchedulerQueuing.java          |  70 ++++++++++
 2 files changed, 70 insertions(+), 128 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/647b7527/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 6eea77b..38df208 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
@@ -70,7 +70,6 @@ import org.apache.hadoop.yarn.api.records.ContainerRetryContext;
 import org.apache.hadoop.yarn.api.records.ContainerRetryPolicy;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
-import org.apache.hadoop.yarn.api.records.ExecutionType;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LocalResourceType;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
@@ -105,7 +104,6 @@ import org.apache.hadoop.yarn.server.nodemanager.executor.ContainerSignalContext
 import org.apache.hadoop.yarn.server.nodemanager.executor.ContainerStartContext;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.junit.Assert;
-import org.junit.Before;
 import org.junit.Test;
 import org.mockito.ArgumentCaptor;
 import org.mockito.Mockito;
@@ -142,14 +140,6 @@ public class TestContainerManager extends BaseContainerManagerTest {
     exec.setConf(conf);
     return spy(exec);
   }
-
-  @Override
-  @Before
-  public void setup() throws IOException {
-    conf.setInt(
-        YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH, 0);
-    super.setup();
-  }
   
   @Override
   protected ContainerManagerImpl
@@ -1945,122 +1935,4 @@ public class TestContainerManager extends BaseContainerManagerTest {
     Assert.assertTrue(response.getFailedRequests().get(cId).getMessage()
         .contains("Null resource visibility for local resource"));
   }
-
-  @Test
-  public void testContainerUpdateExecTypeOpportunisticToGuaranteed()
-      throws IOException, YarnException, InterruptedException {
-    delayContainers = true;
-    containerManager.start();
-    // Construct the Container-id
-    ContainerId cId = createContainerId(0);
-    ContainerLaunchContext containerLaunchContext =
-        recordFactory.newRecordInstance(ContainerLaunchContext.class);
-
-    StartContainerRequest scRequest =
-        StartContainerRequest.newInstance(
-            containerLaunchContext,
-            createContainerToken(cId, DUMMY_RM_IDENTIFIER,
-                context.getNodeId(), user, BuilderUtils.newResource(512, 1),
-                context.getContainerTokenSecretManager(), null,
-                ExecutionType.OPPORTUNISTIC));
-    List<StartContainerRequest> list = new ArrayList<>();
-    list.add(scRequest);
-    StartContainersRequest allRequests =
-        StartContainersRequest.newInstance(list);
-    containerManager.startContainers(allRequests);
-    // Make sure the container reaches RUNNING state
-    BaseContainerManagerTest.waitForNMContainerState(containerManager, cId,
-        org.apache.hadoop.yarn.server.nodemanager.
-            containermanager.container.ContainerState.RUNNING);
-    // Construct container resource increase request,
-    List<Token> updateTokens = new ArrayList<>();
-    Token containerToken =
-        createContainerToken(cId, 1, DUMMY_RM_IDENTIFIER, context.getNodeId(),
-            user, BuilderUtils.newResource(512, 1),
-            context.getContainerTokenSecretManager(), null,
-            ExecutionType.GUARANTEED);
-    updateTokens.add(containerToken);
-    ContainerUpdateRequest updateRequest =
-        ContainerUpdateRequest.newInstance(updateTokens);
-    ContainerUpdateResponse updateResponse =
-        containerManager.updateContainer(updateRequest);
-
-    Assert.assertEquals(
-        1, updateResponse.getSuccessfullyUpdatedContainers().size());
-    Assert.assertTrue(updateResponse.getFailedRequests().isEmpty());
-
-    //Make sure the container is running
-    List<ContainerId> statList = new ArrayList<ContainerId>();
-    statList.add(cId);
-    GetContainerStatusesRequest statRequest =
-        GetContainerStatusesRequest.newInstance(statList);
-    List<ContainerStatus> containerStatuses = containerManager
-        .getContainerStatuses(statRequest).getContainerStatuses();
-    Assert.assertEquals(1, containerStatuses.size());
-    for (ContainerStatus status : containerStatuses) {
-      Assert.assertEquals(
-          org.apache.hadoop.yarn.api.records.ContainerState.RUNNING,
-          status.getState());
-      Assert.assertEquals(ExecutionType.GUARANTEED, status.getExecutionType());
-    }
-  }
-
-  @Test
-  public void testContainerUpdateExecTypeGuaranteedToOpportunistic()
-      throws IOException, YarnException, InterruptedException {
-    delayContainers = true;
-    containerManager.start();
-    // Construct the Container-id
-    ContainerId cId = createContainerId(0);
-    ContainerLaunchContext containerLaunchContext =
-        recordFactory.newRecordInstance(ContainerLaunchContext.class);
-
-    StartContainerRequest scRequest =
-        StartContainerRequest.newInstance(
-            containerLaunchContext,
-            createContainerToken(cId, DUMMY_RM_IDENTIFIER,
-                context.getNodeId(), user, BuilderUtils.newResource(512, 1),
-                context.getContainerTokenSecretManager(), null));
-    List<StartContainerRequest> list = new ArrayList<>();
-    list.add(scRequest);
-    StartContainersRequest allRequests =
-        StartContainersRequest.newInstance(list);
-    containerManager.startContainers(allRequests);
-    // Make sure the container reaches RUNNING state
-    BaseContainerManagerTest.waitForNMContainerState(containerManager, cId,
-        org.apache.hadoop.yarn.server.nodemanager.
-            containermanager.container.ContainerState.RUNNING);
-    // Construct container resource increase request,
-    List<Token> updateTokens = new ArrayList<>();
-    Token containerToken =
-        createContainerToken(cId, 1, DUMMY_RM_IDENTIFIER, context.getNodeId(),
-            user, BuilderUtils.newResource(512, 1),
-            context.getContainerTokenSecretManager(), null,
-            ExecutionType.OPPORTUNISTIC);
-    updateTokens.add(containerToken);
-    ContainerUpdateRequest updateRequest =
-        ContainerUpdateRequest.newInstance(updateTokens);
-    ContainerUpdateResponse updateResponse =
-        containerManager.updateContainer(updateRequest);
-
-    Assert.assertEquals(
-        1, updateResponse.getSuccessfullyUpdatedContainers().size());
-    Assert.assertTrue(updateResponse.getFailedRequests().isEmpty());
-
-    //Make sure the container is running
-    List<ContainerId> statList = new ArrayList<ContainerId>();
-    statList.add(cId);
-    GetContainerStatusesRequest statRequest =
-        GetContainerStatusesRequest.newInstance(statList);
-    List<ContainerStatus> containerStatuses = containerManager
-        .getContainerStatuses(statRequest).getContainerStatuses();
-    Assert.assertEquals(1, containerStatuses.size());
-    for (ContainerStatus status : containerStatuses) {
-      Assert.assertEquals(
-          org.apache.hadoop.yarn.api.records.ContainerState.RUNNING,
-          status.getState());
-      Assert
-          .assertEquals(ExecutionType.OPPORTUNISTIC, status.getExecutionType());
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/647b7527/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 f3fc724..7c74049 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
@@ -1147,7 +1147,77 @@ public class TestContainerSchedulerQueuing extends BaseContainerManagerTest {
     waitForContainerState(containerManager, createContainerId(1),
         org.apache.hadoop.yarn.api.records.ContainerState.RUNNING);
 
+    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()) {
+        Assert.assertEquals(
+            ExecutionType.GUARANTEED, status.getExecutionType());
+      }
+    }
+
     // Ensure no containers are queued.
     Assert.assertEquals(0, containerScheduler.getNumQueuedContainers());
   }
+
+  @Test
+  public void testContainerUpdateExecTypeGuaranteedToOpportunistic()
+      throws IOException, YarnException, InterruptedException {
+    delayContainers = true;
+    containerManager.start();
+    // Construct the Container-id
+    ContainerId cId = createContainerId(0);
+    ContainerLaunchContext containerLaunchContext =
+        recordFactory.newRecordInstance(ContainerLaunchContext.class);
+
+    StartContainerRequest scRequest =
+        StartContainerRequest.newInstance(
+            containerLaunchContext,
+            createContainerToken(cId, DUMMY_RM_IDENTIFIER,
+                context.getNodeId(), user, BuilderUtils.newResource(512, 1),
+                context.getContainerTokenSecretManager(), null));
+    List<StartContainerRequest> list = new ArrayList<>();
+    list.add(scRequest);
+    StartContainersRequest allRequests =
+        StartContainersRequest.newInstance(list);
+    containerManager.startContainers(allRequests);
+    // Make sure the container reaches RUNNING state
+    BaseContainerManagerTest.waitForNMContainerState(containerManager, cId,
+        org.apache.hadoop.yarn.server.nodemanager.
+            containermanager.container.ContainerState.RUNNING);
+    // Construct container resource increase request,
+    List<Token> updateTokens = new ArrayList<>();
+    Token containerToken =
+        createContainerToken(cId, 1, DUMMY_RM_IDENTIFIER, context.getNodeId(),
+            user, BuilderUtils.newResource(512, 1),
+            context.getContainerTokenSecretManager(), null,
+            ExecutionType.OPPORTUNISTIC);
+    updateTokens.add(containerToken);
+    ContainerUpdateRequest updateRequest =
+        ContainerUpdateRequest.newInstance(updateTokens);
+    ContainerUpdateResponse updateResponse =
+        containerManager.updateContainer(updateRequest);
+
+    Assert.assertEquals(
+        1, updateResponse.getSuccessfullyUpdatedContainers().size());
+    Assert.assertTrue(updateResponse.getFailedRequests().isEmpty());
+
+    //Make sure the container is running
+    List<ContainerId> statList = new ArrayList<ContainerId>();
+    statList.add(cId);
+    GetContainerStatusesRequest statRequest =
+        GetContainerStatusesRequest.newInstance(statList);
+    List<ContainerStatus> containerStatuses = containerManager
+        .getContainerStatuses(statRequest).getContainerStatuses();
+    Assert.assertEquals(1, containerStatuses.size());
+    for (ContainerStatus status : containerStatuses) {
+      Assert.assertEquals(
+          org.apache.hadoop.yarn.api.records.ContainerState.RUNNING,
+          status.getState());
+      Assert
+          .assertEquals(ExecutionType.OPPORTUNISTIC, status.getExecutionType());
+    }
+  }
 }


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

Branch: refs/heads/YARN-5734
Commit: a6a9bae3057c39a6acf06514d23a950a45e7f08f
Parents: a12f09b
Author: Jonathan Hung <jh...@linkedin.com>
Authored: Mon Jan 30 19:03:48 2017 -0800
Committer: Jonathan Hung <jh...@linkedin.com>
Committed: Wed Sep 20 17:39:06 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/a6a9bae3/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/a6a9bae3/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/a6a9bae3/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/a6a9bae3/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/a6a9bae3/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/a6a9bae3/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 a039935..cd9b74e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
@@ -248,13 +248,13 @@ public class TestCapacityScheduler {
 
   @Test (timeout = 30000)
   public void testConfValidation() throws Exception {
-    ResourceScheduler scheduler = new CapacityScheduler();
+    CapacityScheduler scheduler = new CapacityScheduler();
     scheduler.setRMContext(resourceManager.getRMContext());
     Configuration conf = new YarnConfiguration();
     conf.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 2048);
     conf.setInt(YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_MB, 1024);
     try {
-      scheduler.reinitialize(conf, mockContext);
+      scheduler.init(conf);
       fail("Exception is expected because the min memory allocation is" +
         " larger than the max memory allocation.");
     } catch (YarnRuntimeException e) {


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


[30/50] [abbrv] hadoop git commit: HDFS-12445. Correct spellings of choosen to chosen. Contributed by hu xiaodong.

Posted by jh...@apache.org.
HDFS-12445. Correct spellings of choosen to chosen. Contributed by hu xiaodong.


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

Branch: refs/heads/YARN-5734
Commit: 51edaacd09d86419f99ca96545a1393db1f43f73
Parents: 59830ca
Author: Andrew Wang <wa...@apache.org>
Authored: Tue Sep 19 13:48:23 2017 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Tue Sep 19 13:48:23 2017 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hdfs/server/blockmanagement/BlockManager.java  | 4 ++--
 .../org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java | 2 +-
 .../java/org/apache/hadoop/examples/dancing/DancingLinks.java    | 2 +-
 .../org/apache/hadoop/examples/dancing/DistributedPentomino.java | 4 ++--
 .../main/java/org/apache/hadoop/examples/dancing/Pentomino.java  | 2 +-
 5 files changed, 7 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/51edaacd/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index f33ec63..0545bb2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -3551,8 +3551,8 @@ public class BlockManager implements BlockStatsMXBean {
     List<DatanodeStorageInfo> replicasToDelete = replicator
         .chooseReplicasToDelete(nonExcess, nonExcess, replication, excessTypes,
             addedNode, delNodeHint);
-    for (DatanodeStorageInfo choosenReplica : replicasToDelete) {
-      processChosenExcessRedundancy(nonExcess, choosenReplica, storedBlock);
+    for (DatanodeStorageInfo chosenReplica : replicasToDelete) {
+      processChosenExcessRedundancy(nonExcess, chosenReplica, storedBlock);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/51edaacd/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
index b6c1318..1860565 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
@@ -178,7 +178,7 @@ public class TestDeadDatanode {
         clientNode, new HashSet<>(), 256 * 1024 * 1024L, null, (byte) 7,
         BlockType.CONTIGUOUS, null, null);
     for (DatanodeStorageInfo datanodeStorageInfo : results) {
-      assertFalse("Dead node should not be choosen", datanodeStorageInfo
+      assertFalse("Dead node should not be chosen", datanodeStorageInfo
           .getDatanodeDescriptor().equals(clientNode));
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/51edaacd/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/dancing/DancingLinks.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/dancing/DancingLinks.java b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/dancing/DancingLinks.java
index 537b4d4..eef4461 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/dancing/DancingLinks.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/dancing/DancingLinks.java
@@ -368,7 +368,7 @@ public class DancingLinks<ColumnName> {
 
   /**
    * Make one move from a prefix
-   * @param goalRow the row that should be choosen
+   * @param goalRow the row that should be chosen
    * @return the row that was found
    */
   private Node<ColumnName> advance(int goalRow) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/51edaacd/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/dancing/DistributedPentomino.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/dancing/DistributedPentomino.java b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/dancing/DistributedPentomino.java
index 29f1eb2..d4fe6dc 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/dancing/DistributedPentomino.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/dancing/DistributedPentomino.java
@@ -39,8 +39,8 @@ import com.google.common.base.Charsets;
  * Launch a distributed pentomino solver.
  * It generates a complete list of prefixes of length N with each unique prefix
  * as a separate line. A prefix is a sequence of N integers that denote the 
- * index of the row that is choosen for each column in order. Note that the
- * next column is heuristically choosen by the solver, so it is dependant on
+ * index of the row that is chosen for each column in order. Note that the
+ * next column is heuristically chosen by the solver, so it is dependant on
  * the previous choice. That file is given as the input to
  * map/reduce. The output key/value are the move prefix/solution as Text/Text.
  */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/51edaacd/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/dancing/Pentomino.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/dancing/Pentomino.java b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/dancing/Pentomino.java
index 2485728..a30d62c 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/dancing/Pentomino.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/dancing/Pentomino.java
@@ -411,7 +411,7 @@ public class Pentomino {
   /**
    * Find all of the solutions that start with the given prefix. The printer
    * is given each solution as it is found.
-   * @param split a list of row indexes that should be choosen for each row
+   * @param split a list of row indexes that should be chosen for each row
    *        in order
    * @return the number of solutions found
    */


---------------------------------------------------------------------
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-7199. Fix TestAMRMClientContainerRequest.testOpportunisticAndGuaranteedRequests. (Botong Huang via asuresh)

Posted by jh...@apache.org.
YARN-7199. Fix TestAMRMClientContainerRequest.testOpportunisticAndGuaranteedRequests. (Botong Huang via asuresh)


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

Branch: refs/heads/YARN-5734
Commit: 29dd55153e37471d9c177f4bd173f1d02bc96410
Parents: 0adc047
Author: Arun Suresh <as...@apache.org>
Authored: Mon Sep 18 11:26:44 2017 -0700
Committer: Arun Suresh <as...@apache.org>
Committed: Mon Sep 18 11:26:44 2017 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/yarn/client/api/AMRMClient.java  | 9 +++++++--
 1 file changed, 7 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/29dd5515/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java
index 815915e..e86bd12 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java
@@ -118,8 +118,8 @@ public abstract class AMRMClient<T extends AMRMClient.ContainerRequest> extends
     private String nodeLabelsExpression;
     private ExecutionTypeRequest executionTypeRequest =
         ExecutionTypeRequest.newInstance();
-    private String resourceProfile;
-    
+    private String resourceProfile = ProfileCapability.DEFAULT_PROFILE;
+
     /**
      * Instantiates a {@link ContainerRequest} with the given constraints and
      * locality relaxation enabled.
@@ -540,6 +540,11 @@ public abstract class AMRMClient<T extends AMRMClient.ContainerRequest> extends
         return this;
       }
 
+      public ContainerRequestBuilder resourceProfile(String resourceProfile) {
+        containerRequest.resourceProfile = resourceProfile;
+        return this;
+      }
+
       public ContainerRequest build() {
         containerRequest.sanityCheck();
         return containerRequest;


---------------------------------------------------------------------
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: HDFS-12479. Some misuses of lock in DFSStripedOutputStream. Contributed by Huafeng Wang

Posted by jh...@apache.org.
HDFS-12479. Some misuses of lock in DFSStripedOutputStream. Contributed by Huafeng Wang


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

Branch: refs/heads/YARN-5734
Commit: dba7a7dd9d70adfab36a78eb55059c54e553a5cb
Parents: 2018538
Author: Kai Zheng <ka...@intel.com>
Authored: Tue Sep 19 17:45:41 2017 +0800
Committer: Kai Zheng <ka...@intel.com>
Committed: Tue Sep 19 17:45:41 2017 +0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java | 9 ++++-----
 1 file changed, 4 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/dba7a7dd/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
index 44db3a6..66eec7a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
@@ -63,6 +63,7 @@ import java.util.Set;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.Callable;
 import java.util.concurrent.CompletionService;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorCompletionService;
 import java.util.concurrent.ExecutorService;
@@ -85,11 +86,10 @@ public class DFSStripedOutputStream extends DFSOutputStream
     private final List<BlockingQueue<T>> queues;
 
     MultipleBlockingQueue(int numQueue, int queueSize) {
-      List<BlockingQueue<T>> list = new ArrayList<>(numQueue);
+      queues = new ArrayList<>(numQueue);
       for (int i = 0; i < numQueue; i++) {
-        list.add(new LinkedBlockingQueue<T>(queueSize));
+        queues.add(new LinkedBlockingQueue<T>(queueSize));
       }
-      queues = Collections.synchronizedList(list);
     }
 
     void offer(int i, T object) {
@@ -156,8 +156,7 @@ public class DFSStripedOutputStream extends DFSOutputStream
       followingBlocks = new MultipleBlockingQueue<>(numAllBlocks, 1);
       endBlocks = new MultipleBlockingQueue<>(numAllBlocks, 1);
       newBlocks = new MultipleBlockingQueue<>(numAllBlocks, 1);
-      updateStreamerMap = Collections.synchronizedMap(
-          new HashMap<StripedDataStreamer, Boolean>(numAllBlocks));
+      updateStreamerMap = new ConcurrentHashMap<>(numAllBlocks);
       streamerUpdateResult = new MultipleBlockingQueue<>(numAllBlocks, 1);
     }
 


---------------------------------------------------------------------
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: HADOOP-7308. Remove unused TaskLogAppender configurations from log4j.properties. Contributed by Todd Lipcon and J.Andreina.

Posted by jh...@apache.org.
HADOOP-7308. Remove unused TaskLogAppender configurations from log4j.properties. Contributed by Todd Lipcon and J.Andreina.


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

Branch: refs/heads/YARN-5734
Commit: 7e58b2478ce10f54b9b9a647f22a69dd528a81e6
Parents: a9019e1
Author: Akira Ajisaka <aa...@apache.org>
Authored: Wed Sep 20 21:07:45 2017 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Wed Sep 20 21:07:49 2017 +0900

----------------------------------------------------------------------
 .../hadoop-common/src/main/conf/log4j.properties        | 12 ------------
 1 file changed, 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e58b247/hadoop-common-project/hadoop-common/src/main/conf/log4j.properties
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/conf/log4j.properties b/hadoop-common-project/hadoop-common/src/main/conf/log4j.properties
index bc1fa6c..5f4b22b 100644
--- a/hadoop-common-project/hadoop-common/src/main/conf/log4j.properties
+++ b/hadoop-common-project/hadoop-common/src/main/conf/log4j.properties
@@ -78,19 +78,7 @@ log4j.appender.console.layout.ConversionPattern=%d{ISO8601} %p %c{2}: %m%n
 #
 # TaskLog Appender
 #
-
-#Default values
-hadoop.tasklog.taskid=null
-hadoop.tasklog.iscleanup=false
-hadoop.tasklog.noKeepSplits=4
-hadoop.tasklog.totalLogFileSize=100
-hadoop.tasklog.purgeLogSplits=true
-hadoop.tasklog.logsRetainHours=12
-
 log4j.appender.TLA=org.apache.hadoop.mapred.TaskLogAppender
-log4j.appender.TLA.taskId=${hadoop.tasklog.taskid}
-log4j.appender.TLA.isCleanup=${hadoop.tasklog.iscleanup}
-log4j.appender.TLA.totalLogFileSize=${hadoop.tasklog.totalLogFileSize}
 
 log4j.appender.TLA.layout=org.apache.log4j.PatternLayout
 log4j.appender.TLA.layout.ConversionPattern=%d{ISO8601} %p %c: %m%n


---------------------------------------------------------------------
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: YARN-7203. Add container ExecutionType into ContainerReport. (Botong Huang via asuresh)

Posted by jh...@apache.org.
YARN-7203. Add container ExecutionType into ContainerReport. (Botong Huang via asuresh)


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

Branch: refs/heads/YARN-5734
Commit: 56ef5279c1db93d03b2f1e04badbfe804f548918
Parents: 3cf3540
Author: Arun Suresh <as...@apache.org>
Authored: Mon Sep 18 15:49:31 2017 -0700
Committer: Arun Suresh <as...@apache.org>
Committed: Mon Sep 18 15:49:31 2017 -0700

----------------------------------------------------------------------
 .../yarn/api/records/ContainerReport.java       | 26 ++++++++++++++++++++
 .../src/main/proto/yarn_protos.proto            |  1 +
 .../yarn/client/api/impl/TestYarnClient.java    |  1 +
 .../records/impl/pb/ContainerReportPBImpl.java  | 20 +++++++++++++++
 .../rmcontainer/RMContainerImpl.java            |  2 +-
 5 files changed, 49 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/56ef5279/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerReport.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerReport.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerReport.java
index 11d7bca..31d2812 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerReport.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerReport.java
@@ -52,6 +52,18 @@ public abstract class ContainerReport {
       long creationTime, long finishTime, String diagnosticInfo, String logUrl,
       int containerExitStatus, ContainerState containerState,
       String nodeHttpAddress) {
+    return newInstance(containerId, allocatedResource, assignedNode, priority,
+        creationTime, finishTime, diagnosticInfo, logUrl, containerExitStatus,
+        containerState, nodeHttpAddress, ExecutionType.GUARANTEED);
+  }
+
+  @Private
+  @Unstable
+  public static ContainerReport newInstance(ContainerId containerId,
+      Resource allocatedResource, NodeId assignedNode, Priority priority,
+      long creationTime, long finishTime, String diagnosticInfo, String logUrl,
+      int containerExitStatus, ContainerState containerState,
+      String nodeHttpAddress, ExecutionType executionType) {
     ContainerReport report = Records.newRecord(ContainerReport.class);
     report.setContainerId(containerId);
     report.setAllocatedResource(allocatedResource);
@@ -64,6 +76,7 @@ public abstract class ContainerReport {
     report.setContainerExitStatus(containerExitStatus);
     report.setContainerState(containerState);
     report.setNodeHttpAddress(nodeHttpAddress);
+    report.setExecutionType(executionType);
     return report;
   }
 
@@ -209,4 +222,17 @@ public abstract class ContainerReport {
   @Private
   @Unstable
   public abstract void setNodeHttpAddress(String nodeHttpAddress);
+
+  /**
+   * Get the execution type of the container.
+   *
+   * @return the execution type of the container
+   */
+  @Public
+  @Unstable
+  public abstract ExecutionType getExecutionType();
+
+  @Private
+  @Unstable
+  public abstract void setExecutionType(ExecutionType executionType);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/56ef5279/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
index 066441c..fb340d1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
@@ -137,6 +137,7 @@ message ContainerReportProto {
   optional int32 container_exit_status = 9;
   optional ContainerStateProto container_state = 10;
   optional string node_http_address = 11;
+  optional ExecutionTypeProto executionType = 12 [default = GUARANTEED];
 }
 
 enum YarnApplicationStateProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/56ef5279/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java
index cd0e472..4e5d8cd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java
@@ -601,6 +601,7 @@ public class TestYarnClient {
     Assert.assertEquals(report.getContainerId().toString(),
         (ContainerId.newContainerId(expectedReports.get(0)
             .getCurrentApplicationAttemptId(), 3)).toString());
+    Assert.assertNotNull(report.getExecutionType());
     client.stop();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/56ef5279/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerReportPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerReportPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerReportPBImpl.java
index 5d435da..2b58c70 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerReportPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerReportPBImpl.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.yarn.api.records.impl.pb;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerReport;
 import org.apache.hadoop.yarn.api.records.ContainerState;
+import org.apache.hadoop.yarn.api.records.ExecutionType;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
@@ -355,4 +356,23 @@ public class ContainerReportPBImpl extends ContainerReport {
     }
     builder.setNodeHttpAddress(nodeHttpAddress);
   }
+
+  @Override
+  public ExecutionType getExecutionType() {
+    ContainerReportProtoOrBuilder p = viaProto ? proto : builder;
+    if (!p.hasExecutionType()) {
+      return ExecutionType.GUARANTEED;  // default value
+    }
+    return ProtoUtils.convertFromProtoFormat(p.getExecutionType());
+  }
+
+  @Override
+  public void setExecutionType(ExecutionType executionType) {
+    maybeInitBuilder();
+    if (executionType == null) {
+      builder.clearExecutionType();
+      return;
+    }
+    builder.setExecutionType(ProtoUtils.convertToProtoFormat(executionType));
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/56ef5279/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.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/rmcontainer/RMContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
index 8c165de..a43459c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
@@ -756,7 +756,7 @@ public class RMContainerImpl implements RMContainer {
           this.getAllocatedSchedulerKey().getPriority(), this.getCreationTime(),
           this.getFinishTime(), this.getDiagnosticsInfo(), this.getLogURL(),
           this.getContainerExitStatus(), this.getContainerState(),
-          this.getNodeHttpAddress());
+          this.getNodeHttpAddress(), this.getExecutionType());
     } finally {
       this.readLock.unlock();
     }


---------------------------------------------------------------------
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: HDFS-12473. Change hosts JSON file format.

Posted by jh...@apache.org.
HDFS-12473. Change hosts JSON file format.


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

Branch: refs/heads/YARN-5734
Commit: 230b85d5865b7e08fb7aaeab45295b5b966011ef
Parents: 7e58b24
Author: Ming Ma <mi...@twitter.com>
Authored: Wed Sep 20 09:03:59 2017 -0700
Committer: Ming Ma <mi...@twitter.com>
Committed: Wed Sep 20 09:03:59 2017 -0700

----------------------------------------------------------------------
 .../hdfs/util/CombinedHostsFileReader.java      | 67 ++++++++++++++------
 .../hdfs/util/CombinedHostsFileWriter.java      | 23 ++++---
 .../CombinedHostFileManager.java                |  3 +-
 .../hdfs/util/TestCombinedHostsFileReader.java  | 44 ++++++++-----
 .../src/test/resources/dfs.hosts.json           | 16 +++--
 .../src/test/resources/legacy.dfs.hosts.json    |  7 ++
 6 files changed, 102 insertions(+), 58 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/230b85d5/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/CombinedHostsFileReader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/CombinedHostsFileReader.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/CombinedHostsFileReader.java
index 8da5655..aa8e4c1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/CombinedHostsFileReader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/CombinedHostsFileReader.java
@@ -19,58 +19,85 @@
 package org.apache.hadoop.hdfs.util;
 
 import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.databind.JsonMappingException;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.ObjectReader;
+
 import java.io.FileInputStream;
 import java.io.InputStreamReader;
 import java.io.IOException;
 import java.io.Reader;
+import java.util.ArrayList;
 import java.util.Iterator;
-import java.util.Set;
-import java.util.HashSet;
+import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdfs.protocol.DatanodeAdminProperties;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 /**
- * Reader support for JSON based datanode configuration, an alternative
+ * Reader support for JSON-based datanode configuration, an alternative format
  * to the exclude/include files configuration.
- * The JSON file format is the array of elements where each element
+ * The JSON file format defines the array of elements where each element
  * in the array describes the properties of a datanode. The properties of
- * a datanode is defined in {@link DatanodeAdminProperties}. For example,
+ * a datanode is defined by {@link DatanodeAdminProperties}. For example,
  *
- * {"hostName": "host1"}
- * {"hostName": "host2", "port": 50, "upgradeDomain": "ud0"}
- * {"hostName": "host3", "port": 0, "adminState": "DECOMMISSIONED"}
+ * [
+ *   {"hostName": "host1"},
+ *   {"hostName": "host2", "port": 50, "upgradeDomain": "ud0"},
+ *   {"hostName": "host3", "port": 0, "adminState": "DECOMMISSIONED"}
+ * ]
  */
 @InterfaceAudience.LimitedPrivate({"HDFS"})
 @InterfaceStability.Unstable
 public final class CombinedHostsFileReader {
-  private static final ObjectReader READER =
-      new ObjectMapper().readerFor(DatanodeAdminProperties.class);
-  private static final JsonFactory JSON_FACTORY = new JsonFactory();
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(CombinedHostsFileReader.class);
 
   private CombinedHostsFileReader() {
   }
 
   /**
    * Deserialize a set of DatanodeAdminProperties from a json file.
-   * @param hostsFile the input json file to read from.
+   * @param hostsFile the input json file to read from
    * @return the set of DatanodeAdminProperties
    * @throws IOException
    */
-  public static Set<DatanodeAdminProperties>
+  public static DatanodeAdminProperties[]
       readFile(final String hostsFile) throws IOException {
-    HashSet<DatanodeAdminProperties> allDNs = new HashSet<>();
+    DatanodeAdminProperties[] allDNs = new DatanodeAdminProperties[0];
+    ObjectMapper objectMapper = new ObjectMapper();
+    boolean tryOldFormat = false;
     try (Reader input =
-         new InputStreamReader(new FileInputStream(hostsFile), "UTF-8")) {
-      Iterator<DatanodeAdminProperties> iterator =
-          READER.readValues(JSON_FACTORY.createParser(input));
-      while (iterator.hasNext()) {
-        DatanodeAdminProperties properties = iterator.next();
-        allDNs.add(properties);
+        new InputStreamReader(new FileInputStream(hostsFile), "UTF-8")) {
+      allDNs = objectMapper.readValue(input, DatanodeAdminProperties[].class);
+    } catch (JsonMappingException jme) {
+      // The old format doesn't have json top-level token to enclose the array.
+      // For backward compatibility, try parsing the old format.
+      tryOldFormat = true;
+      LOG.warn("{} has invalid JSON format." +
+          "Try the old format without top-level token defined.", hostsFile);
+    }
+
+    if (tryOldFormat) {
+      ObjectReader objectReader =
+          objectMapper.readerFor(DatanodeAdminProperties.class);
+      JsonFactory jsonFactory = new JsonFactory();
+      List<DatanodeAdminProperties> all = new ArrayList<>();
+      try (Reader input =
+          new InputStreamReader(new FileInputStream(hostsFile), "UTF-8")) {
+        Iterator<DatanodeAdminProperties> iterator =
+            objectReader.readValues(jsonFactory.createParser(input));
+        while (iterator.hasNext()) {
+          DatanodeAdminProperties properties = iterator.next();
+          all.add(properties);
+        }
       }
+      allDNs = all.toArray(new DatanodeAdminProperties[all.size()]);
     }
     return allDNs;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/230b85d5/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/CombinedHostsFileWriter.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/CombinedHostsFileWriter.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/CombinedHostsFileWriter.java
index d771191..2d9bbf5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/CombinedHostsFileWriter.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/CombinedHostsFileWriter.java
@@ -32,20 +32,21 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdfs.protocol.DatanodeAdminProperties;
 
 /**
- * Writer support for JSON based datanode configuration, an alternative
+ * Writer support for JSON-based datanode configuration, an alternative format
  * to the exclude/include files configuration.
- * The JSON file format is the array of elements where each element
+ * The JSON file format defines the array of elements where each element
  * in the array describes the properties of a datanode. The properties of
- * a datanode is defined in {@link DatanodeAdminProperties}. For example,
+ * a datanode is defined by {@link DatanodeAdminProperties}. For example,
  *
- * {"hostName": "host1"}
- * {"hostName": "host2", "port": 50, "upgradeDomain": "ud0"}
- * {"hostName": "host3", "port": 0, "adminState": "DECOMMISSIONED"}
+ * [
+ *   {"hostName": "host1"},
+ *   {"hostName": "host2", "port": 50, "upgradeDomain": "ud0"},
+ *   {"hostName": "host3", "port": 0, "adminState": "DECOMMISSIONED"}
+ * ]
  */
 @InterfaceAudience.LimitedPrivate({"HDFS"})
 @InterfaceStability.Unstable
 public final class CombinedHostsFileWriter {
-  private static final ObjectMapper MAPPER = new ObjectMapper();
   private CombinedHostsFileWriter() {
   }
 
@@ -57,13 +58,11 @@ public final class CombinedHostsFileWriter {
    */
   public static void writeFile(final String hostsFile,
       final Set<DatanodeAdminProperties> allDNs) throws IOException {
-    StringBuilder configs = new StringBuilder();
+    final ObjectMapper objectMapper = new ObjectMapper();
+
     try (Writer output =
        new OutputStreamWriter(new FileOutputStream(hostsFile), "UTF-8")) {
-      for (DatanodeAdminProperties datanodeAdminProperties: allDNs) {
-        configs.append(MAPPER.writeValueAsString(datanodeAdminProperties));
-      }
-      output.write(configs.toString());
+      objectMapper.writeValue(output, allDNs);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/230b85d5/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CombinedHostFileManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CombinedHostFileManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CombinedHostFileManager.java
index 6f9c35e..d6a0972 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CombinedHostFileManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CombinedHostFileManager.java
@@ -39,7 +39,6 @@ import java.net.InetSocketAddress;
 import java.util.Collection;
 import java.util.Iterator;
 import java.util.Map;
-import java.util.Set;
 
 import com.google.common.base.Predicate;
 
@@ -212,7 +211,7 @@ public class CombinedHostFileManager extends HostConfigManager {
   }
   private void refresh(final String hostsFile) throws IOException {
     HostProperties hostProps = new HostProperties();
-    Set<DatanodeAdminProperties> all =
+    DatanodeAdminProperties[] all =
         CombinedHostsFileReader.readFile(hostsFile);
     for(DatanodeAdminProperties properties : all) {
       InetSocketAddress addr = parseEntry(hostsFile,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/230b85d5/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestCombinedHostsFileReader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestCombinedHostsFileReader.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestCombinedHostsFileReader.java
index b48784f..cf02180 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestCombinedHostsFileReader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestCombinedHostsFileReader.java
@@ -20,8 +20,6 @@ package org.apache.hadoop.hdfs.util;
 import java.io.File;
 import java.io.FileWriter;
 
-import java.util.Set;
-
 import org.apache.hadoop.hdfs.protocol.DatanodeAdminProperties;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Before;
@@ -30,19 +28,21 @@ import org.junit.Test;
 
 import static org.junit.Assert.assertEquals;
 
-/*
- * Test for JSON based HostsFileReader
+/**
+ * Test for JSON based HostsFileReader.
  */
 public class TestCombinedHostsFileReader {
 
   // Using /test/build/data/tmp directory to store temporary files
-  static final String HOSTS_TEST_DIR = GenericTestUtils.getTestDir()
+  static final String HOSTSTESTDIR = GenericTestUtils.getTestDir()
       .getAbsolutePath();
-  File NEW_FILE = new File(HOSTS_TEST_DIR, "dfs.hosts.new.json");
+  private final File newFile = new File(HOSTSTESTDIR, "dfs.hosts.new.json");
 
-  static final String TEST_CACHE_DATA_DIR =
+  static final String TESTCACHEDATADIR =
       System.getProperty("test.cache.data", "build/test/cache");
-  File EXISTING_FILE = new File(TEST_CACHE_DATA_DIR, "dfs.hosts.json");
+  private final File jsonFile = new File(TESTCACHEDATADIR, "dfs.hosts.json");
+  private final File legacyFile =
+      new File(TESTCACHEDATADIR, "legacy.dfs.hosts.json");
 
   @Before
   public void setUp() throws Exception {
@@ -51,18 +51,28 @@ public class TestCombinedHostsFileReader {
   @After
   public void tearDown() throws Exception {
     // Delete test file after running tests
-    NEW_FILE.delete();
+    newFile.delete();
+
+  }
 
+  /*
+   * Load the legacy test json file
+   */
+  @Test
+  public void testLoadLegacyJsonFile() throws Exception {
+    DatanodeAdminProperties[] all =
+        CombinedHostsFileReader.readFile(legacyFile.getAbsolutePath());
+    assertEquals(7, all.length);
   }
 
   /*
-   * Load the existing test json file
+   * Load the test json file
    */
   @Test
   public void testLoadExistingJsonFile() throws Exception {
-    Set<DatanodeAdminProperties> all =
-        CombinedHostsFileReader.readFile(EXISTING_FILE.getAbsolutePath());
-    assertEquals(7, all.size());
+    DatanodeAdminProperties[] all =
+        CombinedHostsFileReader.readFile(jsonFile.getAbsolutePath());
+    assertEquals(7, all.length);
   }
 
   /*
@@ -70,11 +80,11 @@ public class TestCombinedHostsFileReader {
    */
   @Test
   public void testEmptyCombinedHostsFileReader() throws Exception {
-    FileWriter hosts = new FileWriter(NEW_FILE);
+    FileWriter hosts = new FileWriter(newFile);
     hosts.write("");
     hosts.close();
-    Set<DatanodeAdminProperties> all =
-        CombinedHostsFileReader.readFile(NEW_FILE.getAbsolutePath());
-    assertEquals(0, all.size());
+    DatanodeAdminProperties[] all =
+        CombinedHostsFileReader.readFile(newFile.getAbsolutePath());
+    assertEquals(0, all.length);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/230b85d5/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/dfs.hosts.json
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/dfs.hosts.json b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/dfs.hosts.json
index 9c852e0..615b17f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/dfs.hosts.json
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/dfs.hosts.json
@@ -1,7 +1,9 @@
-{"hostName": "host1"}
-{"hostName": "host2", "upgradeDomain": "ud0"}
-{"hostName": "host3", "adminState": "DECOMMISSIONED"}
-{"hostName": "host4", "upgradeDomain": "ud2", "adminState": "DECOMMISSIONED"}
-{"hostName": "host5", "port": 8090}
-{"hostName": "host6", "adminState": "IN_MAINTENANCE"}
-{"hostName": "host7", "adminState": "IN_MAINTENANCE", "maintenanceExpireTimeInMS": "112233"}
+[
+  {"hostName": "host1"},
+  {"hostName": "host2", "upgradeDomain": "ud0"},
+  {"hostName": "host3", "adminState": "DECOMMISSIONED"},
+  {"hostName": "host4", "upgradeDomain": "ud2", "adminState": "DECOMMISSIONED"},
+  {"hostName": "host5", "port": 8090},
+  {"hostName": "host6", "adminState": "IN_MAINTENANCE"},
+  {"hostName": "host7", "adminState": "IN_MAINTENANCE", "maintenanceExpireTimeInMS": "112233"}
+]

http://git-wip-us.apache.org/repos/asf/hadoop/blob/230b85d5/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/legacy.dfs.hosts.json
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/legacy.dfs.hosts.json b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/legacy.dfs.hosts.json
new file mode 100644
index 0000000..9c852e0
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/legacy.dfs.hosts.json
@@ -0,0 +1,7 @@
+{"hostName": "host1"}
+{"hostName": "host2", "upgradeDomain": "ud0"}
+{"hostName": "host3", "adminState": "DECOMMISSIONED"}
+{"hostName": "host4", "upgradeDomain": "ud2", "adminState": "DECOMMISSIONED"}
+{"hostName": "host5", "port": 8090}
+{"hostName": "host6", "adminState": "IN_MAINTENANCE"}
+{"hostName": "host7", "adminState": "IN_MAINTENANCE", "maintenanceExpireTimeInMS": "112233"}


---------------------------------------------------------------------
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-6499. Remove the doc about Schedulable#redistributeShare(). (Contributed by Chetna Chaudhari via Yufei Gu)

Posted by jh...@apache.org.
YARN-6499. Remove the doc about Schedulable#redistributeShare(). (Contributed by Chetna Chaudhari via Yufei Gu)


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

Branch: refs/heads/YARN-5734
Commit: a9019e1fb753f15c1927e3f9355996fd6544c14f
Parents: 647b752
Author: Yufei Gu <yu...@apache.org>
Authored: Tue Sep 19 18:27:37 2017 -0700
Committer: Yufei Gu <yu...@apache.org>
Committed: Tue Sep 19 18:28:31 2017 -0700

----------------------------------------------------------------------
 .../yarn/server/resourcemanager/scheduler/fair/Schedulable.java  | 4 ----
 1 file changed, 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a9019e1f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/Schedulable.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/Schedulable.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/Schedulable.java
index 4d6af98..bd1ff7a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/Schedulable.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/Schedulable.java
@@ -41,10 +41,6 @@ import org.apache.hadoop.yarn.api.records.Resource;
  * - updateDemand() is called periodically to compute the demand of the various
  *   jobs and queues, which may be expensive (e.g. jobs must iterate through all
  *   their tasks to count failed tasks, tasks that can be speculated, etc).
- * - redistributeShare() is called after demands are updated and a Schedulable's
- *   fair share has been set by its parent to let it distribute its share among
- *   the other Schedulables within it (e.g. for queues that want to perform fair
- *   sharing among their jobs).
  */
 @Private
 @Unstable


---------------------------------------------------------------------
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: MAPREDUCE-6960. Shuffle Handler prints disk error stack traces for every read failure.

Posted by jh...@apache.org.
MAPREDUCE-6960. Shuffle Handler prints disk error stack traces for every read failure.


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

Branch: refs/heads/YARN-5734
Commit: 595d478408104bdfe1f08efd79930e18862fafbb
Parents: 3a20deb
Author: Eric Payne <ep...@apache.org>
Authored: Tue Sep 19 10:35:15 2017 -0500
Committer: Eric Payne <ep...@apache.org>
Committed: Tue Sep 19 10:35:15 2017 -0500

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/mapred/ShuffleHandler.java    | 7 ++++++-
 1 file changed, 6 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/595d4784/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java
index b7f2c6d..0eeae19 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java
@@ -80,6 +80,7 @@ import org.apache.hadoop.metrics2.lib.MutableGaugeInt;
 import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
 import org.apache.hadoop.security.ssl.SSLFactory;
 import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.util.DiskChecker;
 import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.util.concurrent.HadoopExecutors;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -1088,7 +1089,11 @@ public class ShuffleHandler extends AuxiliaryService {
           }
           nextMap.addListener(new ReduceMapFileCount(reduceContext));
         } catch (IOException e) {
-          LOG.error("Shuffle error :", e);
+          if (e instanceof DiskChecker.DiskErrorException) {
+            LOG.error("Shuffle error :" + e);
+          } else {
+            LOG.error("Shuffle error :", e);
+          }
           String errorMessage = getErrorMessage(e);
           sendError(reduceContext.getCtx(), errorMessage,
               INTERNAL_SERVER_ERROR);


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

Branch: refs/heads/YARN-5734
Commit: 6684c9af744bec5d8a4948dbb7f8bfd6e96ca304
Parents: 1ac2ccf
Author: Xuan <xg...@apache.org>
Authored: Fri Jul 7 14:16:46 2017 -0700
Committer: Jonathan Hung <jh...@linkedin.com>
Committed: Wed Sep 20 17:40:53 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/6684c9af/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..331fcfe 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/6684c9af/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/6684c9af/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/6684c9af/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/6684c9af/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/6684c9af/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/6684c9af/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/6684c9af/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/6684c9af/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/6684c9af/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/6684c9af/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/6684c9af/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/6684c9af/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/6684c9af/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/6684c9af/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/6684c9af/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/6684c9af/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/6684c9af/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/6684c9af/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


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

Branch: refs/heads/YARN-5734
Commit: 1ac2ccf997d90f7a9ed7b87e980d23536cb871f8
Parents: 78baf50
Author: Xuan <xg...@apache.org>
Authored: Mon Jun 5 16:30:38 2017 -0700
Committer: Jonathan Hung <jh...@linkedin.com>
Committed: Wed Sep 20 17:40:53 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/1ac2ccf9/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/1ac2ccf9/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/1ac2ccf9/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/1ac2ccf9/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/1ac2ccf9/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/1ac2ccf9/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/1ac2ccf9/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/1ac2ccf9/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/1ac2ccf9/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/1ac2ccf9/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/1ac2ccf9/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/1ac2ccf9/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/1ac2ccf9/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


[15/50] [abbrv] hadoop git commit: YARN-6570. No logs were found for running application, running container. Contributed by Junping Du

Posted by jh...@apache.org.
YARN-6570. No logs were found for running application, running
container. Contributed by Junping Du


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

Branch: refs/heads/YARN-5734
Commit: 7c732924a889cd280e972882619a1827877fbafa
Parents: 29dd551
Author: Xuan <xg...@apache.org>
Authored: Mon Sep 18 14:04:05 2017 -0700
Committer: Xuan <xg...@apache.org>
Committed: Mon Sep 18 14:04:05 2017 -0700

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7c732924/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 df107a7..836e70e 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
@@ -616,6 +616,7 @@ 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/7c732924/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 8e4522b..9e59449 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,7 +159,8 @@ public class TestEventFlow {
     containerManager.startContainers(allRequests);
 
     BaseContainerManagerTest.waitForContainerState(containerManager, cID,
-        Arrays.asList(ContainerState.RUNNING, ContainerState.SCHEDULED), 20);
+        Arrays.asList(ContainerState.RUNNING, ContainerState.SCHEDULED,
+            ContainerState.NEW), 20);
 
     List<ContainerId> containerIds = new ArrayList<ContainerId>();
     containerIds.add(cID);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7c732924/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 64e6cf0..b44b500 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,6 +122,8 @@ 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
@@ -131,6 +133,7 @@ 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


[12/50] [abbrv] hadoop git commit: Revert "YARN-7162. Remove XML excludes file format (rkanter)" - wrong commit message

Posted by jh...@apache.org.
Revert "YARN-7162. Remove XML excludes file format (rkanter)" - wrong commit message

This reverts commit 3a8d57a0a2e047b34be82f602a2b6cf5593d2125.


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

Branch: refs/heads/YARN-5734
Commit: 5f496683fb00ba26a6bf5a506ae87d4bc4088727
Parents: a2dcba1
Author: Robert Kanter <rk...@apache.org>
Authored: Mon Sep 18 10:32:08 2017 -0700
Committer: Robert Kanter <rk...@apache.org>
Committed: Mon Sep 18 10:32:08 2017 -0700

----------------------------------------------------------------------
 .../hadoop-mapreduce-client-core/pom.xml        |  4 --
 .../hadoop/mapreduce/JobResourceUploader.java   | 17 --------
 .../apache/hadoop/mapreduce/MRJobConfig.java    |  5 ---
 .../src/main/resources/mapred-default.xml       |  9 ----
 .../mapreduce/TestJobResourceUploader.java      | 46 --------------------
 5 files changed, 81 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f496683/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml
index ce5fdc8..c34f7bd 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml
@@ -44,10 +44,6 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-hdfs-client</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-hdfs</artifactId>
       <scope>test</scope>
     </dependency>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f496683/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobResourceUploader.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobResourceUploader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobResourceUploader.java
index d9bf988..f1cad57 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobResourceUploader.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobResourceUploader.java
@@ -36,8 +36,6 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.DistributedFileSystem;
-import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
 import org.apache.hadoop.mapreduce.filecache.ClientDistributedCacheManager;
 import org.apache.hadoop.mapreduce.filecache.DistributedCache;
 
@@ -96,11 +94,6 @@ class JobResourceUploader {
         new FsPermission(JobSubmissionFiles.JOB_DIR_PERMISSION);
     mkdirs(jtFs, submitJobDir, mapredSysPerms);
 
-    if (!conf.getBoolean(MRJobConfig.MR_AM_STAGING_DIR_ERASURECODING_ENABLED,
-        MRJobConfig.DEFAULT_MR_AM_STAGING_ERASURECODING_ENABLED)) {
-      disableErasureCodingForPath(jtFs, submitJobDir);
-    }
-
     Collection<String> files = conf.getStringCollection("tmpfiles");
     Collection<String> libjars = conf.getStringCollection("tmpjars");
     Collection<String> archives = conf.getStringCollection("tmparchives");
@@ -582,14 +575,4 @@ class JobResourceUploader {
     }
     return finalPath;
   }
-
-  private void disableErasureCodingForPath(FileSystem fs, Path path)
-      throws IOException {
-    if (jtFs instanceof DistributedFileSystem) {
-      LOG.info("Disabling Erasure Coding for path: " + path);
-      DistributedFileSystem dfs = (DistributedFileSystem) jtFs;
-      dfs.setErasureCodingPolicy(path,
-          SystemErasureCodingPolicies.getReplicationPolicy().getName());
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f496683/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..2023ba3 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
@@ -1037,9 +1037,4 @@ public interface MRJobConfig {
   String FINISH_JOB_WHEN_REDUCERS_DONE =
       "mapreduce.job.finish-when-all-reducers-done";
   boolean DEFAULT_FINISH_JOB_WHEN_REDUCERS_DONE = true;
-
-  String MR_AM_STAGING_DIR_ERASURECODING_ENABLED =
-      MR_AM_STAGING_DIR + "erasurecoding.enabled";
-
-  boolean DEFAULT_MR_AM_STAGING_ERASURECODING_ENABLED = false;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f496683/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
index 6b6faf2..ee9b906 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
@@ -1261,15 +1261,6 @@
 </property>
 
 <property>
-  <name>yarn.app.mapreduce.am.staging-dir.erasurecoding.enabled</name>
-  <value>false</value>
-  <description>Whether Erasure Coding should be enabled for
-  files that are copied to the MR staging area. This is a job-level
-  setting.
-  </description>
-</property>
-
-<property>
   <name>mapreduce.am.max-attempts</name>
   <value>2</value>
   <description>The maximum number of application attempts. It is a

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f496683/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestJobResourceUploader.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestJobResourceUploader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestJobResourceUploader.java
index d0d7a34..20b7b7d 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestJobResourceUploader.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestJobResourceUploader.java
@@ -20,11 +20,6 @@ package org.apache.hadoop.mapreduce;
 
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
-import static org.mockito.Mockito.never;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.times;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.eq;
 
 import java.io.IOException;
 import java.net.URI;
@@ -41,12 +36,9 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.DistributedFileSystem;
-import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
 import org.apache.hadoop.mapred.JobConf;
 import org.junit.Assert;
 import org.junit.Test;
-import org.mockito.verification.VerificationMode;
 
 /**
  * A class for unit testing JobResourceUploader.
@@ -365,40 +357,6 @@ public class TestJobResourceUploader {
         expectedArchivesWithFrags, expectedJobJar);
   }
 
-  @Test
-  public void testErasureCodingDefault() throws IOException {
-    testErasureCodingSetting(true);
-  }
-
-  @Test
-  public void testErasureCodingDisabled() throws IOException {
-    testErasureCodingSetting(false);
-  }
-
-  private void testErasureCodingSetting(boolean defaultBehavior)
-      throws IOException {
-    JobConf jConf = new JobConf();
-    // don't set to false if EC remains disabled to check default setting
-    if (!defaultBehavior) {
-      jConf.setBoolean(MRJobConfig.MR_AM_STAGING_DIR_ERASURECODING_ENABLED,
-          true);
-    }
-
-    DistributedFileSystem fs = mock(DistributedFileSystem.class);
-    Path path = new Path("/");
-    when(fs.makeQualified(any(Path.class))).thenReturn(path);
-    JobResourceUploader uploader = new StubedUploader(fs, true);
-    Job job = Job.getInstance(jConf);
-
-    uploader.uploadResources(job, new Path("/test"));
-
-    String replicationPolicyName = SystemErasureCodingPolicies
-        .getReplicationPolicy().getName();
-    VerificationMode mode = defaultBehavior ? times(1) : never();
-    verify(fs, mode).setErasureCodingPolicy(eq(path),
-        eq(replicationPolicyName));
-  }
-
   private void runTmpResourcePathTest(JobResourceUploader uploader,
       ResourceConf rConf, JobConf jConf, String[] expectedFiles,
       String[] expectedArchives, String expectedJobJar) throws IOException {
@@ -740,10 +698,6 @@ public class TestJobResourceUploader {
       super(FileSystem.getLocal(conf), useWildcard);
     }
 
-    StubedUploader(FileSystem fs, boolean useWildcard) throws IOException {
-      super(fs, useWildcard);
-    }
-
     @Override
     FileStatus getFileStatus(Map<URI, FileStatus> statCache, Configuration job,
         Path p) throws IOException {


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

Branch: refs/heads/YARN-5734
Commit: ba537954b38af3c0c26490cdbc801721b8875c88
Parents: b06711c
Author: Xuan <xg...@apache.org>
Authored: Tue Aug 1 08:48:04 2017 -0700
Committer: Jonathan Hung <jh...@linkedin.com>
Committed: Wed Sep 20 17:40:54 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/ba537954/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/ba537954/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/ba537954/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/ba537954/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/ba537954/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/ba537954/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/ba537954/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


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

Branch: refs/heads/YARN-5734
Commit: b06711cb1f6cdbe5f60fbd1f0cbf7890e1ef779d
Parents: e462f10
Author: Xuan <xg...@apache.org>
Authored: Mon Jul 31 16:48:40 2017 -0700
Committer: Jonathan Hung <jh...@linkedin.com>
Committed: Wed Sep 20 17:40:54 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/b06711cb/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 e1062d7..a33d85d 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/b06711cb/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 86aa15e..4529f20 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
@@ -3371,4 +3371,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/b06711cb/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/b06711cb/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/b06711cb/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/b06711cb/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/b06711cb/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/b06711cb/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


[18/50] [abbrv] hadoop git commit: HADOOP-14835. mvn site build throws SAX errors. Contributed by Andrew Wang and Sean Mackrory.

Posted by jh...@apache.org.
HADOOP-14835. mvn site build throws SAX errors. Contributed by Andrew Wang and Sean Mackrory.


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

Branch: refs/heads/YARN-5734
Commit: 3cf3540f19b5fd1a174690db9f1b7be2977d96ba
Parents: b3d6130
Author: Andrew Wang <wa...@apache.org>
Authored: Mon Sep 18 15:13:42 2017 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Mon Sep 18 15:13:42 2017 -0700

----------------------------------------------------------------------
 BUILDING.txt                                       |  2 ++
 dev-support/bin/create-release                     |  1 +
 .../hadoop-mapreduce-client/pom.xml                | 17 ++++++++++++++++-
 hadoop-project-dist/pom.xml                        | 17 ++++++++++++++++-
 hadoop-project/pom.xml                             |  2 ++
 hadoop-yarn-project/hadoop-yarn/pom.xml            | 17 ++++++++++++++++-
 6 files changed, 53 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3cf3540f/BUILDING.txt
----------------------------------------------------------------------
diff --git a/BUILDING.txt b/BUILDING.txt
index 14deec8..47aaab4 100644
--- a/BUILDING.txt
+++ b/BUILDING.txt
@@ -308,6 +308,8 @@ Create a local staging version of the website (in /tmp/hadoop-site)
 
   $ mvn clean site -Preleasedocs; mvn site:stage -DstagingDirectory=/tmp/hadoop-site
 
+Note that the site needs to be built in a second pass after other artifacts.
+
 ----------------------------------------------------------------------------------
 Installing Hadoop
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3cf3540f/dev-support/bin/create-release
----------------------------------------------------------------------
diff --git a/dev-support/bin/create-release b/dev-support/bin/create-release
index b22e90b..b98c058 100755
--- a/dev-support/bin/create-release
+++ b/dev-support/bin/create-release
@@ -564,6 +564,7 @@ function makearelease
     "${MVN}" "${MVN_ARGS[@]}" install \
       site site:stage \
       -DskipTests \
+      -DskipShade \
       -Pdist,src \
       "${DOCFLAGS}"
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3cf3540f/hadoop-mapreduce-project/hadoop-mapreduce-client/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/pom.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/pom.xml
index aa7c7b1..274a821 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/pom.xml
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/pom.xml
@@ -196,6 +196,13 @@
         <jdiff.stability>-unstable</jdiff.stability>
         <jdiff.javadoc.maxmemory>512m</jdiff.javadoc.maxmemory>
       </properties>
+      <dependencies>
+        <dependency>
+          <groupId>xerces</groupId>
+          <artifactId>xercesImpl</artifactId>
+          <version>${xerces.jdiff.version}</version>
+        </dependency>
+      </dependencies>
       <build>
         <plugins>
           <plugin>
@@ -238,6 +245,14 @@
                       <outputDirectory>${project.build.directory}</outputDirectory>
                       <destFileName>hadoop-annotations.jar</destFileName>
                     </artifactItem>
+                    <artifactItem>
+                      <groupId>xerces</groupId>
+                      <artifactId>xercesImpl</artifactId>
+                      <version>${xerces.version.jdiff}</version>
+                      <overWrite>false</overWrite>
+                      <outputDirectory>${project.build.directory}</outputDirectory>
+                      <destFileName>xerces.jar</destFileName>
+                    </artifactItem>
                   </artifactItems>
                 </configuration>
               </execution>
@@ -275,7 +290,7 @@
                            sourceFiles="${dev-support.relative.dir}/jdiff/Null.java"
                            maxmemory="${jdiff.javadoc.maxmemory}">
                     <doclet name="org.apache.hadoop.classification.tools.IncludePublicAnnotationsJDiffDoclet"
-                            path="${project.build.directory}/hadoop-annotations.jar:${project.build.directory}/jdiff.jar">
+                            path="${project.build.directory}/hadoop-annotations.jar:${project.build.directory}/jdiff.jar:${project.build.directory}/xerces.jar">
                       <param name="-oldapi" value="${project.name} ${jdiff.stable.api}"/>
                       <param name="-newapi" value="${project.name} ${project.version}"/>
                       <param name="-oldapidir" value="${basedir}/${dev-support.relative.dir}/jdiff"/>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3cf3540f/hadoop-project-dist/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project-dist/pom.xml b/hadoop-project-dist/pom.xml
index addc2a5..8815dd4 100644
--- a/hadoop-project-dist/pom.xml
+++ b/hadoop-project-dist/pom.xml
@@ -152,6 +152,13 @@
         <jdiff.compatibility></jdiff.compatibility>
         <jdiff.javadoc.maxmemory>512m</jdiff.javadoc.maxmemory>
       </properties>
+      <dependencies>
+        <dependency>
+          <groupId>xerces</groupId>
+          <artifactId>xercesImpl</artifactId>
+          <version>${xerces.jdiff.version}</version>
+        </dependency>
+      </dependencies>
       <build>
         <plugins>
           <plugin>
@@ -194,6 +201,14 @@
                       <outputDirectory>${project.build.directory}</outputDirectory>
                       <destFileName>hadoop-annotations.jar</destFileName>
                     </artifactItem>
+                    <artifactItem>
+                      <groupId>xerces</groupId>
+                      <artifactId>xercesImpl</artifactId>
+                      <version>${xerces.jdiff.version}</version>
+                      <overWrite>false</overWrite>
+                      <outputDirectory>${project.build.directory}</outputDirectory>
+                      <destFileName>xerces.jar</destFileName>
+                    </artifactItem>
                   </artifactItems>
                 </configuration>
               </execution>
@@ -259,7 +274,7 @@
                              sourceFiles="${basedir}/dev-support/jdiff/Null.java"
                              maxmemory="${jdiff.javadoc.maxmemory}">
                       <doclet name="org.apache.hadoop.classification.tools.IncludePublicAnnotationsJDiffDoclet"
-                              path="${project.build.directory}/hadoop-annotations.jar:${project.build.directory}/jdiff.jar">
+                              path="${project.build.directory}/hadoop-annotations.jar:${project.build.directory}/jdiff.jar:${project.build.directory}/xerces.jar">
                         <param name="-oldapi" value="${project.name} ${jdiff.stable.api}"/>
                         <param name="-newapi" value="${project.name} ${project.version}"/>
                         <param name="-oldapidir" value="${basedir}/dev-support/jdiff"/>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3cf3540f/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index b8fb961..a698126 100755
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -45,6 +45,8 @@
     <!-- These 2 versions are defined here because they are used -->
     <!-- JDIFF generation from embedded ant in the antrun plugin -->
     <jdiff.version>1.0.9</jdiff.version>
+    <!-- Version number for xerces used by JDiff -->
+    <xerces.jdiff.version>2.11.0</xerces.jdiff.version>
 
     <kafka.version>0.8.2.1</kafka.version>
     <hbase.version>1.2.6</hbase.version>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3cf3540f/hadoop-yarn-project/hadoop-yarn/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/pom.xml b/hadoop-yarn-project/hadoop-yarn/pom.xml
index be435d5..288f4bc 100644
--- a/hadoop-yarn-project/hadoop-yarn/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/pom.xml
@@ -113,6 +113,13 @@
         <jdiff.compatibility></jdiff.compatibility>
         <jdiff.javadoc.maxmemory>512m</jdiff.javadoc.maxmemory>
       </properties>
+      <dependencies>
+        <dependency>
+          <groupId>xerces</groupId>
+          <artifactId>xercesImpl</artifactId>
+          <version>${xerces.jdiff.version}</version>
+        </dependency>
+      </dependencies>
       <build>
         <plugins>
           <plugin>
@@ -155,6 +162,14 @@
                       <outputDirectory>${project.build.directory}</outputDirectory>
                       <destFileName>hadoop-annotations.jar</destFileName>
                     </artifactItem>
+                    <artifactItem>
+                      <groupId>xerces</groupId>
+                      <artifactId>xercesImpl</artifactId>
+                      <version>${xerces.version.jdiff}</version>
+                      <overWrite>false</overWrite>
+                      <outputDirectory>${project.build.directory}</outputDirectory>
+                      <destFileName>xerces.jar</destFileName>
+                    </artifactItem>
                   </artifactItems>
                 </configuration>
               </execution>
@@ -193,7 +208,7 @@
                       sourceFiles="${dev-support.relative.dir}/jdiff/Null.java"
                              maxmemory="${jdiff.javadoc.maxmemory}">
                       <doclet name="org.apache.hadoop.classification.tools.IncludePublicAnnotationsJDiffDoclet"
-                              path="${project.build.directory}/hadoop-annotations.jar:${project.build.directory}/jdiff.jar">
+                              path="${project.build.directory}/hadoop-annotations.jar:${project.build.directory}/jdiff.jar:${project.build.directory}/xerces.jar">
                         <param name="-oldapi" value="${project.name} ${jdiff.stable.api}"/>
                         <param name="-newapi" value="${project.name} ${project.version}"/>
                         <param name="-oldapidir" value="${basedir}/${dev-support.relative.dir}/jdiff"/>


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


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

Branch: refs/heads/YARN-5734
Commit: c59418fb19094e9c735e631e098637d3b5eba3b3
Parents: ba53795
Author: Xuan <xg...@apache.org>
Authored: Wed Aug 23 11:11:41 2017 -0700
Committer: Jonathan Hung <jh...@linkedin.com>
Committed: Wed Sep 20 17:40:54 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/c59418fb/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/c59418fb/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


[25/50] [abbrv] hadoop git commit: Revert "MAPREDUCE-6958. Shuffle audit logger should log size of shuffle transfer. Contributed by Jason Lowe"

Posted by jh...@apache.org.
Revert "MAPREDUCE-6958. Shuffle audit logger should log size of shuffle transfer. Contributed by Jason Lowe"

This reverts commit b3d61304f2fa4a99526f7a60ccaac9f262083079.


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

Branch: refs/heads/YARN-5734
Commit: ea845ba58c585647c4be8d30d9b814f098e34a12
Parents: aa6e8d2
Author: Jason Lowe <jl...@apache.org>
Authored: Tue Sep 19 08:45:05 2017 -0500
Committer: Jason Lowe <jl...@apache.org>
Committed: Tue Sep 19 08:45:05 2017 -0500

----------------------------------------------------------------------
 .../org/apache/hadoop/mapred/ShuffleHandler.java  | 18 +++++++-----------
 1 file changed, 7 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea845ba5/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java
index 06a3e42..863da7e 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java
@@ -992,6 +992,13 @@ public class ShuffleHandler extends AuxiliaryService {
         return;
       }
 
+      // this audit log is disabled by default,
+      // to turn it on please enable this audit log
+      // on log4j.properties by uncommenting the setting
+      if (AUDITLOG.isDebugEnabled()) {
+        AUDITLOG.debug("shuffle for " + jobQ.get(0) + " mappers: " + mapIds +
+                         " reducer " + reduceQ.get(0));
+      }
       int reduceId;
       String jobId;
       try {
@@ -1176,17 +1183,6 @@ public class ShuffleHandler extends AuxiliaryService {
 
       // Now set the response headers.
       setResponseHeaders(response, keepAliveParam, contentLength);
-
-      // this audit log is disabled by default,
-      // to turn it on please enable this audit log
-      // on log4j.properties by uncommenting the setting
-      if (AUDITLOG.isDebugEnabled()) {
-        StringBuilder sb = new StringBuilder("shuffle for ").append(jobId);
-        sb.append(" mappers: ").append(mapIds);
-        sb.append(" reducer ").append(reduce);
-        sb.append(" length ").append(contentLength);
-        AUDITLOG.debug(sb.toString());
-      }
     }
 
     protected void setResponseHeaders(HttpResponse response,


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


[36/50] [abbrv] hadoop git commit: HDFS-11035. Better documentation for maintenace mode and upgrade domain.

Posted by jh...@apache.org.
HDFS-11035. Better documentation for maintenace mode and upgrade domain.


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

Branch: refs/heads/YARN-5734
Commit: ce943eb17a4218d8ac1f5293c6726122371d8442
Parents: 230b85d
Author: Ming Ma <mi...@twitter.com>
Authored: Wed Sep 20 09:36:33 2017 -0700
Committer: Ming Ma <mi...@twitter.com>
Committed: Wed Sep 20 09:36:33 2017 -0700

----------------------------------------------------------------------
 .../src/site/markdown/HdfsDataNodeAdminGuide.md | 165 ++++++++++++++++++
 .../src/site/markdown/HdfsUpgradeDomain.md      | 167 +++++++++++++++++++
 hadoop-project/src/site/site.xml                |   4 +-
 3 files changed, 335 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ce943eb1/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsDataNodeAdminGuide.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsDataNodeAdminGuide.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsDataNodeAdminGuide.md
new file mode 100644
index 0000000..d6f288e
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsDataNodeAdminGuide.md
@@ -0,0 +1,165 @@
+<!---
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+   http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License. See accompanying LICENSE file.
+-->
+
+HDFS DataNode Admin Guide
+=================
+
+<!-- MACRO{toc|fromDepth=0|toDepth=3} -->
+
+Overview
+--------
+
+The Hadoop Distributed File System (HDFS) namenode maintains states of all datanodes.
+There are two types of states. The fist type describes the liveness of a datanode indicating if
+the node is live, dead or stale. The second type describes the admin state indicating if the node
+is in service, decommissioned or under maintenance.
+
+When an administrator decommission a datanode, the datanode will first be transitioned into
+`DECOMMISSION_INPROGRESS` state. After all blocks belonging to that datanode have been fully replicated elsewhere
+based on each block's replication factor. the datanode will be transitioned to `DECOMMISSIONED` state. After that,
+the administrator can shutdown the node to perform long-term repair and maintenance that could take days or weeks.
+After the machine has been repaired, the machine can be recommissioned back to the cluster.
+
+Sometimes administrators only need to take datanodes down for minutes/hours to perform short-term repair/maintenance.
+In such scenario, the HDFS block replication overhead incurred by decommission might not be necessary and a light-weight process is desirable.
+And that is what maintenance state is used for. When an administrator put a datanode in maintenance state, the datanode will first be transitioned
+to `ENTERING_MAINTENANCE` state. As long as all blocks belonging to that datanode is minimally replicated elsewhere, the datanode
+will immediately be transitioned to `IN_MAINTENANCE` state. After the maintenance has completed, the administrator can take the datanode
+out of the maintenance state. In addition, maintenance state supports timeout that allows administrators to config the maximum duration in
+which a datanode is allowed to stay in maintenance state. After the timeout, the datanode will be transitioned out of maintenance state
+automatically by HDFS without human intervention.
+
+In summary, datanode admin operations include the followings:
+
+* Decommission
+* Recommission
+* Putting nodes in maintenance state
+* Taking nodes out of maintenance state
+
+And datanode admin states include the followings:
+
+* `NORMAL` The node is in service.
+* `DECOMMISSIONED` The node has been decommissioned.
+* `DECOMMISSION_INPROGRESS` The node is being transitioned to DECOMMISSIONED state.
+* `IN_MAINTENANCE` The node in in maintenance state.
+* `ENTERING_MAINTENANCE` The node is being transitioned to maintenance state.
+
+
+Host-level settings
+-----------
+
+To perform any of datanode admin operations, there are two steps.
+
+* Update host-level configuration files to indicate the desired admin states of targeted datanodes. There are two supported formats for configuration files.
+    * Hostname-only configuration. Each line includes the hostname/ip address for a datanode. That is the default format.
+    * JSON-based configuration. The configuration is in JSON format. Each element maps to one datanode and each datanode can have multiple properties. This format is required to put datanodes to maintenance states.
+
+* Run the following command to have namenode reload the host-level configuration files.
+`hdfs dfsadmin [-refreshNodes]`
+
+### Hostname-only configuration
+This is the default configuration used by the namenode. It only supports node decommission and recommission; it doesn't support admin operations related to maintenance state. Use `dfs.hosts` and `dfs.hosts.exclude` as explained in [hdfs-default.xml](./hdfs-default.xml).
+
+In the following example, `host1` and `host2` need to be in service.
+`host3` and `host4` need to be in decommissioned state.
+
+dfs.hosts file
+```text
+host1
+host2
+host3
+host4
+```
+dfs.hosts.exclude file
+```text
+host3
+host4
+```
+
+### JSON-based configuration
+
+JSON-based format is the new configuration format that supports generic properties on datanodes. Set the following
+configurations to enable JSON-based format as explained in [hdfs-default.xml](./hdfs-default.xml).
+
+
+| Setting | Value |
+|:---- |:---- |
+|`dfs.namenode.hosts.provider.classname`| `org.apache.hadoop.hdfs.server.blockmanagement.CombinedHostFileManager`|
+|`dfs.hosts`| the path of the json hosts file |
+
+Here is the list of currently supported properties by HDFS.
+
+
+| Property | Description |
+|:---- |:---- |
+|`hostName`| Required. The host name of the datanode. |
+|`upgradeDomain`| Optional. The upgrade domain id of the datanode. |
+|`adminState`| Optional. The expected admin state. The default value is `NORMAL`; `DECOMMISSIONED` for decommission; `IN_MAINTENANCE` for maintenance state. |
+|`port`| Optional. the port number of the datanode |
+|`maintenanceExpireTimeInMS`| Optional. The epoch time in milliseconds until which the datanode will remain in maintenance state. The default value is forever. |
+
+In the following example, `host1` and `host2` need to in service. `host3` need to be in decommissioned state. `host4` need to be in in maintenance state.
+
+dfs.hosts file
+```json
+[
+  {
+    "hostName": "host1"
+  },
+  {
+    "hostName": "host2",
+    "upgradeDomain": "ud0"
+  },
+  {
+    "hostName": "host3",
+    "adminState": "DECOMMISSIONED"
+  },
+  {
+    "hostName": "host4",
+    "upgradeDomain": "ud2",
+    "adminState": "IN_MAINTENANCE"
+  }
+]
+```
+
+
+Cluster-level settings
+-----------
+
+There are several cluster-level settings related to datanode administration.
+For common use cases, you should rely on the default values. Please refer to
+[hdfs-default.xml](./hdfs-default.xml) for descriptions and default values.
+
+```text
+dfs.namenode.maintenance.replication.min
+dfs.namenode.decommission.interval
+dfs.namenode.decommission.blocks.per.interval
+dfs.namenode.decommission.max.concurrent.tracked.nodes
+```
+
+Metrics
+-----------
+
+Admin states are part of the namenode's webUI and JMX. As explained in [HDFSCommands.html](./HDFSCommands.html), you can also verify admin states using the following commands.
+
+Use `dfsadmin` to check admin states at the cluster level.
+
+`hdfs dfsadmin -report`
+
+Use `fsck` to check admin states of datanodes storing data at a specific path. For backward compatibility, a special flag is required to return maintenance states.
+
+```text
+hdfs fsck <path> // only show decommission state
+hdfs fsck <path> -maintenance // include maintenance state
+```

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ce943eb1/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsUpgradeDomain.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsUpgradeDomain.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsUpgradeDomain.md
new file mode 100644
index 0000000..15a4bae
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsUpgradeDomain.md
@@ -0,0 +1,167 @@
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+      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.
+-->
+
+HDFS Upgrade Domain
+====================
+
+<!-- MACRO{toc|fromDepth=0|toDepth=3} -->
+
+
+Introduction
+------------
+
+The current default HDFS block placement policy guarantees that a block’s 3 replicas will be placed
+on at least 2 racks. Specifically one replica is placed on one rack and the other two replicas
+are placed on another rack during write pipeline. This is a good compromise between rack diversity and write-pipeline efficiency. Note that
+subsequent load balancing or machine membership change might cause 3 replicas of a block to be distributed
+across 3 different racks. Thus any 3 datanodes in different racks could store 3 replicas of a block.
+
+
+However, the default placement policy impacts how we should perform datanode rolling upgrade.
+[HDFS Rolling Upgrade document](./HdfsRollingUpgrade.html) explains how the datanodes can be upgraded in a rolling
+fashion without downtime. Because any 3 datanodes in different racks could store all the replicas of a block, it is
+important to perform sequential restart of datanodes one at a time in order to minimize the impact on data availability
+and read/write operations. Upgrading one rack at a time is another option; but that will increase the chance of
+data unavailability if there is machine failure at another rack during the upgrade.
+
+The side effect of this sequential datanode rolling upgrade strategy is longer
+upgrade duration for larger clusters.
+
+
+Architecture
+-------
+
+To address the limitation of block placement policy on rolling upgrade, the concept of upgrade domain
+has been added to HDFS via a new block placement policy. The idea is to group datanodes in a new
+dimension called upgrade domain, in addition to the existing rack-based grouping.
+For example, we can assign all datanodes in the first position of any rack to upgrade domain ud_01,
+nodes in the second position to upgrade domain ud_02 and so on.
+
+The namenode provides BlockPlacementPolicy interface to support any custom block placement besides
+the default block placement policy. A new upgrade domain block placement policy based on this interface
+is available in HDFS. It will make sure replicas of any given block are distributed across machines from different upgrade domains.
+By default, 3 replicas of any given block are placed on 3 different upgrade domains. This means all datanodes belonging to
+a specific upgrade domain collectively won't store more than one replica of any block.
+
+With upgrade domain block placement policy in place, we can upgrade all datanodes belonging to one upgrade domain at the
+same time without impacting data availability. Only after finishing upgrading one upgrade domain we move to the next
+upgrade domain until all upgrade domains have been upgraded. Such procedure will ensure no two replicas of any given
+block will be upgraded at the same time. This means we can upgrade many machines at the same time for a large cluster.
+And as the cluster continues to scale, new machines will be added to the existing upgrade domains without impact the
+parallelism of the upgrade.
+
+For an existing cluster with the default block placement policy, after switching to the new upgrade domain block
+placement policy, any newly created blocks will conform the new policy. The old blocks allocated based on the old policy
+need to migrated the new policy. There is a migrator tool you can use. See HDFS-8789 for details.
+
+
+Settings
+-------
+
+To enable upgrade domain on your clusters, please follow these steps:
+
+* Assign datanodes to individual upgrade domain groups.
+* Enable upgrade domain block placement policy.
+* Migrate blocks allocated based on old block placement policy to the new upgrade domain policy.
+
+### Upgrade domain id assignment
+
+How a datanode maps to an upgrade domain id is defined by administrators and specific to the cluster layout.
+A common way to use the rack position of the machine as its upgrade domain id.
+
+To configure mapping from host name to its upgrade domain id, we need to use json-based host configuration file.
+by setting the following property as explained in [hdfs-default.xml](./hdfs-default.xml).
+
+| Setting | Value |
+|:---- |:---- |
+|`dfs.namenode.hosts.provider.classname` | `org.apache.hadoop.hdfs.server.blockmanagement.CombinedHostFileManager`|
+|`dfs.hosts`| the path of the json hosts file |
+
+The json hosts file defines the property for all hosts. In the following example,
+there are 4 datanodes in 2 racks; the machines at rack position 01 belong to upgrade domain 01;
+the machines at rack position 02 belong to upgrade domain 02.
+
+```json
+[
+  {
+    "hostName": "dcA­rackA­01",
+    "upgradeDomain": "01"
+  },
+  {
+    "hostName": "dcA­rackA­02",
+    "upgradeDomain": "02"
+  },
+  {
+    "hostName": "dcA­rackB­01",
+    "upgradeDomain": "01"
+  },
+  {
+    "hostName": "dcA­rackB­02",
+    "upgradeDomain": "02"
+  }
+]
+```
+
+
+### Enable upgrade domain block placement policy
+
+After each datanode has been assigned an upgrade domain id, the next step is to enable
+upgrade domain block placement policy with the following configuration as explained in [hdfs-default.xml](./hdfs-default.xml).
+
+| Setting | Value |
+|:---- |:---- |
+|`dfs.block.replicator.classname`| `org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyWithUpgradeDomain` |
+
+After restarting of namenode, the new policy will be used for any new block allocation.
+
+
+### Migration
+
+If you change the block placement policy of an existing cluster, you will need to make sure the
+blocks allocated prior to the block placement policy change conform the new block placement policy.
+
+HDFS-8789 provides the initial draft patch of a client-side migration tool. After the tool is committed,
+we will be able to describe how to use the tool.
+
+
+Rolling restart based on upgrade domains
+-------
+
+During cluster administration, we might need to restart datanodes to pick up new configuration, new hadoop release
+or JVM version and so on. With upgrade domains enabled and all blocks on the cluster conform to the new policy, we can now
+restart datanodes in batches, one upgrade domain at a time. Whether it is manual process or via automation, the steps are
+
+* Group datanodes by upgrade domains based on dfsadmin or JMX's datanode information.
+* For each upgrade domain
+    * (Optional) put all the nodes in that upgrade domain to maintenance state (refer to [HdfsDataNodeAdminGuide.html](./HdfsDataNodeAdminGuide.html)).
+    * Restart all those nodes.
+    * Check if all datanodes are healthy after restart. Unhealthy nodes should be decommissioned.
+    * (Optional) Take all those nodes out of maintenance state.
+
+
+Metrics
+-----------
+
+Upgrade domains are part of namenode's JMX. As explained in [HDFSCommands.html](./HDFSCommands.html), you can also verify upgrade domains using the following commands.
+
+Use `dfsadmin` to check upgrade domains at the cluster level.
+
+`hdfs dfsadmin -report`
+
+Use `fsck` to check upgrade domains of datanodes storing data at a specific path.
+
+`hdfs fsck <path> -files -blocks -upgradedomains`

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ce943eb1/hadoop-project/src/site/site.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/src/site/site.xml b/hadoop-project/src/site/site.xml
index 4685e2a..a88f0e3 100644
--- a/hadoop-project/src/site/site.xml
+++ b/hadoop-project/src/site/site.xml
@@ -101,7 +101,9 @@
       <item name="Synthetic Load Generator" href="hadoop-project-dist/hadoop-hdfs/SLGUserGuide.html"/>
       <item name="Erasure Coding" href="hadoop-project-dist/hadoop-hdfs/HDFSErasureCoding.html"/>
       <item name="Disk Balancer" href="hadoop-project-dist/hadoop-hdfs/HDFSDiskbalancer.html"/>
-   </menu>
+      <item name="Upgrade Domain" href="hadoop-project-dist/hadoop-hdfs/HdfsUpgradeDomain.html"/>
+      <item name="DataNode Admin" href="hadoop-project-dist/hadoop-hdfs/HdfsDataNodeAdminGuide.html"/>
+    </menu>
 
     <menu name="MapReduce" inherit="top">
       <item name="Tutorial" href="hadoop-mapreduce-client/hadoop-mapreduce-client-core/MapReduceTutorial.html"/>


---------------------------------------------------------------------
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: MAPREDUCE-6958. Shuffle audit logger should log size of shuffle transfer. Contributed by Jason Lowe

Posted by jh...@apache.org.
MAPREDUCE-6958. Shuffle audit logger should log size of shuffle transfer. 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/3a20debd
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/3a20debd
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/3a20debd

Branch: refs/heads/YARN-5734
Commit: 3a20debddeac69596ceb5b36f8413529ea8570e6
Parents: ea845ba
Author: Jason Lowe <jl...@apache.org>
Authored: Tue Sep 19 09:13:17 2017 -0500
Committer: Jason Lowe <jl...@apache.org>
Committed: Tue Sep 19 09:13:17 2017 -0500

----------------------------------------------------------------------
 .../org/apache/hadoop/mapred/ShuffleHandler.java  | 18 +++++++++++-------
 1 file changed, 11 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a20debd/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java
index 863da7e..b7f2c6d 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java
@@ -992,13 +992,6 @@ public class ShuffleHandler extends AuxiliaryService {
         return;
       }
 
-      // this audit log is disabled by default,
-      // to turn it on please enable this audit log
-      // on log4j.properties by uncommenting the setting
-      if (AUDITLOG.isDebugEnabled()) {
-        AUDITLOG.debug("shuffle for " + jobQ.get(0) + " mappers: " + mapIds +
-                         " reducer " + reduceQ.get(0));
-      }
       int reduceId;
       String jobId;
       try {
@@ -1183,6 +1176,17 @@ public class ShuffleHandler extends AuxiliaryService {
 
       // Now set the response headers.
       setResponseHeaders(response, keepAliveParam, contentLength);
+
+      // this audit log is disabled by default,
+      // to turn it on please enable this audit log
+      // on log4j.properties by uncommenting the setting
+      if (AUDITLOG.isDebugEnabled()) {
+        StringBuilder sb = new StringBuilder("shuffle for ");
+        sb.append(jobId).append(" reducer ").append(reduce);
+        sb.append(" length ").append(contentLength);
+        sb.append(" mappers: ").append(mapIds);
+        AUDITLOG.debug(sb.toString());
+      }
     }
 
     protected void setResponseHeaders(HttpResponse response,


---------------------------------------------------------------------
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-12460. Make addErasureCodingPolicy an idempotent operation. Contributed by Sammi Chen

Posted by jh...@apache.org.
HDFS-12460. Make addErasureCodingPolicy an idempotent operation. Contributed by Sammi Chen


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

Branch: refs/heads/YARN-5734
Commit: 0f9af246e89e4ad3c4d7ff2c1d7ec9b397494a03
Parents: e81596d
Author: Kai Zheng <ka...@intel.com>
Authored: Mon Sep 18 18:07:12 2017 +0800
Committer: Kai Zheng <ka...@intel.com>
Committed: Mon Sep 18 18:07:12 2017 +0800

----------------------------------------------------------------------
 .../hdfs/server/namenode/ErasureCodingPolicyManager.java      | 7 ++++---
 .../org/apache/hadoop/hdfs/TestErasureCodingPolicies.java     | 2 +-
 .../hadoop/hdfs/server/namenode/TestNamenodeRetryCache.java   | 4 ++--
 .../hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java  | 4 ++--
 4 files changed, 9 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0f9af246/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
index 3a46c30..90699b4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
@@ -242,14 +242,15 @@ public final class ErasureCodingPolicyManager {
         policy.getSchema(), policy.getCellSize());
     for (ErasureCodingPolicy p : getPolicies()) {
       if (p.getName().equals(assignedNewName)) {
-        throw new HadoopIllegalArgumentException("The policy name " +
-            assignedNewName + " already exists");
+        LOG.info("The policy name " + assignedNewName + " already exists");
+        return p;
       }
       if (p.getSchema().equals(policy.getSchema()) &&
           p.getCellSize() == policy.getCellSize()) {
-        throw new HadoopIllegalArgumentException("A policy with same schema "
+        LOG.info("A policy with same schema "
             + policy.getSchema().toString() + " and cell size "
             + p.getCellSize() + " already exists");
+        return p;
       }
     }
     policy.setName(assignedNewName);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0f9af246/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
index 19277c4..4f2040b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
@@ -718,7 +718,7 @@ public class TestErasureCodingPolicies {
     policyArray  = new ErasureCodingPolicy[]{policy0};
     responses = fs.addErasureCodingPolicies(policyArray);
     assertEquals(1, responses.length);
-    assertFalse(responses[0].isSucceed());
+    assertTrue(responses[0].isSucceed());
 
     // Test add policy successfully
     newPolicy =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0f9af246/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeRetryCache.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeRetryCache.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeRetryCache.java
index d217813..42ff698 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeRetryCache.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeRetryCache.java
@@ -436,7 +436,7 @@ public class TestNamenodeRetryCache {
 
     LightWeightCache<CacheEntry, CacheEntry> cacheSet = 
         (LightWeightCache<CacheEntry, CacheEntry>) namesystem.getRetryCache().getCacheSet();
-    assertEquals("Retry cache size is wrong", 26, cacheSet.size());
+    assertEquals("Retry cache size is wrong", 34, cacheSet.size());
     
     Map<CacheEntry, CacheEntry> oldEntries = 
         new HashMap<CacheEntry, CacheEntry>();
@@ -455,7 +455,7 @@ public class TestNamenodeRetryCache {
     assertTrue(namesystem.hasRetryCache());
     cacheSet = (LightWeightCache<CacheEntry, CacheEntry>) namesystem
         .getRetryCache().getCacheSet();
-    assertEquals("Retry cache size is wrong", 26, cacheSet.size());
+    assertEquals("Retry cache size is wrong", 34, cacheSet.size());
     iter = cacheSet.iterator();
     while (iter.hasNext()) {
       CacheEntry entry = iter.next();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0f9af246/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
index b40006b..1d114d6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
@@ -166,7 +166,7 @@ public class TestRetryCacheWithHA {
     FSNamesystem fsn0 = cluster.getNamesystem(0);
     LightWeightCache<CacheEntry, CacheEntry> cacheSet = 
         (LightWeightCache<CacheEntry, CacheEntry>) fsn0.getRetryCache().getCacheSet();
-    assertEquals("Retry cache size is wrong", 26, cacheSet.size());
+    assertEquals("Retry cache size is wrong", 34, cacheSet.size());
     
     Map<CacheEntry, CacheEntry> oldEntries = 
         new HashMap<CacheEntry, CacheEntry>();
@@ -187,7 +187,7 @@ public class TestRetryCacheWithHA {
     FSNamesystem fsn1 = cluster.getNamesystem(1);
     cacheSet = (LightWeightCache<CacheEntry, CacheEntry>) fsn1
         .getRetryCache().getCacheSet();
-    assertEquals("Retry cache size is wrong", 26, cacheSet.size());
+    assertEquals("Retry cache size is wrong", 34, cacheSet.size());
     iter = cacheSet.iterator();
     while (iter.hasNext()) {
       CacheEntry entry = iter.next();


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


[21/50] [abbrv] hadoop git commit: HDFS-12480. TestNameNodeMetrics#testTransactionAndCheckpointMetrics Fails in trunk. Contributed by Hanisha Koneru

Posted by jh...@apache.org.
HDFS-12480. TestNameNodeMetrics#testTransactionAndCheckpointMetrics Fails in trunk. 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/31b58406
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/31b58406
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/31b58406

Branch: refs/heads/YARN-5734
Commit: 31b58406ac369716ef1665b7d60a3409117bdf9d
Parents: 595d478
Author: Brahma Reddy Battula <br...@apache.org>
Authored: Tue Sep 19 10:37:07 2017 +0530
Committer: Brahma Reddy Battula <br...@apache.org>
Committed: Tue Sep 19 10:37:07 2017 +0530

----------------------------------------------------------------------
 .../namenode/metrics/TestNameNodeMetrics.java     | 18 +++++++++---------
 1 file changed, 9 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/31b58406/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java
index 077a5f8..db9adbe 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java
@@ -851,22 +851,22 @@ public class TestNameNodeMetrics {
         getMetrics(NS_METRICS));
     
     assertGauge("LastCheckpointTime", lastCkptTime, getMetrics(NS_METRICS));
-    assertGauge("LastWrittenTransactionId", 3L, getMetrics(NS_METRICS));
-    assertGauge("TransactionsSinceLastCheckpoint", 3L, getMetrics(NS_METRICS));
-    assertGauge("TransactionsSinceLastLogRoll", 3L, getMetrics(NS_METRICS));
+    assertGauge("LastWrittenTransactionId", 4L, getMetrics(NS_METRICS));
+    assertGauge("TransactionsSinceLastCheckpoint", 4L, getMetrics(NS_METRICS));
+    assertGauge("TransactionsSinceLastLogRoll", 4L, getMetrics(NS_METRICS));
     
     fs.mkdirs(new Path(TEST_ROOT_DIR_PATH, "/tmp"));
     
     assertGauge("LastCheckpointTime", lastCkptTime, getMetrics(NS_METRICS));
-    assertGauge("LastWrittenTransactionId", 4L, getMetrics(NS_METRICS));
-    assertGauge("TransactionsSinceLastCheckpoint", 4L, getMetrics(NS_METRICS));
-    assertGauge("TransactionsSinceLastLogRoll", 4L, getMetrics(NS_METRICS));
+    assertGauge("LastWrittenTransactionId", 5L, getMetrics(NS_METRICS));
+    assertGauge("TransactionsSinceLastCheckpoint", 5L, getMetrics(NS_METRICS));
+    assertGauge("TransactionsSinceLastLogRoll", 5L, getMetrics(NS_METRICS));
     
     cluster.getNameNodeRpc().rollEditLog();
     
     assertGauge("LastCheckpointTime", lastCkptTime, getMetrics(NS_METRICS));
-    assertGauge("LastWrittenTransactionId", 6L, getMetrics(NS_METRICS));
-    assertGauge("TransactionsSinceLastCheckpoint", 6L, getMetrics(NS_METRICS));
+    assertGauge("LastWrittenTransactionId", 7L, getMetrics(NS_METRICS));
+    assertGauge("TransactionsSinceLastCheckpoint", 7L, getMetrics(NS_METRICS));
     assertGauge("TransactionsSinceLastLogRoll", 1L, getMetrics(NS_METRICS));
     
     cluster.getNameNodeRpc().setSafeMode(SafeModeAction.SAFEMODE_ENTER, false);
@@ -876,7 +876,7 @@ public class TestNameNodeMetrics {
     long newLastCkptTime = MetricsAsserts.getLongGauge("LastCheckpointTime",
         getMetrics(NS_METRICS));
     assertTrue(lastCkptTime < newLastCkptTime);
-    assertGauge("LastWrittenTransactionId", 8L, getMetrics(NS_METRICS));
+    assertGauge("LastWrittenTransactionId", 9L, getMetrics(NS_METRICS));
     assertGauge("TransactionsSinceLastCheckpoint", 1L, getMetrics(NS_METRICS));
     assertGauge("TransactionsSinceLastLogRoll", 1L, getMetrics(NS_METRICS));
   }


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

Branch: refs/heads/YARN-5734
Commit: 8d9ba97e0e3a6ec1d722d011db6fba223c07a691
Parents: a6a9bae
Author: Xuan <xg...@apache.org>
Authored: Fri Feb 24 15:58:12 2017 -0800
Committer: Jonathan Hung <jh...@linkedin.com>
Committed: Wed Sep 20 17:39:06 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/8d9ba97e/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/8d9ba97e/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/8d9ba97e/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


[28/50] [abbrv] hadoop git commit: HDFS-12449. TestReconstructStripedFile.testNNSendsErasureCodingTasks randomly cannot finish in 60s. (SammiChen via lei)

Posted by jh...@apache.org.
HDFS-12449. TestReconstructStripedFile.testNNSendsErasureCodingTasks randomly cannot finish in 60s. (SammiChen via lei)


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

Branch: refs/heads/YARN-5734
Commit: 7bbeacb75e93261dbda0e8efcde510e5fcf83efb
Parents: fda1221
Author: Lei Xu <le...@apache.org>
Authored: Tue Sep 19 11:50:01 2017 -0700
Committer: Lei Xu <le...@apache.org>
Committed: Tue Sep 19 11:50:01 2017 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/TestReconstructStripedFile.java  | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7bbeacb7/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReconstructStripedFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReconstructStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReconstructStripedFile.java
index 72b1412..713a10b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReconstructStripedFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReconstructStripedFile.java
@@ -456,8 +456,8 @@ public class TestReconstructStripedFile {
     ErasureCodingPolicy policy = StripedFileTestUtil.getDefaultECPolicy();
     fs.getClient().setErasureCodingPolicy("/", policy.getName());
 
-    final int fileLen = cellSize * ecPolicy.getNumDataUnits() * 2;
-    for (int i = 0; i < 100; i++) {
+    final int fileLen = cellSize * ecPolicy.getNumDataUnits();
+    for (int i = 0; i < 50; i++) {
       writeFile(fs, "/ec-file-" + i, fileLen);
     }
 


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


[04/50] [abbrv] hadoop git commit: YARN-6977. Node information is not provided for non am containers in RM logs. (Suma Shivaprasad via wangda)

Posted by jh...@apache.org.
YARN-6977. Node information is not provided for non am containers in RM logs. (Suma Shivaprasad via wangda)

Change-Id: I0c44d09a560446dee2ba68c2b9ae69fce0ec1d3e
(cherry picked from commit 8a42e922fad613f3cf1cc6cb0f3fa72546a9cc56)


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

Branch: refs/heads/YARN-5734
Commit: 958e8c0e257216c82f68fee726e5280a919da94a
Parents: ef8cd5d
Author: Wangda Tan <wa...@apache.org>
Authored: Fri Sep 15 21:24:11 2017 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Fri Sep 15 21:29:31 2017 -0700

----------------------------------------------------------------------
 .../resourcemanager/scheduler/SchedulerNode.java       |  8 --------
 .../scheduler/common/fica/FiCaSchedulerNode.java       | 13 +++++++++++++
 .../scheduler/fair/FSSchedulerNode.java                | 10 ++++++++++
 3 files changed, 23 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/958e8c0e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.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/SchedulerNode.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java
index 272537c..90fa3e4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java
@@ -170,14 +170,6 @@ public abstract class SchedulerNode {
 
     launchedContainers.put(container.getId(),
         new ContainerInfo(rmContainer, launchedOnNode));
-
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Assigned container " + container.getId() + " of capacity "
-              + container.getResource() + " on host " + rmNode.getNodeAddress()
-              + ", which has " + numContainers + " containers, "
-              + getAllocatedResource() + " used and " + getUnallocatedResource()
-              + " available after allocation");
-    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/958e8c0e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerNode.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerNode.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerNode.java
index c26a11b..7277779 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerNode.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerNode.java
@@ -160,4 +160,17 @@ public class FiCaSchedulerNode extends SchedulerNode {
   public synchronized Map<ContainerId, RMContainer> getKillableContainers() {
     return Collections.unmodifiableMap(killableContainers);
   }
+
+  protected synchronized void allocateContainer(RMContainer rmContainer,
+      boolean launchedOnNode) {
+    super.allocateContainer(rmContainer, launchedOnNode);
+
+    final Container container = rmContainer.getContainer();
+    LOG.info("Assigned container " + container.getId() + " of capacity "
+          + container.getResource() + " on host " + getRMNode().getNodeAddress()
+          + ", which has " + getNumContainers() + " containers, "
+          + getAllocatedResource() + " used and " + getUnallocatedResource()
+          + " available after allocation");
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/958e8c0e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSSchedulerNode.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSSchedulerNode.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSSchedulerNode.java
index 93646f4..44ec9c3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSSchedulerNode.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSSchedulerNode.java
@@ -25,6 +25,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.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
@@ -239,6 +240,15 @@ public class FSSchedulerNode extends SchedulerNode {
   protected synchronized void allocateContainer(RMContainer rmContainer,
                                                 boolean launchedOnNode) {
     super.allocateContainer(rmContainer, launchedOnNode);
+    if (LOG.isDebugEnabled()) {
+      final Container container = rmContainer.getContainer();
+      LOG.debug("Assigned container " + container.getId() + " of capacity "
+          + container.getResource() + " on host " + getRMNode().getNodeAddress()
+          + ", which has " + getNumContainers() + " containers, "
+          + getAllocatedResource() + " used and " + getUnallocatedResource()
+          + " available after allocation");
+    }
+
     Resource allocated = rmContainer.getAllocatedResource();
     if (!Resources.isNone(allocated)) {
       // check for satisfied preemption request and update bookkeeping


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


[05/50] [abbrv] hadoop git commit: YARN-7149. Cross-queue preemption sometimes starves an underserved queue. (Eric Payne via wangda)

Posted by jh...@apache.org.
YARN-7149. Cross-queue preemption sometimes starves an underserved queue. (Eric Payne via wangda)

Change-Id: Ib269991dbebce160378e8372ee6d24849c4a5ed6
(cherry picked from commit 3dfa937a1fadfc62947755872515f549b3b15e6a)


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

Branch: refs/heads/YARN-5734
Commit: 38c14ef8d8a094a7101917eb77d90f5e62324f61
Parents: 958e8c0
Author: Wangda Tan <wa...@apache.org>
Authored: Fri Sep 15 21:25:21 2017 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Fri Sep 15 21:29:39 2017 -0700

----------------------------------------------------------------------
 .../scheduler/capacity/UsersManager.java        |  4 +-
 .../capacity/TestContainerAllocation.java       | 50 ++++++++++++++++++++
 .../scheduler/capacity/TestLeafQueue.java       |  8 ++--
 3 files changed, 57 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c14ef8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/UsersManager.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/UsersManager.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/UsersManager.java
index 5f7d185..33f30b0 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/UsersManager.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/UsersManager.java
@@ -731,7 +731,9 @@ public class UsersManager implements AbstractUsersManager {
      * should be higher than queue-hard-limit * ulMin
      */
     float usersSummedByWeight = activeUsersTimesWeights;
-    Resource resourceUsed = totalResUsageForActiveUsers.getUsed(nodePartition);
+    Resource resourceUsed = Resources.add(
+                            totalResUsageForActiveUsers.getUsed(nodePartition),
+                            required);
 
     // For non-activeUser calculation, consider all users count.
     if (!activeUser) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c14ef8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.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/TestContainerAllocation.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/TestContainerAllocation.java
index dd6b25b..906febf 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/TestContainerAllocation.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/TestContainerAllocation.java
@@ -24,6 +24,7 @@ import java.util.List;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.security.SecurityUtilTestHelper;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.api.records.Container;
@@ -887,4 +888,53 @@ public class TestContainerAllocation {
 
     rm1.close();
   }
+
+
+
+  @Test(timeout = 60000)
+  public void testUserLimitAllocationMultipleContainers() throws Exception {
+    CapacitySchedulerConfiguration newConf =
+        (CapacitySchedulerConfiguration) TestUtils
+            .getConfigurationWithMultipleQueues(conf);
+    newConf.setUserLimit("root.c", 50);
+    MockRM rm1 = new MockRM(newConf);
+
+    rm1.getRMContext().setNodeLabelManager(mgr);
+    rm1.start();
+    MockNM nm1 = rm1.registerNode("h1:1234", 1000 * GB);
+
+    // launch app from 1st user to queue C, AM container should be launched in nm1
+    RMApp app1 = rm1.submitApp(2 * GB, "app", "user1", null, "c");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
+
+    // launch app from 2nd user to queue C, AM container should be launched in nm1
+    RMApp app2 = rm1.submitApp(2 * GB, "app", "user2", null, "c");
+    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm1, nm1);
+
+    // Each application asks 1000 * 5GB containers
+    am1.allocate("*", 5 * GB, 1000, null);
+    am1.allocate("h1", 5 * GB, 1000, null);
+    am1.allocate(NetworkTopology.DEFAULT_RACK, 5 * GB, 1000, null);
+
+    // Each application asks 1000 * 5GB containers
+    am2.allocate("*", 5 * GB, 1000, null);
+    am2.allocate("h1", 5 * GB, 1000, null);
+    am2.allocate(NetworkTopology.DEFAULT_RACK, 5 * GB, 1000, null);
+
+    CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
+    RMNode rmNode1 = rm1.getRMContext().getRMNodes().get(nm1.getNodeId());
+
+    FiCaSchedulerApp schedulerApp1 =
+        cs.getApplicationAttempt(am1.getApplicationAttemptId());
+    FiCaSchedulerApp schedulerApp2 =
+        cs.getApplicationAttempt(am2.getApplicationAttemptId());
+
+    // container will be allocated to am1
+    // App1 will get 2 container allocated (plus AM container)
+    cs.handle(new NodeUpdateSchedulerEvent(rmNode1));
+    Assert.assertEquals(101, schedulerApp1.getLiveContainers().size());
+    Assert.assertEquals(100, schedulerApp2.getLiveContainers().size());
+
+    rm1.close();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/38c14ef8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.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/TestLeafQueue.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/TestLeafQueue.java
index d45f756..a32352b 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/TestLeafQueue.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/TestLeafQueue.java
@@ -1252,7 +1252,7 @@ public class TestLeafQueue {
     //app4 is user 0
     //maxqueue 16G, userlimit 7G, used 8G, headroom 5G
     //(8G used is 6G from this test case - app4, 2 from last test case, app_1)
-    assertEquals(0*GB, app_4.getHeadroom().getMemorySize());
+    assertEquals(1*GB, app_4.getHeadroom().getMemorySize());
   }
 
   @Test
@@ -1436,7 +1436,7 @@ public class TestLeafQueue {
     assertEquals(2*GB, app_0.getCurrentConsumption().getMemorySize());
     assertEquals(0*GB, app_1.getCurrentConsumption().getMemorySize());
     // TODO, fix headroom in the future patch
-    assertEquals(0*GB, app_0.getHeadroom().getMemorySize());
+    assertEquals(1*GB, app_0.getHeadroom().getMemorySize());
       // User limit = 2G, 2 in use
     assertEquals(0*GB, app_1.getHeadroom().getMemorySize());
       // the application is not yet active
@@ -1449,8 +1449,8 @@ public class TestLeafQueue {
     assertEquals(3*GB, a.getUsedResources().getMemorySize());
     assertEquals(2*GB, app_0.getCurrentConsumption().getMemorySize());
     assertEquals(1*GB, app_1.getCurrentConsumption().getMemorySize());
-    assertEquals(0*GB, app_0.getHeadroom().getMemorySize()); // 4G - 3G
-    assertEquals(0*GB, app_1.getHeadroom().getMemorySize()); // 4G - 3G
+    assertEquals(1*GB, app_0.getHeadroom().getMemorySize()); // 4G - 3G
+    assertEquals(1*GB, app_1.getHeadroom().getMemorySize()); // 4G - 3G
     
     // Submit requests for app_1 and set max-cap
     a.setMaxCapacity(.1f);


---------------------------------------------------------------------
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: HDFS-11799. Introduce a config to allow setting up write pipeline with fewer nodes than replication factor. Contributed by Brahma Reddy Battula

Posted by jh...@apache.org.
HDFS-11799. Introduce a config to allow setting up write pipeline with fewer nodes than replication factor. Contributed by Brahma Reddy Battula


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

Branch: refs/heads/YARN-5734
Commit: fda1221c55101d97ac62e1ee4e3ddf9a915d5363
Parents: 31b5840
Author: Brahma Reddy Battula <br...@apache.org>
Authored: Tue Sep 19 11:25:45 2017 +0530
Committer: Brahma Reddy Battula <br...@apache.org>
Committed: Tue Sep 19 11:25:45 2017 +0530

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |  13 +-
 .../org/apache/hadoop/hdfs/DataStreamer.java    |  31 +-
 .../hdfs/client/HdfsClientConfigKeys.java       |   2 +
 .../src/main/resources/hdfs-default.xml         |  17 ++
 .../TestReplaceDatanodeFailureReplication.java  | 291 +++++++++++++++++++
 .../hadoop/tools/TestHdfsConfigFields.java      |   4 +-
 6 files changed, 354 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fda1221c/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index 772049d..7e8e95b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -223,6 +223,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   final String clientName;
   final SocketFactory socketFactory;
   final ReplaceDatanodeOnFailure dtpReplaceDatanodeOnFailure;
+  final short dtpReplaceDatanodeOnFailureReplication;
   private final FileSystem.Statistics stats;
   private final URI namenodeUri;
   private final Random r = new Random();
@@ -305,7 +306,17 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     this.socketFactory = NetUtils.getSocketFactory(conf, ClientProtocol.class);
     this.dtpReplaceDatanodeOnFailure = ReplaceDatanodeOnFailure.get(conf);
     this.smallBufferSize = DFSUtilClient.getSmallBufferSize(conf);
-
+    this.dtpReplaceDatanodeOnFailureReplication = (short) conf
+        .getInt(HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.
+                MIN_REPLICATION,
+            HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.
+                MIN_REPLICATION_DEFAULT);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(
+          "Sets " + HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.
+              MIN_REPLICATION + " to "
+              + dtpReplaceDatanodeOnFailureReplication);
+    }
     this.ugi = UserGroupInformation.getCurrentUser();
 
     this.namenodeUri = nameNodeUri;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fda1221c/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
index 4eafca1..99fa5f3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
@@ -1384,7 +1384,36 @@ class DataStreamer extends Daemon {
       setPipeline(lb);
 
       //find the new datanode
-      final int d = findNewDatanode(original);
+      final int d;
+      try {
+        d = findNewDatanode(original);
+      } catch (IOException ioe) {
+        // check the minimal number of nodes available to decide whether to
+        // continue the write.
+
+        //if live block location datanodes is greater than or equal to
+        // HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.
+        // MIN_REPLICATION threshold value, continue writing to the
+        // remaining nodes. Otherwise throw exception.
+        //
+        // If HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.
+        // MIN_REPLICATION is set to 0 or less than zero, an exception will be
+        // thrown if a replacement could not be found.
+
+        if (dfsClient.dtpReplaceDatanodeOnFailureReplication > 0 && nodes.length
+            >= dfsClient.dtpReplaceDatanodeOnFailureReplication) {
+          DFSClient.LOG.warn(
+              "Failed to find a new datanode to add to the write pipeline, "
+                  + " continue to write to the pipeline with " + nodes.length
+                  + " nodes since it's no less than minimum replication: "
+                  + dfsClient.dtpReplaceDatanodeOnFailureReplication
+                  + " configured by "
+                  + BlockWrite.ReplaceDatanodeOnFailure.MIN_REPLICATION
+                  + ".", ioe);
+          return;
+        }
+        throw ioe;
+      }
       //transfer replica. pick a source from the original nodes
       final DatanodeInfo src = original[tried % original.length];
       final DatanodeInfo[] targets = {nodes[d]};

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fda1221c/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
index e99b099..97cb68b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
@@ -320,6 +320,8 @@ public interface HdfsClientConfigKeys {
       String  POLICY_DEFAULT = "DEFAULT";
       String  BEST_EFFORT_KEY = PREFIX + "best-effort";
       boolean BEST_EFFORT_DEFAULT = false;
+      String MIN_REPLICATION = PREFIX + "min-replication";
+      short MIN_REPLICATION_DEFAULT = 0;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fda1221c/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
index af40a34..9327a2c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
@@ -658,6 +658,23 @@
   </description>
 </property>
 
+  <property>
+    <name>dfs.client.block.write.replace-datanode-on-failure.min-replication</name>
+    <value>0</value>
+    <description>
+      The minimum number of replications that are needed to not to fail
+      the write pipeline if new datanodes can not be found to replace
+      failed datanodes (could be due to network failure) in the write pipeline.
+      If the number of the remaining datanodes in the write pipeline is greater
+      than or equal to this property value, continue writing to the remaining nodes.
+      Otherwise throw exception.
+
+      If this is set to 0, an exception will be thrown, when a replacement
+      can not be found.
+      See also dfs.client.block.write.replace-datanode-on-failure.policy
+    </description>
+  </property>
+
 <property>
   <name>dfs.blockreport.intervalMsec</name>
   <value>21600000</value>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fda1221c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReplaceDatanodeFailureReplication.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReplaceDatanodeFailureReplication.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReplaceDatanodeFailureReplication.java
new file mode 100644
index 0000000..9591cb4
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReplaceDatanodeFailureReplication.java
@@ -0,0 +1,291 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.concurrent.TimeoutException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
+import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
+import org.apache.hadoop.hdfs.protocol.datatransfer.ReplaceDatanodeOnFailure;
+import org.apache.hadoop.hdfs.protocol.datatransfer.ReplaceDatanodeOnFailure.Policy;
+import org.apache.hadoop.io.IOUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Verify the behaviours of HdfsClientConfigKeys.BlockWrite.
+ * ReplaceDatanodeOnFailure.MIN_REPLICATION.if live block location datanodes is
+ * greater than or equal to
+ * 'dfs.client.block.write.replace-datanode-on-failure.min.replication'
+ * threshold value, if yes continue writing to the two remaining nodes.
+ * Otherwise it will throw exception.
+ * <p>
+ * If this HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.
+ * MIN_REPLICATION is set to 0 or less than zero, an exception will be thrown
+ * if a replacement could not be found.
+ */
+public class TestReplaceDatanodeFailureReplication {
+  static final Log LOG = LogFactory
+      .getLog(TestReplaceDatanodeFailureReplication.class);
+
+  static final String DIR =
+      "/" + TestReplaceDatanodeFailureReplication.class.getSimpleName() + "/";
+  static final short REPLICATION = 3;
+  final private static String RACK0 = "/rack0";
+
+  /**
+   * Test fail last datanode in the pipeline.
+   */
+  @Test
+  public void testLastDatanodeFailureInPipeline() throws Exception {
+    testWriteFileAndVerifyAfterDNStop(2, 1, 10, false);
+  }
+
+  /**
+   * Test fail first datanode in the pipeline.
+   */
+  @Test
+  public void testFirstDatanodeFailureInPipeline() throws Exception {
+    testWriteFileAndVerifyAfterDNStop(2, 0, 10, false);
+  }
+
+  /**
+   * Test fail all the datanodes except first in the pipeline.
+   */
+  @Test
+  public void testWithOnlyFirstDatanodeIsAlive() throws Exception {
+    testWriteFileAndVerifyAfterDNStop(1, 1, 1, true);
+  }
+
+  /**
+   * Test fail all the datanodes except lastnode in the pipeline.
+   */
+  @Test
+  public void testWithOnlyLastDatanodeIsAlive() throws Exception {
+    testWriteFileAndVerifyAfterDNStop(1, 0, 1, true);
+  }
+
+  /**
+   * Test when number of live nodes are less than the
+   * "dfs.client.block.write.replace-datanode-on-failure.min.replication".
+   */
+  @Test
+  public void testLessNumberOfLiveDatanodesThanWriteReplaceDatanodeOnFailureRF()
+      throws Exception {
+    final MiniDFSCluster cluster = setupCluster(2);
+
+    try {
+      final DistributedFileSystem fs = cluster.getFileSystem();
+      final Path dir = new Path(DIR);
+
+      final SlowWriter[] slowwriters = new SlowWriter[1];
+      for (int i = 1; i <= slowwriters.length; i++) {
+        // create slow writers in different speed
+        slowwriters[i - 1] = new SlowWriter(fs, new Path(dir, "file" + i),
+            i * 200L);
+      }
+
+      for (SlowWriter s : slowwriters) {
+        s.start();
+      }
+
+      // Let slow writers write something.
+      // Some of them are too slow and will be not yet started.
+      sleepSeconds(1);
+
+      // stop an old datanode
+      cluster.stopDataNode(0);
+      cluster.stopDataNode(0);
+
+      // Let the slow writer writes a few more seconds
+      // Everyone should have written something.
+      sleepSeconds(20);
+
+      // check replication and interrupt.
+      for (SlowWriter s : slowwriters) {
+        try {
+          s.out.getCurrentBlockReplication();
+          Assert.fail(
+              "Must throw exception as failed to add a new datanode for write "
+                  + "pipeline, minimum failure replication");
+        } catch (IOException e) {
+          // expected
+        }
+        s.interruptRunning();
+      }
+
+      // close files
+      for (SlowWriter s : slowwriters) {
+        s.joinAndClose();
+      }
+
+      // Verify the file
+      verifyFileContent(fs, slowwriters);
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  private MiniDFSCluster setupCluster(int failRF) throws IOException {
+    final Configuration conf = new HdfsConfiguration();
+    conf.setInt(HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.
+        MIN_REPLICATION, failRF);
+    // always replace a datanode
+    ReplaceDatanodeOnFailure.write(Policy.ALWAYS, false, conf);
+
+    final String[] racks = new String[REPLICATION];
+    Arrays.fill(racks, RACK0);
+    return new MiniDFSCluster.Builder(conf).racks(racks)
+        .numDataNodes(REPLICATION).build();
+  }
+
+  private void testWriteFileAndVerifyAfterDNStop(int failRF, int dnindex,
+      int slowWrites, boolean failPipeLine)
+      throws IOException, InterruptedException, TimeoutException {
+    final MiniDFSCluster cluster = setupCluster(failRF);
+    try {
+      final DistributedFileSystem fs = cluster.getFileSystem();
+      final Path dir = new Path(DIR);
+
+      final SlowWriter[] slowwriters = new SlowWriter[slowWrites];
+      for (int i = 1; i <= slowwriters.length; i++) {
+        // create slow writers in different speed
+        slowwriters[i - 1] = new SlowWriter(fs, new Path(dir, "file" + i),
+            i * 200L);
+      }
+
+      for (SlowWriter s : slowwriters) {
+        s.start();
+      }
+
+      // Let slow writers write something.
+      // Some of them are too slow and will be not yet started.
+      sleepSeconds(3);
+
+      // stop an datanode
+      cluster.stopDataNode(dnindex);
+      if (failPipeLine) {
+        cluster.stopDataNode(dnindex);
+      }
+
+      // Let the slow writer writes a few more seconds
+      // Everyone should have written something.
+      sleepSeconds(5);
+      cluster.waitFirstBRCompleted(0, 10000);
+      // check replication and interrupt.
+      for (SlowWriter s : slowwriters) {
+        Assert.assertEquals(failRF, s.out.getCurrentBlockReplication());
+        s.interruptRunning();
+      }
+
+      // close files
+      for (SlowWriter s : slowwriters) {
+        s.joinAndClose();
+      }
+
+      // Verify the file
+      verifyFileContent(fs, slowwriters);
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  private void verifyFileContent(DistributedFileSystem fs,
+      SlowWriter[] slowwriters) throws IOException {
+    LOG.info("Verify the file");
+    for (int i = 0; i < slowwriters.length; i++) {
+      LOG.info(slowwriters[i].filepath + ": length=" + fs
+          .getFileStatus(slowwriters[i].filepath).getLen());
+      FSDataInputStream in = null;
+      try {
+        in = fs.open(slowwriters[i].filepath);
+        for (int j = 0, x;; j++) {
+          x = in.read();
+          if ((x) != -1) {
+            Assert.assertEquals(j, x);
+          } else {
+            return;
+          }
+        }
+      } finally {
+        IOUtils.closeStream(in);
+      }
+    }
+  }
+
+  static void sleepSeconds(final int waittime) throws InterruptedException {
+    LOG.info("Wait " + waittime + " seconds");
+    Thread.sleep(waittime * 1000L);
+  }
+
+  static class SlowWriter extends Thread {
+    private final Path filepath;
+    private final HdfsDataOutputStream out;
+    private final long sleepms;
+    private volatile boolean running = true;
+
+    SlowWriter(DistributedFileSystem fs, Path filepath, final long sleepms)
+        throws IOException {
+      super(SlowWriter.class.getSimpleName() + ":" + filepath);
+      this.filepath = filepath;
+      this.out = (HdfsDataOutputStream) fs.create(filepath, REPLICATION);
+      this.sleepms = sleepms;
+    }
+
+    @Override public void run() {
+      int i = 0;
+      try {
+        sleep(sleepms);
+        for (; running; i++) {
+          LOG.info(getName() + " writes " + i);
+          out.write(i);
+          out.hflush();
+          sleep(sleepms);
+        }
+      } catch (InterruptedException e) {
+        LOG.info(getName() + " interrupted:" + e);
+      } catch (IOException e) {
+        throw new RuntimeException(getName(), e);
+      } finally {
+        LOG.info(getName() + " terminated: i=" + i);
+      }
+    }
+
+    void interruptRunning() {
+      running = false;
+      interrupt();
+    }
+
+    void joinAndClose() throws InterruptedException {
+      LOG.info(getName() + " join and close");
+      join();
+      IOUtils.closeStream(out);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fda1221c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestHdfsConfigFields.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestHdfsConfigFields.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestHdfsConfigFields.java
index 233dc5a..47db565 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestHdfsConfigFields.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestHdfsConfigFields.java
@@ -41,8 +41,8 @@ public class TestHdfsConfigFields extends TestConfigurationFieldsBase {
   public void initializeMemberVariables() {
     xmlFilename = new String("hdfs-default.xml");
     configurationClasses = new Class[] { HdfsClientConfigKeys.class,
-        HdfsClientConfigKeys.StripedRead.class,
-        DFSConfigKeys.class};
+        HdfsClientConfigKeys.StripedRead.class, DFSConfigKeys.class,
+        HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.class };
 
     // Set error modes
     errorIfMissingConfigProps = true;


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


[11/50] [abbrv] hadoop git commit: HDFS-12470. DiskBalancer: Some tests create plan files under system directory. Contributed by Hanisha Koneru.

Posted by jh...@apache.org.
HDFS-12470. DiskBalancer: Some tests create plan files under system directory. 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/a2dcba18
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/a2dcba18
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/a2dcba18

Branch: refs/heads/YARN-5734
Commit: a2dcba18531c6fa4b76325f5132773f12ddfc6d5
Parents: a4f9c7c
Author: Arpit Agarwal <ar...@apache.org>
Authored: Mon Sep 18 09:53:24 2017 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Mon Sep 18 09:53:24 2017 -0700

----------------------------------------------------------------------
 .../server/diskbalancer/command/TestDiskBalancerCommand.java    | 5 ++++-
 1 file changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a2dcba18/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java
index b0b0b0c..1cebae0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java
@@ -476,9 +476,12 @@ public class TestDiskBalancerCommand {
   public void testPlanJsonNode() throws Exception {
     final String planArg = String.format("-%s %s", PLAN,
         "a87654a9-54c7-4693-8dd9-c9c7021dc340");
+    final Path testPath = new Path(
+        PathUtils.getTestPath(getClass()),
+        GenericTestUtils.getMethodName());
     final String cmdLine = String
         .format(
-            "hdfs diskbalancer %s", planArg);
+            "hdfs diskbalancer -out %s %s", testPath, planArg);
     runCommand(cmdLine);
   }
 


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

Branch: refs/heads/YARN-5734
Commit: 78baf500b2f4bc7de732fe64820e776d000cec55
Parents: fbcc60c
Author: Wangda Tan <wa...@apache.org>
Authored: Mon May 22 13:38:31 2017 -0700
Committer: Jonathan Hung <jh...@linkedin.com>
Committed: Wed Sep 20 17:40:53 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/78baf500/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 2f77316..e1062d7 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/78baf500/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 f05e005..86aa15e 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
@@ -3360,4 +3360,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/78baf500/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/78baf500/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/78baf500/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/78baf500/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/78baf500/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/78baf500/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/78baf500/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/78baf500/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/78baf500/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/78baf500/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/78baf500/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


[16/50] [abbrv] hadoop git commit: HADOOP-14771. hadoop-client does not include hadoop-yarn-client. (Ajay Kumar via Haibo Chen)

Posted by jh...@apache.org.
HADOOP-14771. hadoop-client does not include hadoop-yarn-client. (Ajay Kumar via Haibo Chen)


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

Branch: refs/heads/YARN-5734
Commit: 1ee25278c891e95ba2ab142e5b78aebd752ea163
Parents: 7c73292
Author: Haibo Chen <ha...@apache.org>
Authored: Mon Sep 18 14:25:35 2017 -0700
Committer: Haibo Chen <ha...@apache.org>
Committed: Mon Sep 18 14:25:35 2017 -0700

----------------------------------------------------------------------
 hadoop-client-modules/hadoop-client/pom.xml | 31 ++++++++++++++++++++++++
 1 file changed, 31 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1ee25278/hadoop-client-modules/hadoop-client/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-client-modules/hadoop-client/pom.xml b/hadoop-client-modules/hadoop-client/pom.xml
index bed3f5c..6500ebf 100644
--- a/hadoop-client-modules/hadoop-client/pom.xml
+++ b/hadoop-client-modules/hadoop-client/pom.xml
@@ -179,6 +179,37 @@
 
     <dependency>
       <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-client</artifactId>
+      <scope>compile</scope>
+      <exclusions>
+        <!--Excluding hadoop-yarn-api & hadoop-annotations as they are already
+        included as direct dependencies. Guava,commons-cli and log4j are
+        transitive dependencies -->
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-yarn-api</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-annotations</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.google.guava</groupId>
+          <artifactId>guava</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>commons-cli</groupId>
+          <artifactId>commons-cli</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>log4j</groupId>
+          <artifactId>log4j</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-mapreduce-client-core</artifactId>
       <scope>compile</scope>
       <exclusions>


---------------------------------------------------------------------
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-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/034e6f4f/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


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

Branch: refs/heads/YARN-5734
Commit: 034e6f4f80046ace4987da021dee8f8d2533a1f3
Parents: c59418f
Author: Wangda Tan <wa...@apache.org>
Authored: Mon Sep 18 09:53:42 2017 -0700
Committer: Jonathan Hung <jh...@linkedin.com>
Committed: Wed Sep 20 17:40:54 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/034e6f4f/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 a33d85d..6e6089f 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/034e6f4f/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 4529f20..0ccf6f4 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
@@ -3393,11 +3393,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/034e6f4f/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/034e6f4f/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/034e6f4f/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/034e6f4f/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/034e6f4f/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/034e6f4f/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/034e6f4f/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/034e6f4f/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/034e6f4f/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/034e6f4f/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/034e6f4f/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/034e6f4f/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/034e6f4f/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/034e6f4f/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/034e6f4f/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/034e6f4f/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/034e6f4f/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