Procházet zdrojové kódy

Merging r1558604 through r1559901 from trunk.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-5698@1559904 13f79535-47bb-0310-9956-ffa450edef68
Jing Zhao před 11 roky
rodič
revize
6696bdd5db
57 změnil soubory, kde provedl 2241 přidání a 185 odebrání
  1. 4 4
      BUILDING.txt
  2. 18 6
      hadoop-common-project/hadoop-common/CHANGES.txt
  3. 2 0
      hadoop-common-project/hadoop-common/src/main/bin/hadoop
  4. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/JavaKeyStoreProvider.java
  5. 11 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProvider.java
  6. 474 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyShell.java
  7. 5 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/UserProvider.java
  8. 16 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java
  9. 176 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestKeyShell.java
  10. 13 2
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  11. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
  12. 64 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LayoutFlags.java
  13. 5 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LayoutVersion.java
  14. 11 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java
  15. 2 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java
  16. 7 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOpCodes.java
  17. 5 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
  18. 5 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java
  19. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java
  20. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
  21. binární
      hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored
  22. 84 84
      hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml
  23. 16 0
      hadoop-mapreduce-project/CHANGES.txt
  24. 9 15
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/MapReduceChildJVM.java
  25. 0 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java
  26. 46 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestTaskAttemptListenerImpl.java
  27. 13 5
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/TypeConverter.java
  28. 7 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/jobhistory/JHAdminConfig.java
  29. 11 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRApps.java
  30. 9 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/TestTypeConverter.java
  31. 9 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
  32. 50 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
  33. 10 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/pom.xml
  34. 110 31
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryFileManager.java
  35. 139 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestHistoryFileManager.java
  36. 3 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java
  37. 9 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientRedirect.java
  38. 14 6
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestNetworkedJob.java
  39. 123 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobs.java
  40. 244 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobsWithProfiler.java
  41. 7 1
      hadoop-project/pom.xml
  42. 9 0
      hadoop-yarn-project/CHANGES.txt
  43. 15 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocol.java
  44. 69 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/MoveApplicationAcrossQueuesRequest.java
  45. 47 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/MoveApplicationAcrossQueuesResponse.java
  46. 3 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
  47. 1 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/applicationclient_protocol.proto
  48. 8 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
  49. 65 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ContainerRollingLogAppender.java
  50. 21 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ApplicationClientProtocolPBClientImpl.java
  51. 21 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ApplicationClientProtocolPBServiceImpl.java
  52. 158 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/MoveApplicationAcrossQueuesRequestPBImpl.java
  53. 68 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/MoveApplicationAcrossQueuesResponsePBImpl.java
  54. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
  55. 3 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LinuxContainerExecutor.java
  56. 7 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/resources/container-log4j.properties
  57. 8 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java

+ 4 - 4
BUILDING.txt

@@ -183,11 +183,11 @@ Building on Windows
 Requirements:
 Requirements:
 
 
 * Windows System
 * Windows System
-* JDK 1.6
-* Maven 3.0
-* Windows SDK or Visual Studio 2010 Professional
-* ProtocolBuffer 2.4.1+ (for MapReduce and HDFS)
+* JDK 1.6+
+* Maven 3.0 or later
 * Findbugs 1.3.9 (if running findbugs)
 * Findbugs 1.3.9 (if running findbugs)
+* ProtocolBuffer 2.5.0
+* Windows SDK or Visual Studio 2010 Professional
 * Unix command-line tools from GnuWin32 or Cygwin: sh, mkdir, rm, cp, tar, gzip
 * Unix command-line tools from GnuWin32 or Cygwin: sh, mkdir, rm, cp, tar, gzip
 * Internet connection for first build (to fetch all Maven and Hadoop dependencies)
 * Internet connection for first build (to fetch all Maven and Hadoop dependencies)
 
 

+ 18 - 6
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -98,7 +98,8 @@ Trunk (Unreleased)
     HADOOP-8844. Add a plaintext fs -text test-case.
     HADOOP-8844. Add a plaintext fs -text test-case.
     (Akira AJISAKA via harsh)
     (Akira AJISAKA via harsh)
 
 
-    HADOOP-9432 Add support for markdown .md files in site documentation (stevel)
+    HADOOP-9432 Add support for markdown .md files in site documentation 
+    (stevel)
 
 
     HADOOP-9186.  test-patch.sh should report build failure to JIRA.
     HADOOP-9186.  test-patch.sh should report build failure to JIRA.
     (Binglin Chang via Colin Patrick McCabe)
     (Binglin Chang via Colin Patrick McCabe)
@@ -110,6 +111,8 @@ Trunk (Unreleased)
 
 
     HADOOP-10201. Add listing to KeyProvider API. (Larry McCay via omalley)
     HADOOP-10201. Add listing to KeyProvider API. (Larry McCay via omalley)
 
 
+    HADOOP-10177. Create CLI tools for managing keys. (Larry McCay via omalley)
+
   BUG FIXES
   BUG FIXES
 
 
     HADOOP-9451. Fault single-layer config if node group topology is enabled.
     HADOOP-9451. Fault single-layer config if node group topology is enabled.
@@ -117,8 +120,8 @@ Trunk (Unreleased)
 
 
     HADOOP-8419. Fixed GzipCode NPE reset for IBM JDK. (Yu Li via eyang)
     HADOOP-8419. Fixed GzipCode NPE reset for IBM JDK. (Yu Li via eyang)
 
 
-    HADOOP-8177. MBeans shouldn't try to register when it fails to create MBeanName.
-    (Devaraj K via umamahesh)
+    HADOOP-8177. MBeans shouldn't try to register when it fails to create 
+    MBeanName. (Devaraj K via umamahesh)
 
 
     HADOOP-8018.  Hudson auto test for HDFS has started throwing javadoc
     HADOOP-8018.  Hudson auto test for HDFS has started throwing javadoc
     (Jon Eagles via bobby)
     (Jon Eagles via bobby)
@@ -287,9 +290,6 @@ Trunk (Unreleased)
 
 
     HADOOP-10044 Improve the javadoc of rpc code (sanjay Radia)
     HADOOP-10044 Improve the javadoc of rpc code (sanjay Radia)
 
 
-    HADOOP-10125. no need to process RPC request if the client connection
-    has been dropped (Ming Ma via brandonli)
-
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
     HADOOP-7761. Improve the performance of raw comparisons. (todd)
     HADOOP-7761. Improve the performance of raw comparisons. (todd)
@@ -523,6 +523,13 @@ Release 2.4.0 - UNRELEASED
     HADOOP-10236. Fix typo in o.a.h.ipc.Client#checkResponse. (Akira Ajisaka
     HADOOP-10236. Fix typo in o.a.h.ipc.Client#checkResponse. (Akira Ajisaka
     via suresh)
     via suresh)
 
 
+    HADOOP-10146. Workaround JDK7 Process fd close bug (daryn)
+
+    HADOOP-10125. no need to process RPC request if the client connection
+    has been dropped (Ming Ma via brandonli)
+
+    HADOOP-10235. Hadoop tarball has 2 versions of stax-api JARs. (tucu)
+
 Release 2.3.0 - UNRELEASED
 Release 2.3.0 - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES
@@ -613,6 +620,11 @@ Release 2.3.0 - UNRELEASED
 
 
     HADOOP-10234. "hadoop.cmd jar" does not propagate exit code. (cnauroth)
     HADOOP-10234. "hadoop.cmd jar" does not propagate exit code. (cnauroth)
 
 
+    HADOOP-10240. Windows build instructions incorrectly state requirement of
+    protoc 2.4.1 instead of 2.5.0. (cnauroth)
+
+    HADOOP-10112. har file listing doesn't work with wild card. (brandonli)
+
 Release 2.2.0 - 2013-10-13
 Release 2.2.0 - 2013-10-13
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 2 - 0
hadoop-common-project/hadoop-common/src/main/bin/hadoop

@@ -104,6 +104,8 @@ case $COMMAND in
       CLASS=org.apache.hadoop.util.VersionInfo
       CLASS=org.apache.hadoop.util.VersionInfo
     elif [ "$COMMAND" = "jar" ] ; then
     elif [ "$COMMAND" = "jar" ] ; then
       CLASS=org.apache.hadoop.util.RunJar
       CLASS=org.apache.hadoop.util.RunJar
+    elif [ "$COMMAND" = "key" ] ; then
+      CLASS=org.apache.hadoop.crypto.key.KeyShell
     elif [ "$COMMAND" = "checknative" ] ; then
     elif [ "$COMMAND" = "checknative" ] ; then
       CLASS=org.apache.hadoop.util.NativeLibraryChecker
       CLASS=org.apache.hadoop.util.NativeLibraryChecker
     elif [ "$COMMAND" = "distcp" ] ; then
     elif [ "$COMMAND" = "distcp" ] ; then

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/JavaKeyStoreProvider.java

@@ -77,7 +77,7 @@ public class JavaKeyStoreProvider extends KeyProvider {
   private JavaKeyStoreProvider(URI uri, Configuration conf) throws IOException {
   private JavaKeyStoreProvider(URI uri, Configuration conf) throws IOException {
     this.uri = uri;
     this.uri = uri;
     path = unnestUri(uri);
     path = unnestUri(uri);
-    fs = FileSystem.get(conf);
+    fs = path.getFileSystem(conf);
     // Get the password from the user's environment
     // Get the password from the user's environment
     String pw = System.getenv(KEYSTORE_PASSWORD_NAME);
     String pw = System.getenv(KEYSTORE_PASSWORD_NAME);
     if (pw == null) {
     if (pw == null) {

+ 11 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProvider.java

@@ -244,6 +244,17 @@ public abstract class KeyProvider {
     return new Options(conf);
     return new Options(conf);
   }
   }
 
 
+  /**
+   * Indicates whether this provider represents a store
+   * that is intended for transient use - such as the UserProvider
+   * is. These providers are generally used to provide access to
+   * keying material rather than for long term storage.
+   * @return true if transient, false otherwise
+   */
+  public boolean isTransient() {
+    return false;
+  }
+
   /**
   /**
    * Get the key material for a specific version of the key. This method is used
    * Get the key material for a specific version of the key. This method is used
    * when decrypting data.
    * when decrypting data.

+ 474 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyShell.java

@@ -0,0 +1,474 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.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.crypto.key;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.security.InvalidParameterException;
+import java.security.NoSuchAlgorithmException;
+import java.util.List;
+
+import javax.crypto.KeyGenerator;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.crypto.key.KeyProvider.Metadata;
+import org.apache.hadoop.crypto.key.KeyProvider.Options;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+
+/**
+ * This program is the CLI utility for the KeyProvider facilities in Hadoop.
+ */
+public class KeyShell extends Configured implements Tool {
+  final static private String USAGE_PREFIX = "Usage: hadoop key " +
+  		"[generic options]\n";
+  final static private String COMMANDS =
+      "   [--help]\n" +
+      "   [" + CreateCommand.USAGE + "]\n" +
+      "   [" + RollCommand.USAGE + "]\n" +
+      "   [" + DeleteCommand.USAGE + "]\n" +
+      "   [" + ListCommand.USAGE + "]\n";
+
+  private boolean interactive = false;
+  private Command command = null;
+
+  /** allows stdout to be captured if necessary */
+  public PrintStream out = System.out;
+  /** allows stderr to be captured if necessary */
+  public PrintStream err = System.err;
+
+  private boolean userSuppliedProvider = false;
+
+  @Override
+  public int run(String[] args) throws Exception {
+    int exitCode = 0;
+    try {
+      exitCode = init(args);
+      if (exitCode != 0) {
+        return exitCode;
+      }
+      if (command.validate()) {
+          command.execute();
+      } else {
+        exitCode = -1;
+      }
+    } catch (Exception e) {
+      e.printStackTrace(err);
+      return -1;
+    }
+    return exitCode;
+  }
+
+  /**
+   * Parse the command line arguments and initialize the data
+   * <pre>
+   * % hadoop key create keyName [--size size] [--cipher algorithm]
+   *    [--provider providerPath]
+   * % hadoop key roll keyName [--provider providerPath]
+   * % hadoop key list [-provider providerPath]
+   * % hadoop key delete keyName [--provider providerPath] [-i]
+   * </pre>
+   * @param args
+   * @return
+   * @throws IOException
+   */
+  private int init(String[] args) throws IOException {
+    for (int i = 0; i < args.length; i++) { // parse command line
+      if (args[i].equals("create")) {
+        String keyName = args[++i];
+        command = new CreateCommand(keyName);
+        if (keyName.equals("--help")) {
+          printKeyShellUsage();
+          return -1;
+        }
+      } else if (args[i].equals("delete")) {
+        String keyName = args[++i];
+        command = new DeleteCommand(keyName);
+        if (keyName.equals("--help")) {
+          printKeyShellUsage();
+          return -1;
+        }
+      } else if (args[i].equals("roll")) {
+        String keyName = args[++i];
+        command = new RollCommand(keyName);
+        if (keyName.equals("--help")) {
+          printKeyShellUsage();
+          return -1;
+        }
+      } else if (args[i].equals("list")) {
+        command = new ListCommand();
+      } else if (args[i].equals("--size")) {
+        getConf().set(KeyProvider.DEFAULT_BITLENGTH_NAME, args[++i]);
+      } else if (args[i].equals("--cipher")) {
+        getConf().set(KeyProvider.DEFAULT_CIPHER_NAME, args[++i]);
+      } else if (args[i].equals("--provider")) {
+        userSuppliedProvider = true;
+        getConf().set(KeyProviderFactory.KEY_PROVIDER_PATH, args[++i]);
+      } else if (args[i].equals("-i") || (args[i].equals("--interactive"))) {
+        interactive = true;
+      } else if (args[i].equals("--help")) {
+        printKeyShellUsage();
+        return -1;
+      } else {
+        printKeyShellUsage();
+        ToolRunner.printGenericCommandUsage(System.err);
+        return -1;
+      }
+    }
+    return 0;
+  }
+
+  private void printKeyShellUsage() {
+    out.println(USAGE_PREFIX + COMMANDS);
+    if (command != null) {
+      out.println(command.getUsage());
+    }
+    else {
+      out.println("=========================================================" +
+      		"======");
+      out.println(CreateCommand.USAGE + ":\n\n" + CreateCommand.DESC);
+      out.println("=========================================================" +
+          "======");
+      out.println(RollCommand.USAGE + ":\n\n" + RollCommand.DESC);
+      out.println("=========================================================" +
+          "======");
+      out.println(DeleteCommand.USAGE + ":\n\n" + DeleteCommand.DESC);
+      out.println("=========================================================" +
+          "======");
+      out.println(ListCommand.USAGE + ":\n\n" + ListCommand.DESC);
+    }
+  }
+
+  private abstract class Command {
+    protected KeyProvider provider = null;
+
+    public boolean validate() {
+      return true;
+    }
+
+    protected KeyProvider getKeyProvider() {
+      KeyProvider provider = null;
+      List<KeyProvider> providers;
+      try {
+        providers = KeyProviderFactory.getProviders(getConf());
+        if (userSuppliedProvider) {
+          provider = providers.get(0);
+        }
+        else {
+          for (KeyProvider p : providers) {
+            if (!p.isTransient()) {
+              provider = p;
+              break;
+            }
+          }
+        }
+      } catch (IOException e) {
+        e.printStackTrace(err);
+      }
+      return provider;
+    }
+
+    protected byte[] generateKey(int size, String algorithm)
+        throws NoSuchAlgorithmException {
+      out.println("Generating key using size: " + size + " and algorithm: "
+          + algorithm);
+      KeyGenerator keyGenerator = KeyGenerator.getInstance(algorithm);
+      keyGenerator.init(size);
+      byte[] key = keyGenerator.generateKey().getEncoded();
+      return key;
+    }
+
+    protected void printProviderWritten() {
+        out.println(provider.getClass().getName() + " has been updated.");
+    }
+
+    protected void warnIfTransientProvider() {
+      if (provider.isTransient()) {
+        out.println("WARNING: you are modifying a transient provider.");
+      }
+    }
+
+    public abstract void execute() throws Exception;
+
+    public abstract String getUsage();
+  }
+
+  private class ListCommand extends Command {
+    public static final String USAGE = "list <keyname> [--provider] [--help]";
+    public static final String DESC =
+        "The list subcommand displays the keynames contained within \n" +
+        "a particular provider - as configured in core-site.xml or " +
+        "indicated\nthrough the --provider argument.";
+
+    public boolean validate() {
+      boolean rc = true;
+      provider = getKeyProvider();
+      if (provider == null) {
+        out.println("There are no non-transient KeyProviders configured.\n"
+            + "Consider using the --provider option to indicate the provider\n"
+            + "to use. If you want to list a transient provider then you\n"
+            + "you MUST use the --provider argument.");
+        rc = false;
+      }
+      return rc;
+    }
+
+    public void execute() throws IOException {
+      List<String> keys;
+      try {
+        keys = provider.getKeys();
+        out.println("Listing keys for KeyProvider: " + provider.toString());
+        for (String keyName : keys) {
+          out.println(keyName);
+        }
+      } catch (IOException e) {
+        out.println("Cannot list keys for KeyProvider: " + provider.toString()
+            + ": " + e.getMessage());
+        throw e;
+      }
+    }
+
+    @Override
+    public String getUsage() {
+      return USAGE + ":\n\n" + DESC;
+    }
+  }
+
+  private class RollCommand extends Command {
+    public static final String USAGE = "roll <keyname> [--provider] [--help]";
+    public static final String DESC =
+        "The roll subcommand creates a new version of the key specified\n" +
+        "through the <keyname> argument within the provider indicated using\n" +
+        "the --provider argument";
+
+    String keyName = null;
+
+    public RollCommand(String keyName) {
+      this.keyName = keyName;
+    }
+
+    public boolean validate() {
+      boolean rc = true;
+      provider = getKeyProvider();
+      if (provider == null) {
+        out.println("There are no valid KeyProviders configured.\n"
+            + "Key will not be rolled.\n"
+            + "Consider using the --provider option to indicate the provider"
+            + " to use.");
+        rc = false;
+      }
+      if (keyName == null) {
+        out.println("There is no keyName specified. Please provide the" +
+            "mandatory <keyname>. See the usage description with --help.");
+        rc = false;
+      }
+      return rc;
+    }
+
+    public void execute() throws NoSuchAlgorithmException, IOException {
+      try {
+        Metadata md = provider.getMetadata(keyName);
+        warnIfTransientProvider();
+        out.println("Rolling key version from KeyProvider: "
+            + provider.toString() + " for key name: " + keyName);
+        try {
+          byte[] material = null;
+          material = generateKey(md.getBitLength(), md.getAlgorithm());
+          provider.rollNewVersion(keyName, material);
+          out.println(keyName + " has been successfully rolled.");
+          provider.flush();
+          printProviderWritten();
+        } catch (NoSuchAlgorithmException e) {
+          out.println("Cannot roll key: " + keyName + " within KeyProvider: "
+              + provider.toString());
+          throw e;
+        }
+      } catch (IOException e1) {
+        out.println("Cannot roll key: " + keyName + " within KeyProvider: "
+            + provider.toString());
+        throw e1;
+      }
+    }
+
+    @Override
+    public String getUsage() {
+      return USAGE + ":\n\n" + DESC;
+    }
+  }
+
+  private class DeleteCommand extends Command {
+    public static final String USAGE = "delete <keyname> [--provider] [--help]";
+    public static final String DESC =
+        "The delete subcommand deletes all of the versions of the key\n" +
+        "specified as the <keyname> argument from within the provider\n" +
+        "indicated through the --provider argument";
+
+    String keyName = null;
+    boolean cont = true;
+
+    public DeleteCommand(String keyName) {
+      this.keyName = keyName;
+    }
+
+    @Override
+    public boolean validate() {
+      provider = getKeyProvider();
+      if (provider == null) {
+        out.println("There are no valid KeyProviders configured.\n"
+            + "Nothing will be deleted.\n"
+            + "Consider using the --provider option to indicate the provider"
+            + " to use.");
+        return false;
+      }
+      if (keyName == null) {
+        out.println("There is no keyName specified. Please provide the" +
+            "mandatory <keyname>. See the usage description with --help.");
+        return false;
+      }
+      if (interactive) {
+        try {
+          cont = ToolRunner
+              .confirmPrompt("You are about to DELETE all versions of "
+                  + "the key: " + keyName + " from KeyProvider "
+                  + provider.toString() + ". Continue?:");
+          if (!cont) {
+            out.println("Nothing has been be deleted.");
+          }
+          return cont;
+        } catch (IOException e) {
+          out.println(keyName + " will not be deleted.");
+          e.printStackTrace(err);
+        }
+      }
+      return true;
+    }
+
+    public void execute() throws IOException {
+      warnIfTransientProvider();
+      out.println("Deleting key: " + keyName + " from KeyProvider: "
+          + provider.toString());
+      if (cont) {
+        try {
+          provider.deleteKey(keyName);
+          out.println(keyName + " has been successfully deleted.");
+          provider.flush();
+          printProviderWritten();
+        } catch (IOException e) {
+          out.println(keyName + "has NOT been deleted.");
+          throw e;
+        }
+      }
+    }
+
+    @Override
+    public String getUsage() {
+      return USAGE + ":\n\n" + DESC;
+    }
+  }
+
+  private class CreateCommand extends Command {
+    public static final String USAGE = "create <keyname> [--cipher] " +
+    		"[--size] [--provider] [--help]";
+    public static final String DESC =
+        "The create subcommand creates a new key for the name specified\n" +
+        "as the <keyname> argument within the provider indicated through\n" +
+        "the --provider argument. You may also indicate the specific\n" +
+        "cipher through the --cipher argument. The default for cipher is\n" +
+        "currently \"AES/CTR/NoPadding\". The default keysize is \"256\".\n" +
+        "You may also indicate the requested key length through the --size\n" +
+        "argument.";
+
+    String keyName = null;
+
+    public CreateCommand(String keyName) {
+      this.keyName = keyName;
+    }
+
+    public boolean validate() {
+      boolean rc = true;
+      provider = getKeyProvider();
+      if (provider == null) {
+        out.println("There are no valid KeyProviders configured.\nKey" +
+        		" will not be created.\n"
+            + "Consider using the --provider option to indicate the provider" +
+            " to use.");
+        rc = false;
+      }
+      if (keyName == null) {
+        out.println("There is no keyName specified. Please provide the" +
+        		"mandatory <keyname>. See the usage description with --help.");
+        rc = false;
+      }
+      return rc;
+    }
+
+    public void execute() throws IOException, NoSuchAlgorithmException {
+      warnIfTransientProvider();
+      try {
+        Options options = KeyProvider.options(getConf());
+        String alg = getAlgorithm(options.getCipher());
+        byte[] material = generateKey(options.getBitLength(), alg);
+        provider.createKey(keyName, material, options);
+        out.println(keyName + " has been successfully created.");
+        provider.flush();
+        printProviderWritten();
+      } catch (InvalidParameterException e) {
+        out.println(keyName + " has NOT been created. " + e.getMessage());
+        throw e;
+      } catch (IOException e) {
+        out.println(keyName + " has NOT been created. " + e.getMessage());
+        throw e;
+      } catch (NoSuchAlgorithmException e) {
+        out.println(keyName + " has NOT been created. " + e.getMessage());
+        throw e;
+      }
+    }
+
+    /**
+     * Get the algorithm from the cipher.
+     * @return the algorithm name
+     */
+    public String getAlgorithm(String cipher) {
+      int slash = cipher.indexOf('/');
+      if (slash == - 1) {
+        return cipher;
+      } else {
+        return cipher.substring(0, slash);
+      }
+    }
+
+    @Override
+    public String getUsage() {
+      return USAGE + ":\n\n" + DESC;
+    }
+  }
+
+  /**
+   * Main program.
+   *
+   * @param args
+   *          Command line arguments
+   * @throws Exception
+   */
+  public static void main(String[] args) throws Exception {
+    int res = ToolRunner.run(new Configuration(), new KeyShell(), args);
+    System.exit(res);
+  }
+}

+ 5 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/UserProvider.java

@@ -49,6 +49,11 @@ public class UserProvider extends KeyProvider {
     credentials = user.getCredentials();
     credentials = user.getCredentials();
   }
   }
 
 
+  @Override
+  public boolean isTransient() {
+    return true;
+  }
+
   @Override
   @Override
   public KeyVersion getKeyVersion(String versionName) {
   public KeyVersion getKeyVersion(String versionName) {
     byte[] bytes = credentials.getSecretKey(new Text(versionName));
     byte[] bytes = credentials.getSecretKey(new Text(versionName));

+ 16 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java

@@ -21,6 +21,7 @@ import java.io.BufferedReader;
 import java.io.File;
 import java.io.File;
 import java.io.IOException;
 import java.io.IOException;
 import java.io.InputStreamReader;
 import java.io.InputStreamReader;
+import java.io.InputStream;
 import java.util.Arrays;
 import java.util.Arrays;
 import java.util.Map;
 import java.util.Map;
 import java.util.Timer;
 import java.util.Timer;
@@ -511,7 +512,17 @@ abstract public class Shell {
       }
       }
       // close the input stream
       // close the input stream
       try {
       try {
-        inReader.close();
+        // JDK 7 tries to automatically drain the input streams for us
+        // when the process exits, but since close is not synchronized,
+        // it creates a race if we close the stream first and the same
+        // fd is recycled.  the stream draining thread will attempt to
+        // drain that fd!!  it may block, OOM, or cause bizarre behavior
+        // see: https://bugs.openjdk.java.net/browse/JDK-8024521
+        //      issue is fixed in build 7u60
+        InputStream stdout = process.getInputStream();
+        synchronized (stdout) {
+          inReader.close();
+        }
       } catch (IOException ioe) {
       } catch (IOException ioe) {
         LOG.warn("Error while closing the input stream", ioe);
         LOG.warn("Error while closing the input stream", ioe);
       }
       }
@@ -524,7 +535,10 @@ abstract public class Shell {
         LOG.warn("Interrupted while joining errThread");
         LOG.warn("Interrupted while joining errThread");
       }
       }
       try {
       try {
-        errReader.close();
+        InputStream stderr = process.getErrorStream();
+        synchronized (stderr) {
+          errReader.close();
+        }
       } catch (IOException ioe) {
       } catch (IOException ioe) {
         LOG.warn("Error while closing the error stream", ioe);
         LOG.warn("Error while closing the error stream", ioe);
       }
       }

+ 176 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestKeyShell.java

@@ -0,0 +1,176 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.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.crypto.key;
+
+import static org.junit.Assert.*;
+
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.PrintStream;
+
+import org.apache.hadoop.conf.Configuration;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestKeyShell {
+  private final ByteArrayOutputStream outContent = new ByteArrayOutputStream();
+  private final ByteArrayOutputStream errContent = new ByteArrayOutputStream();
+  private static final File tmpDir =
+      new File(System.getProperty("test.build.data", "/tmp"), "key");
+  
+  @Before
+  public void setup() throws Exception {
+    System.setOut(new PrintStream(outContent));
+    System.setErr(new PrintStream(errContent));
+  }
+  
+  @Test
+  public void testKeySuccessfulKeyLifecycle() throws Exception {
+    outContent.flush();
+    String[] args1 = {"create", "key1", "--provider", 
+        "jceks://file" + tmpDir + "/keystore.jceks"};
+    int rc = 0;
+    KeyShell ks = new KeyShell();
+    ks.setConf(new Configuration());
+    rc = ks.run(args1);
+    assertEquals(0, rc);
+    assertTrue(outContent.toString().contains("key1 has been successfully " +
+    		"created."));
+
+    outContent.flush();
+    String[] args2 = {"list", "--provider", 
+        "jceks://file" + tmpDir + "/keystore.jceks"};
+    rc = ks.run(args2);
+    assertEquals(0, rc);
+    assertTrue(outContent.toString().contains("key1"));
+
+    outContent.flush();
+    String[] args3 = {"roll", "key1", "--provider", 
+        "jceks://file" + tmpDir + "/keystore.jceks"};
+    rc = ks.run(args3);
+    assertEquals(0, rc);
+    assertTrue(outContent.toString().contains("key1 has been successfully " +
+    		"rolled."));
+
+    outContent.flush();
+    String[] args4 = {"delete", "key1", "--provider", 
+        "jceks://file" + tmpDir + "/keystore.jceks"};
+    rc = ks.run(args4);
+    assertEquals(0, rc);
+    assertTrue(outContent.toString().contains("key1 has been successfully " +
+    		"deleted."));
+
+    outContent.flush();
+    String[] args5 = {"list", "--provider", 
+        "jceks://file" + tmpDir + "/keystore.jceks"};
+    rc = ks.run(args5);
+    assertEquals(0, rc);
+    assertTrue(outContent.toString().contains("key1"));
+  }
+  
+  @Test
+  public void testInvalidKeySize() throws Exception {
+    String[] args1 = {"create", "key1", "--size", "56", "--provider", 
+        "jceks://file" + tmpDir + "/keystore.jceks"};
+    
+    int rc = 0;
+    KeyShell ks = new KeyShell();
+    ks.setConf(new Configuration());
+    rc = ks.run(args1);
+    assertEquals(-1, rc);
+    assertTrue(outContent.toString().contains("key1 has NOT been created."));
+  }
+
+  @Test
+  public void testInvalidCipher() throws Exception {
+    String[] args1 = {"create", "key1", "--cipher", "LJM", "--provider", 
+        "jceks://file" + tmpDir + "/keystore.jceks"};
+    
+    int rc = 0;
+    KeyShell ks = new KeyShell();
+    ks.setConf(new Configuration());
+    rc = ks.run(args1);
+    assertEquals(-1, rc);
+    assertTrue(outContent.toString().contains("key1 has NOT been created."));
+  }
+
+  @Test
+  public void testInvalidProvider() throws Exception {
+    String[] args1 = {"create", "key1", "--cipher", "AES", "--provider", 
+      "sdff://file/tmp/keystore.jceks"};
+    
+    int rc = 0;
+    KeyShell ks = new KeyShell();
+    ks.setConf(new Configuration());
+    rc = ks.run(args1);
+    assertEquals(-1, rc);
+    assertTrue(outContent.toString().contains("There are no valid " +
+    		"KeyProviders configured."));
+  }
+
+  @Test
+  public void testTransientProviderWarning() throws Exception {
+    String[] args1 = {"create", "key1", "--cipher", "AES", "--provider", 
+      "user:///"};
+    
+    int rc = 0;
+    KeyShell ks = new KeyShell();
+    ks.setConf(new Configuration());
+    rc = ks.run(args1);
+    assertEquals(0, rc);
+    assertTrue(outContent.toString().contains("WARNING: you are modifying a " +
+    		"transient provider."));
+  }
+  
+  @Test
+  public void testTransientProviderOnlyConfig() throws Exception {
+    String[] args1 = {"create", "key1"};
+    
+    int rc = 0;
+    KeyShell ks = new KeyShell();
+    Configuration config = new Configuration();
+    config.set(KeyProviderFactory.KEY_PROVIDER_PATH, "user:///");
+    ks.setConf(config);
+    rc = ks.run(args1);
+    assertEquals(-1, rc);
+    assertTrue(outContent.toString().contains("There are no valid " +
+    		"KeyProviders configured."));
+  }
+
+  @Test
+  public void testFullCipher() throws Exception {
+    String[] args1 = {"create", "key1", "--cipher", "AES/CBC/pkcs5Padding", 
+        "--provider", "jceks://file" + tmpDir + "/keystore.jceks"};
+    
+    int rc = 0;
+    KeyShell ks = new KeyShell();
+    ks.setConf(new Configuration());
+    rc = ks.run(args1);
+    assertEquals(0, rc);
+    assertTrue(outContent.toString().contains("key1 has been successfully " +
+    		"created."));
+
+    outContent.flush();
+    String[] args2 = {"delete", "key1", "--provider", 
+        "jceks://file" + tmpDir + "/keystore.jceks"};
+    rc = ks.run(args2);
+    assertEquals(0, rc);
+    assertTrue(outContent.toString().contains("key1 has been successfully " +
+    		"deleted."));
+  }
+}

+ 13 - 2
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -484,6 +484,9 @@ Trunk (Unreleased)
     HDFS-5775. Consolidate the code for serialization in CacheManager
     HDFS-5775. Consolidate the code for serialization in CacheManager
     (Haohui Mai via brandonli)
     (Haohui Mai via brandonli)
 
 
+    HDFS-5794. Fix the inconsistency of layout version number of 
+    ADD_DATANODE_AND_STORAGE_UUIDS between trunk and branch-2. (jing9)
+
 Release 2.4.0 - UNRELEASED
 Release 2.4.0 - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES
@@ -518,6 +521,9 @@ Release 2.4.0 - UNRELEASED
     as a collection of storages (see breakdown of tasks below for features and
     as a collection of storages (see breakdown of tasks below for features and
     contributors).
     contributors).
 
 
+    HDFS-5784. reserve space in edit log header and fsimage header for feature
+    flag section (cmccabe)
+
   IMPROVEMENTS
   IMPROVEMENTS
 
 
     HDFS-5267. Remove volatile from LightWeightHashSet. (Junping Du via llu)
     HDFS-5267. Remove volatile from LightWeightHashSet. (Junping Du via llu)
@@ -687,6 +693,8 @@ Release 2.4.0 - UNRELEASED
     InvalidEncryptionKeyException in fetchBlockByteRange (Liang Xie via Colin
     InvalidEncryptionKeyException in fetchBlockByteRange (Liang Xie via Colin
     Patrick McCabe)
     Patrick McCabe)
 
 
+    HDFS-5704. Change OP_UPDATE_BLOCKS with a new OP_ADD_BLOCK. (jing9)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
     HDFS-5239.  Allow FSNamesystem lock fairness to be configurable (daryn)
     HDFS-5239.  Allow FSNamesystem lock fairness to be configurable (daryn)
@@ -765,6 +773,11 @@ Release 2.4.0 - UNRELEASED
     HDFS-5579. Under construction files make DataNode decommission take very long
     HDFS-5579. Under construction files make DataNode decommission take very long
     hours. (zhaoyunjiong via jing9)
     hours. (zhaoyunjiong via jing9)
 
 
+    HDFS-5777. Update LayoutVersion for the new editlog op OP_ADD_BLOCK. (jing9)
+
+    HDFS-5800. Fix a typo in DFSClient.renewLease().  (Kousuke Saruta
+    via szetszwo)
+
   BREAKDOWN OF HDFS-2832 SUBTASKS AND RELATED JIRAS
   BREAKDOWN OF HDFS-2832 SUBTASKS AND RELATED JIRAS
 
 
     HDFS-4985. Add storage type to the protocol and expose it in block report
     HDFS-4985. Add storage type to the protocol and expose it in block report
@@ -940,8 +953,6 @@ Release 2.3.0 - UNRELEASED
     HDFS-5677. Need error checking for HA cluster configuration.
     HDFS-5677. Need error checking for HA cluster configuration.
     (Vincent Sheffer via cos)
     (Vincent Sheffer via cos)
 
 
-    HDFS-5704. Change OP_UPDATE_BLOCKS with a new OP_ADD_BLOCK. (jing9)
-
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
   BUG FIXES
   BUG FIXES

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -770,7 +770,7 @@ public class DFSClient implements java.io.Closeable {
         final long elapsed = Time.now() - getLastLeaseRenewal();
         final long elapsed = Time.now() - getLastLeaseRenewal();
         if (elapsed > HdfsConstants.LEASE_HARDLIMIT_PERIOD) {
         if (elapsed > HdfsConstants.LEASE_HARDLIMIT_PERIOD) {
           LOG.warn("Failed to renew lease for " + clientName + " for "
           LOG.warn("Failed to renew lease for " + clientName + " for "
-              + (elapsed/1000) + " seconds (>= soft-limit ="
+              + (elapsed/1000) + " seconds (>= hard-limit ="
               + (HdfsConstants.LEASE_HARDLIMIT_PERIOD/1000) + " seconds.) "
               + (HdfsConstants.LEASE_HARDLIMIT_PERIOD/1000) + " seconds.) "
               + "Closing all files being written ...", e);
               + "Closing all files being written ...", e);
           closeAllFilesBeingWritten(true);
           closeAllFilesBeingWritten(true);

+ 64 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LayoutFlags.java

@@ -0,0 +1,64 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.protocol;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Sets;
+
+/**
+ * LayoutFlags represent features which the FSImage and edit logs can either
+ * support or not, independently of layout version.
+ * 
+ * Note: all flags starting with 'test' are reserved for unit test purposes.
+ */
+@InterfaceAudience.Private
+public class LayoutFlags {
+  /**
+   * Load a LayoutFlags object from a stream.
+   *
+   * @param in            The stream to read from.
+   * @throws IOException
+   */
+  public static LayoutFlags read(DataInputStream in)
+      throws IOException {
+    int length = in.readInt();
+    if (length < 0) {
+      throw new IOException("The length of the feature flag section " +
+          "was negative at " + length + " bytes.");
+    } else if (length > 0) {
+      throw new IOException("Found feature flags which we can't handle. " +
+          "Please upgrade your software.");
+    }
+    return new LayoutFlags();
+  }
+
+  private LayoutFlags() {
+  }
+
+  public static void write(DataOutputStream out) throws IOException {
+    out.writeInt(0);
+  }
+}

+ 5 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LayoutVersion.java

@@ -107,12 +107,14 @@ public class LayoutVersion {
         "block IDs in the edits log and image files"),
         "block IDs in the edits log and image files"),
     EDITLOG_SUPPORT_RETRYCACHE(-47, "Record ClientId and CallId in editlog to " 
     EDITLOG_SUPPORT_RETRYCACHE(-47, "Record ClientId and CallId in editlog to " 
         + "enable rebuilding retry cache in case of HA failover"),
         + "enable rebuilding retry cache in case of HA failover"),
-    CACHING(-48, "Support for cache pools and path-based caching"),
+    EDITLOG_ADD_BLOCK(-48, "Add new editlog that only records allocation of "
+        + "the new block instead of the entire block list"),
     ADD_DATANODE_AND_STORAGE_UUIDS(-49, "Replace StorageID with DatanodeUuid."
     ADD_DATANODE_AND_STORAGE_UUIDS(-49, "Replace StorageID with DatanodeUuid."
         + " Use distinct StorageUuid per storage directory."),
         + " Use distinct StorageUuid per storage directory."),
-    PROTOBUF_FORMAT(-50, "Use protobuf to serialize FSImage");
+    ADD_LAYOUT_FLAGS(-50, "Add support for layout flags."),
+    CACHING(-51, "Support for cache pools and path-based caching"),
+    PROTOBUF_FORMAT(-52, "Use protobuf to serialize FSImage");
 
 
-    
     final int lv;
     final int lv;
     final int ancestorLV;
     final int ancestorLV;
     final String description;
     final String description;

+ 11 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java

@@ -34,6 +34,9 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.LayoutFlags;
+import org.apache.hadoop.hdfs.protocol.LayoutVersion;
+import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.namenode.TransferFsImage.HttpGetFailedException;
 import org.apache.hadoop.hdfs.server.namenode.TransferFsImage.HttpGetFailedException;
 import org.apache.hadoop.hdfs.web.URLConnectionFactory;
 import org.apache.hadoop.hdfs.web.URLConnectionFactory;
@@ -146,6 +149,14 @@ public class EditLogFileInputStream extends EditLogInputStream {
       } catch (EOFException eofe) {
       } catch (EOFException eofe) {
         throw new LogHeaderCorruptException("No header found in log");
         throw new LogHeaderCorruptException("No header found in log");
       }
       }
+      if (LayoutVersion.supports(Feature.ADD_LAYOUT_FLAGS, logVersion)) {
+        try {
+          LayoutFlags.read(dataIn);
+        } catch (EOFException eofe) {
+          throw new LogHeaderCorruptException("EOF while reading layout " +
+              "flags from log");
+        }
+      }
       reader = new FSEditLogOp.Reader(dataIn, tracker, logVersion);
       reader = new FSEditLogOp.Reader(dataIn, tracker, logVersion);
       reader.setMaxOpSize(maxOpSize);
       reader.setMaxOpSize(maxOpSize);
       state = State.OPEN;
       state = State.OPEN;

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java

@@ -32,6 +32,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.LayoutFlags;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IOUtils;
 
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.annotations.VisibleForTesting;
@@ -132,6 +133,7 @@ public class EditLogFileOutputStream extends EditLogOutputStream {
   @VisibleForTesting
   @VisibleForTesting
   public static void writeHeader(DataOutputStream out) throws IOException {
   public static void writeHeader(DataOutputStream out) throws IOException {
     out.writeInt(HdfsConstants.LAYOUT_VERSION);
     out.writeInt(HdfsConstants.LAYOUT_VERSION);
+    LayoutFlags.write(out);
   }
   }
 
 
   @Override
   @Override

+ 7 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOpCodes.java

@@ -60,13 +60,13 @@ public enum FSEditLogOpCodes {
   OP_DISALLOW_SNAPSHOT          ((byte) 30),
   OP_DISALLOW_SNAPSHOT          ((byte) 30),
   OP_SET_GENSTAMP_V2            ((byte) 31),
   OP_SET_GENSTAMP_V2            ((byte) 31),
   OP_ALLOCATE_BLOCK_ID          ((byte) 32),
   OP_ALLOCATE_BLOCK_ID          ((byte) 32),
-  OP_ADD_CACHE_DIRECTIVE       ((byte) 33),
-  OP_REMOVE_CACHE_DIRECTIVE    ((byte) 34),
-  OP_ADD_CACHE_POOL                       ((byte) 35),
-  OP_MODIFY_CACHE_POOL                    ((byte) 36),
-  OP_REMOVE_CACHE_POOL                    ((byte) 37),
-  OP_MODIFY_CACHE_DIRECTIVE     ((byte) 38),
-  OP_ADD_BLOCK                  ((byte) 39),
+  OP_ADD_BLOCK                  ((byte) 33),
+  OP_ADD_CACHE_DIRECTIVE       ((byte) 34),
+  OP_REMOVE_CACHE_DIRECTIVE    ((byte) 35),
+  OP_ADD_CACHE_POOL                       ((byte) 36),
+  OP_MODIFY_CACHE_POOL                    ((byte) 37),
+  OP_REMOVE_CACHE_POOL                    ((byte) 38),
+  OP_MODIFY_CACHE_DIRECTIVE     ((byte) 39),
 
 
   // Note that fromByte(..) depends on OP_INVALID being at the last position.  
   // Note that fromByte(..) depends on OP_INVALID being at the last position.  
   OP_INVALID                    ((byte) -1);
   OP_INVALID                    ((byte) -1);

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java

@@ -48,6 +48,7 @@ import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
+import org.apache.hadoop.hdfs.protocol.LayoutFlags;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
@@ -324,6 +325,9 @@ public class FSImageFormat {
         }
         }
         boolean supportSnapshot = LayoutVersion.supports(Feature.SNAPSHOT,
         boolean supportSnapshot = LayoutVersion.supports(Feature.SNAPSHOT,
             imgVersion);
             imgVersion);
+        if (LayoutVersion.supports(Feature.ADD_LAYOUT_FLAGS, imgVersion)) {
+          LayoutFlags.read(in);
+        }
 
 
         // read namespaceID: first appeared in version -2
         // read namespaceID: first appeared in version -2
         in.readInt();
         in.readInt();
@@ -1053,6 +1057,7 @@ public class FSImageFormat {
       DataOutputStream out = new DataOutputStream(fos);
       DataOutputStream out = new DataOutputStream(fos);
       try {
       try {
         out.writeInt(HdfsConstants.LAYOUT_VERSION);
         out.writeInt(HdfsConstants.LAYOUT_VERSION);
+        LayoutFlags.write(out);
         // We use the non-locked version of getNamespaceInfo here since
         // We use the non-locked version of getNamespaceInfo here since
         // the coordinating thread of saveNamespace already has read-locked
         // the coordinating thread of saveNamespace already has read-locked
         // the namespace for us. If we attempt to take another readlock
         // the namespace for us. If we attempt to take another readlock

+ 5 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java

@@ -28,6 +28,7 @@ import java.util.Map;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
+import org.apache.hadoop.hdfs.protocol.LayoutFlags;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
@@ -126,7 +127,7 @@ class ImageLoaderCurrent implements ImageLoader {
                                       new SimpleDateFormat("yyyy-MM-dd HH:mm");
                                       new SimpleDateFormat("yyyy-MM-dd HH:mm");
   private static int[] versions = { -16, -17, -18, -19, -20, -21, -22, -23,
   private static int[] versions = { -16, -17, -18, -19, -20, -21, -22, -23,
       -24, -25, -26, -27, -28, -30, -31, -32, -33, -34, -35, -36, -37, -38, -39,
       -24, -25, -26, -27, -28, -30, -31, -32, -33, -34, -35, -36, -37, -38, -39,
-      -40, -41, -42, -43, -44, -45, -46, -47, -48, -49, -50 };
+      -40, -41, -42, -43, -44, -45, -46, -47, -48, -49, -50, -51, -52 };
   private int imageVersion = 0;
   private int imageVersion = 0;
   
   
   private final Map<Long, Boolean> subtreeMap = new HashMap<Long, Boolean>();
   private final Map<Long, Boolean> subtreeMap = new HashMap<Long, Boolean>();
@@ -157,6 +158,9 @@ class ImageLoaderCurrent implements ImageLoader {
       imageVersion = in.readInt();
       imageVersion = in.readInt();
       if( !canLoadVersion(imageVersion))
       if( !canLoadVersion(imageVersion))
         throw new IOException("Cannot process fslayout version " + imageVersion);
         throw new IOException("Cannot process fslayout version " + imageVersion);
+      if (LayoutVersion.supports(Feature.ADD_LAYOUT_FLAGS, imageVersion)) {
+        LayoutFlags.read(in);
+      }
 
 
       v.visit(ImageElement.IMAGE_VERSION, imageVersion);
       v.visit(ImageElement.IMAGE_VERSION, imageVersion);
       v.visit(ImageElement.NAMESPACE_ID, in.readInt());
       v.visit(ImageElement.NAMESPACE_ID, in.readInt());

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java

@@ -191,6 +191,7 @@ public class TestJournalNode {
         "/getJournal?segmentTxId=1&jid=" + journalId));
         "/getJournal?segmentTxId=1&jid=" + journalId));
     byte[] expected = Bytes.concat(
     byte[] expected = Bytes.concat(
             Ints.toByteArray(HdfsConstants.LAYOUT_VERSION),
             Ints.toByteArray(HdfsConstants.LAYOUT_VERSION),
+            (new byte[] { 0, 0, 0, 0 }), // layout flags section
             EDITS_DATA);
             EDITS_DATA);
 
 
     assertArrayEquals(expected, retrievedViaHttp);
     assertArrayEquals(expected, retrievedViaHttp);

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java

@@ -377,8 +377,9 @@ public class TestFSEditLogLoader {
     File testDir = new File(TEST_DIR, "testValidateEmptyEditLog");
     File testDir = new File(TEST_DIR, "testValidateEmptyEditLog");
     SortedMap<Long, Long> offsetToTxId = Maps.newTreeMap();
     SortedMap<Long, Long> offsetToTxId = Maps.newTreeMap();
     File logFile = prepareUnfinalizedTestEditLog(testDir, 0, offsetToTxId);
     File logFile = prepareUnfinalizedTestEditLog(testDir, 0, offsetToTxId);
-    // Truncate the file so that there is nothing except the header
-    truncateFile(logFile, 4);
+    // Truncate the file so that there is nothing except the header and
+    // layout flags section.
+    truncateFile(logFile, 8);
     EditLogValidation validation =
     EditLogValidation validation =
         EditLogFileInputStream.validateEditLog(logFile);
         EditLogFileInputStream.validateEditLog(logFile);
     assertTrue(!validation.hasCorruptHeader());
     assertTrue(!validation.hasCorruptHeader());

binární
hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored


+ 84 - 84
hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml

@@ -1,6 +1,6 @@
 <?xml version="1.0" encoding="UTF-8"?>
 <?xml version="1.0" encoding="UTF-8"?>
 <EDITS>
 <EDITS>
-  <EDITS_VERSION>-49</EDITS_VERSION>
+  <EDITS_VERSION>-51</EDITS_VERSION>
   <RECORD>
   <RECORD>
     <OPCODE>OP_START_LOG_SEGMENT</OPCODE>
     <OPCODE>OP_START_LOG_SEGMENT</OPCODE>
     <DATA>
     <DATA>
@@ -13,8 +13,8 @@
       <TXID>2</TXID>
       <TXID>2</TXID>
       <DELEGATION_KEY>
       <DELEGATION_KEY>
         <KEY_ID>1</KEY_ID>
         <KEY_ID>1</KEY_ID>
-        <EXPIRY_DATE>1389736494300</EXPIRY_DATE>
-        <KEY>d1a0861e6b9e394e</KEY>
+        <EXPIRY_DATE>1390519460949</EXPIRY_DATE>
+        <KEY>dc8d30edc97df67d</KEY>
       </DELEGATION_KEY>
       </DELEGATION_KEY>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
@@ -24,8 +24,8 @@
       <TXID>3</TXID>
       <TXID>3</TXID>
       <DELEGATION_KEY>
       <DELEGATION_KEY>
         <KEY_ID>2</KEY_ID>
         <KEY_ID>2</KEY_ID>
-        <EXPIRY_DATE>1389736494302</EXPIRY_DATE>
-        <KEY>8239b8f0ed7e6ce6</KEY>
+        <EXPIRY_DATE>1390519460952</EXPIRY_DATE>
+        <KEY>096bc20b6debed03</KEY>
       </DELEGATION_KEY>
       </DELEGATION_KEY>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
@@ -37,18 +37,18 @@
       <INODEID>16386</INODEID>
       <INODEID>16386</INODEID>
       <PATH>/file_create</PATH>
       <PATH>/file_create</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1389045295288</MTIME>
-      <ATIME>1389045295288</ATIME>
+      <MTIME>1389828264873</MTIME>
+      <ATIME>1389828264873</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-216163116_1</CLIENT_NAME>
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_16108824_1</CLIENT_NAME>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <PERMISSION_STATUS>
       <PERMISSION_STATUS>
         <USERNAME>jing</USERNAME>
         <USERNAME>jing</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
       </PERMISSION_STATUS>
-      <RPC_CLIENTID>48c96601-9238-4d1f-b78b-ef0f1e922ba2</RPC_CLIENTID>
-      <RPC_CALLID>7</RPC_CALLID>
+      <RPC_CLIENTID>b5928e80-e373-4807-a688-f94483d08ce5</RPC_CLIENTID>
+      <RPC_CALLID>9</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
   <RECORD>
   <RECORD>
@@ -59,8 +59,8 @@
       <INODEID>0</INODEID>
       <INODEID>0</INODEID>
       <PATH>/file_create</PATH>
       <PATH>/file_create</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1389045295307</MTIME>
-      <ATIME>1389045295288</ATIME>
+      <MTIME>1389828265699</MTIME>
+      <ATIME>1389828264873</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_MACHINE></CLIENT_MACHINE>
       <CLIENT_MACHINE></CLIENT_MACHINE>
@@ -78,9 +78,9 @@
       <LENGTH>0</LENGTH>
       <LENGTH>0</LENGTH>
       <SRC>/file_create</SRC>
       <SRC>/file_create</SRC>
       <DST>/file_moved</DST>
       <DST>/file_moved</DST>
-      <TIMESTAMP>1389045295311</TIMESTAMP>
-      <RPC_CLIENTID>48c96601-9238-4d1f-b78b-ef0f1e922ba2</RPC_CLIENTID>
-      <RPC_CALLID>9</RPC_CALLID>
+      <TIMESTAMP>1389828265705</TIMESTAMP>
+      <RPC_CLIENTID>b5928e80-e373-4807-a688-f94483d08ce5</RPC_CLIENTID>
+      <RPC_CALLID>11</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
   <RECORD>
   <RECORD>
@@ -89,9 +89,9 @@
       <TXID>7</TXID>
       <TXID>7</TXID>
       <LENGTH>0</LENGTH>
       <LENGTH>0</LENGTH>
       <PATH>/file_moved</PATH>
       <PATH>/file_moved</PATH>
-      <TIMESTAMP>1389045295318</TIMESTAMP>
-      <RPC_CLIENTID>48c96601-9238-4d1f-b78b-ef0f1e922ba2</RPC_CLIENTID>
-      <RPC_CALLID>10</RPC_CALLID>
+      <TIMESTAMP>1389828265712</TIMESTAMP>
+      <RPC_CLIENTID>b5928e80-e373-4807-a688-f94483d08ce5</RPC_CLIENTID>
+      <RPC_CALLID>12</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
   <RECORD>
   <RECORD>
@@ -101,7 +101,7 @@
       <LENGTH>0</LENGTH>
       <LENGTH>0</LENGTH>
       <INODEID>16387</INODEID>
       <INODEID>16387</INODEID>
       <PATH>/directory_mkdir</PATH>
       <PATH>/directory_mkdir</PATH>
-      <TIMESTAMP>1389045295326</TIMESTAMP>
+      <TIMESTAMP>1389828265722</TIMESTAMP>
       <PERMISSION_STATUS>
       <PERMISSION_STATUS>
         <USERNAME>jing</USERNAME>
         <USERNAME>jing</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <GROUPNAME>supergroup</GROUPNAME>
@@ -136,8 +136,8 @@
       <TXID>12</TXID>
       <TXID>12</TXID>
       <SNAPSHOTROOT>/directory_mkdir</SNAPSHOTROOT>
       <SNAPSHOTROOT>/directory_mkdir</SNAPSHOTROOT>
       <SNAPSHOTNAME>snapshot1</SNAPSHOTNAME>
       <SNAPSHOTNAME>snapshot1</SNAPSHOTNAME>
-      <RPC_CLIENTID>48c96601-9238-4d1f-b78b-ef0f1e922ba2</RPC_CLIENTID>
-      <RPC_CALLID>15</RPC_CALLID>
+      <RPC_CLIENTID>b5928e80-e373-4807-a688-f94483d08ce5</RPC_CLIENTID>
+      <RPC_CALLID>17</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
   <RECORD>
   <RECORD>
@@ -147,8 +147,8 @@
       <SNAPSHOTROOT>/directory_mkdir</SNAPSHOTROOT>
       <SNAPSHOTROOT>/directory_mkdir</SNAPSHOTROOT>
       <SNAPSHOTOLDNAME>snapshot1</SNAPSHOTOLDNAME>
       <SNAPSHOTOLDNAME>snapshot1</SNAPSHOTOLDNAME>
       <SNAPSHOTNEWNAME>snapshot2</SNAPSHOTNEWNAME>
       <SNAPSHOTNEWNAME>snapshot2</SNAPSHOTNEWNAME>
-      <RPC_CLIENTID>48c96601-9238-4d1f-b78b-ef0f1e922ba2</RPC_CLIENTID>
-      <RPC_CALLID>16</RPC_CALLID>
+      <RPC_CLIENTID>b5928e80-e373-4807-a688-f94483d08ce5</RPC_CLIENTID>
+      <RPC_CALLID>18</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
   <RECORD>
   <RECORD>
@@ -157,8 +157,8 @@
       <TXID>14</TXID>
       <TXID>14</TXID>
       <SNAPSHOTROOT>/directory_mkdir</SNAPSHOTROOT>
       <SNAPSHOTROOT>/directory_mkdir</SNAPSHOTROOT>
       <SNAPSHOTNAME>snapshot2</SNAPSHOTNAME>
       <SNAPSHOTNAME>snapshot2</SNAPSHOTNAME>
-      <RPC_CLIENTID>48c96601-9238-4d1f-b78b-ef0f1e922ba2</RPC_CLIENTID>
-      <RPC_CALLID>17</RPC_CALLID>
+      <RPC_CLIENTID>b5928e80-e373-4807-a688-f94483d08ce5</RPC_CLIENTID>
+      <RPC_CALLID>19</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
   <RECORD>
   <RECORD>
@@ -169,18 +169,18 @@
       <INODEID>16388</INODEID>
       <INODEID>16388</INODEID>
       <PATH>/file_create</PATH>
       <PATH>/file_create</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1389045295354</MTIME>
-      <ATIME>1389045295354</ATIME>
+      <MTIME>1389828265757</MTIME>
+      <ATIME>1389828265757</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-216163116_1</CLIENT_NAME>
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_16108824_1</CLIENT_NAME>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <PERMISSION_STATUS>
       <PERMISSION_STATUS>
         <USERNAME>jing</USERNAME>
         <USERNAME>jing</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
       </PERMISSION_STATUS>
-      <RPC_CLIENTID>48c96601-9238-4d1f-b78b-ef0f1e922ba2</RPC_CLIENTID>
-      <RPC_CALLID>18</RPC_CALLID>
+      <RPC_CLIENTID>b5928e80-e373-4807-a688-f94483d08ce5</RPC_CLIENTID>
+      <RPC_CALLID>20</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
   <RECORD>
   <RECORD>
@@ -191,8 +191,8 @@
       <INODEID>0</INODEID>
       <INODEID>0</INODEID>
       <PATH>/file_create</PATH>
       <PATH>/file_create</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1389045295357</MTIME>
-      <ATIME>1389045295354</ATIME>
+      <MTIME>1389828265759</MTIME>
+      <ATIME>1389828265757</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_MACHINE></CLIENT_MACHINE>
       <CLIENT_MACHINE></CLIENT_MACHINE>
@@ -253,10 +253,10 @@
       <LENGTH>0</LENGTH>
       <LENGTH>0</LENGTH>
       <SRC>/file_create</SRC>
       <SRC>/file_create</SRC>
       <DST>/file_moved</DST>
       <DST>/file_moved</DST>
-      <TIMESTAMP>1389045295378</TIMESTAMP>
+      <TIMESTAMP>1389828265782</TIMESTAMP>
       <OPTIONS>NONE</OPTIONS>
       <OPTIONS>NONE</OPTIONS>
-      <RPC_CLIENTID>48c96601-9238-4d1f-b78b-ef0f1e922ba2</RPC_CLIENTID>
-      <RPC_CALLID>25</RPC_CALLID>
+      <RPC_CLIENTID>b5928e80-e373-4807-a688-f94483d08ce5</RPC_CLIENTID>
+      <RPC_CALLID>27</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
   <RECORD>
   <RECORD>
@@ -267,18 +267,18 @@
       <INODEID>16389</INODEID>
       <INODEID>16389</INODEID>
       <PATH>/file_concat_target</PATH>
       <PATH>/file_concat_target</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1389045295383</MTIME>
-      <ATIME>1389045295383</ATIME>
+      <MTIME>1389828265787</MTIME>
+      <ATIME>1389828265787</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-216163116_1</CLIENT_NAME>
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_16108824_1</CLIENT_NAME>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <PERMISSION_STATUS>
       <PERMISSION_STATUS>
         <USERNAME>jing</USERNAME>
         <USERNAME>jing</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
       </PERMISSION_STATUS>
-      <RPC_CLIENTID>48c96601-9238-4d1f-b78b-ef0f1e922ba2</RPC_CLIENTID>
-      <RPC_CALLID>27</RPC_CALLID>
+      <RPC_CLIENTID>b5928e80-e373-4807-a688-f94483d08ce5</RPC_CLIENTID>
+      <RPC_CALLID>29</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
   <RECORD>
   <RECORD>
@@ -383,8 +383,8 @@
       <INODEID>0</INODEID>
       <INODEID>0</INODEID>
       <PATH>/file_concat_target</PATH>
       <PATH>/file_concat_target</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1389045295484</MTIME>
-      <ATIME>1389045295383</ATIME>
+      <MTIME>1389828266540</MTIME>
+      <ATIME>1389828265787</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_MACHINE></CLIENT_MACHINE>
       <CLIENT_MACHINE></CLIENT_MACHINE>
@@ -418,18 +418,18 @@
       <INODEID>16390</INODEID>
       <INODEID>16390</INODEID>
       <PATH>/file_concat_0</PATH>
       <PATH>/file_concat_0</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1389045295486</MTIME>
-      <ATIME>1389045295486</ATIME>
+      <MTIME>1389828266544</MTIME>
+      <ATIME>1389828266544</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-216163116_1</CLIENT_NAME>
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_16108824_1</CLIENT_NAME>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <PERMISSION_STATUS>
       <PERMISSION_STATUS>
         <USERNAME>jing</USERNAME>
         <USERNAME>jing</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
       </PERMISSION_STATUS>
-      <RPC_CLIENTID>48c96601-9238-4d1f-b78b-ef0f1e922ba2</RPC_CLIENTID>
-      <RPC_CALLID>40</RPC_CALLID>
+      <RPC_CLIENTID>b5928e80-e373-4807-a688-f94483d08ce5</RPC_CLIENTID>
+      <RPC_CALLID>41</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
   <RECORD>
   <RECORD>
@@ -534,8 +534,8 @@
       <INODEID>0</INODEID>
       <INODEID>0</INODEID>
       <PATH>/file_concat_0</PATH>
       <PATH>/file_concat_0</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1389045295507</MTIME>
-      <ATIME>1389045295486</ATIME>
+      <MTIME>1389828266569</MTIME>
+      <ATIME>1389828266544</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_MACHINE></CLIENT_MACHINE>
       <CLIENT_MACHINE></CLIENT_MACHINE>
@@ -569,18 +569,18 @@
       <INODEID>16391</INODEID>
       <INODEID>16391</INODEID>
       <PATH>/file_concat_1</PATH>
       <PATH>/file_concat_1</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1389045295509</MTIME>
-      <ATIME>1389045295509</ATIME>
+      <MTIME>1389828266572</MTIME>
+      <ATIME>1389828266572</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-216163116_1</CLIENT_NAME>
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_16108824_1</CLIENT_NAME>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <PERMISSION_STATUS>
       <PERMISSION_STATUS>
         <USERNAME>jing</USERNAME>
         <USERNAME>jing</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
       </PERMISSION_STATUS>
-      <RPC_CLIENTID>48c96601-9238-4d1f-b78b-ef0f1e922ba2</RPC_CLIENTID>
-      <RPC_CALLID>52</RPC_CALLID>
+      <RPC_CLIENTID>b5928e80-e373-4807-a688-f94483d08ce5</RPC_CLIENTID>
+      <RPC_CALLID>53</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
   <RECORD>
   <RECORD>
@@ -685,8 +685,8 @@
       <INODEID>0</INODEID>
       <INODEID>0</INODEID>
       <PATH>/file_concat_1</PATH>
       <PATH>/file_concat_1</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1389045295532</MTIME>
-      <ATIME>1389045295509</ATIME>
+      <MTIME>1389828266599</MTIME>
+      <ATIME>1389828266572</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_MACHINE></CLIENT_MACHINE>
       <CLIENT_MACHINE></CLIENT_MACHINE>
@@ -718,13 +718,13 @@
       <TXID>56</TXID>
       <TXID>56</TXID>
       <LENGTH>0</LENGTH>
       <LENGTH>0</LENGTH>
       <TRG>/file_concat_target</TRG>
       <TRG>/file_concat_target</TRG>
-      <TIMESTAMP>1389045295535</TIMESTAMP>
+      <TIMESTAMP>1389828266603</TIMESTAMP>
       <SOURCES>
       <SOURCES>
         <SOURCE1>/file_concat_0</SOURCE1>
         <SOURCE1>/file_concat_0</SOURCE1>
         <SOURCE2>/file_concat_1</SOURCE2>
         <SOURCE2>/file_concat_1</SOURCE2>
       </SOURCES>
       </SOURCES>
-      <RPC_CLIENTID>48c96601-9238-4d1f-b78b-ef0f1e922ba2</RPC_CLIENTID>
-      <RPC_CALLID>63</RPC_CALLID>
+      <RPC_CLIENTID>b5928e80-e373-4807-a688-f94483d08ce5</RPC_CLIENTID>
+      <RPC_CALLID>64</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
   <RECORD>
   <RECORD>
@@ -735,15 +735,15 @@
       <INODEID>16392</INODEID>
       <INODEID>16392</INODEID>
       <PATH>/file_symlink</PATH>
       <PATH>/file_symlink</PATH>
       <VALUE>/file_concat_target</VALUE>
       <VALUE>/file_concat_target</VALUE>
-      <MTIME>1389045295540</MTIME>
-      <ATIME>1389045295540</ATIME>
+      <MTIME>1389828266633</MTIME>
+      <ATIME>1389828266633</ATIME>
       <PERMISSION_STATUS>
       <PERMISSION_STATUS>
         <USERNAME>jing</USERNAME>
         <USERNAME>jing</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>511</MODE>
         <MODE>511</MODE>
       </PERMISSION_STATUS>
       </PERMISSION_STATUS>
-      <RPC_CLIENTID>48c96601-9238-4d1f-b78b-ef0f1e922ba2</RPC_CLIENTID>
-      <RPC_CALLID>64</RPC_CALLID>
+      <RPC_CLIENTID>b5928e80-e373-4807-a688-f94483d08ce5</RPC_CLIENTID>
+      <RPC_CALLID>66</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
   <RECORD>
   <RECORD>
@@ -754,18 +754,18 @@
       <INODEID>16393</INODEID>
       <INODEID>16393</INODEID>
       <PATH>/hard-lease-recovery-test</PATH>
       <PATH>/hard-lease-recovery-test</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1389045295543</MTIME>
-      <ATIME>1389045295543</ATIME>
+      <MTIME>1389828266637</MTIME>
+      <ATIME>1389828266637</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME>DFSClient_NONMAPREDUCE_-216163116_1</CLIENT_NAME>
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_16108824_1</CLIENT_NAME>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <PERMISSION_STATUS>
       <PERMISSION_STATUS>
         <USERNAME>jing</USERNAME>
         <USERNAME>jing</USERNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
       </PERMISSION_STATUS>
-      <RPC_CLIENTID>48c96601-9238-4d1f-b78b-ef0f1e922ba2</RPC_CLIENTID>
-      <RPC_CALLID>65</RPC_CALLID>
+      <RPC_CLIENTID>b5928e80-e373-4807-a688-f94483d08ce5</RPC_CLIENTID>
+      <RPC_CALLID>67</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
   <RECORD>
   <RECORD>
@@ -821,7 +821,7 @@
     <OPCODE>OP_REASSIGN_LEASE</OPCODE>
     <OPCODE>OP_REASSIGN_LEASE</OPCODE>
     <DATA>
     <DATA>
       <TXID>64</TXID>
       <TXID>64</TXID>
-      <LEASEHOLDER>DFSClient_NONMAPREDUCE_-216163116_1</LEASEHOLDER>
+      <LEASEHOLDER>DFSClient_NONMAPREDUCE_16108824_1</LEASEHOLDER>
       <PATH>/hard-lease-recovery-test</PATH>
       <PATH>/hard-lease-recovery-test</PATH>
       <NEWHOLDER>HDFS_NameNode</NEWHOLDER>
       <NEWHOLDER>HDFS_NameNode</NEWHOLDER>
     </DATA>
     </DATA>
@@ -834,8 +834,8 @@
       <INODEID>0</INODEID>
       <INODEID>0</INODEID>
       <PATH>/hard-lease-recovery-test</PATH>
       <PATH>/hard-lease-recovery-test</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
-      <MTIME>1389045298180</MTIME>
-      <ATIME>1389045295543</ATIME>
+      <MTIME>1389828269751</MTIME>
+      <ATIME>1389828266637</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_MACHINE></CLIENT_MACHINE>
       <CLIENT_MACHINE></CLIENT_MACHINE>
@@ -861,8 +861,8 @@
       <MODE>493</MODE>
       <MODE>493</MODE>
       <LIMIT>9223372036854775807</LIMIT>
       <LIMIT>9223372036854775807</LIMIT>
       <MAXRELATIVEEXPIRY>2305843009213693951</MAXRELATIVEEXPIRY>
       <MAXRELATIVEEXPIRY>2305843009213693951</MAXRELATIVEEXPIRY>
-      <RPC_CLIENTID>48c96601-9238-4d1f-b78b-ef0f1e922ba2</RPC_CLIENTID>
-      <RPC_CALLID>72</RPC_CALLID>
+      <RPC_CLIENTID>b5928e80-e373-4807-a688-f94483d08ce5</RPC_CLIENTID>
+      <RPC_CALLID>74</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
   <RECORD>
   <RECORD>
@@ -871,8 +871,8 @@
       <TXID>67</TXID>
       <TXID>67</TXID>
       <POOLNAME>pool1</POOLNAME>
       <POOLNAME>pool1</POOLNAME>
       <LIMIT>99</LIMIT>
       <LIMIT>99</LIMIT>
-      <RPC_CLIENTID>48c96601-9238-4d1f-b78b-ef0f1e922ba2</RPC_CLIENTID>
-      <RPC_CALLID>73</RPC_CALLID>
+      <RPC_CLIENTID>b5928e80-e373-4807-a688-f94483d08ce5</RPC_CLIENTID>
+      <RPC_CALLID>75</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
   <RECORD>
   <RECORD>
@@ -883,9 +883,9 @@
       <PATH>/path</PATH>
       <PATH>/path</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
       <POOL>pool1</POOL>
       <POOL>pool1</POOL>
-      <EXPIRATION>2305844398258992525</EXPIRATION>
-      <RPC_CLIENTID>48c96601-9238-4d1f-b78b-ef0f1e922ba2</RPC_CLIENTID>
-      <RPC_CALLID>74</RPC_CALLID>
+      <EXPIRATION>2305844399041964876</EXPIRATION>
+      <RPC_CLIENTID>b5928e80-e373-4807-a688-f94483d08ce5</RPC_CLIENTID>
+      <RPC_CALLID>76</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
   <RECORD>
   <RECORD>
@@ -894,8 +894,8 @@
       <TXID>69</TXID>
       <TXID>69</TXID>
       <ID>1</ID>
       <ID>1</ID>
       <REPLICATION>2</REPLICATION>
       <REPLICATION>2</REPLICATION>
-      <RPC_CLIENTID>48c96601-9238-4d1f-b78b-ef0f1e922ba2</RPC_CLIENTID>
-      <RPC_CALLID>75</RPC_CALLID>
+      <RPC_CLIENTID>b5928e80-e373-4807-a688-f94483d08ce5</RPC_CLIENTID>
+      <RPC_CALLID>77</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
   <RECORD>
   <RECORD>
@@ -903,8 +903,8 @@
     <DATA>
     <DATA>
       <TXID>70</TXID>
       <TXID>70</TXID>
       <ID>1</ID>
       <ID>1</ID>
-      <RPC_CLIENTID>48c96601-9238-4d1f-b78b-ef0f1e922ba2</RPC_CLIENTID>
-      <RPC_CALLID>76</RPC_CALLID>
+      <RPC_CLIENTID>b5928e80-e373-4807-a688-f94483d08ce5</RPC_CLIENTID>
+      <RPC_CALLID>78</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
   <RECORD>
   <RECORD>
@@ -912,8 +912,8 @@
     <DATA>
     <DATA>
       <TXID>71</TXID>
       <TXID>71</TXID>
       <POOLNAME>pool1</POOLNAME>
       <POOLNAME>pool1</POOLNAME>
-      <RPC_CLIENTID>48c96601-9238-4d1f-b78b-ef0f1e922ba2</RPC_CLIENTID>
-      <RPC_CALLID>77</RPC_CALLID>
+      <RPC_CLIENTID>b5928e80-e373-4807-a688-f94483d08ce5</RPC_CLIENTID>
+      <RPC_CALLID>79</RPC_CALLID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
   <RECORD>
   <RECORD>

+ 16 - 0
hadoop-mapreduce-project/CHANGES.txt

@@ -146,6 +146,8 @@ Trunk (Unreleased)
     MAPREDUCE-5191. TestQueue#testQueue fails with timeout on Windows. (Ivan
     MAPREDUCE-5191. TestQueue#testQueue fails with timeout on Windows. (Ivan
     Mitic via hitesh)
     Mitic via hitesh)
 
 
+    MAPREDUCE-5717. Task pings are interpreted as task progress (jlowe)
+
 Release 2.4.0 - UNRELEASED
 Release 2.4.0 - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES
@@ -198,6 +200,12 @@ Release 2.4.0 - UNRELEASED
 
 
     MAPREDUCE-3310. Custom grouping comparator cannot be set for Combiners (tucu)
     MAPREDUCE-3310. Custom grouping comparator cannot be set for Combiners (tucu)
 
 
+    MAPREDUCE-5672. Provide optional RollingFileAppender for container log4j
+    (syslog) (Gera Shegalov via jlowe)
+
+    MAPREDUCE-5725. Make explicit that TestNetworkedJob relies on the Capacity
+    Scheduler (Sandy Ryza)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
     MAPREDUCE-5484. YarnChild unnecessarily loads job conf twice (Sandy Ryza)
     MAPREDUCE-5484. YarnChild unnecessarily loads job conf twice (Sandy Ryza)
@@ -272,6 +280,11 @@ Release 2.4.0 - UNRELEASED
     MAPREDUCE-5689. MRAppMaster does not preempt reducers when scheduled maps 
     MAPREDUCE-5689. MRAppMaster does not preempt reducers when scheduled maps 
     cannot be fulfilled. (lohit via kasha)
     cannot be fulfilled. (lohit via kasha)
 
 
+    MAPREDUCE-5724. JobHistoryServer does not start if HDFS is not running. 
+    (tucu)
+
+    MAPREDUCE-5729. mapred job -list throws NPE (kasha)
+
 Release 2.3.0 - UNRELEASED
 Release 2.3.0 - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES
@@ -328,6 +341,9 @@ Release 2.3.0 - UNRELEASED
     MAPREDUCE-5674. Missing start and finish time in mapred.JobStatus.
     MAPREDUCE-5674. Missing start and finish time in mapred.JobStatus.
     (Chuan Liu via cnauroth)
     (Chuan Liu via cnauroth)
 
 
+    MAPREDUCE-5650. Job fails when hprof mapreduce.task.profile.map/reduce.params
+    is specified (Gera Shegalov via Sandy Ryza)
+
 Release 2.2.0 - 2013-10-13
 Release 2.2.0 - 2013-10-13
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 9 - 15
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/MapReduceChildJVM.java

@@ -149,8 +149,10 @@ public class MapReduceChildJVM {
   private static void setupLog4jProperties(Task task,
   private static void setupLog4jProperties(Task task,
       Vector<String> vargs,
       Vector<String> vargs,
       long logSize) {
       long logSize) {
-    String logLevel = getChildLogLevel(task.conf, task.isMapTask()); 
-    MRApps.addLog4jSystemProperties(logLevel, logSize, vargs);
+    String logLevel = getChildLogLevel(task.conf, task.isMapTask());
+    int numBackups = task.conf.getInt(MRJobConfig.TASK_LOG_BACKUPS,
+        MRJobConfig.DEFAULT_TASK_LOG_BACKUPS);
+    MRApps.addLog4jSystemProperties(logLevel, logSize, numBackups, vargs);
   }
   }
 
 
   public static List<String> getVMCommand(
   public static List<String> getVMCommand(
@@ -210,19 +212,11 @@ public class MapReduceChildJVM {
     if (conf.getProfileEnabled()) {
     if (conf.getProfileEnabled()) {
       if (conf.getProfileTaskRange(task.isMapTask()
       if (conf.getProfileTaskRange(task.isMapTask()
                                    ).isIncluded(task.getPartition())) {
                                    ).isIncluded(task.getPartition())) {
-        vargs.add(
-            String.format(
-                conf.getProfileParams(), 
-                getTaskLogFile(TaskLog.LogName.PROFILE)
-                )
-            );
-        if (task.isMapTask()) {
-          vargs.add(conf.get(MRJobConfig.TASK_MAP_PROFILE_PARAMS, ""));
-        }
-        else {
-          vargs.add(conf.get(MRJobConfig.TASK_REDUCE_PROFILE_PARAMS, ""));
-        }
-        
+        final String profileParams = conf.get(task.isMapTask()
+            ? MRJobConfig.TASK_MAP_PROFILE_PARAMS
+            : MRJobConfig.TASK_REDUCE_PROFILE_PARAMS, conf.getProfileParams());
+        vargs.add(String.format(profileParams,
+            getTaskLogFile(TaskLog.LogName.PROFILE)));
       }
       }
     }
     }
 
 

+ 0 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java

@@ -361,7 +361,6 @@ public class TaskAttemptListenerImpl extends CompositeService
     if (taskStatus == null) {
     if (taskStatus == null) {
       //We are using statusUpdate only as a simple ping
       //We are using statusUpdate only as a simple ping
       LOG.info("Ping from " + taskAttemptID.toString());
       LOG.info("Ping from " + taskAttemptID.toString());
-      taskHeartbeatHandler.progressing(yarnAttemptID);
       return feedback;
       return feedback;
     }
     }
 
 

+ 46 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestTaskAttemptListenerImpl.java

@@ -381,4 +381,50 @@ public class TestTaskAttemptListenerImpl {
 
 
   }
   }
 
 
+  @SuppressWarnings("rawtypes")
+  @Test
+  public void testStatusUpdateProgress()
+      throws IOException, InterruptedException {
+    AppContext appCtx = mock(AppContext.class);
+    JobTokenSecretManager secret = mock(JobTokenSecretManager.class);
+    RMHeartbeatHandler rmHeartbeatHandler =
+        mock(RMHeartbeatHandler.class);
+    TaskHeartbeatHandler hbHandler = mock(TaskHeartbeatHandler.class);
+    Dispatcher dispatcher = mock(Dispatcher.class);
+    EventHandler ea = mock(EventHandler.class);
+    when(dispatcher.getEventHandler()).thenReturn(ea);
+
+    when(appCtx.getEventHandler()).thenReturn(ea);
+    CheckpointAMPreemptionPolicy policy = new CheckpointAMPreemptionPolicy();
+    policy.init(appCtx);
+    MockTaskAttemptListenerImpl listener =
+      new MockTaskAttemptListenerImpl(appCtx, secret,
+          rmHeartbeatHandler, hbHandler, policy);
+    Configuration conf = new Configuration();
+    listener.init(conf);
+    listener.start();
+    JVMId id = new JVMId("foo",1, true, 1);
+    WrappedJvmID wid = new WrappedJvmID(id.getJobId(), id.isMap, id.getId());
+
+    TaskAttemptID attemptID = new TaskAttemptID("1", 1, TaskType.MAP, 1, 1);
+    TaskAttemptId attemptId = TypeConverter.toYarn(attemptID);
+    Task task = mock(Task.class);
+    listener.registerPendingTask(task, wid);
+    listener.registerLaunchedTask(attemptId, wid);
+    verify(hbHandler).register(attemptId);
+
+    // make sure a ping doesn't report progress
+    AMFeedback feedback = listener.statusUpdate(attemptID, null);
+    assertTrue(feedback.getTaskFound());
+    verify(hbHandler, never()).progressing(eq(attemptId));
+
+    // make sure a status update does report progress
+    MapTaskStatus mockStatus = new MapTaskStatus(attemptID, 0.0f, 1,
+        TaskStatus.State.RUNNING, "", "RUNNING", "", TaskStatus.Phase.MAP,
+        new Counters());
+    feedback = listener.statusUpdate(attemptID, mockStatus);
+    assertTrue(feedback.getTaskFound());
+    verify(hbHandler).progressing(eq(attemptId));
+    listener.close();
+  }
 }
 }

+ 13 - 5
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/TypeConverter.java

@@ -43,6 +43,7 @@ import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueACL;
@@ -445,11 +446,18 @@ public class TypeConverter {
     jobStatus.setStartTime(application.getStartTime());
     jobStatus.setStartTime(application.getStartTime());
     jobStatus.setFinishTime(application.getFinishTime());
     jobStatus.setFinishTime(application.getFinishTime());
     jobStatus.setFailureInfo(application.getDiagnostics());
     jobStatus.setFailureInfo(application.getDiagnostics());
-    jobStatus.setNeededMem(application.getApplicationResourceUsageReport().getNeededResources().getMemory());
-    jobStatus.setNumReservedSlots(application.getApplicationResourceUsageReport().getNumReservedContainers());
-    jobStatus.setNumUsedSlots(application.getApplicationResourceUsageReport().getNumUsedContainers());
-    jobStatus.setReservedMem(application.getApplicationResourceUsageReport().getReservedResources().getMemory());
-    jobStatus.setUsedMem(application.getApplicationResourceUsageReport().getUsedResources().getMemory());
+    ApplicationResourceUsageReport resourceUsageReport =
+        application.getApplicationResourceUsageReport();
+    if (resourceUsageReport != null) {
+      jobStatus.setNeededMem(
+          resourceUsageReport.getNeededResources().getMemory());
+      jobStatus.setNumReservedSlots(
+          resourceUsageReport.getNumReservedContainers());
+      jobStatus.setNumUsedSlots(resourceUsageReport.getNumUsedContainers());
+      jobStatus.setReservedMem(
+          resourceUsageReport.getReservedResources().getMemory());
+      jobStatus.setUsedMem(resourceUsageReport.getUsedResources().getMemory());
+    }
     return jobStatus;
     return jobStatus;
   }
   }
 
 

+ 7 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/jobhistory/JHAdminConfig.java

@@ -77,6 +77,13 @@ public class JHAdminConfig {
   public static final String MR_HISTORY_DONE_DIR =
   public static final String MR_HISTORY_DONE_DIR =
     MR_HISTORY_PREFIX + "done-dir";
     MR_HISTORY_PREFIX + "done-dir";
 
 
+  /**
+   * Maximum time the History server will wait for the FileSystem for History
+   * files to become available. Default value is -1, forever.
+   */
+  public static final String MR_HISTORY_MAX_START_WAIT_TIME =
+      MR_HISTORY_PREFIX + "maximum-start-wait-time-millis";
+  public static final long DEFAULT_MR_HISTORY_MAX_START_WAIT_TIME = -1;
   /**
   /**
    *  Path where history files should be stored after a job finished and before
    *  Path where history files should be stored after a job finished and before
    *  they are pulled into the job history server.
    *  they are pulled into the job history server.

+ 11 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRApps.java

@@ -61,6 +61,7 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.util.ApplicationClassLoader;
 import org.apache.hadoop.yarn.util.ApplicationClassLoader;
 import org.apache.hadoop.yarn.util.Apps;
 import org.apache.hadoop.yarn.util.Apps;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.apache.log4j.RollingFileAppender;
 
 
 /**
 /**
  * Helper class for MR applications
  * Helper class for MR applications
@@ -476,16 +477,24 @@ public class MRApps extends Apps {
    * Add the JVM system properties necessary to configure {@link ContainerLogAppender}.
    * Add the JVM system properties necessary to configure {@link ContainerLogAppender}.
    * @param logLevel the desired log level (eg INFO/WARN/DEBUG)
    * @param logLevel the desired log level (eg INFO/WARN/DEBUG)
    * @param logSize See {@link ContainerLogAppender#setTotalLogFileSize(long)}
    * @param logSize See {@link ContainerLogAppender#setTotalLogFileSize(long)}
+   * @param numBackups See {@link RollingFileAppender#setMaxBackupIndex(int)}
    * @param vargs the argument list to append to
    * @param vargs the argument list to append to
    */
    */
   public static void addLog4jSystemProperties(
   public static void addLog4jSystemProperties(
-      String logLevel, long logSize, List<String> vargs) {
+      String logLevel, long logSize, int numBackups, List<String> vargs) {
     vargs.add("-Dlog4j.configuration=container-log4j.properties");
     vargs.add("-Dlog4j.configuration=container-log4j.properties");
     vargs.add("-D" + YarnConfiguration.YARN_APP_CONTAINER_LOG_DIR + "=" +
     vargs.add("-D" + YarnConfiguration.YARN_APP_CONTAINER_LOG_DIR + "=" +
         ApplicationConstants.LOG_DIR_EXPANSION_VAR);
         ApplicationConstants.LOG_DIR_EXPANSION_VAR);
     vargs.add(
     vargs.add(
         "-D" + YarnConfiguration.YARN_APP_CONTAINER_LOG_SIZE + "=" + logSize);
         "-D" + YarnConfiguration.YARN_APP_CONTAINER_LOG_SIZE + "=" + logSize);
-    vargs.add("-Dhadoop.root.logger=" + logLevel + ",CLA"); 
+    if (logSize > 0L && numBackups > 0) {
+      // log should be rolled
+      vargs.add("-D" + YarnConfiguration.YARN_APP_CONTAINER_LOG_BACKUPS + "="
+          + numBackups);
+      vargs.add("-Dhadoop.root.logger=" + logLevel + ",CRLA");
+    } else {
+      vargs.add("-Dhadoop.root.logger=" + logLevel + ",CLA");
+    }
   }
   }
 
 
   /**
   /**

+ 9 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/TestTypeConverter.java

@@ -23,8 +23,6 @@ import static org.mockito.Mockito.when;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.List;
 
 
-import junit.framework.Assert;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.JobStatus.State;
 import org.apache.hadoop.mapreduce.JobStatus.State;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
@@ -40,6 +38,7 @@ import org.apache.hadoop.yarn.api.records.QueueState;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.util.Records;
 import org.apache.hadoop.yarn.util.Records;
+import org.junit.Assert;
 import org.junit.Test;
 import org.junit.Test;
 import org.mockito.Mockito;
 import org.mockito.Mockito;
 
 
@@ -112,6 +111,14 @@ public class TestTypeConverter {
     when(mockReport.getUser()).thenReturn("dummy-user");
     when(mockReport.getUser()).thenReturn("dummy-user");
     when(mockReport.getQueue()).thenReturn("dummy-queue");
     when(mockReport.getQueue()).thenReturn("dummy-queue");
     String jobFile = "dummy-path/job.xml";
     String jobFile = "dummy-path/job.xml";
+
+    try {
+      JobStatus status = TypeConverter.fromYarn(mockReport, jobFile);
+    } catch (NullPointerException npe) {
+      Assert.fail("Type converstion from YARN fails for jobs without " +
+          "ApplicationUsageReport");
+    }
+
     ApplicationResourceUsageReport appUsageRpt = Records
     ApplicationResourceUsageReport appUsageRpt = Records
         .newRecord(ApplicationResourceUsageReport.class);
         .newRecord(ApplicationResourceUsageReport.class);
     Resource r = Records.newRecord(Resource.class);
     Resource r = Records.newRecord(Resource.class);

+ 9 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java

@@ -412,6 +412,10 @@ public interface MRJobConfig {
     MR_AM_PREFIX+"log.level";
     MR_AM_PREFIX+"log.level";
   public static final String DEFAULT_MR_AM_LOG_LEVEL = "INFO";
   public static final String DEFAULT_MR_AM_LOG_LEVEL = "INFO";
 
 
+  public static final String MR_AM_LOG_BACKUPS =
+      MR_AM_PREFIX + "container.log.backups";
+  public static final int DEFAULT_MR_AM_LOG_BACKUPS = 0; // don't roll
+
   /**The number of splits when reporting progress in MR*/
   /**The number of splits when reporting progress in MR*/
   public static final String MR_AM_NUM_PROGRESS_SPLITS = 
   public static final String MR_AM_NUM_PROGRESS_SPLITS = 
     MR_AM_PREFIX+"num-progress-splits";
     MR_AM_PREFIX+"num-progress-splits";
@@ -692,7 +696,11 @@ public interface MRJobConfig {
       + "$HADOOP_MAPRED_HOME/share/hadoop/mapreduce/lib/*";
       + "$HADOOP_MAPRED_HOME/share/hadoop/mapreduce/lib/*";
 
 
   public static final String WORKFLOW_ID = "mapreduce.workflow.id";
   public static final String WORKFLOW_ID = "mapreduce.workflow.id";
-  
+
+  public static final String TASK_LOG_BACKUPS =
+      MR_PREFIX + "task.container.log.backups";
+  public static final int DEFAULT_TASK_LOG_BACKUPS = 0; // don't roll
+
   public static final String WORKFLOW_NAME = "mapreduce.workflow.name";
   public static final String WORKFLOW_NAME = "mapreduce.workflow.name";
   
   
   public static final String WORKFLOW_NODE_NAME =
   public static final String WORKFLOW_NODE_NAME =

+ 50 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml

@@ -510,6 +510,31 @@
   </description>
   </description>
 </property>
 </property>
 
 
+<property>
+  <name>yarn.app.mapreduce.task.container.log.backups</name>
+  <value>0</value>
+  <description>Number of backup files for task logs when using
+    ContainerRollingLogAppender (CRLA). See
+    org.apache.log4j.RollingFileAppender.maxBackupIndex. By default,
+    ContainerLogAppender (CLA) is used, and container logs are not rolled. CRLA
+    is enabled for tasks when both mapreduce.task.userlog.limit.kb and
+    yarn.app.mapreduce.task.container.log.backups are greater than zero.
+  </description>
+</property>
+
+<property>
+  <name>yarn.app.mapreduce.am.container.log.backups</name>
+  <value>0</value>
+  <description>Number of backup files for the ApplicationMaster logs when using
+    ContainerRollingLogAppender (CRLA). See
+    org.apache.log4j.RollingFileAppender.maxBackupIndex. By default,
+    ContainerLogAppender (CLA) is used, and container logs are not rolled. CRLA
+    is enabled for the ApplicationMaster when both
+    mapreduce.task.userlog.limit.kb and
+    yarn.app.mapreduce.am.container.log.backups are greater than zero.
+  </description>
+</property>
+
 <property>
 <property>
   <name>mapreduce.job.maxtaskfailures.per.tracker</name>
   <name>mapreduce.job.maxtaskfailures.per.tracker</name>
   <value>3</value>
   <value>3</value>
@@ -577,6 +602,31 @@
     </description>
     </description>
   </property>
   </property>
 
 
+  <property>
+    <name>mapreduce.task.profile.params</name>
+    <value></value>
+    <description>JVM profiler parameters used to profile map and reduce task
+      attempts. This string may contain a single format specifier %s that will
+      be replaced by the path to profile.out in the task attempt log directory.
+      To specify different profiling options for map tasks and reduce tasks,
+      more specific parameters mapreduce.task.profile.map.params and
+      mapreduce.task.profile.reduce.params should be used.</description>
+  </property>
+
+  <property>
+    <name>mapreduce.task.profile.map.params</name>
+    <value>${mapreduce.task.profile.params}</value>
+    <description>Map-task-specific JVM profiler parameters. See
+      mapreduce.task.profile.params</description>
+  </property>
+
+  <property>
+    <name>mapreduce.task.profile.reduce.params</name>
+    <value>${mapreduce.task.profile.params}</value>
+    <description>Reduce-task-specific JVM profiler parameters. See
+      mapreduce.task.profile.params</description>
+  </property>
+
   <property>
   <property>
     <name>mapreduce.task.skip.start.attempts</name>
     <name>mapreduce.task.skip.start.attempts</name>
     <value>2</value>
     <value>2</value>

+ 10 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/pom.xml

@@ -33,6 +33,10 @@
   </properties>
   </properties>
 
 
   <dependencies>
   <dependencies>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-hdfs</artifactId>
+    </dependency>
     <dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-mapreduce-client-common</artifactId>
       <artifactId>hadoop-mapreduce-client-common</artifactId>
@@ -53,6 +57,12 @@
       <type>test-jar</type>
       <type>test-jar</type>
       <scope>test</scope>
       <scope>test</scope>
     </dependency>
     </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-hdfs</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
   </dependencies>
 
 
   <build>
   <build>

+ 110 - 31
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryFileManager.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.mapreduce.v2.hs;
 
 
 import java.io.FileNotFoundException;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.IOException;
+import java.net.ConnectException;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Collections;
@@ -69,6 +70,8 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hadoop.yarn.util.Clock;
+import org.apache.hadoop.yarn.util.SystemClock;
 
 
 /**
 /**
  * This class provides a way to interact with history files in a thread safe
  * This class provides a way to interact with history files in a thread safe
@@ -464,7 +467,8 @@ public class HistoryFileManager extends AbstractService {
 
 
   private JobACLsManager aclsMgr;
   private JobACLsManager aclsMgr;
 
 
-  private Configuration conf;
+  @VisibleForTesting
+  Configuration conf;
 
 
   private String serialNumberFormat;
   private String serialNumberFormat;
 
 
@@ -491,36 +495,10 @@ public class HistoryFileManager extends AbstractService {
         + (JobHistoryUtils.SERIAL_NUMBER_DIRECTORY_DIGITS + serialNumberLowDigits)
         + (JobHistoryUtils.SERIAL_NUMBER_DIRECTORY_DIGITS + serialNumberLowDigits)
         + "d");
         + "d");
 
 
-    String doneDirPrefix = null;
-    doneDirPrefix = JobHistoryUtils
-        .getConfiguredHistoryServerDoneDirPrefix(conf);
-    try {
-      doneDirPrefixPath = FileContext.getFileContext(conf).makeQualified(
-          new Path(doneDirPrefix));
-      doneDirFc = FileContext.getFileContext(doneDirPrefixPath.toUri(), conf);
-      doneDirFc.setUMask(JobHistoryUtils.HISTORY_DONE_DIR_UMASK);
-      mkdir(doneDirFc, doneDirPrefixPath, new FsPermission(
-          JobHistoryUtils.HISTORY_DONE_DIR_PERMISSION));
-    } catch (IOException e) {
-      throw new YarnRuntimeException("Error creating done directory: ["
-          + doneDirPrefixPath + "]", e);
-    }
-
-    String intermediateDoneDirPrefix = null;
-    intermediateDoneDirPrefix = JobHistoryUtils
-        .getConfiguredHistoryIntermediateDoneDirPrefix(conf);
-    try {
-      intermediateDoneDirPath = FileContext.getFileContext(conf).makeQualified(
-          new Path(intermediateDoneDirPrefix));
-      intermediateDoneDirFc = FileContext.getFileContext(
-          intermediateDoneDirPath.toUri(), conf);
-      mkdir(intermediateDoneDirFc, intermediateDoneDirPath, new FsPermission(
-          JobHistoryUtils.HISTORY_INTERMEDIATE_DONE_DIR_PERMISSIONS.toShort()));
-    } catch (IOException e) {
-      LOG.info("error creating done directory on dfs " + e);
-      throw new YarnRuntimeException("Error creating intermediate done directory: ["
-          + intermediateDoneDirPath + "]", e);
-    }
+    long maxFSWaitTime = conf.getLong(
+        JHAdminConfig.MR_HISTORY_MAX_START_WAIT_TIME,
+        JHAdminConfig.DEFAULT_MR_HISTORY_MAX_START_WAIT_TIME);
+    createHistoryDirs(new SystemClock(), 10 * 1000, maxFSWaitTime);
 
 
     this.aclsMgr = new JobACLsManager(conf);
     this.aclsMgr = new JobACLsManager(conf);
 
 
@@ -544,6 +522,107 @@ public class HistoryFileManager extends AbstractService {
     super.serviceInit(conf);
     super.serviceInit(conf);
   }
   }
 
 
+  @VisibleForTesting
+  void createHistoryDirs(Clock clock, long intervalCheckMillis,
+      long timeOutMillis) throws IOException {
+    long start = clock.getTime();
+    boolean done = false;
+    int counter = 0;
+    while (!done &&
+        ((timeOutMillis == -1) || (clock.getTime() - start < timeOutMillis))) {
+      done = tryCreatingHistoryDirs(counter++ % 3 == 0); // log every 3 attempts, 30sec
+      try {
+        Thread.sleep(intervalCheckMillis);
+      } catch (InterruptedException ex) {
+        throw new YarnRuntimeException(ex);
+      }
+    }
+    if (!done) {
+      throw new YarnRuntimeException("Timed out '" + timeOutMillis+
+              "ms' waiting for FileSystem to become available");
+    }
+  }
+
+  /**
+   * DistributedFileSystem returns a RemoteException with a message stating
+   * SafeModeException in it. So this is only way to check it is because of
+   * being in safe mode.
+   */
+  private boolean isBecauseSafeMode(Throwable ex) {
+    return ex.toString().contains("SafeModeException");
+  }
+
+  /**
+   * Returns TRUE if the history dirs were created, FALSE if they could not
+   * be created because the FileSystem is not reachable or in safe mode and
+   * throws and exception otherwise.
+   */
+  @VisibleForTesting
+  boolean tryCreatingHistoryDirs(boolean logWait) throws IOException {
+    boolean succeeded = true;
+    String doneDirPrefix = JobHistoryUtils.
+        getConfiguredHistoryServerDoneDirPrefix(conf);
+    try {
+      doneDirPrefixPath = FileContext.getFileContext(conf).makeQualified(
+          new Path(doneDirPrefix));
+      doneDirFc = FileContext.getFileContext(doneDirPrefixPath.toUri(), conf);
+      doneDirFc.setUMask(JobHistoryUtils.HISTORY_DONE_DIR_UMASK);
+      mkdir(doneDirFc, doneDirPrefixPath, new FsPermission(
+          JobHistoryUtils.HISTORY_DONE_DIR_PERMISSION));
+    } catch (ConnectException ex) {
+      if (logWait) {
+        LOG.info("Waiting for FileSystem at " +
+            doneDirPrefixPath.toUri().getAuthority()  + "to be available");
+      }
+      succeeded = false;
+    } catch (IOException e) {
+      if (isBecauseSafeMode(e)) {
+        succeeded = false;
+        if (logWait) {
+          LOG.info("Waiting for FileSystem at " +
+              doneDirPrefixPath.toUri().getAuthority() +
+              "to be out of safe mode");
+        }
+      } else {
+        throw new YarnRuntimeException("Error creating done directory: ["
+            + doneDirPrefixPath + "]", e);
+      }
+    }
+    if (succeeded) {
+      String intermediateDoneDirPrefix = JobHistoryUtils.
+          getConfiguredHistoryIntermediateDoneDirPrefix(conf);
+      try {
+        intermediateDoneDirPath = FileContext.getFileContext(conf).makeQualified(
+            new Path(intermediateDoneDirPrefix));
+        intermediateDoneDirFc = FileContext.getFileContext(
+            intermediateDoneDirPath.toUri(), conf);
+        mkdir(intermediateDoneDirFc, intermediateDoneDirPath, new FsPermission(
+            JobHistoryUtils.HISTORY_INTERMEDIATE_DONE_DIR_PERMISSIONS.toShort()));
+      } catch (ConnectException ex) {
+        succeeded = false;
+        if (logWait) {
+          LOG.info("Waiting for FileSystem at " +
+              intermediateDoneDirPath.toUri().getAuthority() +
+              "to be available");
+        }
+      } catch (IOException e) {
+        if (isBecauseSafeMode(e)) {
+          succeeded = false;
+          if (logWait) {
+            LOG.info("Waiting for FileSystem at " +
+                intermediateDoneDirPath.toUri().getAuthority() +
+                "to be out of safe mode");
+          }
+        } else {
+          throw new YarnRuntimeException(
+              "Error creating intermediate done directory: ["
+              + intermediateDoneDirPath + "]", e);
+        }
+      }
+    }
+    return succeeded;
+  }
+
   @Override
   @Override
   public void serviceStop() throws Exception {
   public void serviceStop() throws Exception {
     ShutdownThreadsHelper.shutdownExecutorService(moveToDoneExecutor);
     ShutdownThreadsHelper.shutdownExecutorService(moveToDoneExecutor);

+ 139 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestHistoryFileManager.java

@@ -0,0 +1,139 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.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.mapreduce.v2.hs;
+
+
+import junit.framework.Assert;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.mapreduce.v2.app.ControlledClock;
+import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.hadoop.yarn.util.Clock;
+import org.apache.hadoop.yarn.util.SystemClock;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.UUID;
+
+public class TestHistoryFileManager {
+  private static MiniDFSCluster dfsCluster = null;
+
+  @BeforeClass
+  public static void setUpClass() throws Exception {
+    Configuration conf = new HdfsConfiguration();
+    dfsCluster = new MiniDFSCluster.Builder(conf).build();
+  }
+
+  @AfterClass
+  public static void cleanUpClass() throws Exception {
+    dfsCluster.shutdown();
+  }
+
+  private void testTryCreateHistoryDirs(Configuration conf, boolean expected)
+      throws Exception {
+    conf.set(JHAdminConfig.MR_HISTORY_DONE_DIR, "/" + UUID.randomUUID());
+    conf.set(JHAdminConfig.MR_HISTORY_INTERMEDIATE_DONE_DIR, "/" + UUID.randomUUID());
+    HistoryFileManager hfm = new HistoryFileManager();
+    hfm.conf = conf;
+    Assert.assertEquals(expected, hfm.tryCreatingHistoryDirs(false));
+  }
+
+  @Test
+  public void testCreateDirsWithoutFileSystem() throws Exception {
+    Configuration conf = new YarnConfiguration();
+    conf.set(FileSystem.FS_DEFAULT_NAME_KEY, "hdfs://localhost:1");
+    testTryCreateHistoryDirs(conf, false);
+  }
+
+  @Test
+  public void testCreateDirsWithFileSystem() throws Exception {
+    dfsCluster.getFileSystem().setSafeMode(
+        HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
+    Assert.assertFalse(dfsCluster.getFileSystem().isInSafeMode());
+    testTryCreateHistoryDirs(dfsCluster.getConfiguration(0), true);
+  }
+
+  @Test
+  public void testCreateDirsWithFileSystemInSafeMode() throws Exception {
+    dfsCluster.getFileSystem().setSafeMode(
+        HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
+    Assert.assertTrue(dfsCluster.getFileSystem().isInSafeMode());
+    testTryCreateHistoryDirs(dfsCluster.getConfiguration(0), false);
+  }
+
+  private void testCreateHistoryDirs(Configuration conf, Clock clock)
+      throws Exception {
+    conf.set(JHAdminConfig.MR_HISTORY_DONE_DIR, "/" + UUID.randomUUID());
+    conf.set(JHAdminConfig.MR_HISTORY_INTERMEDIATE_DONE_DIR, "/" + UUID.randomUUID());
+    HistoryFileManager hfm = new HistoryFileManager();
+    hfm.conf = conf;
+    hfm.createHistoryDirs(clock, 500, 2000);
+  }
+
+  @Test
+  public void testCreateDirsWithFileSystemBecomingAvailBeforeTimeout()
+      throws Exception {
+    dfsCluster.getFileSystem().setSafeMode(
+        HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
+    Assert.assertTrue(dfsCluster.getFileSystem().isInSafeMode());
+    new Thread() {
+      @Override
+      public void run() {
+        try {
+          Thread.sleep(500);
+          dfsCluster.getFileSystem().setSafeMode(
+              HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
+          Assert.assertTrue(dfsCluster.getFileSystem().isInSafeMode());
+        } catch (Exception ex) {
+          Assert.fail(ex.toString());
+        }
+      }
+    }.start();
+    testCreateHistoryDirs(dfsCluster.getConfiguration(0), new SystemClock());
+  }
+
+  @Test(expected = YarnRuntimeException.class)
+  public void testCreateDirsWithFileSystemNotBecomingAvailBeforeTimeout()
+      throws Exception {
+    dfsCluster.getFileSystem().setSafeMode(
+        HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
+    Assert.assertTrue(dfsCluster.getFileSystem().isInSafeMode());
+    final ControlledClock clock = new ControlledClock(new SystemClock());
+    clock.setTime(1);
+    new Thread() {
+      @Override
+      public void run() {
+        try {
+          Thread.sleep(500);
+          clock.setTime(3000);
+        } catch (Exception ex) {
+          Assert.fail(ex.toString());
+        }
+      }
+    }.start();
+    testCreateHistoryDirs(dfsCluster.getConfiguration(0), clock);
+  }
+
+}

+ 3 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java

@@ -392,7 +392,9 @@ public class YARNRunner implements ClientProtocol {
     long logSize = TaskLog.getTaskLogLength(new JobConf(conf));
     long logSize = TaskLog.getTaskLogLength(new JobConf(conf));
     String logLevel = jobConf.get(
     String logLevel = jobConf.get(
         MRJobConfig.MR_AM_LOG_LEVEL, MRJobConfig.DEFAULT_MR_AM_LOG_LEVEL);
         MRJobConfig.MR_AM_LOG_LEVEL, MRJobConfig.DEFAULT_MR_AM_LOG_LEVEL);
-    MRApps.addLog4jSystemProperties(logLevel, logSize, vargs);
+    int numBackups = jobConf.getInt(MRJobConfig.MR_AM_LOG_BACKUPS,
+        MRJobConfig.DEFAULT_MR_AM_LOG_BACKUPS);
+    MRApps.addLog4jSystemProperties(logLevel, logSize, numBackups, vargs);
 
 
     // Check for Java Lib Path usage in MAP and REDUCE configs
     // Check for Java Lib Path usage in MAP and REDUCE configs
     warnForJavaLibPath(conf.get(MRJobConfig.MAP_JAVA_OPTS,""), "map", 
     warnForJavaLibPath(conf.get(MRJobConfig.MAP_JAVA_OPTS,""), "map", 

+ 9 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientRedirect.java

@@ -90,6 +90,8 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.MoveApplicationAcrossQueuesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.MoveApplicationAcrossQueuesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
@@ -99,6 +101,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
@@ -354,6 +357,12 @@ public class TestClientRedirect {
         CancelDelegationTokenRequest request) throws IOException {
         CancelDelegationTokenRequest request) throws IOException {
       return null;
       return null;
     }
     }
+
+    @Override
+    public MoveApplicationAcrossQueuesResponse moveApplicationAcrossQueues(
+        MoveApplicationAcrossQueuesRequest request) throws YarnException, IOException {
+      return null;
+    }
   }
   }
 
 
   class HistoryService extends AMService implements HSClientProtocol {
   class HistoryService extends AMService implements HSClientProtocol {

+ 14 - 6
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestNetworkedJob.java

@@ -45,7 +45,9 @@ import org.apache.hadoop.mapred.lib.IdentityMapper;
 import org.apache.hadoop.mapred.lib.IdentityReducer;
 import org.apache.hadoop.mapred.lib.IdentityReducer;
 import org.apache.hadoop.mapreduce.Cluster.JobTrackerStatus;
 import org.apache.hadoop.mapreduce.Cluster.JobTrackerStatus;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
 import org.junit.Test;
 import org.junit.Test;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
@@ -76,8 +78,7 @@ public class TestNetworkedJob {
     FileSystem fileSys = null;
     FileSystem fileSys = null;
 
 
     try {
     try {
-      mr = MiniMRClientClusterFactory.create(this.getClass(), 2,
-          new Configuration());
+      mr = createMiniClusterWithCapacityScheduler();
 
 
       JobConf job = new JobConf(mr.getConfig());
       JobConf job = new JobConf(mr.getConfig());
 
 
@@ -129,8 +130,7 @@ public class TestNetworkedJob {
     FileSystem fileSys = null;
     FileSystem fileSys = null;
 
 
     try {
     try {
-      Configuration conf = new Configuration();
-      mr = MiniMRClientClusterFactory.create(this.getClass(), 2, conf);
+      mr = createMiniClusterWithCapacityScheduler();
 
 
       JobConf job = new JobConf(mr.getConfig());
       JobConf job = new JobConf(mr.getConfig());
 
 
@@ -315,8 +315,7 @@ public class TestNetworkedJob {
     FileSystem fileSys = null;
     FileSystem fileSys = null;
     PrintStream oldOut = System.out;
     PrintStream oldOut = System.out;
     try {
     try {
-      Configuration conf = new Configuration();
-      mr = MiniMRClientClusterFactory.create(this.getClass(), 2, conf);
+      mr = createMiniClusterWithCapacityScheduler();
 
 
       JobConf job = new JobConf(mr.getConfig());
       JobConf job = new JobConf(mr.getConfig());
 
 
@@ -392,4 +391,13 @@ public class TestNetworkedJob {
       }
       }
     }
     }
   }
   }
+  
+  private MiniMRClientCluster createMiniClusterWithCapacityScheduler()
+      throws IOException {
+    Configuration conf = new Configuration();
+    // Expected queue names depending on Capacity Scheduler queue naming
+    conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
+        CapacityScheduler.class);
+    return MiniMRClientClusterFactory.create(this.getClass(), 2, conf);
+  }
 }
 }

+ 123 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobs.java

@@ -23,10 +23,12 @@ import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileInputStream;
 import java.io.FileNotFoundException;
 import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
 import java.io.FileOutputStream;
+import java.io.InputStreamReader;
 import java.io.IOException;
 import java.io.IOException;
 import java.io.StringReader;
 import java.io.StringReader;
 import java.net.URI;
 import java.net.URI;
 import java.security.PrivilegedExceptionAction;
 import java.security.PrivilegedExceptionAction;
+import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Map;
 import java.util.jar.JarOutputStream;
 import java.util.jar.JarOutputStream;
@@ -53,6 +55,8 @@ import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.TaskLog;
 import org.apache.hadoop.mapreduce.Counters;
 import org.apache.hadoop.mapreduce.Counters;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.JobCounter;
 import org.apache.hadoop.mapreduce.JobCounter;
@@ -65,17 +69,22 @@ import org.apache.hadoop.mapreduce.TaskCompletionEvent;
 import org.apache.hadoop.mapreduce.TaskID;
 import org.apache.hadoop.mapreduce.TaskID;
 import org.apache.hadoop.mapreduce.TaskReport;
 import org.apache.hadoop.mapreduce.TaskReport;
 import org.apache.hadoop.mapreduce.TaskType;
 import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.mapreduce.TypeConverter;
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
 import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
 import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
 import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
+import org.apache.hadoop.mapreduce.v2.api.records.JobId;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.util.JarFinder;
 import org.apache.hadoop.util.JarFinder;
 import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.util.Shell;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
+import org.apache.log4j.Level;
 import org.junit.AfterClass;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.BeforeClass;
@@ -84,6 +93,9 @@ import org.junit.Test;
 public class TestMRJobs {
 public class TestMRJobs {
 
 
   private static final Log LOG = LogFactory.getLog(TestMRJobs.class);
   private static final Log LOG = LogFactory.getLog(TestMRJobs.class);
+  private static final EnumSet<RMAppState> TERMINAL_RM_APP_STATES =
+      EnumSet.of(RMAppState.FINISHED, RMAppState.FAILED, RMAppState.KILLED);
+  private static final int NUM_NODE_MGRS = 3;
 
 
   protected static MiniMRYarnCluster mrCluster;
   protected static MiniMRYarnCluster mrCluster;
   protected static MiniDFSCluster dfsCluster;
   protected static MiniDFSCluster dfsCluster;
@@ -122,7 +134,8 @@ public class TestMRJobs {
     }
     }
 
 
     if (mrCluster == null) {
     if (mrCluster == null) {
-      mrCluster = new MiniMRYarnCluster(TestMRJobs.class.getName(), 3);
+      mrCluster = new MiniMRYarnCluster(TestMRJobs.class.getName(),
+          NUM_NODE_MGRS);
       Configuration conf = new Configuration();
       Configuration conf = new Configuration();
       conf.set("fs.defaultFS", remoteFs.getUri().toString());   // use HDFS
       conf.set("fs.defaultFS", remoteFs.getUri().toString());   // use HDFS
       conf.set(MRJobConfig.MR_AM_STAGING_DIR, "/apps_staging_dir");
       conf.set(MRJobConfig.MR_AM_STAGING_DIR, "/apps_staging_dir");
@@ -416,6 +429,115 @@ public class TestMRJobs {
     // TODO later:  add explicit "isUber()" checks of some sort
     // TODO later:  add explicit "isUber()" checks of some sort
   }
   }
 
 
+  @Test(timeout = 120000)
+  public void testContainerRollingLog() throws IOException,
+      InterruptedException, ClassNotFoundException {
+    if (!(new File(MiniMRYarnCluster.APPJAR)).exists()) {
+      LOG.info("MRAppJar " + MiniMRYarnCluster.APPJAR
+          + " not found. Not running test.");
+      return;
+    }
+
+    final SleepJob sleepJob = new SleepJob();
+    final JobConf sleepConf = new JobConf(mrCluster.getConfig());
+    sleepConf.set(MRJobConfig.MAP_LOG_LEVEL, Level.ALL.toString());
+    sleepConf.set(MRJobConfig.MR_AM_LOG_LEVEL, Level.ALL.toString());
+    sleepConf.setLong(MRJobConfig.TASK_USERLOG_LIMIT, 1);
+    sleepConf.setInt(MRJobConfig.TASK_LOG_BACKUPS, 3);
+    sleepConf.setInt(MRJobConfig.MR_AM_LOG_BACKUPS, 7);
+    sleepJob.setConf(sleepConf);
+
+    final Job job = sleepJob.createJob(1, 0, 1L, 100, 0L, 0);
+    job.setJarByClass(SleepJob.class);
+    job.addFileToClassPath(APP_JAR); // The AppMaster jar itself.
+    job.waitForCompletion(true);
+    final JobId jobId = TypeConverter.toYarn(job.getJobID());
+    final ApplicationId appID = jobId.getAppId();
+    int pollElapsed = 0;
+    while (true) {
+      Thread.sleep(1000);
+      pollElapsed += 1000;
+      if (TERMINAL_RM_APP_STATES.contains(
+        mrCluster.getResourceManager().getRMContext().getRMApps().get(appID)
+            .getState())) {
+        break;
+      }
+      if (pollElapsed >= 60000) {
+        LOG.warn("application did not reach terminal state within 60 seconds");
+        break;
+      }
+    }
+    Assert.assertEquals(RMAppState.FINISHED, mrCluster.getResourceManager()
+        .getRMContext().getRMApps().get(appID).getState());
+
+    // Job finished, verify logs
+    //
+
+    final String appIdStr = appID.toString();
+    final String appIdSuffix = appIdStr.substring("application_".length(),
+        appIdStr.length());
+    final String containerGlob = "container_" + appIdSuffix + "_*_*";
+    final String syslogGlob = appIdStr
+        + Path.SEPARATOR + containerGlob
+        + Path.SEPARATOR + TaskLog.LogName.SYSLOG;
+    int numAppMasters = 0;
+    int numMapTasks = 0;
+
+    for (int i = 0; i < NUM_NODE_MGRS; i++) {
+      final Configuration nmConf = mrCluster.getNodeManager(i).getConfig();
+      for (String logDir :
+               nmConf.getTrimmedStrings(YarnConfiguration.NM_LOG_DIRS)) {
+        final Path absSyslogGlob =
+            new Path(logDir + Path.SEPARATOR + syslogGlob);
+        LOG.info("Checking for glob: " + absSyslogGlob);
+        final FileStatus[] syslogs = localFs.globStatus(absSyslogGlob);
+        for (FileStatus slog : syslogs) {
+          // check all syslogs for the container
+          //
+          final FileStatus[] sysSiblings = localFs.globStatus(new Path(
+              slog.getPath().getParent(), TaskLog.LogName.SYSLOG + "*"));
+          boolean foundAppMaster = false;
+          floop:
+          for (FileStatus f : sysSiblings) {
+            final BufferedReader reader = new BufferedReader(
+                new InputStreamReader(localFs.open(f.getPath())));
+            String line;
+            try {
+              while ((line = reader.readLine()) != null) {
+                if (line.contains(MRJobConfig.APPLICATION_MASTER_CLASS)) {
+                  foundAppMaster = true;
+                  break floop;
+                }
+              }
+            } finally {
+              reader.close();
+            }
+          }
+
+          if (foundAppMaster) {
+            numAppMasters++;
+          } else {
+            numMapTasks++;
+          }
+
+          Assert.assertSame("Number of sylog* files",
+              foundAppMaster
+                ? sleepConf.getInt(MRJobConfig.MR_AM_LOG_BACKUPS, 0) + 1
+                : sleepConf.getInt(MRJobConfig.TASK_LOG_BACKUPS, 0) + 1,
+              sysSiblings.length);
+        }
+      }
+    }
+    // Make sure we checked non-empty set
+    //
+    Assert.assertEquals("No AppMaster log found!", 1, numAppMasters);
+    if (sleepConf.getBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, false)) {
+      Assert.assertEquals("MapTask log with uber found!", 0, numMapTasks);
+    } else {
+      Assert.assertEquals("No MapTask log found!", 1, numMapTasks);
+    }
+  }
+
   public static class DistributedCacheChecker extends
   public static class DistributedCacheChecker extends
       Mapper<LongWritable, Text, NullWritable, NullWritable> {
       Mapper<LongWritable, Text, NullWritable, NullWritable> {
 
 

+ 244 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobsWithProfiler.java

@@ -0,0 +1,244 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.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.mapreduce.v2;
+
+import java.io.*;
+import java.util.*;
+
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import junit.framework.Assert;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.SleepJob;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.mapred.*;
+import org.apache.hadoop.mapreduce.*;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.v2.api.records.JobId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestMRJobsWithProfiler {
+
+  private static final Log LOG =
+    LogFactory.getLog(TestMRJobsWithProfiler.class);
+
+  private static final EnumSet<RMAppState> TERMINAL_RM_APP_STATES =
+    EnumSet.of(RMAppState.FINISHED, RMAppState.FAILED, RMAppState.KILLED);
+
+  private static MiniMRYarnCluster mrCluster;
+
+  private static final Configuration CONF = new Configuration();
+  private static final FileSystem localFs;
+  static {
+    try {
+      localFs = FileSystem.getLocal(CONF);
+    } catch (IOException io) {
+      throw new RuntimeException("problem getting local fs", io);
+    }
+  }
+
+  private static final Path TEST_ROOT_DIR =
+    new Path("target",  TestMRJobs.class.getName() + "-tmpDir").
+      makeQualified(localFs.getUri(), localFs.getWorkingDirectory());
+
+  private static final Path APP_JAR = new Path(TEST_ROOT_DIR, "MRAppJar.jar");
+
+  @Before
+  public void setup() throws InterruptedException, IOException {
+
+    if (!(new File(MiniMRYarnCluster.APPJAR)).exists()) {
+      LOG.info("MRAppJar " + MiniMRYarnCluster.APPJAR
+               + " not found. Not running test.");
+      return;
+    }
+
+    if (mrCluster == null) {
+      mrCluster = new MiniMRYarnCluster(getClass().getName());
+      mrCluster.init(CONF);
+      mrCluster.start();
+    }
+
+    // Copy MRAppJar and make it private. TODO: FIXME. This is a hack to
+    // workaround the absent public discache.
+    localFs.copyFromLocalFile(new Path(MiniMRYarnCluster.APPJAR), APP_JAR);
+    localFs.setPermission(APP_JAR, new FsPermission("700"));
+  }
+
+  @After
+  public void tearDown() {
+    if (!(new File(MiniMRYarnCluster.APPJAR)).exists()) {
+      LOG.info("MRAppJar " + MiniMRYarnCluster.APPJAR
+          + " not found. Not running test.");
+      return;
+    }
+
+    if (mrCluster != null) {
+      mrCluster.stop();
+    }
+  }
+
+
+  @Test (timeout = 120000)
+  public void testProfiler() throws IOException, InterruptedException,
+      ClassNotFoundException {
+    if (!(new File(MiniMRYarnCluster.APPJAR)).exists()) {
+      LOG.info("MRAppJar " + MiniMRYarnCluster.APPJAR
+        + " not found. Not running test.");
+      return;
+    }
+
+    final SleepJob sleepJob = new SleepJob();
+    final JobConf sleepConf = new JobConf(mrCluster.getConfig());
+
+    sleepConf.setProfileEnabled(true);
+    // profile map split 1
+    sleepConf.setProfileTaskRange(true, "1");
+    // profile reduce of map output partitions 1
+    sleepConf.setProfileTaskRange(false, "1");
+
+    // use hprof for map to profile.out
+    sleepConf.set(MRJobConfig.TASK_MAP_PROFILE_PARAMS,
+        "-agentlib:hprof=cpu=times,heap=sites,force=n,thread=y,verbose=n,"
+      + "file=%s");
+
+    // use Xprof for reduce to stdout
+    sleepConf.set(MRJobConfig.TASK_REDUCE_PROFILE_PARAMS, "-Xprof");
+    sleepJob.setConf(sleepConf);
+
+    // 2-map-2-reduce SleepJob
+    final Job job = sleepJob.createJob(2, 2, 500, 1, 500, 1);
+    job.setJarByClass(SleepJob.class);
+    job.addFileToClassPath(APP_JAR); // The AppMaster jar itself.
+    job.waitForCompletion(true);
+    final JobId jobId = TypeConverter.toYarn(job.getJobID());
+    final ApplicationId appID = jobId.getAppId();
+    int pollElapsed = 0;
+    while (true) {
+      Thread.sleep(1000);
+      pollElapsed += 1000;
+
+      if (TERMINAL_RM_APP_STATES.contains(
+        mrCluster.getResourceManager().getRMContext().getRMApps().get(appID)
+          .getState())) {
+        break;
+      }
+
+      if (pollElapsed >= 60000) {
+        LOG.warn("application did not reach terminal state within 60 seconds");
+        break;
+      }
+    }
+    Assert.assertEquals(RMAppState.FINISHED, mrCluster.getResourceManager()
+      .getRMContext().getRMApps().get(appID).getState());
+
+    // Job finished, verify logs
+    //
+    final Configuration nmConf = mrCluster.getNodeManager(0).getConfig();
+
+    final String appIdStr = appID.toString();
+    final String appIdSuffix = appIdStr.substring(
+      "application_".length(), appIdStr.length());
+    final String containerGlob = "container_" + appIdSuffix + "_*_*";
+
+    final Map<TaskAttemptID,Path> taLogDirs = new HashMap<TaskAttemptID,Path>();
+    final Pattern taskPattern = Pattern.compile(
+        ".*Task:(attempt_"
+      + appIdSuffix + "_[rm]_" + "[0-9]+_[0-9]+).*");
+    for (String logDir :
+         nmConf.getTrimmedStrings(YarnConfiguration.NM_LOG_DIRS))
+    {
+      // filter out MRAppMaster and create attemptId->logDir map
+      //
+      for (FileStatus fileStatus :
+          localFs.globStatus(new Path(logDir
+            + Path.SEPARATOR + appIdStr
+            + Path.SEPARATOR + containerGlob
+            + Path.SEPARATOR + TaskLog.LogName.SYSLOG)))
+      {
+        final BufferedReader br = new BufferedReader(
+          new InputStreamReader(localFs.open(fileStatus.getPath())));
+        String line;
+        while ((line = br.readLine()) != null) {
+          final Matcher m = taskPattern.matcher(line);
+          if (m.matches()) {
+            // found Task done message
+            taLogDirs.put(TaskAttemptID.forName(m.group(1)),
+              fileStatus.getPath().getParent());
+            break;
+          }
+        }
+        br.close();
+      }
+    }
+
+    Assert.assertEquals(4, taLogDirs.size());  // all 4 attempts found
+
+    for (Map.Entry<TaskAttemptID,Path> dirEntry : taLogDirs.entrySet()) {
+      final TaskAttemptID tid = dirEntry.getKey();
+      final Path profilePath = new Path(dirEntry.getValue(),
+        TaskLog.LogName.PROFILE.toString());
+      final Path stdoutPath = new Path(dirEntry.getValue(),
+        TaskLog.LogName.STDOUT.toString());
+      if (tid.getTaskType() == TaskType.MAP) {
+        if (tid.getTaskID().getId() == 1) {
+          // verify profile.out
+          final BufferedReader br = new BufferedReader(new InputStreamReader(
+            localFs.open(profilePath)));
+          final String line = br.readLine();
+          Assert.assertTrue("No hprof content found!",
+            line !=null && line.startsWith("JAVA PROFILE"));
+          br.close();
+          Assert.assertEquals(0L, localFs.getFileStatus(stdoutPath).getLen());
+        } else {
+          Assert.assertFalse("hprof file should not exist",
+            localFs.exists(profilePath));
+        }
+      } else {
+        Assert.assertFalse("hprof file should not exist",
+          localFs.exists(profilePath));
+        if (tid.getTaskID().getId() == 1) {
+          final BufferedReader br = new BufferedReader(new InputStreamReader(
+            localFs.open(stdoutPath)));
+          boolean flatProfFound = false;
+          String line;
+          while ((line = br.readLine()) != null) {
+            if (line.startsWith("Flat profile")) {
+              flatProfFound = true;
+              break;
+            }
+          }
+          br.close();
+          Assert.assertTrue("Xprof flat profile not found!", flatProfFound);
+        } else {
+          Assert.assertEquals(0L, localFs.getFileStatus(stdoutPath).getLen());
+        }
+      }
+    }
+  }
+}

+ 7 - 1
hadoop-project/pom.xml

@@ -422,7 +422,7 @@
         <version>${jersey.version}</version>
         <version>${jersey.version}</version>
         <exclusions>
         <exclusions>
           <exclusion>
           <exclusion>
-            <groupId>javax.xml.stream</groupId>
+            <groupId>stax</groupId>
             <artifactId>stax-api</artifactId>
             <artifactId>stax-api</artifactId>
           </exclusion>
           </exclusion>
         </exclusions>
         </exclusions>
@@ -770,6 +770,12 @@
         <groupId>org.codehaus.jettison</groupId>
         <groupId>org.codehaus.jettison</groupId>
         <artifactId>jettison</artifactId>
         <artifactId>jettison</artifactId>
         <version>1.1</version>
         <version>1.1</version>
+        <exclusions>
+          <exclusion>
+            <groupId>stax</groupId>
+            <artifactId>stax-api</artifactId>
+          </exclusion>
+        </exclusions>
       </dependency>
       </dependency>
       <dependency>
       <dependency>
         <groupId>com.sun.jersey</groupId>
         <groupId>com.sun.jersey</groupId>

+ 9 - 0
hadoop-yarn-project/CHANGES.txt

@@ -6,6 +6,9 @@ Trunk - Unreleased
 
 
   NEW FEATURES
   NEW FEATURES
 
 
+    YARN-1496. Protocol additions to allow moving apps between queues (Sandy
+    Ryza)
+
   IMPROVEMENTS
   IMPROVEMENTS
 
 
   OPTIMIZATIONS
   OPTIMIZATIONS
@@ -343,6 +346,12 @@ Release 2.4.0 - UNRELEASED
     YARN-1351. Invalid string format in Fair Scheduler log warn message
     YARN-1351. Invalid string format in Fair Scheduler log warn message
     (Konstantin Weitz via Sandy Ryza)
     (Konstantin Weitz via Sandy Ryza)
 
 
+    YARN-1608. LinuxContainerExecutor has a few DEBUG messages at INFO level
+    (kasha)
+
+    YARN-1606. Fix the default value of yarn.resourcemanager.zk-timeout-ms 
+    in yarn-default.xml (kasha)
+
 Release 2.3.0 - UNRELEASED
 Release 2.3.0 - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 15 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocol.java

@@ -44,6 +44,8 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.MoveApplicationAcrossQueuesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.MoveApplicationAcrossQueuesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
@@ -339,4 +341,17 @@ public interface ApplicationClientProtocol {
   public CancelDelegationTokenResponse cancelDelegationToken(
   public CancelDelegationTokenResponse cancelDelegationToken(
       CancelDelegationTokenRequest request) throws YarnException,
       CancelDelegationTokenRequest request) throws YarnException,
       IOException;
       IOException;
+  
+  /**
+   * Move an application to a new queue.
+   * 
+   * @param request the application ID and the target queue
+   * @return an empty response
+   * @throws YarnException
+   * @throws IOException
+   */
+  @Public
+  @Unstable
+  public MoveApplicationAcrossQueuesResponse moveApplicationAcrossQueues(
+      MoveApplicationAcrossQueuesRequest request) throws YarnException, IOException;
 }
 }

+ 69 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/MoveApplicationAcrossQueuesRequest.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.api.protocolrecords;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * <p>The request sent by the client to the <code>ResourceManager</code>
+ * to move a submitted application to a different queue.</p>
+ * 
+ * <p>The request includes the {@link ApplicationId} of the application to be
+ * moved and the queue to place it in.</p>
+ * 
+ * @see ApplicationClientProtocol#moveApplicationAcrossQueues(MoveApplicationAcrossQueuesRequest)
+ */
+@Public
+@Unstable
+public abstract class MoveApplicationAcrossQueuesRequest {
+  public static MoveApplicationAcrossQueuesRequest newInstance(ApplicationId appId, String queue) {
+    MoveApplicationAcrossQueuesRequest request =
+        Records.newRecord(MoveApplicationAcrossQueuesRequest.class);
+    request.setApplicationId(appId);
+    request.setTargetQueue(queue);
+    return request;
+  }
+  
+  /**
+   * Get the <code>ApplicationId</code> of the application to be moved.
+   * @return <code>ApplicationId</code> of the application to be moved
+   */
+  public abstract ApplicationId getApplicationId();
+  
+  /**
+   * Set the <code>ApplicationId</code> of the application to be moved.
+   * @param appId <code>ApplicationId</code> of the application to be moved
+   */
+  public abstract void setApplicationId(ApplicationId appId);
+  
+  /**
+   * Get the queue to place the application in.
+   * @return the name of the queue to place the application in
+   */
+  public abstract String getTargetQueue();
+
+  /**
+   * Get the queue to place the application in.
+   * @param queue the name of the queue to place the application in
+   */
+  public abstract void setTargetQueue(String queue);
+}

+ 47 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/MoveApplicationAcrossQueuesResponse.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.api.protocolrecords;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * <p>
+ * The response sent by the <code>ResourceManager</code> to the client moving
+ * a submitted application to a different queue.
+ * </p>
+ * <p>
+ * A response without exception means that the move has completed successfully.
+ * </p>
+ * 
+ * @see ApplicationClientProtocol#moveApplicationAcrossQueues(MoveApplicationAcrossQueuesRequest)
+ */
+@Public
+@Unstable
+public class MoveApplicationAcrossQueuesResponse {
+  @Private
+  @Unstable
+  public MoveApplicationAcrossQueuesResponse newInstance() {
+    MoveApplicationAcrossQueuesResponse response =
+        Records.newRecord(MoveApplicationAcrossQueuesResponse.class);
+    return response;
+  }
+}

+ 3 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java

@@ -928,6 +928,9 @@ public class YarnConfiguration extends Configuration {
   public static final String YARN_APP_CONTAINER_LOG_SIZE =
   public static final String YARN_APP_CONTAINER_LOG_SIZE =
       YARN_PREFIX + "app.container.log.filesize";
       YARN_PREFIX + "app.container.log.filesize";
 
 
+  public static final String YARN_APP_CONTAINER_LOG_BACKUPS =
+      YARN_PREFIX + "app.container.log.backups";
+
   ////////////////////////////////
   ////////////////////////////////
   // Other Configs
   // Other Configs
   ////////////////////////////////
   ////////////////////////////////

+ 1 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/applicationclient_protocol.proto

@@ -44,5 +44,6 @@ service ApplicationClientProtocolService {
   rpc getDelegationToken(hadoop.common.GetDelegationTokenRequestProto) returns (hadoop.common.GetDelegationTokenResponseProto);
   rpc getDelegationToken(hadoop.common.GetDelegationTokenRequestProto) returns (hadoop.common.GetDelegationTokenResponseProto);
   rpc renewDelegationToken(hadoop.common.RenewDelegationTokenRequestProto) returns (hadoop.common.RenewDelegationTokenResponseProto);
   rpc renewDelegationToken(hadoop.common.RenewDelegationTokenRequestProto) returns (hadoop.common.RenewDelegationTokenResponseProto);
   rpc cancelDelegationToken(hadoop.common.CancelDelegationTokenRequestProto) returns (hadoop.common.CancelDelegationTokenResponseProto);
   rpc cancelDelegationToken(hadoop.common.CancelDelegationTokenRequestProto) returns (hadoop.common.CancelDelegationTokenResponseProto);
+  rpc moveApplicationAcrossQueues(MoveApplicationAcrossQueuesRequestProto) returns (MoveApplicationAcrossQueuesResponseProto);
 }
 }
 
 

+ 8 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto

@@ -127,6 +127,14 @@ message GetClusterMetricsResponseProto {
   optional YarnClusterMetricsProto cluster_metrics = 1;
   optional YarnClusterMetricsProto cluster_metrics = 1;
 }
 }
 
 
+message MoveApplicationAcrossQueuesRequestProto {
+  required ApplicationIdProto application_id = 1;
+  required string target_queue = 2;
+}
+
+message MoveApplicationAcrossQueuesResponseProto {
+}
+
 message GetApplicationsRequestProto {
 message GetApplicationsRequestProto {
   repeated string application_types = 1;
   repeated string application_types = 1;
   repeated YarnApplicationStateProto application_states = 2;
   repeated YarnApplicationStateProto application_states = 2;

+ 65 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ContainerRollingLogAppender.java

@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.log4j.RollingFileAppender;
+
+import java.io.File;
+import java.io.Flushable;
+
+/**
+ * A simple log4j-appender for container's logs.
+ *
+ */
+@Public
+@Unstable
+public class ContainerRollingLogAppender extends RollingFileAppender
+  implements Flushable {
+  private String containerLogDir;
+
+  @Override
+  public void activateOptions() {
+    synchronized (this) {
+      setFile(new File(this.containerLogDir, "syslog").toString());
+      setAppend(true);
+      super.activateOptions();
+    }
+  }
+
+  @Override
+  public void flush() {
+    if (qw != null) {
+      qw.flush();
+    }
+  }
+
+  /**
+   * Getter/Setter methods for log4j.
+   */
+
+  public String getContainerLogDir() {
+    return this.containerLogDir;
+  }
+
+  public void setContainerLogDir(String containerLogDir) {
+    this.containerLogDir = containerLogDir;
+  }
+}

+ 21 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ApplicationClientProtocolPBClientImpl.java

@@ -51,6 +51,8 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.MoveApplicationAcrossQueuesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.MoveApplicationAcrossQueuesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
@@ -75,6 +77,8 @@ import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetQueueUserAclsInfoRe
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetQueueUserAclsInfoResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetQueueUserAclsInfoResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.KillApplicationRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.KillApplicationRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.KillApplicationResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.KillApplicationResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.MoveApplicationAcrossQueuesRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.MoveApplicationAcrossQueuesResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RenewDelegationTokenRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RenewDelegationTokenRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RenewDelegationTokenResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RenewDelegationTokenResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SubmitApplicationRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SubmitApplicationRequestPBImpl;
@@ -89,6 +93,7 @@ import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetNewApplicationRequestPr
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetQueueInfoRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetQueueInfoRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetQueueUserAclsInfoRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetQueueUserAclsInfoRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.KillApplicationRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.KillApplicationRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.MoveApplicationAcrossQueuesRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.SubmitApplicationRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.SubmitApplicationRequestProto;
 
 
 import com.google.protobuf.ServiceException;
 import com.google.protobuf.ServiceException;
@@ -291,4 +296,20 @@ public class ApplicationClientProtocolPBClientImpl implements ApplicationClientP
       return null;
       return null;
     }
     }
   }
   }
+  
+  @Override
+  public MoveApplicationAcrossQueuesResponse moveApplicationAcrossQueues(
+      MoveApplicationAcrossQueuesRequest request) throws YarnException,
+      IOException {
+    MoveApplicationAcrossQueuesRequestProto requestProto =
+        ((MoveApplicationAcrossQueuesRequestPBImpl) request).getProto();
+    try {
+      return new MoveApplicationAcrossQueuesResponsePBImpl(
+          proxy.moveApplicationAcrossQueues(null, requestProto));
+
+    } catch (ServiceException e) {
+      RPCUtil.unwrapAndThrowException(e);
+      return null;
+    }
+  }
 }
 }

+ 21 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ApplicationClientProtocolPBServiceImpl.java

@@ -39,6 +39,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.MoveApplicationAcrossQueuesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.CancelDelegationTokenRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.CancelDelegationTokenRequestPBImpl;
@@ -61,6 +62,8 @@ import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetQueueUserAclsInfoRe
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetQueueUserAclsInfoResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetQueueUserAclsInfoResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.KillApplicationRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.KillApplicationRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.KillApplicationResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.KillApplicationResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.MoveApplicationAcrossQueuesRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.MoveApplicationAcrossQueuesResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RenewDelegationTokenRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RenewDelegationTokenRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RenewDelegationTokenResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RenewDelegationTokenResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SubmitApplicationRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SubmitApplicationRequestPBImpl;
@@ -82,6 +85,8 @@ import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetQueueUserAclsInfoReques
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetQueueUserAclsInfoResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetQueueUserAclsInfoResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.KillApplicationRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.KillApplicationRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.KillApplicationResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.KillApplicationResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.MoveApplicationAcrossQueuesRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.MoveApplicationAcrossQueuesResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.SubmitApplicationRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.SubmitApplicationRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.SubmitApplicationResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.SubmitApplicationResponseProto;
 
 
@@ -278,4 +283,20 @@ public class ApplicationClientProtocolPBServiceImpl implements ApplicationClient
         throw new ServiceException(e);
         throw new ServiceException(e);
       }
       }
   }
   }
+  
+  @Override
+  public MoveApplicationAcrossQueuesResponseProto moveApplicationAcrossQueues(
+      RpcController controller, MoveApplicationAcrossQueuesRequestProto proto)
+      throws ServiceException {
+    MoveApplicationAcrossQueuesRequestPBImpl request =
+        new MoveApplicationAcrossQueuesRequestPBImpl(proto);
+    try {
+      MoveApplicationAcrossQueuesResponse response = real.moveApplicationAcrossQueues(request);
+      return ((MoveApplicationAcrossQueuesResponsePBImpl)response).getProto();
+    } catch (YarnException e) {
+      throw new ServiceException(e);
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
 }
 }

+ 158 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/MoveApplicationAcrossQueuesRequestPBImpl.java

@@ -0,0 +1,158 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.api.protocolrecords.impl.pb;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.protocolrecords.MoveApplicationAcrossQueuesRequest;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl;
+import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.MoveApplicationAcrossQueuesRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.MoveApplicationAcrossQueuesRequestProtoOrBuilder;
+
+import com.google.protobuf.TextFormat;
+
+@Private
+@Unstable
+public class MoveApplicationAcrossQueuesRequestPBImpl extends MoveApplicationAcrossQueuesRequest {
+  MoveApplicationAcrossQueuesRequestProto proto = MoveApplicationAcrossQueuesRequestProto.getDefaultInstance();
+  MoveApplicationAcrossQueuesRequestProto.Builder builder = null;
+  boolean viaProto = false;
+  
+  private ApplicationId applicationId;
+  private String targetQueue;
+  
+  public MoveApplicationAcrossQueuesRequestPBImpl() {
+    builder = MoveApplicationAcrossQueuesRequestProto.newBuilder();
+  }
+
+  public MoveApplicationAcrossQueuesRequestPBImpl(MoveApplicationAcrossQueuesRequestProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+  
+  public MoveApplicationAcrossQueuesRequestProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+  
+  @Override
+  public ApplicationId getApplicationId() {
+    if (this.applicationId != null) {
+      return this.applicationId;
+    }
+    
+    MoveApplicationAcrossQueuesRequestProtoOrBuilder p = viaProto ? proto : builder;
+    if (!p.hasApplicationId()) {
+      return null;
+    }
+    
+    this.applicationId = convertFromProtoFormat(p.getApplicationId());
+    return this.applicationId;
+  }
+  
+  @Override
+  public void setApplicationId(ApplicationId appId) {
+    maybeInitBuilder();
+    if (applicationId == null) {
+      builder.clearApplicationId();
+    }
+    applicationId = appId;
+  }
+  
+  @Override
+  public String getTargetQueue() {
+    if (this.targetQueue != null) {
+      return this.targetQueue;
+    }
+    
+    MoveApplicationAcrossQueuesRequestProtoOrBuilder p = viaProto ? proto : builder;
+    if (!p.hasApplicationId()) {
+      return null;
+    }
+    
+    this.targetQueue = p.getTargetQueue();
+    return this.targetQueue;
+  }
+  
+  @Override
+  public void setTargetQueue(String queue) {
+    maybeInitBuilder();
+    if (applicationId == null) {
+      builder.clearTargetQueue();
+    }
+    targetQueue = queue;
+  }
+  
+  private void mergeLocalToBuilder() {
+    if (applicationId != null) {
+      builder.setApplicationId(convertToProtoFormat(this.applicationId));
+    }
+    if (targetQueue != null) {
+      builder.setTargetQueue(this.targetQueue);
+    }
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = MoveApplicationAcrossQueuesRequestProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+  
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null) {
+      return false;
+    }
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
+  }
+  
+  private ApplicationIdPBImpl convertFromProtoFormat(ApplicationIdProto p) {
+    return new ApplicationIdPBImpl(p);
+  }
+
+  private ApplicationIdProto convertToProtoFormat(ApplicationId t) {
+    return ((ApplicationIdPBImpl)t).getProto();
+  }
+}

+ 68 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/MoveApplicationAcrossQueuesResponsePBImpl.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.yarn.api.protocolrecords.impl.pb;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.protocolrecords.MoveApplicationAcrossQueuesResponse;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.MoveApplicationAcrossQueuesResponseProto;
+
+import com.google.protobuf.TextFormat;
+
+@Private
+@Unstable
+public class MoveApplicationAcrossQueuesResponsePBImpl extends MoveApplicationAcrossQueuesResponse {
+  MoveApplicationAcrossQueuesResponseProto proto = MoveApplicationAcrossQueuesResponseProto.getDefaultInstance();
+  MoveApplicationAcrossQueuesResponseProto.Builder builder = null;
+  boolean viaProto = false;
+  
+  public MoveApplicationAcrossQueuesResponsePBImpl() {
+    builder = MoveApplicationAcrossQueuesResponseProto.newBuilder();
+  }
+
+  public MoveApplicationAcrossQueuesResponsePBImpl(MoveApplicationAcrossQueuesResponseProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+  
+  public MoveApplicationAcrossQueuesResponseProto getProto() {
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null)
+      return false;
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return TextFormat.shortDebugString(getProto());
+  }
+}

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml

@@ -335,7 +335,7 @@
     Expirations happens when the cluster does not hear from the client within
     Expirations happens when the cluster does not hear from the client within
     the specified session timeout period (i.e. no heartbeat).</description>
     the specified session timeout period (i.e. no heartbeat).</description>
     <name>yarn.resourcemanager.zk-timeout-ms</name>
     <name>yarn.resourcemanager.zk-timeout-ms</name>
-    <value>60000</value>
+    <value>10000</value>
   </property>
   </property>
 
 
   <property>
   <property>

+ 3 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LinuxContainerExecutor.java

@@ -217,8 +217,6 @@ public class LinuxContainerExecutor extends ContainerExecutor {
     }
     }
     String[] commandArray = command.toArray(new String[command.size()]);
     String[] commandArray = command.toArray(new String[command.size()]);
     ShellCommandExecutor shExec = new ShellCommandExecutor(commandArray);
     ShellCommandExecutor shExec = new ShellCommandExecutor(commandArray);
-    // TODO: DEBUG
-    LOG.info("initApplication: " + Arrays.toString(commandArray));
     if (LOG.isDebugEnabled()) {
     if (LOG.isDebugEnabled()) {
       LOG.debug("initApplication: " + Arrays.toString(commandArray));
       LOG.debug("initApplication: " + Arrays.toString(commandArray));
     }
     }
@@ -274,8 +272,9 @@ public class LinuxContainerExecutor extends ContainerExecutor {
         String[] commandArray = command.toArray(new String[command.size()]);
         String[] commandArray = command.toArray(new String[command.size()]);
         shExec = new ShellCommandExecutor(commandArray, null, // NM's cwd
         shExec = new ShellCommandExecutor(commandArray, null, // NM's cwd
             container.getLaunchContext().getEnvironment()); // sanitized env
             container.getLaunchContext().getEnvironment()); // sanitized env
-        // DEBUG
-        LOG.info("launchContainer: " + Arrays.toString(commandArray));
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("launchContainer: " + Arrays.toString(commandArray));
+        }
         shExec.execute();
         shExec.execute();
         if (LOG.isDebugEnabled()) {
         if (LOG.isDebugEnabled()) {
           logOutput(shExec.getOutput());
           logOutput(shExec.getOutput());
@@ -374,7 +373,6 @@ public class LinuxContainerExecutor extends ContainerExecutor {
     }
     }
     String[] commandArray = command.toArray(new String[command.size()]);
     String[] commandArray = command.toArray(new String[command.size()]);
     ShellCommandExecutor shExec = new ShellCommandExecutor(commandArray);
     ShellCommandExecutor shExec = new ShellCommandExecutor(commandArray);
-    LOG.info(" -- DEBUG -- deleteAsUser: " + Arrays.toString(commandArray));
     if (LOG.isDebugEnabled()) {
     if (LOG.isDebugEnabled()) {
       LOG.debug("deleteAsUser: " + Arrays.toString(commandArray));
       LOG.debug("deleteAsUser: " + Arrays.toString(commandArray));
     }
     }

+ 7 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/resources/container-log4j.properties

@@ -35,6 +35,13 @@ log4j.appender.CLA.totalLogFileSize=${yarn.app.container.log.filesize}
 log4j.appender.CLA.layout=org.apache.log4j.PatternLayout
 log4j.appender.CLA.layout=org.apache.log4j.PatternLayout
 log4j.appender.CLA.layout.ConversionPattern=%d{ISO8601} %p [%t] %c: %m%n
 log4j.appender.CLA.layout.ConversionPattern=%d{ISO8601} %p [%t] %c: %m%n
 
 
+log4j.appender.CRLA=org.apache.hadoop.yarn.ContainerRollingLogAppender
+log4j.appender.CRLA.containerLogDir=${yarn.app.container.log.dir}
+log4j.appender.CRLA.maximumFileSize=${yarn.app.container.log.filesize}
+log4j.appender.CRLA.maxBackupIndex=${yarn.app.container.log.backups}
+log4j.appender.CRLA.layout=org.apache.log4j.PatternLayout
+log4j.appender.CRLA.layout.ConversionPattern=%d{ISO8601} %p [%t] %c: %m%n
+
 #
 #
 # Event Counter Appender
 # Event Counter Appender
 # Sends counts of logging messages at different severity levels to Hadoop Metrics.
 # Sends counts of logging messages at different severity levels to Hadoop Metrics.

+ 8 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java

@@ -64,6 +64,8 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.MoveApplicationAcrossQueuesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.MoveApplicationAcrossQueuesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
@@ -683,6 +685,12 @@ public class ClientRMService extends AbstractService implements
       throw RPCUtil.getRemoteException(e);
       throw RPCUtil.getRemoteException(e);
     }
     }
   }
   }
+  
+  @Override
+  public MoveApplicationAcrossQueuesResponse moveApplicationAcrossQueues(
+      MoveApplicationAcrossQueuesRequest request) throws YarnException {
+    throw new UnsupportedOperationException("Move not yet supported");
+  }
 
 
   private String getRenewerForToken(Token<RMDelegationTokenIdentifier> token)
   private String getRenewerForToken(Token<RMDelegationTokenIdentifier> token)
       throws IOException {
       throws IOException {