Browse Source

Merge r1555021 through r1559304 from trunk.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-5535@1559305 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 11 years ago
parent
commit
1025121874
74 changed files with 3200 additions and 2333 deletions
  1. 4 4
      BUILDING.txt
  2. 1 2
      hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-dist.xml
  3. 23 3
      hadoop-common-project/hadoop-common/CHANGES.txt
  4. 2 0
      hadoop-common-project/hadoop-common/src/main/bin/hadoop
  5. 1 1
      hadoop-common-project/hadoop-common/src/main/bin/hadoop.cmd
  6. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/JavaKeyStoreProvider.java
  7. 11 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProvider.java
  8. 474 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyShell.java
  9. 5 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/UserProvider.java
  10. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
  11. 4 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
  12. 16 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java
  13. 176 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestKeyShell.java
  14. 23 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  15. 2 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReader.java
  16. 25 18
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java
  17. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
  18. 64 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LayoutFlags.java
  19. 5 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LayoutVersion.java
  20. 107 96
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenSecretManager.java
  21. 103 91
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CacheManager.java
  22. 11 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java
  23. 2 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java
  24. 20 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  25. 13 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
  26. 51 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
  27. 104 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
  28. 8 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOpCodes.java
  29. 9 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
  30. 5 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  31. 5 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java
  32. 47 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderLocal.java
  33. 23 12
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppendRestart.java
  34. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java
  35. 157 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlock.java
  36. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
  37. 0 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/TestOfflineEditsViewer.java
  38. 355 41
      hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml
  39. 11 0
      hadoop-mapreduce-project/CHANGES.txt
  40. 9 15
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/MapReduceChildJVM.java
  41. 0 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java
  42. 46 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestTaskAttemptListenerImpl.java
  43. 7 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/jobhistory/JHAdminConfig.java
  44. 11 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRApps.java
  45. 9 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
  46. 50 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
  47. 10 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/pom.xml
  48. 110 31
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryFileManager.java
  49. 139 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestHistoryFileManager.java
  50. 3 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java
  51. 9 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientRedirect.java
  52. 123 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobs.java
  53. 244 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobsWithProfiler.java
  54. 7 1
      hadoop-project/pom.xml
  55. 17 0
      hadoop-yarn-project/CHANGES.txt
  56. 15 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocol.java
  57. 69 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/MoveApplicationAcrossQueuesRequest.java
  58. 47 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/MoveApplicationAcrossQueuesResponse.java
  59. 3 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
  60. 1 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/applicationclient_protocol.proto
  61. 8 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
  62. 65 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ContainerRollingLogAppender.java
  63. 21 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ApplicationClientProtocolPBClientImpl.java
  64. 21 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ApplicationClientProtocolPBServiceImpl.java
  65. 158 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/MoveApplicationAcrossQueuesRequestPBImpl.java
  66. 68 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/MoveApplicationAcrossQueuesResponsePBImpl.java
  67. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
  68. 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
  69. 7 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/resources/container-log4j.properties
  70. 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
  71. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationFileLoaderService.java
  72. 0 1361
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java.orig
  73. 0 615
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java.orig
  74. 36 1
      hadoop-yarn-project/pom.xml

+ 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)
 
 

+ 1 - 2
hadoop-assemblies/src/main/resources/assemblies/hadoop-yarn-dist.xml

@@ -207,8 +207,7 @@
       <outputDirectory>/share/hadoop/${hadoop.component}/lib</outputDirectory>
       <outputDirectory>/share/hadoop/${hadoop.component}/lib</outputDirectory>
       <!-- Exclude hadoop artifacts. They will be found via HADOOP* env -->
       <!-- Exclude hadoop artifacts. They will be found via HADOOP* env -->
       <excludes>
       <excludes>
-        <exclude>org.apache.hadoop:hadoop-common</exclude>
-        <exclude>org.apache.hadoop:hadoop-hdfs</exclude>
+        <exclude>org.apache.hadoop:*</exclude>
         <!-- use slf4j from common to avoid multiple binding warnings -->
         <!-- use slf4j from common to avoid multiple binding warnings -->
         <exclude>org.slf4j:slf4j-api</exclude>
         <exclude>org.slf4j:slf4j-api</exclude>
         <exclude>org.slf4j:slf4j-log4j12</exclude>
         <exclude>org.slf4j:slf4j-log4j12</exclude>

+ 23 - 3
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)
@@ -517,6 +520,16 @@ Release 2.4.0 - UNRELEASED
     HADOOP-10223. MiniKdc#main() should close the FileReader it creates. 
     HADOOP-10223. MiniKdc#main() should close the FileReader it creates. 
     (Ted Yu via tucu)
     (Ted Yu via tucu)
 
 
+    HADOOP-10236. Fix typo in o.a.h.ipc.Client#checkResponse. (Akira Ajisaka
+    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
@@ -605,6 +618,13 @@ Release 2.3.0 - UNRELEASED
     HADOOP-10178. Configuration deprecation always emit "deprecated" warnings
     HADOOP-10178. Configuration deprecation always emit "deprecated" warnings
     when a new key is used. (Shanyu Zhao via cnauroth)
     when a new key is used. (Shanyu Zhao via 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/bin/hadoop.cmd

@@ -143,7 +143,7 @@ call :updatepath %HADOOP_BIN_PATH%
 
 
   call %JAVA% %JAVA_HEAP_MAX% %HADOOP_OPTS% -classpath %CLASSPATH% %CLASS% %hadoop-command-arguments%
   call %JAVA% %JAVA_HEAP_MAX% %HADOOP_OPTS% -classpath %CLASSPATH% %CLASS% %hadoop-command-arguments%
 
 
-  goto :eof
+  exit /b %ERRORLEVEL%
 
 
 :fs 
 :fs 
   set CLASS=org.apache.hadoop.fs.FsShell
   set CLASS=org.apache.hadoop.fs.FsShell

+ 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));

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java

@@ -286,7 +286,7 @@ public class Client {
       if (!Arrays.equals(id, RpcConstants.DUMMY_CLIENT_ID)) {
       if (!Arrays.equals(id, RpcConstants.DUMMY_CLIENT_ID)) {
         if (!Arrays.equals(id, clientId)) {
         if (!Arrays.equals(id, clientId)) {
           throw new IOException("Client IDs not matched: local ID="
           throw new IOException("Client IDs not matched: local ID="
-              + StringUtils.byteToHexString(clientId) + ", ID in reponse="
+              + StringUtils.byteToHexString(clientId) + ", ID in response="
               + StringUtils.byteToHexString(header.getClientId().toByteArray()));
               + StringUtils.byteToHexString(header.getClientId().toByteArray()));
         }
         }
       }
       }

+ 4 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java

@@ -2021,6 +2021,10 @@ public abstract class Server {
           if (LOG.isDebugEnabled()) {
           if (LOG.isDebugEnabled()) {
             LOG.debug(Thread.currentThread().getName() + ": " + call + " for RpcKind " + call.rpcKind);
             LOG.debug(Thread.currentThread().getName() + ": " + call + " for RpcKind " + call.rpcKind);
           }
           }
+          if (!call.connection.channel.isOpen()) {
+            LOG.info(Thread.currentThread().getName() + ": skipped " + call);
+            continue;
+          }
           String errorClass = null;
           String errorClass = null;
           String error = null;
           String error = null;
           RpcStatusProto returnStatus = RpcStatusProto.SUCCESS;
           RpcStatusProto returnStatus = RpcStatusProto.SUCCESS;

+ 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."));
+  }
+}

+ 23 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -478,6 +478,15 @@ Trunk (Unreleased)
 
 
     HDFS-5726. Fix compilation error in AbstractINodeDiff for JDK7. (jing9)
     HDFS-5726. Fix compilation error in AbstractINodeDiff for JDK7. (jing9)
 
 
+    HDFS-5768. Consolidate the serialization code in DelegationTokenSecretManager 
+    (Haohui Mai via brandonli)
+
+    HDFS-5775. Consolidate the code for serialization in CacheManager
+    (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
@@ -512,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)
@@ -674,6 +686,15 @@ Release 2.4.0 - UNRELEASED
 
 
     HDFS-5220. Expose group resolution time as metric (jxiang via cmccabe)
     HDFS-5220. Expose group resolution time as metric (jxiang via cmccabe)
 
 
+    HDFS-5762. BlockReaderLocal doesn't return -1 on EOF when doing zero-length
+    reads (Colin Patrick McCabe)
+
+    HDFS-5766. In DFSInputStream, do not add datanode to deadNodes after
+    InvalidEncryptionKeyException in fetchBlockByteRange (Liang Xie via Colin
+    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)
@@ -752,6 +773,8 @@ 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)
+
   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

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReader.java

@@ -39,6 +39,8 @@ public interface BlockReader extends ByteBufferReadable {
    * "Read should not modify user buffer before successful read"
    * "Read should not modify user buffer before successful read"
    * because it first reads the data to user buffer and then checks
    * because it first reads the data to user buffer and then checks
    * the checksum.
    * the checksum.
+   * Note: this must return -1 on EOF, even in the case of a 0-byte read.
+   * See HDFS-5762 for details.
    */
    */
   int read(byte[] buf, int off, int len) throws IOException;
   int read(byte[] buf, int off, int len) throws IOException;
 
 

+ 25 - 18
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java

@@ -328,10 +328,12 @@ class BlockReaderLocal implements BlockReader {
 
 
   private synchronized int drainDataBuf(ByteBuffer buf)
   private synchronized int drainDataBuf(ByteBuffer buf)
       throws IOException {
       throws IOException {
-    if (dataBuf == null) return 0;
+    if (dataBuf == null) return -1;
     int oldLimit = dataBuf.limit();
     int oldLimit = dataBuf.limit();
     int nRead = Math.min(dataBuf.remaining(), buf.remaining());
     int nRead = Math.min(dataBuf.remaining(), buf.remaining());
-    if (nRead == 0) return 0;
+    if (nRead == 0) {
+      return (dataBuf.remaining() == 0) ? -1 : 0;
+    }
     try {
     try {
       dataBuf.limit(dataBuf.position() + nRead);
       dataBuf.limit(dataBuf.position() + nRead);
       buf.put(dataBuf);
       buf.put(dataBuf);
@@ -444,13 +446,11 @@ class BlockReaderLocal implements BlockReader {
     int total = 0;
     int total = 0;
     while (buf.hasRemaining()) {
     while (buf.hasRemaining()) {
       int nRead = dataIn.read(buf, dataPos);
       int nRead = dataIn.read(buf, dataPos);
-      if (nRead < 0) {
-        break;
-      }
+      if (nRead <= 0) break;
       dataPos += nRead;
       dataPos += nRead;
       total += nRead;
       total += nRead;
     }
     }
-    return (total == 0) ? -1 : total;
+    return (total == 0 && (dataPos == dataIn.size())) ? -1 : total;
   }
   }
 
 
   /**
   /**
@@ -512,15 +512,15 @@ class BlockReaderLocal implements BlockReader {
   private synchronized int readWithBounceBuffer(ByteBuffer buf,
   private synchronized int readWithBounceBuffer(ByteBuffer buf,
         boolean canSkipChecksum) throws IOException {
         boolean canSkipChecksum) throws IOException {
     int total = 0;
     int total = 0;
-    boolean eof = false;
-    while (true) {
-      int bb = drainDataBuf(buf); // drain bounce buffer if possible
+    int bb = drainDataBuf(buf); // drain bounce buffer if possible
+    if (bb >= 0) {
       total += bb;
       total += bb;
-      int needed = buf.remaining();
-      if (eof || (needed == 0)) {
-        break;
-      } else if (buf.isDirect() && (needed >= maxReadaheadLength)
-          && ((dataPos % bytesPerChecksum) == 0)) {
+      if (buf.remaining() == 0) return total;
+    }
+    boolean eof = false;
+    do {
+      if (buf.isDirect() && (buf.remaining() >= maxReadaheadLength)
+            && ((dataPos % bytesPerChecksum) == 0)) {
         // Fast lane: try to read directly into user-supplied buffer, bypassing
         // Fast lane: try to read directly into user-supplied buffer, bypassing
         // bounce buffer.
         // bounce buffer.
         int oldLimit = buf.limit();
         int oldLimit = buf.limit();
@@ -540,9 +540,13 @@ class BlockReaderLocal implements BlockReader {
         if (fillDataBuf(canSkipChecksum)) {
         if (fillDataBuf(canSkipChecksum)) {
           eof = true;
           eof = true;
         }
         }
+        bb = drainDataBuf(buf); // drain bounce buffer if possible
+        if (bb >= 0) {
+          total += bb;
+        }
       }
       }
-    }
-    return total == 0 ? -1 : total;
+    } while ((!eof) && (buf.remaining() > 0));
+    return (eof && total == 0) ? -1 : total;
   }
   }
 
 
   @Override
   @Override
@@ -587,8 +591,10 @@ class BlockReaderLocal implements BlockReader {
     int nRead = dataIn.read(ByteBuffer.wrap(arr, off, len), dataPos);
     int nRead = dataIn.read(ByteBuffer.wrap(arr, off, len), dataPos);
     if (nRead > 0) {
     if (nRead > 0) {
       dataPos += nRead;
       dataPos += nRead;
+    } else if ((nRead == 0) && (dataPos == dataIn.size())) {
+      return -1;
     }
     }
-    return nRead == 0 ? -1 : nRead;
+    return nRead;
   }
   }
 
 
   private synchronized int readWithBounceBuffer(byte arr[], int off, int len,
   private synchronized int readWithBounceBuffer(byte arr[], int off, int len,
@@ -599,9 +605,10 @@ class BlockReaderLocal implements BlockReader {
       dataBuf.limit(maxReadaheadLength);
       dataBuf.limit(maxReadaheadLength);
       fillDataBuf(canSkipChecksum);
       fillDataBuf(canSkipChecksum);
     }
     }
+    if (dataBuf.remaining() == 0) return -1;
     int toRead = Math.min(dataBuf.remaining(), len);
     int toRead = Math.min(dataBuf.remaining(), len);
     dataBuf.get(arr, off, toRead);
     dataBuf.get(arr, off, toRead);
-    return toRead == 0 ? -1 : toRead;
+    return toRead;
   }
   }
 
 
   @Override
   @Override

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

@@ -969,6 +969,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
           // The encryption key used is invalid.
           // The encryption key used is invalid.
           refetchEncryptionKey--;
           refetchEncryptionKey--;
           dfsClient.clearDataEncryptionKey();
           dfsClient.clearDataEncryptionKey();
+          continue;
         } else if (refetchToken > 0 && tokenRefetchNeeded(e, targetAddr)) {
         } else if (refetchToken > 0 && tokenRefetchNeeded(e, targetAddr)) {
           refetchToken--;
           refetchToken--;
           fetchBlockAt(block.getStartOffset());
           fetchBlockAt(block.getStartOffset());

+ 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."),
-    ROLLING_UPGRADE_MARKER(-50, "Upgrade marker for rolling upgrade");
+    ADD_LAYOUT_FLAGS(-50, "Add support for layout flags."),
+    CACHING(-51, "Support for cache pools and path-based caching"),
+    ROLLING_UPGRADE_MARKER(-52, "Upgrade marker for rolling upgrade");
 
 
-    
     final int lv;
     final int lv;
     final int ancestorLV;
     final int ancestorLV;
     final String description;
     final String description;

+ 107 - 96
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenSecretManager.java

@@ -59,6 +59,7 @@ public class DelegationTokenSecretManager
       .getLog(DelegationTokenSecretManager.class);
       .getLog(DelegationTokenSecretManager.class);
   
   
   private final FSNamesystem namesystem;
   private final FSNamesystem namesystem;
+  private final SerializerCompat serializerCompat = new SerializerCompat();
 
 
   public DelegationTokenSecretManager(long delegationKeyUpdateInterval,
   public DelegationTokenSecretManager(long delegationKeyUpdateInterval,
       long delegationTokenMaxLifetime, long delegationTokenRenewInterval,
       long delegationTokenMaxLifetime, long delegationTokenRenewInterval,
@@ -150,24 +151,21 @@ public class DelegationTokenSecretManager
       throw new IOException("No delegation token found for this identifier");
       throw new IOException("No delegation token found for this identifier");
     }
     }
   }
   }
-  
+
   /**
   /**
    * Load SecretManager state from fsimage.
    * Load SecretManager state from fsimage.
    * 
    * 
    * @param in input stream to read fsimage
    * @param in input stream to read fsimage
    * @throws IOException
    * @throws IOException
    */
    */
-  public synchronized void loadSecretManagerState(DataInput in)
+  public synchronized void loadSecretManagerStateCompat(DataInput in)
       throws IOException {
       throws IOException {
     if (running) {
     if (running) {
       // a safety check
       // a safety check
       throw new IOException(
       throw new IOException(
           "Can't load state from image in a running SecretManager.");
           "Can't load state from image in a running SecretManager.");
     }
     }
-    currentId = in.readInt();
-    loadAllKeys(in);
-    delegationTokenSequenceNumber = in.readInt();
-    loadCurrentTokens(in);
+    serializerCompat.load(in);
   }
   }
   
   
   /**
   /**
@@ -177,12 +175,9 @@ public class DelegationTokenSecretManager
    * @param sdPath String storage directory path
    * @param sdPath String storage directory path
    * @throws IOException
    * @throws IOException
    */
    */
-  public synchronized void saveSecretManagerState(DataOutputStream out,
+  public synchronized void saveSecretManagerStateCompat(DataOutputStream out,
       String sdPath) throws IOException {
       String sdPath) throws IOException {
-    out.writeInt(currentId);
-    saveAllKeys(out, sdPath);
-    out.writeInt(delegationTokenSequenceNumber);
-    saveCurrentTokens(out, sdPath);
+    serializerCompat.save(out, sdPath);
   }
   }
   
   
   /**
   /**
@@ -282,91 +277,6 @@ public class DelegationTokenSecretManager
     return allKeys.size();
     return allKeys.size();
   }
   }
 
 
-  /**
-   * Private helper methods to save delegation keys and tokens in fsimage
-   */
-  private synchronized void saveCurrentTokens(DataOutputStream out,
-      String sdPath) throws IOException {
-    StartupProgress prog = NameNode.getStartupProgress();
-    Step step = new Step(StepType.DELEGATION_TOKENS, sdPath);
-    prog.beginStep(Phase.SAVING_CHECKPOINT, step);
-    prog.setTotal(Phase.SAVING_CHECKPOINT, step, currentTokens.size());
-    Counter counter = prog.getCounter(Phase.SAVING_CHECKPOINT, step);
-    out.writeInt(currentTokens.size());
-    Iterator<DelegationTokenIdentifier> iter = currentTokens.keySet()
-        .iterator();
-    while (iter.hasNext()) {
-      DelegationTokenIdentifier id = iter.next();
-      id.write(out);
-      DelegationTokenInformation info = currentTokens.get(id);
-      out.writeLong(info.getRenewDate());
-      counter.increment();
-    }
-    prog.endStep(Phase.SAVING_CHECKPOINT, step);
-  }
-  
-  /*
-   * Save the current state of allKeys
-   */
-  private synchronized void saveAllKeys(DataOutputStream out, String sdPath)
-      throws IOException {
-    StartupProgress prog = NameNode.getStartupProgress();
-    Step step = new Step(StepType.DELEGATION_KEYS, sdPath);
-    prog.beginStep(Phase.SAVING_CHECKPOINT, step);
-    prog.setTotal(Phase.SAVING_CHECKPOINT, step, currentTokens.size());
-    Counter counter = prog.getCounter(Phase.SAVING_CHECKPOINT, step);
-    out.writeInt(allKeys.size());
-    Iterator<Integer> iter = allKeys.keySet().iterator();
-    while (iter.hasNext()) {
-      Integer key = iter.next();
-      allKeys.get(key).write(out);
-      counter.increment();
-    }
-    prog.endStep(Phase.SAVING_CHECKPOINT, step);
-  }
-  
-  /**
-   * Private helper methods to load Delegation tokens from fsimage
-   */
-  private synchronized void loadCurrentTokens(DataInput in)
-      throws IOException {
-    StartupProgress prog = NameNode.getStartupProgress();
-    Step step = new Step(StepType.DELEGATION_TOKENS);
-    prog.beginStep(Phase.LOADING_FSIMAGE, step);
-    int numberOfTokens = in.readInt();
-    prog.setTotal(Phase.LOADING_FSIMAGE, step, numberOfTokens);
-    Counter counter = prog.getCounter(Phase.LOADING_FSIMAGE, step);
-    for (int i = 0; i < numberOfTokens; i++) {
-      DelegationTokenIdentifier id = new DelegationTokenIdentifier();
-      id.readFields(in);
-      long expiryTime = in.readLong();
-      addPersistedDelegationToken(id, expiryTime);
-      counter.increment();
-    }
-    prog.endStep(Phase.LOADING_FSIMAGE, step);
-  }
-
-  /**
-   * Private helper method to load delegation keys from fsimage.
-   * @param in
-   * @throws IOException
-   */
-  private synchronized void loadAllKeys(DataInput in) throws IOException {
-    StartupProgress prog = NameNode.getStartupProgress();
-    Step step = new Step(StepType.DELEGATION_KEYS);
-    prog.beginStep(Phase.LOADING_FSIMAGE, step);
-    int numberOfKeys = in.readInt();
-    prog.setTotal(Phase.LOADING_FSIMAGE, step, numberOfKeys);
-    Counter counter = prog.getCounter(Phase.LOADING_FSIMAGE, step);
-    for (int i = 0; i < numberOfKeys; i++) {
-      DelegationKey value = new DelegationKey();
-      value.readFields(in);
-      addKey(value);
-      counter.increment();
-    }
-    prog.endStep(Phase.LOADING_FSIMAGE, step);
-  }
-
   /**
   /**
    * Call namesystem to update editlogs for new master key.
    * Call namesystem to update editlogs for new master key.
    */
    */
@@ -420,4 +330,105 @@ public class DelegationTokenSecretManager
     c.addToken(new Text(ugi.getShortUserName()), token);
     c.addToken(new Text(ugi.getShortUserName()), token);
     return c;
     return c;
   }
   }
+
+  private final class SerializerCompat {
+    private void load(DataInput in) throws IOException {
+      currentId = in.readInt();
+      loadAllKeys(in);
+      delegationTokenSequenceNumber = in.readInt();
+      loadCurrentTokens(in);
+    }
+
+    private void save(DataOutputStream out, String sdPath) throws IOException {
+      out.writeInt(currentId);
+      saveAllKeys(out, sdPath);
+      out.writeInt(delegationTokenSequenceNumber);
+      saveCurrentTokens(out, sdPath);
+    }
+
+    /**
+     * Private helper methods to save delegation keys and tokens in fsimage
+     */
+    private synchronized void saveCurrentTokens(DataOutputStream out,
+        String sdPath) throws IOException {
+      StartupProgress prog = NameNode.getStartupProgress();
+      Step step = new Step(StepType.DELEGATION_TOKENS, sdPath);
+      prog.beginStep(Phase.SAVING_CHECKPOINT, step);
+      prog.setTotal(Phase.SAVING_CHECKPOINT, step, currentTokens.size());
+      Counter counter = prog.getCounter(Phase.SAVING_CHECKPOINT, step);
+      out.writeInt(currentTokens.size());
+      Iterator<DelegationTokenIdentifier> iter = currentTokens.keySet()
+          .iterator();
+      while (iter.hasNext()) {
+        DelegationTokenIdentifier id = iter.next();
+        id.write(out);
+        DelegationTokenInformation info = currentTokens.get(id);
+        out.writeLong(info.getRenewDate());
+        counter.increment();
+      }
+      prog.endStep(Phase.SAVING_CHECKPOINT, step);
+    }
+
+    /*
+     * Save the current state of allKeys
+     */
+    private synchronized void saveAllKeys(DataOutputStream out, String sdPath)
+        throws IOException {
+      StartupProgress prog = NameNode.getStartupProgress();
+      Step step = new Step(StepType.DELEGATION_KEYS, sdPath);
+      prog.beginStep(Phase.SAVING_CHECKPOINT, step);
+      prog.setTotal(Phase.SAVING_CHECKPOINT, step, currentTokens.size());
+      Counter counter = prog.getCounter(Phase.SAVING_CHECKPOINT, step);
+      out.writeInt(allKeys.size());
+      Iterator<Integer> iter = allKeys.keySet().iterator();
+      while (iter.hasNext()) {
+        Integer key = iter.next();
+        allKeys.get(key).write(out);
+        counter.increment();
+      }
+      prog.endStep(Phase.SAVING_CHECKPOINT, step);
+    }
+
+    /**
+     * Private helper methods to load Delegation tokens from fsimage
+     */
+    private synchronized void loadCurrentTokens(DataInput in)
+        throws IOException {
+      StartupProgress prog = NameNode.getStartupProgress();
+      Step step = new Step(StepType.DELEGATION_TOKENS);
+      prog.beginStep(Phase.LOADING_FSIMAGE, step);
+      int numberOfTokens = in.readInt();
+      prog.setTotal(Phase.LOADING_FSIMAGE, step, numberOfTokens);
+      Counter counter = prog.getCounter(Phase.LOADING_FSIMAGE, step);
+      for (int i = 0; i < numberOfTokens; i++) {
+        DelegationTokenIdentifier id = new DelegationTokenIdentifier();
+        id.readFields(in);
+        long expiryTime = in.readLong();
+        addPersistedDelegationToken(id, expiryTime);
+        counter.increment();
+      }
+      prog.endStep(Phase.LOADING_FSIMAGE, step);
+    }
+
+    /**
+     * Private helper method to load delegation keys from fsimage.
+     * @param in
+     * @throws IOException
+     */
+    private synchronized void loadAllKeys(DataInput in) throws IOException {
+      StartupProgress prog = NameNode.getStartupProgress();
+      Step step = new Step(StepType.DELEGATION_KEYS);
+      prog.beginStep(Phase.LOADING_FSIMAGE, step);
+      int numberOfKeys = in.readInt();
+      prog.setTotal(Phase.LOADING_FSIMAGE, step, numberOfKeys);
+      Counter counter = prog.getCounter(Phase.LOADING_FSIMAGE, step);
+      for (int i = 0; i < numberOfKeys; i++) {
+        DelegationKey value = new DelegationKey();
+        value.readFields(in);
+        addKey(value);
+        counter.increment();
+      }
+      prog.endStep(Phase.LOADING_FSIMAGE, step);
+    }
+  }
 }
 }

+ 103 - 91
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CacheManager.java

@@ -160,6 +160,8 @@ public final class CacheManager {
    */
    */
   private final ReentrantLock crmLock = new ReentrantLock();
   private final ReentrantLock crmLock = new ReentrantLock();
 
 
+  private final SerializerCompat serializerCompat = new SerializerCompat();
+
   /**
   /**
    * The CacheReplicationMonitor.
    * The CacheReplicationMonitor.
    */
    */
@@ -926,11 +928,9 @@ public final class CacheManager {
    * @param sdPath path of the storage directory
    * @param sdPath path of the storage directory
    * @throws IOException
    * @throws IOException
    */
    */
-  public void saveState(DataOutputStream out, String sdPath)
+  public void saveStateCompat(DataOutputStream out, String sdPath)
       throws IOException {
       throws IOException {
-    out.writeLong(nextDirectiveId);
-    savePools(out, sdPath);
-    saveDirectives(out, sdPath);
+    serializerCompat.save(out, sdPath);
   }
   }
 
 
   /**
   /**
@@ -939,105 +939,117 @@ public final class CacheManager {
    * @param in DataInput from which to restore state
    * @param in DataInput from which to restore state
    * @throws IOException
    * @throws IOException
    */
    */
-  public void loadState(DataInput in) throws IOException {
-    nextDirectiveId = in.readLong();
-    // pools need to be loaded first since directives point to their parent pool
-    loadPools(in);
-    loadDirectives(in);
+  public void loadStateCompat(DataInput in) throws IOException {
+    serializerCompat.load(in);
   }
   }
 
 
-  /**
-   * Save cache pools to fsimage
-   */
-  private void savePools(DataOutputStream out,
-      String sdPath) throws IOException {
-    StartupProgress prog = NameNode.getStartupProgress();
-    Step step = new Step(StepType.CACHE_POOLS, sdPath);
-    prog.beginStep(Phase.SAVING_CHECKPOINT, step);
-    prog.setTotal(Phase.SAVING_CHECKPOINT, step, cachePools.size());
-    Counter counter = prog.getCounter(Phase.SAVING_CHECKPOINT, step);
-    out.writeInt(cachePools.size());
-    for (CachePool pool: cachePools.values()) {
-      FSImageSerialization.writeCachePoolInfo(out, pool.getInfo(true));
-      counter.increment();
+  private final class SerializerCompat {
+    private void save(DataOutputStream out, String sdPath) throws IOException {
+      out.writeLong(nextDirectiveId);
+      savePools(out, sdPath);
+      saveDirectives(out, sdPath);
     }
     }
-    prog.endStep(Phase.SAVING_CHECKPOINT, step);
-  }
 
 
-  /*
-   * Save cache entries to fsimage
-   */
-  private void saveDirectives(DataOutputStream out, String sdPath)
-      throws IOException {
-    StartupProgress prog = NameNode.getStartupProgress();
-    Step step = new Step(StepType.CACHE_ENTRIES, sdPath);
-    prog.beginStep(Phase.SAVING_CHECKPOINT, step);
-    prog.setTotal(Phase.SAVING_CHECKPOINT, step, directivesById.size());
-    Counter counter = prog.getCounter(Phase.SAVING_CHECKPOINT, step);
-    out.writeInt(directivesById.size());
-    for (CacheDirective directive : directivesById.values()) {
-      FSImageSerialization.writeCacheDirectiveInfo(out, directive.toInfo());
-      counter.increment();
+    private void load(DataInput in) throws IOException {
+      nextDirectiveId = in.readLong();
+      // pools need to be loaded first since directives point to their parent pool
+      loadPools(in);
+      loadDirectives(in);
     }
     }
-    prog.endStep(Phase.SAVING_CHECKPOINT, step);
-  }
 
 
-  /**
-   * Load cache pools from fsimage
-   */
-  private void loadPools(DataInput in)
-      throws IOException {
-    StartupProgress prog = NameNode.getStartupProgress();
-    Step step = new Step(StepType.CACHE_POOLS);
-    prog.beginStep(Phase.LOADING_FSIMAGE, step);
-    int numberOfPools = in.readInt();
-    prog.setTotal(Phase.LOADING_FSIMAGE, step, numberOfPools);
-    Counter counter = prog.getCounter(Phase.LOADING_FSIMAGE, step);
-    for (int i = 0; i < numberOfPools; i++) {
-      addCachePool(FSImageSerialization.readCachePoolInfo(in));
-      counter.increment();
+    /**
+     * Save cache pools to fsimage
+     */
+    private void savePools(DataOutputStream out,
+        String sdPath) throws IOException {
+      StartupProgress prog = NameNode.getStartupProgress();
+      Step step = new Step(StepType.CACHE_POOLS, sdPath);
+      prog.beginStep(Phase.SAVING_CHECKPOINT, step);
+      prog.setTotal(Phase.SAVING_CHECKPOINT, step, cachePools.size());
+      Counter counter = prog.getCounter(Phase.SAVING_CHECKPOINT, step);
+      out.writeInt(cachePools.size());
+      for (CachePool pool: cachePools.values()) {
+        FSImageSerialization.writeCachePoolInfo(out, pool.getInfo(true));
+        counter.increment();
+      }
+      prog.endStep(Phase.SAVING_CHECKPOINT, step);
     }
     }
-    prog.endStep(Phase.LOADING_FSIMAGE, step);
-  }
 
 
-  /**
-   * Load cache directives from the fsimage
-   */
-  private void loadDirectives(DataInput in) throws IOException {
-    StartupProgress prog = NameNode.getStartupProgress();
-    Step step = new Step(StepType.CACHE_ENTRIES);
-    prog.beginStep(Phase.LOADING_FSIMAGE, step);
-    int numDirectives = in.readInt();
-    prog.setTotal(Phase.LOADING_FSIMAGE, step, numDirectives);
-    Counter counter = prog.getCounter(Phase.LOADING_FSIMAGE, step);
-    for (int i = 0; i < numDirectives; i++) {
-      CacheDirectiveInfo info = FSImageSerialization.readCacheDirectiveInfo(in);
-      // Get pool reference by looking it up in the map
-      final String poolName = info.getPool();
-      CachePool pool = cachePools.get(poolName);
-      if (pool == null) {
-        throw new IOException("Directive refers to pool " + poolName +
-            ", which does not exist.");
+    /*
+     * Save cache entries to fsimage
+     */
+    private void saveDirectives(DataOutputStream out, String sdPath)
+        throws IOException {
+      StartupProgress prog = NameNode.getStartupProgress();
+      Step step = new Step(StepType.CACHE_ENTRIES, sdPath);
+      prog.beginStep(Phase.SAVING_CHECKPOINT, step);
+      prog.setTotal(Phase.SAVING_CHECKPOINT, step, directivesById.size());
+      Counter counter = prog.getCounter(Phase.SAVING_CHECKPOINT, step);
+      out.writeInt(directivesById.size());
+      for (CacheDirective directive : directivesById.values()) {
+        FSImageSerialization.writeCacheDirectiveInfo(out, directive.toInfo());
+        counter.increment();
       }
       }
-      CacheDirective directive =
-          new CacheDirective(info.getId(), info.getPath().toUri().getPath(),
-              info.getReplication(), info.getExpiration().getAbsoluteMillis());
-      boolean addedDirective = pool.getDirectiveList().add(directive);
-      assert addedDirective;
-      if (directivesById.put(directive.getId(), directive) != null) {
-        throw new IOException("A directive with ID " + directive.getId() +
-            " already exists");
+      prog.endStep(Phase.SAVING_CHECKPOINT, step);
+    }
+
+    /**
+     * Load cache pools from fsimage
+     */
+    private void loadPools(DataInput in)
+        throws IOException {
+      StartupProgress prog = NameNode.getStartupProgress();
+      Step step = new Step(StepType.CACHE_POOLS);
+      prog.beginStep(Phase.LOADING_FSIMAGE, step);
+      int numberOfPools = in.readInt();
+      prog.setTotal(Phase.LOADING_FSIMAGE, step, numberOfPools);
+      Counter counter = prog.getCounter(Phase.LOADING_FSIMAGE, step);
+      for (int i = 0; i < numberOfPools; i++) {
+        addCachePool(FSImageSerialization.readCachePoolInfo(in));
+        counter.increment();
       }
       }
-      List<CacheDirective> directives =
-          directivesByPath.get(directive.getPath());
-      if (directives == null) {
-        directives = new LinkedList<CacheDirective>();
-        directivesByPath.put(directive.getPath(), directives);
+      prog.endStep(Phase.LOADING_FSIMAGE, step);
+    }
+
+    /**
+     * Load cache directives from the fsimage
+     */
+    private void loadDirectives(DataInput in) throws IOException {
+      StartupProgress prog = NameNode.getStartupProgress();
+      Step step = new Step(StepType.CACHE_ENTRIES);
+      prog.beginStep(Phase.LOADING_FSIMAGE, step);
+      int numDirectives = in.readInt();
+      prog.setTotal(Phase.LOADING_FSIMAGE, step, numDirectives);
+      Counter counter = prog.getCounter(Phase.LOADING_FSIMAGE, step);
+      for (int i = 0; i < numDirectives; i++) {
+        CacheDirectiveInfo info = FSImageSerialization.readCacheDirectiveInfo(in);
+        // Get pool reference by looking it up in the map
+        final String poolName = info.getPool();
+        CachePool pool = cachePools.get(poolName);
+        if (pool == null) {
+          throw new IOException("Directive refers to pool " + poolName +
+              ", which does not exist.");
+        }
+        CacheDirective directive =
+            new CacheDirective(info.getId(), info.getPath().toUri().getPath(),
+                info.getReplication(), info.getExpiration().getAbsoluteMillis());
+        boolean addedDirective = pool.getDirectiveList().add(directive);
+        assert addedDirective;
+        if (directivesById.put(directive.getId(), directive) != null) {
+          throw new IOException("A directive with ID " + directive.getId() +
+              " already exists");
+        }
+        List<CacheDirective> directives =
+            directivesByPath.get(directive.getPath());
+        if (directives == null) {
+          directives = new LinkedList<CacheDirective>();
+          directivesByPath.put(directive.getPath(), directives);
+        }
+        directives.add(directive);
+        counter.increment();
       }
       }
-      directives.add(directive);
-      counter.increment();
+      prog.endStep(Phase.LOADING_FSIMAGE, step);
     }
     }
-    prog.endStep(Phase.LOADING_FSIMAGE, step);
   }
   }
 
 
   public void waitForRescanIfNeeded() {
   public void waitForRescanIfNeeded() {

+ 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

+ 20 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -417,6 +417,26 @@ public class FSDirectory implements Closeable {
     }
     }
   }
   }
   
   
+  /**
+   * Persist the new block (the last block of the given file).
+   */
+  void persistNewBlock(String path, INodeFile file) {
+    Preconditions.checkArgument(file.isUnderConstruction());
+    waitForReady();
+
+    writeLock();
+    try {
+      fsImage.getEditLog().logAddBlock(path, file);
+    } finally {
+      writeUnlock();
+    }
+    if (NameNode.stateChangeLog.isDebugEnabled()) {
+      NameNode.stateChangeLog.debug("DIR* FSDirectory.persistNewBlock: "
+          + path + " with new block " + file.getLastBlock().toString()
+          + ", current total block count is " + file.getBlocks().length);
+    }
+  }
+  
   /**
   /**
    * Close file.
    * Close file.
    */
    */

+ 13 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java

@@ -40,9 +40,11 @@ import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.Storage.FormatConfirmable;
 import org.apache.hadoop.hdfs.server.common.Storage.FormatConfirmable;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddBlockOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCacheDirectiveInfoOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCacheDirectiveInfoOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCachePoolOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCachePoolOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddOp;
@@ -710,6 +712,17 @@ public class FSEditLog implements LogsPurgeable {
     logEdit(op);
     logEdit(op);
   }
   }
   
   
+  public void logAddBlock(String path, INodeFile file) {
+    Preconditions.checkArgument(file.isUnderConstruction());
+    BlockInfo[] blocks = file.getBlocks();
+    Preconditions.checkState(blocks != null && blocks.length > 0);
+    BlockInfo pBlock = blocks.length > 1 ? blocks[blocks.length - 2] : null;
+    BlockInfo lastBlock = blocks[blocks.length - 1];
+    AddBlockOp op = AddBlockOp.getInstance(cache.get()).setPath(path)
+        .setPenultimateBlock(pBlock).setLastBlock(lastBlock);
+    logEdit(op);
+  }
+  
   public void logUpdateBlocks(String path, INodeFile file, boolean toLogRpcIds) {
   public void logUpdateBlocks(String path, INodeFile file, boolean toLogRpcIds) {
     Preconditions.checkArgument(file.isUnderConstruction());
     Preconditions.checkArgument(file.isUnderConstruction());
     UpdateBlocksOp op = UpdateBlocksOp.getInstance(cache.get())
     UpdateBlocksOp op = UpdateBlocksOp.getInstance(cache.get())

+ 51 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java

@@ -40,6 +40,7 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 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.common.Storage;
 import org.apache.hadoop.hdfs.server.common.Storage;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddBlockOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCacheDirectiveInfoOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCacheDirectiveInfoOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCachePoolOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCachePoolOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCloseOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCloseOp;
@@ -417,7 +418,18 @@ public class FSEditLogLoader {
       }
       }
       break;
       break;
     }
     }
-      
+    case OP_ADD_BLOCK: {
+      AddBlockOp addBlockOp = (AddBlockOp) op;
+      String path = addBlockOp.getPath();
+      if (FSNamesystem.LOG.isDebugEnabled()) {
+        FSNamesystem.LOG.debug(op.opCode + ": " + path +
+            " new block id : " + addBlockOp.getLastBlock().getBlockId());
+      }
+      INodeFile oldFile = INodeFile.valueOf(fsDir.getINode(path), path);
+      // add the new block to the INodeFile
+      addNewBlock(fsDir, addBlockOp, oldFile);
+      break;
+    }
     case OP_SET_REPLICATION: {
     case OP_SET_REPLICATION: {
       SetReplicationOp setReplicationOp = (SetReplicationOp)op;
       SetReplicationOp setReplicationOp = (SetReplicationOp)op;
       short replication = fsNamesys.getBlockManager().adjustReplication(
       short replication = fsNamesys.getBlockManager().adjustReplication(
@@ -722,6 +734,44 @@ public class FSEditLogLoader {
     return sb.toString();
     return sb.toString();
   }
   }
 
 
+  /**
+   * Add a new block into the given INodeFile
+   */
+  private void addNewBlock(FSDirectory fsDir, AddBlockOp op, INodeFile file)
+      throws IOException {
+    BlockInfo[] oldBlocks = file.getBlocks();
+    Block pBlock = op.getPenultimateBlock();
+    Block newBlock= op.getLastBlock();
+    
+    if (pBlock != null) { // the penultimate block is not null
+      Preconditions.checkState(oldBlocks != null && oldBlocks.length > 0);
+      // compare pBlock with the last block of oldBlocks
+      Block oldLastBlock = oldBlocks[oldBlocks.length - 1];
+      if (oldLastBlock.getBlockId() != pBlock.getBlockId()
+          || oldLastBlock.getGenerationStamp() != pBlock.getGenerationStamp()) {
+        throw new IOException(
+            "Mismatched block IDs or generation stamps for the old last block of file "
+                + op.getPath() + ", the old last block is " + oldLastBlock
+                + ", and the block read from editlog is " + pBlock);
+      }
+      
+      oldLastBlock.setNumBytes(pBlock.getNumBytes());
+      if (oldLastBlock instanceof BlockInfoUnderConstruction) {
+        fsNamesys.getBlockManager().forceCompleteBlock(file,
+            (BlockInfoUnderConstruction) oldLastBlock);
+        fsNamesys.getBlockManager().processQueuedMessagesForBlock(pBlock);
+      }
+    } else { // the penultimate block is null
+      Preconditions.checkState(oldBlocks == null || oldBlocks.length == 0);
+    }
+    // add the new block
+    BlockInfo newBI = new BlockInfoUnderConstruction(
+          newBlock, file.getBlockReplication());
+    fsNamesys.getBlockManager().addBlockCollection(newBI, file);
+    file.addBlock(newBI);
+    fsNamesys.getBlockManager().processQueuedMessagesForBlock(newBlock);
+  }
+  
   /**
   /**
    * Update in-memory data structures with new block information.
    * Update in-memory data structures with new block information.
    * @throws IOException
    * @throws IOException

+ 104 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java

@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.server.namenode;
 package org.apache.hadoop.hdfs.server.namenode;
 
 
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_ADD;
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_ADD;
+import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_ADD_BLOCK;
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_ADD_CACHE_DIRECTIVE;
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_ADD_CACHE_DIRECTIVE;
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_ADD_CACHE_POOL;
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_ADD_CACHE_POOL;
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_ALLOCATE_BLOCK_ID;
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_ALLOCATE_BLOCK_ID;
@@ -168,6 +169,7 @@ public abstract class FSEditLogOp {
       inst.put(OP_MODIFY_CACHE_POOL, new ModifyCachePoolOp());
       inst.put(OP_MODIFY_CACHE_POOL, new ModifyCachePoolOp());
       inst.put(OP_REMOVE_CACHE_POOL, new RemoveCachePoolOp());
       inst.put(OP_REMOVE_CACHE_POOL, new RemoveCachePoolOp());
 
 
+      inst.put(OP_ADD_BLOCK, new AddBlockOp());
       inst.put(OP_UPGRADE_MARKER, new UpgradeMarkerOp());
       inst.put(OP_UPGRADE_MARKER, new UpgradeMarkerOp());
     }
     }
     
     
@@ -603,6 +605,108 @@ public abstract class FSEditLogOp {
     }
     }
   }
   }
   
   
+  static class AddBlockOp extends FSEditLogOp {
+    private String path;
+    private Block penultimateBlock;
+    private Block lastBlock;
+    
+    private AddBlockOp() {
+      super(OP_ADD_BLOCK);
+    }
+    
+    static AddBlockOp getInstance(OpInstanceCache cache) {
+      return (AddBlockOp) cache.get(OP_ADD_BLOCK);
+    }
+    
+    AddBlockOp setPath(String path) {
+      this.path = path;
+      return this;
+    }
+    
+    public String getPath() {
+      return path;
+    }
+
+    AddBlockOp setPenultimateBlock(Block pBlock) {
+      this.penultimateBlock = pBlock;
+      return this;
+    }
+    
+    Block getPenultimateBlock() {
+      return penultimateBlock;
+    }
+    
+    AddBlockOp setLastBlock(Block lastBlock) {
+      this.lastBlock = lastBlock;
+      return this;
+    }
+    
+    Block getLastBlock() {
+      return lastBlock;
+    }
+
+    @Override
+    public void writeFields(DataOutputStream out) throws IOException {
+      FSImageSerialization.writeString(path, out);
+      int size = penultimateBlock != null ? 2 : 1;
+      Block[] blocks = new Block[size];
+      if (penultimateBlock != null) {
+        blocks[0] = penultimateBlock;
+      }
+      blocks[size - 1] = lastBlock;
+      FSImageSerialization.writeCompactBlockArray(blocks, out);
+      // clientId and callId
+      writeRpcIds(rpcClientId, rpcCallId, out);
+    }
+    
+    @Override
+    void readFields(DataInputStream in, int logVersion) throws IOException {
+      path = FSImageSerialization.readString(in);
+      Block[] blocks = FSImageSerialization.readCompactBlockArray(in,
+          logVersion);
+      Preconditions.checkState(blocks.length == 2 || blocks.length == 1);
+      penultimateBlock = blocks.length == 1 ? null : blocks[0];
+      lastBlock = blocks[blocks.length - 1];
+      readRpcIds(in, logVersion);
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder();
+      sb.append("AddBlockOp [path=")
+        .append(path)
+        .append(", penultimateBlock=")
+        .append(penultimateBlock == null ? "NULL" : penultimateBlock)
+        .append(", lastBlock=")
+        .append(lastBlock);
+      appendRpcIdsToString(sb, rpcClientId, rpcCallId);
+      sb.append("]");
+      return sb.toString();
+    }
+    
+    @Override
+    protected void toXml(ContentHandler contentHandler) throws SAXException {
+      XMLUtils.addSaxString(contentHandler, "PATH", path);
+      if (penultimateBlock != null) {
+        FSEditLogOp.blockToXml(contentHandler, penultimateBlock);
+      }
+      FSEditLogOp.blockToXml(contentHandler, lastBlock);
+      appendRpcIdsToXml(contentHandler, rpcClientId, rpcCallId);
+    }
+    
+    @Override 
+    void fromXml(Stanza st) throws InvalidXmlException {
+      this.path = st.getValue("PATH");
+      List<Stanza> blocks = st.getChildren("BLOCK");
+      int size = blocks.size();
+      Preconditions.checkState(size == 1 || size == 2);
+      this.penultimateBlock = size == 2 ? 
+          FSEditLogOp.blockFromXml(blocks.get(0)) : null;
+      this.lastBlock = FSEditLogOp.blockFromXml(blocks.get(size - 1));
+      readRpcIdsFromXml(st);
+    }
+  }
+  
   /**
   /**
    * {@literal @AtMostOnce} for {@link ClientProtocol#updatePipeline}, but 
    * {@literal @AtMostOnce} for {@link ClientProtocol#updatePipeline}, but 
    * {@literal @Idempotent} for some other ops.
    * {@literal @Idempotent} for some other ops.

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

@@ -60,13 +60,14 @@ 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_UPGRADE_MARKER             ((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),
+  OP_UPGRADE_MARKER             ((byte) 40),
 
 
   // 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);

+ 9 - 4
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;
@@ -261,6 +262,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();
@@ -870,7 +874,7 @@ public class FSImageFormat {
         //This must not happen if security is turned on.
         //This must not happen if security is turned on.
         return; 
         return; 
       }
       }
-      namesystem.loadSecretManagerState(in);
+      namesystem.loadSecretManagerStateCompat(in);
     }
     }
 
 
     private void loadCacheManagerState(DataInput in) throws IOException {
     private void loadCacheManagerState(DataInput in) throws IOException {
@@ -878,7 +882,7 @@ public class FSImageFormat {
       if (!LayoutVersion.supports(Feature.CACHING, imgVersion)) {
       if (!LayoutVersion.supports(Feature.CACHING, imgVersion)) {
         return;
         return;
       }
       }
-      namesystem.getCacheManager().loadState(in);
+      namesystem.getCacheManager().loadStateCompat(in);
     }
     }
 
 
     private int getLayoutVersion() {
     private int getLayoutVersion() {
@@ -990,6 +994,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
@@ -1032,9 +1037,9 @@ public class FSImageFormat {
         sourceNamesystem.saveFilesUnderConstruction(out, snapshotUCMap);
         sourceNamesystem.saveFilesUnderConstruction(out, snapshotUCMap);
         
         
         context.checkCancelled();
         context.checkCancelled();
-        sourceNamesystem.saveSecretManagerState(out, sdPath);
+        sourceNamesystem.saveSecretManagerStateCompat(out, sdPath);
         context.checkCancelled();
         context.checkCancelled();
-        sourceNamesystem.getCacheManager().saveState(out, sdPath);
+        sourceNamesystem.getCacheManager().saveStateCompat(out, sdPath);
         context.checkCancelled();
         context.checkCancelled();
         out.flush();
         out.flush();
         context.checkCancelled();
         context.checkCancelled();

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

@@ -2647,7 +2647,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       newBlock = createNewBlock();
       newBlock = createNewBlock();
       saveAllocatedBlock(src, inodesInPath, newBlock, targets);
       saveAllocatedBlock(src, inodesInPath, newBlock, targets);
 
 
-      dir.persistBlocks(src, pendingFile, false);
+      dir.persistNewBlock(src, pendingFile);
       offset = pendingFile.computeFileSize();
       offset = pendingFile.computeFileSize();
     } finally {
     } finally {
       writeUnlock();
       writeUnlock();
@@ -6239,16 +6239,16 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    * @param out save state of the secret manager
    * @param out save state of the secret manager
    * @param sdPath String storage directory path
    * @param sdPath String storage directory path
    */
    */
-  void saveSecretManagerState(DataOutputStream out, String sdPath)
+  void saveSecretManagerStateCompat(DataOutputStream out, String sdPath)
       throws IOException {
       throws IOException {
-    dtSecretManager.saveSecretManagerState(out, sdPath);
+    dtSecretManager.saveSecretManagerStateCompat(out, sdPath);
   }
   }
 
 
   /**
   /**
    * @param in load the state of secret manager from input stream
    * @param in load the state of secret manager from input stream
    */
    */
-  void loadSecretManagerState(DataInput in) throws IOException {
-    dtSecretManager.loadSecretManagerState(in);
+  void loadSecretManagerStateCompat(DataInput in) throws IOException {
+    dtSecretManager.loadSecretManagerStateCompat(in);
   }
   }
 
 
   /**
   /**

+ 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());

+ 47 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderLocal.java

@@ -604,6 +604,24 @@ public class TestBlockReaderLocal {
     }
     }
   }
   }
 
 
+  private static class TestBlockReaderLocalReadZeroBytes
+      extends BlockReaderLocalTest {
+    @Override
+    public void doTest(BlockReaderLocal reader, byte original[])
+        throws IOException {
+      byte emptyArr[] = new byte[0];
+      Assert.assertEquals(0, reader.read(emptyArr, 0, 0));
+      ByteBuffer emptyBuf = ByteBuffer.wrap(emptyArr);
+      Assert.assertEquals(0, reader.read(emptyBuf));
+      reader.skip(1);
+      Assert.assertEquals(0, reader.read(emptyArr, 0, 0));
+      Assert.assertEquals(0, reader.read(emptyBuf));
+      reader.skip(BlockReaderLocalTest.TEST_LENGTH - 1);
+      Assert.assertEquals(-1, reader.read(emptyArr, 0, 0));
+      Assert.assertEquals(-1, reader.read(emptyBuf));
+    }
+  }
+
   @Test
   @Test
   public void testBlockReaderLocalOnFileWithoutChecksum()
   public void testBlockReaderLocalOnFileWithoutChecksum()
       throws IOException {
       throws IOException {
@@ -631,6 +649,35 @@ public class TestBlockReaderLocal {
     runBlockReaderLocalTest(new TestBlockReaderLocalOnFileWithoutChecksum(),
     runBlockReaderLocalTest(new TestBlockReaderLocalOnFileWithoutChecksum(),
         false, 0);
         false, 0);
   }
   }
+  
+  @Test
+  public void testBlockReaderLocalReadZeroBytes()
+      throws IOException {
+    runBlockReaderLocalTest(new TestBlockReaderLocalReadZeroBytes(),
+        true, DFSConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT);
+  }
+
+  @Test
+  public void testBlockReaderLocalReadZeroBytesNoChecksum()
+      throws IOException {
+    runBlockReaderLocalTest(new TestBlockReaderLocalReadZeroBytes(),
+        false, DFSConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT);
+  }
+
+  @Test
+  public void testBlockReaderLocalReadZeroBytesNoReadahead()
+      throws IOException {
+    runBlockReaderLocalTest(new TestBlockReaderLocalReadZeroBytes(),
+        true, 0);
+  }
+
+  @Test
+  public void testBlockReaderLocalReadZeroBytesNoChecksumNoReadahead()
+      throws IOException {
+    runBlockReaderLocalTest(new TestBlockReaderLocalReadZeroBytes(),
+        false, 0);
+  }
+  
 
 
   @Test(timeout=60000)
   @Test(timeout=60000)
   public void TestStatisticsForShortCircuitLocalRead() throws Exception {
   public void TestStatisticsForShortCircuitLocalRead() throws Exception {

+ 23 - 12
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppendRestart.java

@@ -97,29 +97,30 @@ public class TestFileAppendRestart {
 
 
       counts = FSImageTestUtil.countEditLogOpTypes(editLog);
       counts = FSImageTestUtil.countEditLogOpTypes(editLog);
       // OP_ADD to create file
       // OP_ADD to create file
-      // OP_UPDATE_BLOCKS for first block
+      // OP_ADD_BLOCK for first block
       // OP_CLOSE to close file
       // OP_CLOSE to close file
       // OP_ADD to reopen file
       // OP_ADD to reopen file
-      // OP_UPDATE_BLOCKS for second block
+      // OP_ADD_BLOCK for second block
       // OP_CLOSE to close file
       // OP_CLOSE to close file
       assertEquals(2, (int)counts.get(FSEditLogOpCodes.OP_ADD).held);
       assertEquals(2, (int)counts.get(FSEditLogOpCodes.OP_ADD).held);
-      assertEquals(2, (int)counts.get(FSEditLogOpCodes.OP_UPDATE_BLOCKS).held);
+      assertEquals(2, (int)counts.get(FSEditLogOpCodes.OP_ADD_BLOCK).held);
       assertEquals(2, (int)counts.get(FSEditLogOpCodes.OP_CLOSE).held);
       assertEquals(2, (int)counts.get(FSEditLogOpCodes.OP_CLOSE).held);
 
 
       Path p2 = new Path("/not-block-boundaries");
       Path p2 = new Path("/not-block-boundaries");
       writeAndAppend(fs, p2, BLOCK_SIZE/2, BLOCK_SIZE);
       writeAndAppend(fs, p2, BLOCK_SIZE/2, BLOCK_SIZE);
       counts = FSImageTestUtil.countEditLogOpTypes(editLog);
       counts = FSImageTestUtil.countEditLogOpTypes(editLog);
       // OP_ADD to create file
       // OP_ADD to create file
-      // OP_UPDATE_BLOCKS for first block
+      // OP_ADD_BLOCK for first block
       // OP_CLOSE to close file
       // OP_CLOSE to close file
       // OP_ADD to re-establish the lease
       // OP_ADD to re-establish the lease
       // OP_UPDATE_BLOCKS from the updatePipeline call (increments genstamp of last block)
       // OP_UPDATE_BLOCKS from the updatePipeline call (increments genstamp of last block)
-      // OP_UPDATE_BLOCKS at the start of the second block
+      // OP_ADD_BLOCK at the start of the second block
       // OP_CLOSE to close file
       // OP_CLOSE to close file
-      // Total: 2 OP_ADDs, 3 OP_UPDATE_BLOCKS, and 2 OP_CLOSEs in addition
-      //        to the ones above
+      // Total: 2 OP_ADDs, 1 OP_UPDATE_BLOCKS, 2 OP_ADD_BLOCKs, and 2 OP_CLOSEs
+       //       in addition to the ones above
       assertEquals(2+2, (int)counts.get(FSEditLogOpCodes.OP_ADD).held);
       assertEquals(2+2, (int)counts.get(FSEditLogOpCodes.OP_ADD).held);
-      assertEquals(2+3, (int)counts.get(FSEditLogOpCodes.OP_UPDATE_BLOCKS).held);
+      assertEquals(1, (int)counts.get(FSEditLogOpCodes.OP_UPDATE_BLOCKS).held);
+      assertEquals(2+2, (int)counts.get(FSEditLogOpCodes.OP_ADD_BLOCK).held);
       assertEquals(2+2, (int)counts.get(FSEditLogOpCodes.OP_CLOSE).held);
       assertEquals(2+2, (int)counts.get(FSEditLogOpCodes.OP_CLOSE).held);
       
       
       cluster.restartNameNode();
       cluster.restartNameNode();
@@ -176,13 +177,14 @@ public class TestFileAppendRestart {
   }
   }
 
 
   /**
   /**
-   * Test to append to the file, when one of datanode in the existing pipeline is down.
-   * @throws Exception
+   * Test to append to the file, when one of datanode in the existing pipeline
+   * is down.
    */
    */
   @Test
   @Test
   public void testAppendWithPipelineRecovery() throws Exception {
   public void testAppendWithPipelineRecovery() throws Exception {
     Configuration conf = new Configuration();
     Configuration conf = new Configuration();
     MiniDFSCluster cluster = null;
     MiniDFSCluster cluster = null;
+    FSDataOutputStream out = null;
     try {
     try {
       cluster = new MiniDFSCluster.Builder(conf).manageDataDfsDirs(true)
       cluster = new MiniDFSCluster.Builder(conf).manageDataDfsDirs(true)
           .manageNameDfsDirs(true).numDataNodes(4)
           .manageNameDfsDirs(true).numDataNodes(4)
@@ -192,11 +194,20 @@ public class TestFileAppendRestart {
 
 
       DistributedFileSystem fs = cluster.getFileSystem();
       DistributedFileSystem fs = cluster.getFileSystem();
       Path path = new Path("/test1");
       Path path = new Path("/test1");
-      DFSTestUtil.createFile(fs, path, 1024, (short) 3, 1l);
+      
+      out = fs.create(path, true, BLOCK_SIZE, (short) 3, BLOCK_SIZE);
+      AppendTestUtil.write(out, 0, 1024);
+      out.close();
 
 
       cluster.stopDataNode(3);
       cluster.stopDataNode(3);
-      DFSTestUtil.appendFile(fs, path, "hello");
+      out = fs.append(path);
+      AppendTestUtil.write(out, 1024, 1024);
+      out.close();
+      
+      cluster.restartNameNode(true);
+      AppendTestUtil.check(fs, path, 2048);
     } finally {
     } finally {
+      IOUtils.closeStream(out);
       if (null != cluster) {
       if (null != cluster) {
         cluster.shutdown();
         cluster.shutdown();
       }
       }

+ 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);

+ 157 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlock.java

@@ -0,0 +1,157 @@
+/**
+ * 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.server.namenode;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+import java.util.EnumSet;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSOutputStream;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test AddBlockOp is written and read correctly
+ */
+public class TestAddBlock {
+  private static final short REPLICATION = 3;
+  private static final int BLOCKSIZE = 1024;
+  
+  private MiniDFSCluster cluster;
+  private Configuration conf;
+
+  @Before
+  public void setup() throws IOException {
+    conf = new Configuration();
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCKSIZE);
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPLICATION)
+        .build();
+    cluster.waitActive();
+  }
+  
+  @After
+  public void tearDown() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+  
+  /**
+   * Test adding new blocks. Restart the NameNode in the test to make sure the
+   * AddBlockOp in the editlog is applied correctly. 
+   */
+  @Test
+  public void testAddBlock() throws Exception {
+    DistributedFileSystem fs = cluster.getFileSystem();
+    
+    final Path file1 = new Path("/file1");
+    final Path file2 = new Path("/file2");
+    final Path file3 = new Path("/file3");
+    final Path file4 = new Path("/file4");
+    
+    DFSTestUtil.createFile(fs, file1, BLOCKSIZE - 1, REPLICATION, 0L);
+    DFSTestUtil.createFile(fs, file2, BLOCKSIZE, REPLICATION, 0L);
+    DFSTestUtil.createFile(fs, file3, BLOCKSIZE * 2 - 1, REPLICATION, 0L);
+    DFSTestUtil.createFile(fs, file4, BLOCKSIZE * 2, REPLICATION, 0L);
+    
+    // restart NameNode
+    cluster.restartNameNode(true);
+    FSDirectory fsdir = cluster.getNamesystem().getFSDirectory();
+    
+    // check file1
+    INodeFile file1Node = fsdir.getINode4Write(file1.toString()).asFile();
+    BlockInfo[] file1Blocks = file1Node.getBlocks();
+    assertEquals(1, file1Blocks.length);
+    assertEquals(BLOCKSIZE - 1, file1Blocks[0].getNumBytes());
+    assertEquals(BlockUCState.COMPLETE, file1Blocks[0].getBlockUCState());
+    
+    // check file2
+    INodeFile file2Node = fsdir.getINode4Write(file2.toString()).asFile();
+    BlockInfo[] file2Blocks = file2Node.getBlocks();
+    assertEquals(1, file2Blocks.length);
+    assertEquals(BLOCKSIZE, file2Blocks[0].getNumBytes());
+    assertEquals(BlockUCState.COMPLETE, file2Blocks[0].getBlockUCState());
+    
+    // check file3
+    INodeFile file3Node = fsdir.getINode4Write(file3.toString()).asFile();
+    BlockInfo[] file3Blocks = file3Node.getBlocks();
+    assertEquals(2, file3Blocks.length);
+    assertEquals(BLOCKSIZE, file3Blocks[0].getNumBytes());
+    assertEquals(BlockUCState.COMPLETE, file3Blocks[0].getBlockUCState());
+    assertEquals(BLOCKSIZE - 1, file3Blocks[1].getNumBytes());
+    assertEquals(BlockUCState.COMPLETE, file3Blocks[1].getBlockUCState());
+    
+    // check file4
+    INodeFile file4Node = fsdir.getINode4Write(file4.toString()).asFile();
+    BlockInfo[] file4Blocks = file4Node.getBlocks();
+    assertEquals(2, file4Blocks.length);
+    assertEquals(BLOCKSIZE, file4Blocks[0].getNumBytes());
+    assertEquals(BlockUCState.COMPLETE, file4Blocks[0].getBlockUCState());
+    assertEquals(BLOCKSIZE, file4Blocks[1].getNumBytes());
+    assertEquals(BlockUCState.COMPLETE, file4Blocks[1].getBlockUCState());
+  }
+  
+  /**
+   * Test adding new blocks but without closing the corresponding the file
+   */
+  @Test
+  public void testAddBlockUC() throws Exception {
+    DistributedFileSystem fs = cluster.getFileSystem();
+    final Path file1 = new Path("/file1");
+    DFSTestUtil.createFile(fs, file1, BLOCKSIZE - 1, REPLICATION, 0L);
+    
+    FSDataOutputStream out = null;
+    try {
+      // append files without closing the streams
+      out = fs.append(file1);
+      String appendContent = "appending-content";
+      out.writeBytes(appendContent);
+      ((DFSOutputStream) out.getWrappedStream()).hsync(
+          EnumSet.of(SyncFlag.UPDATE_LENGTH));
+      
+      // restart NN
+      cluster.restartNameNode(true);
+      FSDirectory fsdir = cluster.getNamesystem().getFSDirectory();
+      
+      INodeFile fileNode = fsdir.getINode4Write(file1.toString()).asFile();
+      BlockInfo[] fileBlocks = fileNode.getBlocks();
+      assertEquals(2, fileBlocks.length);
+      assertEquals(BLOCKSIZE, fileBlocks[0].getNumBytes());
+      assertEquals(BlockUCState.COMPLETE, fileBlocks[0].getBlockUCState());
+      assertEquals(appendContent.length() - 1, fileBlocks[1].getNumBytes());
+      assertEquals(BlockUCState.UNDER_CONSTRUCTION,
+          fileBlocks[1].getBlockUCState());
+    } finally {
+      if (out != null) {
+        out.close();
+      }
+    }
+  }
+}

+ 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());

+ 0 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/TestOfflineEditsViewer.java

@@ -21,7 +21,6 @@ package org.apache.hadoop.hdfs.tools.offlineEditsViewer;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertTrue;
 
 
-import java.io.File;
 import java.io.FileOutputStream;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.nio.ByteBuffer;

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

@@ -1,6 +1,10 @@
 <?xml version="1.0" encoding="UTF-8"?>
 <?xml version="1.0" encoding="UTF-8"?>
 <EDITS>
 <EDITS>
+<<<<<<< .working
   <EDITS_VERSION>-50</EDITS_VERSION>
   <EDITS_VERSION>-50</EDITS_VERSION>
+=======
+  <EDITS_VERSION>-51</EDITS_VERSION>
+>>>>>>> .merge-right.r1559304
   <RECORD>
   <RECORD>
     <OPCODE>OP_START_LOG_SEGMENT</OPCODE>
     <OPCODE>OP_START_LOG_SEGMENT</OPCODE>
     <DATA>
     <DATA>
@@ -13,8 +17,13 @@
       <TXID>2</TXID>
       <TXID>2</TXID>
       <DELEGATION_KEY>
       <DELEGATION_KEY>
         <KEY_ID>1</KEY_ID>
         <KEY_ID>1</KEY_ID>
+<<<<<<< .working
         <EXPIRY_DATE>1389421314720</EXPIRY_DATE>
         <EXPIRY_DATE>1389421314720</EXPIRY_DATE>
         <KEY>d2a03d66ebfac521</KEY>
         <KEY>d2a03d66ebfac521</KEY>
+=======
+        <EXPIRY_DATE>1390519460949</EXPIRY_DATE>
+        <KEY>dc8d30edc97df67d</KEY>
+>>>>>>> .merge-right.r1559304
       </DELEGATION_KEY>
       </DELEGATION_KEY>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
@@ -24,8 +33,13 @@
       <TXID>3</TXID>
       <TXID>3</TXID>
       <DELEGATION_KEY>
       <DELEGATION_KEY>
         <KEY_ID>2</KEY_ID>
         <KEY_ID>2</KEY_ID>
+<<<<<<< .working
         <EXPIRY_DATE>1389421314722</EXPIRY_DATE>
         <EXPIRY_DATE>1389421314722</EXPIRY_DATE>
         <KEY>ef94532092f55aef</KEY>
         <KEY>ef94532092f55aef</KEY>
+=======
+        <EXPIRY_DATE>1390519460952</EXPIRY_DATE>
+        <KEY>096bc20b6debed03</KEY>
+>>>>>>> .merge-right.r1559304
       </DELEGATION_KEY>
       </DELEGATION_KEY>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
@@ -35,20 +49,38 @@
       <TXID>4</TXID>
       <TXID>4</TXID>
       <LENGTH>0</LENGTH>
       <LENGTH>0</LENGTH>
       <INODEID>16386</INODEID>
       <INODEID>16386</INODEID>
-      <PATH>/file_create_u\0001;F431</PATH>
+      <PATH>/file_create</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
+<<<<<<< .working
       <MTIME>1388730115316</MTIME>
       <MTIME>1388730115316</MTIME>
       <ATIME>1388730115316</ATIME>
       <ATIME>1388730115316</ATIME>
+=======
+      <MTIME>1389828264873</MTIME>
+      <ATIME>1389828264873</ATIME>
+>>>>>>> .merge-right.r1559304
       <BLOCKSIZE>512</BLOCKSIZE>
       <BLOCKSIZE>512</BLOCKSIZE>
+<<<<<<< .working
       <CLIENT_NAME>DFSClient_NONMAPREDUCE_381408282_1</CLIENT_NAME>
       <CLIENT_NAME>DFSClient_NONMAPREDUCE_381408282_1</CLIENT_NAME>
+=======
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_16108824_1</CLIENT_NAME>
+>>>>>>> .merge-right.r1559304
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <PERMISSION_STATUS>
       <PERMISSION_STATUS>
+<<<<<<< .working
         <USERNAME>szetszwo</USERNAME>
         <USERNAME>szetszwo</USERNAME>
+=======
+        <USERNAME>jing</USERNAME>
+>>>>>>> .merge-right.r1559304
         <GROUPNAME>supergroup</GROUPNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
       </PERMISSION_STATUS>
+<<<<<<< .working
       <RPC_CLIENTID>8205c453-0c7f-4b25-955a-7786e56bce86</RPC_CLIENTID>
       <RPC_CLIENTID>8205c453-0c7f-4b25-955a-7786e56bce86</RPC_CLIENTID>
       <RPC_CALLID>6</RPC_CALLID>
       <RPC_CALLID>6</RPC_CALLID>
+=======
+      <RPC_CLIENTID>b5928e80-e373-4807-a688-f94483d08ce5</RPC_CLIENTID>
+      <RPC_CALLID>9</RPC_CALLID>
+>>>>>>> .merge-right.r1559304
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
   <RECORD>
   <RECORD>
@@ -57,15 +89,24 @@
       <TXID>5</TXID>
       <TXID>5</TXID>
       <LENGTH>0</LENGTH>
       <LENGTH>0</LENGTH>
       <INODEID>0</INODEID>
       <INODEID>0</INODEID>
-      <PATH>/file_create_u\0001;F431</PATH>
+      <PATH>/file_create</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
+<<<<<<< .working
       <MTIME>1388730115327</MTIME>
       <MTIME>1388730115327</MTIME>
       <ATIME>1388730115316</ATIME>
       <ATIME>1388730115316</ATIME>
+=======
+      <MTIME>1389828265699</MTIME>
+      <ATIME>1389828264873</ATIME>
+>>>>>>> .merge-right.r1559304
       <BLOCKSIZE>512</BLOCKSIZE>
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_MACHINE></CLIENT_MACHINE>
       <CLIENT_MACHINE></CLIENT_MACHINE>
       <PERMISSION_STATUS>
       <PERMISSION_STATUS>
+<<<<<<< .working
         <USERNAME>szetszwo</USERNAME>
         <USERNAME>szetszwo</USERNAME>
+=======
+        <USERNAME>jing</USERNAME>
+>>>>>>> .merge-right.r1559304
         <GROUPNAME>supergroup</GROUPNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
       </PERMISSION_STATUS>
@@ -76,11 +117,17 @@
     <DATA>
     <DATA>
       <TXID>6</TXID>
       <TXID>6</TXID>
       <LENGTH>0</LENGTH>
       <LENGTH>0</LENGTH>
-      <SRC>/file_create_u\0001;F431</SRC>
+      <SRC>/file_create</SRC>
       <DST>/file_moved</DST>
       <DST>/file_moved</DST>
+<<<<<<< .working
       <TIMESTAMP>1388730115331</TIMESTAMP>
       <TIMESTAMP>1388730115331</TIMESTAMP>
       <RPC_CLIENTID>8205c453-0c7f-4b25-955a-7786e56bce86</RPC_CLIENTID>
       <RPC_CLIENTID>8205c453-0c7f-4b25-955a-7786e56bce86</RPC_CLIENTID>
       <RPC_CALLID>8</RPC_CALLID>
       <RPC_CALLID>8</RPC_CALLID>
+=======
+      <TIMESTAMP>1389828265705</TIMESTAMP>
+      <RPC_CLIENTID>b5928e80-e373-4807-a688-f94483d08ce5</RPC_CLIENTID>
+      <RPC_CALLID>11</RPC_CALLID>
+>>>>>>> .merge-right.r1559304
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
   <RECORD>
   <RECORD>
@@ -89,9 +136,15 @@
       <TXID>7</TXID>
       <TXID>7</TXID>
       <LENGTH>0</LENGTH>
       <LENGTH>0</LENGTH>
       <PATH>/file_moved</PATH>
       <PATH>/file_moved</PATH>
+<<<<<<< .working
       <TIMESTAMP>1388730115336</TIMESTAMP>
       <TIMESTAMP>1388730115336</TIMESTAMP>
       <RPC_CLIENTID>8205c453-0c7f-4b25-955a-7786e56bce86</RPC_CLIENTID>
       <RPC_CLIENTID>8205c453-0c7f-4b25-955a-7786e56bce86</RPC_CLIENTID>
       <RPC_CALLID>9</RPC_CALLID>
       <RPC_CALLID>9</RPC_CALLID>
+=======
+      <TIMESTAMP>1389828265712</TIMESTAMP>
+      <RPC_CLIENTID>b5928e80-e373-4807-a688-f94483d08ce5</RPC_CLIENTID>
+      <RPC_CALLID>12</RPC_CALLID>
+>>>>>>> .merge-right.r1559304
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
   <RECORD>
   <RECORD>
@@ -101,9 +154,17 @@
       <LENGTH>0</LENGTH>
       <LENGTH>0</LENGTH>
       <INODEID>16387</INODEID>
       <INODEID>16387</INODEID>
       <PATH>/directory_mkdir</PATH>
       <PATH>/directory_mkdir</PATH>
+<<<<<<< .working
       <TIMESTAMP>1388730115342</TIMESTAMP>
       <TIMESTAMP>1388730115342</TIMESTAMP>
+=======
+      <TIMESTAMP>1389828265722</TIMESTAMP>
+>>>>>>> .merge-right.r1559304
       <PERMISSION_STATUS>
       <PERMISSION_STATUS>
+<<<<<<< .working
         <USERNAME>szetszwo</USERNAME>
         <USERNAME>szetszwo</USERNAME>
+=======
+        <USERNAME>jing</USERNAME>
+>>>>>>> .merge-right.r1559304
         <GROUPNAME>supergroup</GROUPNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>493</MODE>
         <MODE>493</MODE>
       </PERMISSION_STATUS>
       </PERMISSION_STATUS>
@@ -136,8 +197,13 @@
       <TXID>12</TXID>
       <TXID>12</TXID>
       <SNAPSHOTROOT>/directory_mkdir</SNAPSHOTROOT>
       <SNAPSHOTROOT>/directory_mkdir</SNAPSHOTROOT>
       <SNAPSHOTNAME>snapshot1</SNAPSHOTNAME>
       <SNAPSHOTNAME>snapshot1</SNAPSHOTNAME>
+<<<<<<< .working
       <RPC_CLIENTID>8205c453-0c7f-4b25-955a-7786e56bce86</RPC_CLIENTID>
       <RPC_CLIENTID>8205c453-0c7f-4b25-955a-7786e56bce86</RPC_CLIENTID>
       <RPC_CALLID>14</RPC_CALLID>
       <RPC_CALLID>14</RPC_CALLID>
+=======
+      <RPC_CLIENTID>b5928e80-e373-4807-a688-f94483d08ce5</RPC_CLIENTID>
+      <RPC_CALLID>17</RPC_CALLID>
+>>>>>>> .merge-right.r1559304
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
   <RECORD>
   <RECORD>
@@ -147,8 +213,13 @@
       <SNAPSHOTROOT>/directory_mkdir</SNAPSHOTROOT>
       <SNAPSHOTROOT>/directory_mkdir</SNAPSHOTROOT>
       <SNAPSHOTOLDNAME>snapshot1</SNAPSHOTOLDNAME>
       <SNAPSHOTOLDNAME>snapshot1</SNAPSHOTOLDNAME>
       <SNAPSHOTNEWNAME>snapshot2</SNAPSHOTNEWNAME>
       <SNAPSHOTNEWNAME>snapshot2</SNAPSHOTNEWNAME>
+<<<<<<< .working
       <RPC_CLIENTID>8205c453-0c7f-4b25-955a-7786e56bce86</RPC_CLIENTID>
       <RPC_CLIENTID>8205c453-0c7f-4b25-955a-7786e56bce86</RPC_CLIENTID>
       <RPC_CALLID>15</RPC_CALLID>
       <RPC_CALLID>15</RPC_CALLID>
+=======
+      <RPC_CLIENTID>b5928e80-e373-4807-a688-f94483d08ce5</RPC_CLIENTID>
+      <RPC_CALLID>18</RPC_CALLID>
+>>>>>>> .merge-right.r1559304
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
   <RECORD>
   <RECORD>
@@ -157,8 +228,13 @@
       <TXID>14</TXID>
       <TXID>14</TXID>
       <SNAPSHOTROOT>/directory_mkdir</SNAPSHOTROOT>
       <SNAPSHOTROOT>/directory_mkdir</SNAPSHOTROOT>
       <SNAPSHOTNAME>snapshot2</SNAPSHOTNAME>
       <SNAPSHOTNAME>snapshot2</SNAPSHOTNAME>
+<<<<<<< .working
       <RPC_CLIENTID>8205c453-0c7f-4b25-955a-7786e56bce86</RPC_CLIENTID>
       <RPC_CLIENTID>8205c453-0c7f-4b25-955a-7786e56bce86</RPC_CLIENTID>
       <RPC_CALLID>16</RPC_CALLID>
       <RPC_CALLID>16</RPC_CALLID>
+=======
+      <RPC_CLIENTID>b5928e80-e373-4807-a688-f94483d08ce5</RPC_CLIENTID>
+      <RPC_CALLID>19</RPC_CALLID>
+>>>>>>> .merge-right.r1559304
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
   <RECORD>
   <RECORD>
@@ -167,20 +243,38 @@
       <TXID>15</TXID>
       <TXID>15</TXID>
       <LENGTH>0</LENGTH>
       <LENGTH>0</LENGTH>
       <INODEID>16388</INODEID>
       <INODEID>16388</INODEID>
-      <PATH>/file_create_u\0001;F431</PATH>
+      <PATH>/file_create</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
+<<<<<<< .working
       <MTIME>1388730115362</MTIME>
       <MTIME>1388730115362</MTIME>
       <ATIME>1388730115362</ATIME>
       <ATIME>1388730115362</ATIME>
+=======
+      <MTIME>1389828265757</MTIME>
+      <ATIME>1389828265757</ATIME>
+>>>>>>> .merge-right.r1559304
       <BLOCKSIZE>512</BLOCKSIZE>
       <BLOCKSIZE>512</BLOCKSIZE>
+<<<<<<< .working
       <CLIENT_NAME>DFSClient_NONMAPREDUCE_381408282_1</CLIENT_NAME>
       <CLIENT_NAME>DFSClient_NONMAPREDUCE_381408282_1</CLIENT_NAME>
+=======
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_16108824_1</CLIENT_NAME>
+>>>>>>> .merge-right.r1559304
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <PERMISSION_STATUS>
       <PERMISSION_STATUS>
+<<<<<<< .working
         <USERNAME>szetszwo</USERNAME>
         <USERNAME>szetszwo</USERNAME>
+=======
+        <USERNAME>jing</USERNAME>
+>>>>>>> .merge-right.r1559304
         <GROUPNAME>supergroup</GROUPNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
       </PERMISSION_STATUS>
+<<<<<<< .working
       <RPC_CLIENTID>8205c453-0c7f-4b25-955a-7786e56bce86</RPC_CLIENTID>
       <RPC_CLIENTID>8205c453-0c7f-4b25-955a-7786e56bce86</RPC_CLIENTID>
       <RPC_CALLID>17</RPC_CALLID>
       <RPC_CALLID>17</RPC_CALLID>
+=======
+      <RPC_CLIENTID>b5928e80-e373-4807-a688-f94483d08ce5</RPC_CLIENTID>
+      <RPC_CALLID>20</RPC_CALLID>
+>>>>>>> .merge-right.r1559304
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
   <RECORD>
   <RECORD>
@@ -189,15 +283,24 @@
       <TXID>16</TXID>
       <TXID>16</TXID>
       <LENGTH>0</LENGTH>
       <LENGTH>0</LENGTH>
       <INODEID>0</INODEID>
       <INODEID>0</INODEID>
-      <PATH>/file_create_u\0001;F431</PATH>
+      <PATH>/file_create</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
+<<<<<<< .working
       <MTIME>1388730115363</MTIME>
       <MTIME>1388730115363</MTIME>
       <ATIME>1388730115362</ATIME>
       <ATIME>1388730115362</ATIME>
+=======
+      <MTIME>1389828265759</MTIME>
+      <ATIME>1389828265757</ATIME>
+>>>>>>> .merge-right.r1559304
       <BLOCKSIZE>512</BLOCKSIZE>
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_MACHINE></CLIENT_MACHINE>
       <CLIENT_MACHINE></CLIENT_MACHINE>
       <PERMISSION_STATUS>
       <PERMISSION_STATUS>
+<<<<<<< .working
         <USERNAME>szetszwo</USERNAME>
         <USERNAME>szetszwo</USERNAME>
+=======
+        <USERNAME>jing</USERNAME>
+>>>>>>> .merge-right.r1559304
         <GROUPNAME>supergroup</GROUPNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
       </PERMISSION_STATUS>
@@ -207,7 +310,7 @@
     <OPCODE>OP_SET_REPLICATION</OPCODE>
     <OPCODE>OP_SET_REPLICATION</OPCODE>
     <DATA>
     <DATA>
       <TXID>17</TXID>
       <TXID>17</TXID>
-      <PATH>/file_create_u\0001;F431</PATH>
+      <PATH>/file_create</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
@@ -215,7 +318,7 @@
     <OPCODE>OP_SET_PERMISSIONS</OPCODE>
     <OPCODE>OP_SET_PERMISSIONS</OPCODE>
     <DATA>
     <DATA>
       <TXID>18</TXID>
       <TXID>18</TXID>
-      <SRC>/file_create_u\0001;F431</SRC>
+      <SRC>/file_create</SRC>
       <MODE>511</MODE>
       <MODE>511</MODE>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
@@ -223,7 +326,7 @@
     <OPCODE>OP_SET_OWNER</OPCODE>
     <OPCODE>OP_SET_OWNER</OPCODE>
     <DATA>
     <DATA>
       <TXID>19</TXID>
       <TXID>19</TXID>
-      <SRC>/file_create_u\0001;F431</SRC>
+      <SRC>/file_create</SRC>
       <USERNAME>newOwner</USERNAME>
       <USERNAME>newOwner</USERNAME>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
@@ -232,7 +335,7 @@
     <DATA>
     <DATA>
       <TXID>20</TXID>
       <TXID>20</TXID>
       <LENGTH>0</LENGTH>
       <LENGTH>0</LENGTH>
-      <PATH>/file_create_u\0001;F431</PATH>
+      <PATH>/file_create</PATH>
       <MTIME>1285195527000</MTIME>
       <MTIME>1285195527000</MTIME>
       <ATIME>1285195527000</ATIME>
       <ATIME>1285195527000</ATIME>
     </DATA>
     </DATA>
@@ -251,12 +354,21 @@
     <DATA>
     <DATA>
       <TXID>22</TXID>
       <TXID>22</TXID>
       <LENGTH>0</LENGTH>
       <LENGTH>0</LENGTH>
-      <SRC>/file_create_u\0001;F431</SRC>
+      <SRC>/file_create</SRC>
       <DST>/file_moved</DST>
       <DST>/file_moved</DST>
+<<<<<<< .working
       <TIMESTAMP>1388730115378</TIMESTAMP>
       <TIMESTAMP>1388730115378</TIMESTAMP>
+=======
+      <TIMESTAMP>1389828265782</TIMESTAMP>
+>>>>>>> .merge-right.r1559304
       <OPTIONS>NONE</OPTIONS>
       <OPTIONS>NONE</OPTIONS>
+<<<<<<< .working
       <RPC_CLIENTID>8205c453-0c7f-4b25-955a-7786e56bce86</RPC_CLIENTID>
       <RPC_CLIENTID>8205c453-0c7f-4b25-955a-7786e56bce86</RPC_CLIENTID>
       <RPC_CALLID>24</RPC_CALLID>
       <RPC_CALLID>24</RPC_CALLID>
+=======
+      <RPC_CLIENTID>b5928e80-e373-4807-a688-f94483d08ce5</RPC_CLIENTID>
+      <RPC_CALLID>27</RPC_CALLID>
+>>>>>>> .merge-right.r1559304
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
   <RECORD>
   <RECORD>
@@ -267,18 +379,36 @@
       <INODEID>16389</INODEID>
       <INODEID>16389</INODEID>
       <PATH>/file_concat_target</PATH>
       <PATH>/file_concat_target</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
+<<<<<<< .working
       <MTIME>1388730115382</MTIME>
       <MTIME>1388730115382</MTIME>
       <ATIME>1388730115382</ATIME>
       <ATIME>1388730115382</ATIME>
+=======
+      <MTIME>1389828265787</MTIME>
+      <ATIME>1389828265787</ATIME>
+>>>>>>> .merge-right.r1559304
       <BLOCKSIZE>512</BLOCKSIZE>
       <BLOCKSIZE>512</BLOCKSIZE>
+<<<<<<< .working
       <CLIENT_NAME>DFSClient_NONMAPREDUCE_381408282_1</CLIENT_NAME>
       <CLIENT_NAME>DFSClient_NONMAPREDUCE_381408282_1</CLIENT_NAME>
+=======
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_16108824_1</CLIENT_NAME>
+>>>>>>> .merge-right.r1559304
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <PERMISSION_STATUS>
       <PERMISSION_STATUS>
+<<<<<<< .working
         <USERNAME>szetszwo</USERNAME>
         <USERNAME>szetszwo</USERNAME>
+=======
+        <USERNAME>jing</USERNAME>
+>>>>>>> .merge-right.r1559304
         <GROUPNAME>supergroup</GROUPNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
       </PERMISSION_STATUS>
+<<<<<<< .working
       <RPC_CLIENTID>8205c453-0c7f-4b25-955a-7786e56bce86</RPC_CLIENTID>
       <RPC_CLIENTID>8205c453-0c7f-4b25-955a-7786e56bce86</RPC_CLIENTID>
       <RPC_CALLID>26</RPC_CALLID>
       <RPC_CALLID>26</RPC_CALLID>
+=======
+      <RPC_CLIENTID>b5928e80-e373-4807-a688-f94483d08ce5</RPC_CLIENTID>
+      <RPC_CALLID>29</RPC_CALLID>
+>>>>>>> .merge-right.r1559304
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
   <RECORD>
   <RECORD>
@@ -296,7 +426,7 @@
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
   <RECORD>
   <RECORD>
-    <OPCODE>OP_UPDATE_BLOCKS</OPCODE>
+    <OPCODE>OP_ADD_BLOCK</OPCODE>
     <DATA>
     <DATA>
       <TXID>26</TXID>
       <TXID>26</TXID>
       <PATH>/file_concat_target</PATH>
       <PATH>/file_concat_target</PATH>
@@ -324,7 +454,7 @@
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
   <RECORD>
   <RECORD>
-    <OPCODE>OP_UPDATE_BLOCKS</OPCODE>
+    <OPCODE>OP_ADD_BLOCK</OPCODE>
     <DATA>
     <DATA>
       <TXID>29</TXID>
       <TXID>29</TXID>
       <PATH>/file_concat_target</PATH>
       <PATH>/file_concat_target</PATH>
@@ -357,15 +487,10 @@
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
   <RECORD>
   <RECORD>
-    <OPCODE>OP_UPDATE_BLOCKS</OPCODE>
+    <OPCODE>OP_ADD_BLOCK</OPCODE>
     <DATA>
     <DATA>
       <TXID>32</TXID>
       <TXID>32</TXID>
       <PATH>/file_concat_target</PATH>
       <PATH>/file_concat_target</PATH>
-      <BLOCK>
-        <BLOCK_ID>1073741825</BLOCK_ID>
-        <NUM_BYTES>512</NUM_BYTES>
-        <GENSTAMP>1001</GENSTAMP>
-      </BLOCK>
       <BLOCK>
       <BLOCK>
         <BLOCK_ID>1073741826</BLOCK_ID>
         <BLOCK_ID>1073741826</BLOCK_ID>
         <NUM_BYTES>512</NUM_BYTES>
         <NUM_BYTES>512</NUM_BYTES>
@@ -388,8 +513,13 @@
       <INODEID>0</INODEID>
       <INODEID>0</INODEID>
       <PATH>/file_concat_target</PATH>
       <PATH>/file_concat_target</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
+<<<<<<< .working
       <MTIME>1388730115461</MTIME>
       <MTIME>1388730115461</MTIME>
       <ATIME>1388730115382</ATIME>
       <ATIME>1388730115382</ATIME>
+=======
+      <MTIME>1389828266540</MTIME>
+      <ATIME>1389828265787</ATIME>
+>>>>>>> .merge-right.r1559304
       <BLOCKSIZE>512</BLOCKSIZE>
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_MACHINE></CLIENT_MACHINE>
       <CLIENT_MACHINE></CLIENT_MACHINE>
@@ -409,7 +539,11 @@
         <GENSTAMP>1003</GENSTAMP>
         <GENSTAMP>1003</GENSTAMP>
       </BLOCK>
       </BLOCK>
       <PERMISSION_STATUS>
       <PERMISSION_STATUS>
+<<<<<<< .working
         <USERNAME>szetszwo</USERNAME>
         <USERNAME>szetszwo</USERNAME>
+=======
+        <USERNAME>jing</USERNAME>
+>>>>>>> .merge-right.r1559304
         <GROUPNAME>supergroup</GROUPNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
       </PERMISSION_STATUS>
@@ -423,18 +557,36 @@
       <INODEID>16390</INODEID>
       <INODEID>16390</INODEID>
       <PATH>/file_concat_0</PATH>
       <PATH>/file_concat_0</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
+<<<<<<< .working
       <MTIME>1388730115463</MTIME>
       <MTIME>1388730115463</MTIME>
       <ATIME>1388730115463</ATIME>
       <ATIME>1388730115463</ATIME>
+=======
+      <MTIME>1389828266544</MTIME>
+      <ATIME>1389828266544</ATIME>
+>>>>>>> .merge-right.r1559304
       <BLOCKSIZE>512</BLOCKSIZE>
       <BLOCKSIZE>512</BLOCKSIZE>
+<<<<<<< .working
       <CLIENT_NAME>DFSClient_NONMAPREDUCE_381408282_1</CLIENT_NAME>
       <CLIENT_NAME>DFSClient_NONMAPREDUCE_381408282_1</CLIENT_NAME>
+=======
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_16108824_1</CLIENT_NAME>
+>>>>>>> .merge-right.r1559304
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <PERMISSION_STATUS>
       <PERMISSION_STATUS>
+<<<<<<< .working
         <USERNAME>szetszwo</USERNAME>
         <USERNAME>szetszwo</USERNAME>
+=======
+        <USERNAME>jing</USERNAME>
+>>>>>>> .merge-right.r1559304
         <GROUPNAME>supergroup</GROUPNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
       </PERMISSION_STATUS>
+<<<<<<< .working
       <RPC_CLIENTID>8205c453-0c7f-4b25-955a-7786e56bce86</RPC_CLIENTID>
       <RPC_CLIENTID>8205c453-0c7f-4b25-955a-7786e56bce86</RPC_CLIENTID>
       <RPC_CALLID>39</RPC_CALLID>
       <RPC_CALLID>39</RPC_CALLID>
+=======
+      <RPC_CLIENTID>b5928e80-e373-4807-a688-f94483d08ce5</RPC_CLIENTID>
+      <RPC_CALLID>41</RPC_CALLID>
+>>>>>>> .merge-right.r1559304
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
   <RECORD>
   <RECORD>
@@ -452,7 +604,7 @@
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
   <RECORD>
   <RECORD>
-    <OPCODE>OP_UPDATE_BLOCKS</OPCODE>
+    <OPCODE>OP_ADD_BLOCK</OPCODE>
     <DATA>
     <DATA>
       <TXID>37</TXID>
       <TXID>37</TXID>
       <PATH>/file_concat_0</PATH>
       <PATH>/file_concat_0</PATH>
@@ -480,7 +632,7 @@
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
   <RECORD>
   <RECORD>
-    <OPCODE>OP_UPDATE_BLOCKS</OPCODE>
+    <OPCODE>OP_ADD_BLOCK</OPCODE>
     <DATA>
     <DATA>
       <TXID>40</TXID>
       <TXID>40</TXID>
       <PATH>/file_concat_0</PATH>
       <PATH>/file_concat_0</PATH>
@@ -513,15 +665,10 @@
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
   <RECORD>
   <RECORD>
-    <OPCODE>OP_UPDATE_BLOCKS</OPCODE>
+    <OPCODE>OP_ADD_BLOCK</OPCODE>
     <DATA>
     <DATA>
       <TXID>43</TXID>
       <TXID>43</TXID>
       <PATH>/file_concat_0</PATH>
       <PATH>/file_concat_0</PATH>
-      <BLOCK>
-        <BLOCK_ID>1073741828</BLOCK_ID>
-        <NUM_BYTES>512</NUM_BYTES>
-        <GENSTAMP>1004</GENSTAMP>
-      </BLOCK>
       <BLOCK>
       <BLOCK>
         <BLOCK_ID>1073741829</BLOCK_ID>
         <BLOCK_ID>1073741829</BLOCK_ID>
         <NUM_BYTES>512</NUM_BYTES>
         <NUM_BYTES>512</NUM_BYTES>
@@ -544,8 +691,13 @@
       <INODEID>0</INODEID>
       <INODEID>0</INODEID>
       <PATH>/file_concat_0</PATH>
       <PATH>/file_concat_0</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
+<<<<<<< .working
       <MTIME>1388730115477</MTIME>
       <MTIME>1388730115477</MTIME>
       <ATIME>1388730115463</ATIME>
       <ATIME>1388730115463</ATIME>
+=======
+      <MTIME>1389828266569</MTIME>
+      <ATIME>1389828266544</ATIME>
+>>>>>>> .merge-right.r1559304
       <BLOCKSIZE>512</BLOCKSIZE>
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_MACHINE></CLIENT_MACHINE>
       <CLIENT_MACHINE></CLIENT_MACHINE>
@@ -565,7 +717,11 @@
         <GENSTAMP>1006</GENSTAMP>
         <GENSTAMP>1006</GENSTAMP>
       </BLOCK>
       </BLOCK>
       <PERMISSION_STATUS>
       <PERMISSION_STATUS>
+<<<<<<< .working
         <USERNAME>szetszwo</USERNAME>
         <USERNAME>szetszwo</USERNAME>
+=======
+        <USERNAME>jing</USERNAME>
+>>>>>>> .merge-right.r1559304
         <GROUPNAME>supergroup</GROUPNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
       </PERMISSION_STATUS>
@@ -579,18 +735,36 @@
       <INODEID>16391</INODEID>
       <INODEID>16391</INODEID>
       <PATH>/file_concat_1</PATH>
       <PATH>/file_concat_1</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
+<<<<<<< .working
       <MTIME>1388730115479</MTIME>
       <MTIME>1388730115479</MTIME>
       <ATIME>1388730115479</ATIME>
       <ATIME>1388730115479</ATIME>
+=======
+      <MTIME>1389828266572</MTIME>
+      <ATIME>1389828266572</ATIME>
+>>>>>>> .merge-right.r1559304
       <BLOCKSIZE>512</BLOCKSIZE>
       <BLOCKSIZE>512</BLOCKSIZE>
+<<<<<<< .working
       <CLIENT_NAME>DFSClient_NONMAPREDUCE_381408282_1</CLIENT_NAME>
       <CLIENT_NAME>DFSClient_NONMAPREDUCE_381408282_1</CLIENT_NAME>
+=======
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_16108824_1</CLIENT_NAME>
+>>>>>>> .merge-right.r1559304
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <PERMISSION_STATUS>
       <PERMISSION_STATUS>
+<<<<<<< .working
         <USERNAME>szetszwo</USERNAME>
         <USERNAME>szetszwo</USERNAME>
+=======
+        <USERNAME>jing</USERNAME>
+>>>>>>> .merge-right.r1559304
         <GROUPNAME>supergroup</GROUPNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
       </PERMISSION_STATUS>
+<<<<<<< .working
       <RPC_CLIENTID>8205c453-0c7f-4b25-955a-7786e56bce86</RPC_CLIENTID>
       <RPC_CLIENTID>8205c453-0c7f-4b25-955a-7786e56bce86</RPC_CLIENTID>
       <RPC_CALLID>51</RPC_CALLID>
       <RPC_CALLID>51</RPC_CALLID>
+=======
+      <RPC_CLIENTID>b5928e80-e373-4807-a688-f94483d08ce5</RPC_CLIENTID>
+      <RPC_CALLID>53</RPC_CALLID>
+>>>>>>> .merge-right.r1559304
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
   <RECORD>
   <RECORD>
@@ -608,7 +782,7 @@
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
   <RECORD>
   <RECORD>
-    <OPCODE>OP_UPDATE_BLOCKS</OPCODE>
+    <OPCODE>OP_ADD_BLOCK</OPCODE>
     <DATA>
     <DATA>
       <TXID>48</TXID>
       <TXID>48</TXID>
       <PATH>/file_concat_1</PATH>
       <PATH>/file_concat_1</PATH>
@@ -636,7 +810,7 @@
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
   <RECORD>
   <RECORD>
-    <OPCODE>OP_UPDATE_BLOCKS</OPCODE>
+    <OPCODE>OP_ADD_BLOCK</OPCODE>
     <DATA>
     <DATA>
       <TXID>51</TXID>
       <TXID>51</TXID>
       <PATH>/file_concat_1</PATH>
       <PATH>/file_concat_1</PATH>
@@ -669,15 +843,10 @@
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
   <RECORD>
   <RECORD>
-    <OPCODE>OP_UPDATE_BLOCKS</OPCODE>
+    <OPCODE>OP_ADD_BLOCK</OPCODE>
     <DATA>
     <DATA>
       <TXID>54</TXID>
       <TXID>54</TXID>
       <PATH>/file_concat_1</PATH>
       <PATH>/file_concat_1</PATH>
-      <BLOCK>
-        <BLOCK_ID>1073741831</BLOCK_ID>
-        <NUM_BYTES>512</NUM_BYTES>
-        <GENSTAMP>1007</GENSTAMP>
-      </BLOCK>
       <BLOCK>
       <BLOCK>
         <BLOCK_ID>1073741832</BLOCK_ID>
         <BLOCK_ID>1073741832</BLOCK_ID>
         <NUM_BYTES>512</NUM_BYTES>
         <NUM_BYTES>512</NUM_BYTES>
@@ -700,8 +869,13 @@
       <INODEID>0</INODEID>
       <INODEID>0</INODEID>
       <PATH>/file_concat_1</PATH>
       <PATH>/file_concat_1</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
+<<<<<<< .working
       <MTIME>1388730115495</MTIME>
       <MTIME>1388730115495</MTIME>
       <ATIME>1388730115479</ATIME>
       <ATIME>1388730115479</ATIME>
+=======
+      <MTIME>1389828266599</MTIME>
+      <ATIME>1389828266572</ATIME>
+>>>>>>> .merge-right.r1559304
       <BLOCKSIZE>512</BLOCKSIZE>
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_MACHINE></CLIENT_MACHINE>
       <CLIENT_MACHINE></CLIENT_MACHINE>
@@ -721,7 +895,11 @@
         <GENSTAMP>1009</GENSTAMP>
         <GENSTAMP>1009</GENSTAMP>
       </BLOCK>
       </BLOCK>
       <PERMISSION_STATUS>
       <PERMISSION_STATUS>
+<<<<<<< .working
         <USERNAME>szetszwo</USERNAME>
         <USERNAME>szetszwo</USERNAME>
+=======
+        <USERNAME>jing</USERNAME>
+>>>>>>> .merge-right.r1559304
         <GROUPNAME>supergroup</GROUPNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
       </PERMISSION_STATUS>
@@ -733,13 +911,22 @@
       <TXID>56</TXID>
       <TXID>56</TXID>
       <LENGTH>0</LENGTH>
       <LENGTH>0</LENGTH>
       <TRG>/file_concat_target</TRG>
       <TRG>/file_concat_target</TRG>
+<<<<<<< .working
       <TIMESTAMP>1388730115498</TIMESTAMP>
       <TIMESTAMP>1388730115498</TIMESTAMP>
+=======
+      <TIMESTAMP>1389828266603</TIMESTAMP>
+>>>>>>> .merge-right.r1559304
       <SOURCES>
       <SOURCES>
         <SOURCE1>/file_concat_0</SOURCE1>
         <SOURCE1>/file_concat_0</SOURCE1>
         <SOURCE2>/file_concat_1</SOURCE2>
         <SOURCE2>/file_concat_1</SOURCE2>
       </SOURCES>
       </SOURCES>
+<<<<<<< .working
       <RPC_CLIENTID>8205c453-0c7f-4b25-955a-7786e56bce86</RPC_CLIENTID>
       <RPC_CLIENTID>8205c453-0c7f-4b25-955a-7786e56bce86</RPC_CLIENTID>
       <RPC_CALLID>62</RPC_CALLID>
       <RPC_CALLID>62</RPC_CALLID>
+=======
+      <RPC_CLIENTID>b5928e80-e373-4807-a688-f94483d08ce5</RPC_CLIENTID>
+      <RPC_CALLID>64</RPC_CALLID>
+>>>>>>> .merge-right.r1559304
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
   <RECORD>
   <RECORD>
@@ -750,21 +937,36 @@
       <INODEID>16392</INODEID>
       <INODEID>16392</INODEID>
       <PATH>/file_symlink</PATH>
       <PATH>/file_symlink</PATH>
       <VALUE>/file_concat_target</VALUE>
       <VALUE>/file_concat_target</VALUE>
+<<<<<<< .working
       <MTIME>1388730115502</MTIME>
       <MTIME>1388730115502</MTIME>
       <ATIME>1388730115502</ATIME>
       <ATIME>1388730115502</ATIME>
+=======
+      <MTIME>1389828266633</MTIME>
+      <ATIME>1389828266633</ATIME>
+>>>>>>> .merge-right.r1559304
       <PERMISSION_STATUS>
       <PERMISSION_STATUS>
+<<<<<<< .working
         <USERNAME>szetszwo</USERNAME>
         <USERNAME>szetszwo</USERNAME>
+=======
+        <USERNAME>jing</USERNAME>
+>>>>>>> .merge-right.r1559304
         <GROUPNAME>supergroup</GROUPNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>511</MODE>
         <MODE>511</MODE>
       </PERMISSION_STATUS>
       </PERMISSION_STATUS>
+<<<<<<< .working
       <RPC_CLIENTID>8205c453-0c7f-4b25-955a-7786e56bce86</RPC_CLIENTID>
       <RPC_CLIENTID>8205c453-0c7f-4b25-955a-7786e56bce86</RPC_CLIENTID>
       <RPC_CALLID>63</RPC_CALLID>
       <RPC_CALLID>63</RPC_CALLID>
+=======
+      <RPC_CLIENTID>b5928e80-e373-4807-a688-f94483d08ce5</RPC_CLIENTID>
+      <RPC_CALLID>66</RPC_CALLID>
+>>>>>>> .merge-right.r1559304
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
   <RECORD>
   <RECORD>
-    <OPCODE>OP_GET_DELEGATION_TOKEN</OPCODE>
+    <OPCODE>OP_ADD</OPCODE>
     <DATA>
     <DATA>
       <TXID>58</TXID>
       <TXID>58</TXID>
+<<<<<<< .working
       <DELEGATION_TOKEN_IDENTIFIER>
       <DELEGATION_TOKEN_IDENTIFIER>
         <KIND>HDFS_DELEGATION_TOKEN</KIND>
         <KIND>HDFS_DELEGATION_TOKEN</KIND>
         <SEQUENCE_NUMBER>1</SEQUENCE_NUMBER>
         <SEQUENCE_NUMBER>1</SEQUENCE_NUMBER>
@@ -883,42 +1085,62 @@
     <OPCODE>OP_ADD</OPCODE>
     <OPCODE>OP_ADD</OPCODE>
     <DATA>
     <DATA>
       <TXID>67</TXID>
       <TXID>67</TXID>
+=======
+>>>>>>> .merge-right.r1559304
       <LENGTH>0</LENGTH>
       <LENGTH>0</LENGTH>
       <INODEID>16393</INODEID>
       <INODEID>16393</INODEID>
       <PATH>/hard-lease-recovery-test</PATH>
       <PATH>/hard-lease-recovery-test</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
+<<<<<<< .working
       <MTIME>1388730115596</MTIME>
       <MTIME>1388730115596</MTIME>
       <ATIME>1388730115596</ATIME>
       <ATIME>1388730115596</ATIME>
+=======
+      <MTIME>1389828266637</MTIME>
+      <ATIME>1389828266637</ATIME>
+>>>>>>> .merge-right.r1559304
       <BLOCKSIZE>512</BLOCKSIZE>
       <BLOCKSIZE>512</BLOCKSIZE>
+<<<<<<< .working
       <CLIENT_NAME>DFSClient_NONMAPREDUCE_381408282_1</CLIENT_NAME>
       <CLIENT_NAME>DFSClient_NONMAPREDUCE_381408282_1</CLIENT_NAME>
+=======
+      <CLIENT_NAME>DFSClient_NONMAPREDUCE_16108824_1</CLIENT_NAME>
+>>>>>>> .merge-right.r1559304
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
       <PERMISSION_STATUS>
       <PERMISSION_STATUS>
+<<<<<<< .working
         <USERNAME>szetszwo</USERNAME>
         <USERNAME>szetszwo</USERNAME>
+=======
+        <USERNAME>jing</USERNAME>
+>>>>>>> .merge-right.r1559304
         <GROUPNAME>supergroup</GROUPNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
       </PERMISSION_STATUS>
+<<<<<<< .working
       <RPC_CLIENTID>8205c453-0c7f-4b25-955a-7786e56bce86</RPC_CLIENTID>
       <RPC_CLIENTID>8205c453-0c7f-4b25-955a-7786e56bce86</RPC_CLIENTID>
       <RPC_CALLID>73</RPC_CALLID>
       <RPC_CALLID>73</RPC_CALLID>
+=======
+      <RPC_CLIENTID>b5928e80-e373-4807-a688-f94483d08ce5</RPC_CLIENTID>
+      <RPC_CALLID>67</RPC_CALLID>
+>>>>>>> .merge-right.r1559304
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
   <RECORD>
   <RECORD>
     <OPCODE>OP_ALLOCATE_BLOCK_ID</OPCODE>
     <OPCODE>OP_ALLOCATE_BLOCK_ID</OPCODE>
     <DATA>
     <DATA>
-      <TXID>68</TXID>
+      <TXID>59</TXID>
       <BLOCK_ID>1073741834</BLOCK_ID>
       <BLOCK_ID>1073741834</BLOCK_ID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
   <RECORD>
   <RECORD>
     <OPCODE>OP_SET_GENSTAMP_V2</OPCODE>
     <OPCODE>OP_SET_GENSTAMP_V2</OPCODE>
     <DATA>
     <DATA>
-      <TXID>69</TXID>
+      <TXID>60</TXID>
       <GENSTAMPV2>1010</GENSTAMPV2>
       <GENSTAMPV2>1010</GENSTAMPV2>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
   <RECORD>
   <RECORD>
-    <OPCODE>OP_UPDATE_BLOCKS</OPCODE>
+    <OPCODE>OP_ADD_BLOCK</OPCODE>
     <DATA>
     <DATA>
-      <TXID>70</TXID>
+      <TXID>61</TXID>
       <PATH>/hard-lease-recovery-test</PATH>
       <PATH>/hard-lease-recovery-test</PATH>
       <BLOCK>
       <BLOCK>
         <BLOCK_ID>1073741834</BLOCK_ID>
         <BLOCK_ID>1073741834</BLOCK_ID>
@@ -932,7 +1154,7 @@
   <RECORD>
   <RECORD>
     <OPCODE>OP_UPDATE_BLOCKS</OPCODE>
     <OPCODE>OP_UPDATE_BLOCKS</OPCODE>
     <DATA>
     <DATA>
-      <TXID>71</TXID>
+      <TXID>62</TXID>
       <PATH>/hard-lease-recovery-test</PATH>
       <PATH>/hard-lease-recovery-test</PATH>
       <BLOCK>
       <BLOCK>
         <BLOCK_ID>1073741834</BLOCK_ID>
         <BLOCK_ID>1073741834</BLOCK_ID>
@@ -946,29 +1168,45 @@
   <RECORD>
   <RECORD>
     <OPCODE>OP_SET_GENSTAMP_V2</OPCODE>
     <OPCODE>OP_SET_GENSTAMP_V2</OPCODE>
     <DATA>
     <DATA>
-      <TXID>72</TXID>
+      <TXID>63</TXID>
       <GENSTAMPV2>1011</GENSTAMPV2>
       <GENSTAMPV2>1011</GENSTAMPV2>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
   <RECORD>
   <RECORD>
     <OPCODE>OP_REASSIGN_LEASE</OPCODE>
     <OPCODE>OP_REASSIGN_LEASE</OPCODE>
     <DATA>
     <DATA>
+<<<<<<< .working
       <TXID>73</TXID>
       <TXID>73</TXID>
       <LEASEHOLDER>DFSClient_NONMAPREDUCE_381408282_1</LEASEHOLDER>
       <LEASEHOLDER>DFSClient_NONMAPREDUCE_381408282_1</LEASEHOLDER>
+=======
+      <TXID>64</TXID>
+      <LEASEHOLDER>DFSClient_NONMAPREDUCE_16108824_1</LEASEHOLDER>
+>>>>>>> .merge-right.r1559304
       <PATH>/hard-lease-recovery-test</PATH>
       <PATH>/hard-lease-recovery-test</PATH>
       <NEWHOLDER>HDFS_NameNode</NEWHOLDER>
       <NEWHOLDER>HDFS_NameNode</NEWHOLDER>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
   <RECORD>
   <RECORD>
+<<<<<<< .working
     <OPCODE>OP_CLOSE</OPCODE>
     <OPCODE>OP_CLOSE</OPCODE>
     <DATA>
     <DATA>
       <TXID>74</TXID>
       <TXID>74</TXID>
+=======
+    <OPCODE>OP_CLOSE</OPCODE>
+    <DATA>
+      <TXID>65</TXID>
+>>>>>>> .merge-right.r1559304
       <LENGTH>0</LENGTH>
       <LENGTH>0</LENGTH>
       <INODEID>0</INODEID>
       <INODEID>0</INODEID>
       <PATH>/hard-lease-recovery-test</PATH>
       <PATH>/hard-lease-recovery-test</PATH>
       <REPLICATION>1</REPLICATION>
       <REPLICATION>1</REPLICATION>
+<<<<<<< .working
       <MTIME>1388730118281</MTIME>
       <MTIME>1388730118281</MTIME>
       <ATIME>1388730115596</ATIME>
       <ATIME>1388730115596</ATIME>
+=======
+      <MTIME>1389828269751</MTIME>
+      <ATIME>1389828266637</ATIME>
+>>>>>>> .merge-right.r1559304
       <BLOCKSIZE>512</BLOCKSIZE>
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_MACHINE></CLIENT_MACHINE>
       <CLIENT_MACHINE></CLIENT_MACHINE>
@@ -978,22 +1216,98 @@
         <GENSTAMP>1011</GENSTAMP>
         <GENSTAMP>1011</GENSTAMP>
       </BLOCK>
       </BLOCK>
       <PERMISSION_STATUS>
       <PERMISSION_STATUS>
+<<<<<<< .working
         <USERNAME>szetszwo</USERNAME>
         <USERNAME>szetszwo</USERNAME>
+=======
+        <USERNAME>jing</USERNAME>
+>>>>>>> .merge-right.r1559304
         <GROUPNAME>supergroup</GROUPNAME>
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
       </PERMISSION_STATUS>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
   <RECORD>
   <RECORD>
+<<<<<<< .working
     <OPCODE>OP_UPGRADE_MARKER</OPCODE>
     <OPCODE>OP_UPGRADE_MARKER</OPCODE>
     <DATA>
     <DATA>
       <TXID>75</TXID>
       <TXID>75</TXID>
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
   <RECORD>
   <RECORD>
+=======
+    <OPCODE>OP_ADD_CACHE_POOL</OPCODE>
+    <DATA>
+      <TXID>66</TXID>
+      <POOLNAME>pool1</POOLNAME>
+      <OWNERNAME>jing</OWNERNAME>
+      <GROUPNAME>staff</GROUPNAME>
+      <MODE>493</MODE>
+      <LIMIT>9223372036854775807</LIMIT>
+      <MAXRELATIVEEXPIRY>2305843009213693951</MAXRELATIVEEXPIRY>
+      <RPC_CLIENTID>b5928e80-e373-4807-a688-f94483d08ce5</RPC_CLIENTID>
+      <RPC_CALLID>74</RPC_CALLID>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>OP_MODIFY_CACHE_POOL</OPCODE>
+    <DATA>
+      <TXID>67</TXID>
+      <POOLNAME>pool1</POOLNAME>
+      <LIMIT>99</LIMIT>
+      <RPC_CLIENTID>b5928e80-e373-4807-a688-f94483d08ce5</RPC_CLIENTID>
+      <RPC_CALLID>75</RPC_CALLID>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>OP_ADD_CACHE_DIRECTIVE</OPCODE>
+    <DATA>
+      <TXID>68</TXID>
+      <ID>1</ID>
+      <PATH>/path</PATH>
+      <REPLICATION>1</REPLICATION>
+      <POOL>pool1</POOL>
+      <EXPIRATION>2305844399041964876</EXPIRATION>
+      <RPC_CLIENTID>b5928e80-e373-4807-a688-f94483d08ce5</RPC_CLIENTID>
+      <RPC_CALLID>76</RPC_CALLID>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>OP_MODIFY_CACHE_DIRECTIVE</OPCODE>
+    <DATA>
+      <TXID>69</TXID>
+      <ID>1</ID>
+      <REPLICATION>2</REPLICATION>
+      <RPC_CLIENTID>b5928e80-e373-4807-a688-f94483d08ce5</RPC_CLIENTID>
+      <RPC_CALLID>77</RPC_CALLID>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>OP_REMOVE_CACHE_DIRECTIVE</OPCODE>
+    <DATA>
+      <TXID>70</TXID>
+      <ID>1</ID>
+      <RPC_CLIENTID>b5928e80-e373-4807-a688-f94483d08ce5</RPC_CLIENTID>
+      <RPC_CALLID>78</RPC_CALLID>
+    </DATA>
+  </RECORD>
+  <RECORD>
+    <OPCODE>OP_REMOVE_CACHE_POOL</OPCODE>
+    <DATA>
+      <TXID>71</TXID>
+      <POOLNAME>pool1</POOLNAME>
+      <RPC_CLIENTID>b5928e80-e373-4807-a688-f94483d08ce5</RPC_CLIENTID>
+      <RPC_CALLID>79</RPC_CALLID>
+    </DATA>
+  </RECORD>
+  <RECORD>
+>>>>>>> .merge-right.r1559304
     <OPCODE>OP_END_LOG_SEGMENT</OPCODE>
     <OPCODE>OP_END_LOG_SEGMENT</OPCODE>
     <DATA>
     <DATA>
+<<<<<<< .working
       <TXID>76</TXID>
       <TXID>76</TXID>
+=======
+      <TXID>72</TXID>
+>>>>>>> .merge-right.r1559304
     </DATA>
     </DATA>
   </RECORD>
   </RECORD>
 </EDITS>
 </EDITS>

+ 11 - 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,9 @@ 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)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
     MAPREDUCE-5484. YarnChild unnecessarily loads job conf twice (Sandy Ryza)
     MAPREDUCE-5484. YarnChild unnecessarily loads job conf twice (Sandy Ryza)
@@ -272,6 +277,9 @@ 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)
+
 Release 2.3.0 - UNRELEASED
 Release 2.3.0 - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES
@@ -328,6 +336,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();
+  }
 }
 }

+ 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 - 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 {

+ 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>

+ 17 - 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
@@ -335,6 +338,20 @@ Release 2.4.0 - UNRELEASED
 
 
     YARN-1598. HA-related rmadmin commands don't work on a secure cluster (kasha)
     YARN-1598. HA-related rmadmin commands don't work on a secure cluster (kasha)
 
 
+    YARN-1603. Remove two *.orig files which were unexpectedly committed. 
+    (Zhijie Shen via junping_du)
+
+    YARN-1601. 3rd party JARs are missing from hadoop-dist output. (tucu)
+
+    YARN-1351. Invalid string format in Fair Scheduler log warn message
+    (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 {

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationFileLoaderService.java

@@ -387,7 +387,7 @@ public class AllocationFileLoaderService extends AbstractService {
     if (maxQueueResources.containsKey(queueName) && minQueueResources.containsKey(queueName)
     if (maxQueueResources.containsKey(queueName) && minQueueResources.containsKey(queueName)
         && !Resources.fitsIn(minQueueResources.get(queueName),
         && !Resources.fitsIn(minQueueResources.get(queueName),
             maxQueueResources.get(queueName))) {
             maxQueueResources.get(queueName))) {
-      LOG.warn(String.format("Queue %s has max resources %d less than min resources %d",
+      LOG.warn(String.format("Queue %s has max resources %s less than min resources %s",
           queueName, maxQueueResources.get(queueName), minQueueResources.get(queueName)));
           queueName, maxQueueResources.get(queueName), minQueueResources.get(queueName)));
     }
     }
   }
   }

+ 0 - 1361
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java.orig

@@ -1,1361 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
-import org.apache.hadoop.yarn.api.records.Container;
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.ContainerStatus;
-import org.apache.hadoop.yarn.api.records.NodeId;
-import org.apache.hadoop.yarn.api.records.Priority;
-import org.apache.hadoop.yarn.api.records.QueueACL;
-import org.apache.hadoop.yarn.api.records.QueueInfo;
-import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.api.records.ResourceRequest;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
-import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
-import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger;
-import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
-import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
-import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
-import org.apache.hadoop.yarn.server.resourcemanager.resource.ResourceWeights;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppRejectedEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
-import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
-import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
-import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
-import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
-import org.apache.hadoop.yarn.server.resourcemanager.rmnode.UpdatedContainerInfo;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppRemovedSchedulerEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.ContainerExpiredSchedulerEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
-import org.apache.hadoop.yarn.util.Clock;
-import org.apache.hadoop.yarn.util.SystemClock;
-import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
-import org.apache.hadoop.yarn.util.resource.DominantResourceCalculator;
-import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
-import org.apache.hadoop.yarn.util.resource.Resources;
-
-import com.google.common.annotations.VisibleForTesting;
-
-/**
- * A scheduler that schedules resources between a set of queues. The scheduler
- * keeps track of the resources used by each queue, and attempts to maintain
- * fairness by scheduling tasks at queues whose allocations are farthest below
- * an ideal fair distribution.
- * 
- * The fair scheduler supports hierarchical queues. All queues descend from a
- * queue named "root". Available resources are distributed among the children
- * of the root queue in the typical fair scheduling fashion. Then, the children
- * distribute the resources assigned to them to their children in the same
- * fashion.  Applications may only be scheduled on leaf queues. Queues can be
- * specified as children of other queues by placing them as sub-elements of their
- * parents in the fair scheduler configuration file.
- * 
- * A queue's name starts with the names of its parents, with periods as
- * separators.  So a queue named "queue1" under the root named, would be 
- * referred to as "root.queue1", and a queue named "queue2" under a queue
- * named "parent1" would be referred to as "root.parent1.queue2".
- */
-@LimitedPrivate("yarn")
-@Unstable
-@SuppressWarnings("unchecked")
-public class FairScheduler implements ResourceScheduler {
-  private boolean initialized;
-  private FairSchedulerConfiguration conf;
-  private RMContext rmContext;
-  private Resource minimumAllocation;
-  private Resource maximumAllocation;
-  private Resource incrAllocation;
-  private QueueManager queueMgr;
-  private Clock clock;
-  private boolean usePortForNodeName;
-
-  private static final Log LOG = LogFactory.getLog(FairScheduler.class);
-  
-  private static final ResourceCalculator RESOURCE_CALCULATOR =
-      new DefaultResourceCalculator();
-  
-  // Value that container assignment methods return when a container is
-  // reserved
-  public static final Resource CONTAINER_RESERVED = Resources.createResource(-1);
-
-  // How often fair shares are re-calculated (ms)
-  protected long UPDATE_INTERVAL = 500;
-
-  private final static List<Container> EMPTY_CONTAINER_LIST =
-      new ArrayList<Container>();
-
-  private static final Allocation EMPTY_ALLOCATION =
-      new Allocation(EMPTY_CONTAINER_LIST, Resources.createResource(0));
-
-  // Aggregate metrics
-  FSQueueMetrics rootMetrics;
-
-  // Time when we last updated preemption vars
-  protected long lastPreemptionUpdateTime;
-  // Time we last ran preemptTasksIfNecessary
-  private long lastPreemptCheckTime;
-
-  // This stores per-application scheduling information,
-  @VisibleForTesting
-  protected Map<ApplicationId, SchedulerApplication> applications =
-      new ConcurrentHashMap<ApplicationId, SchedulerApplication>();
-
-  // Nodes in the cluster, indexed by NodeId
-  private Map<NodeId, FSSchedulerNode> nodes = 
-      new ConcurrentHashMap<NodeId, FSSchedulerNode>();
-
-  // Aggregate capacity of the cluster
-  private Resource clusterCapacity = 
-      RecordFactoryProvider.getRecordFactory(null).newRecordInstance(Resource.class);
-
-  // How often tasks are preempted 
-  protected long preemptionInterval; 
-  
-  // ms to wait before force killing stuff (must be longer than a couple
-  // of heartbeats to give task-kill commands a chance to act).
-  protected long waitTimeBeforeKill; 
-  
-  // Containers whose AMs have been warned that they will be preempted soon.
-  private List<RMContainer> warnedContainers = new ArrayList<RMContainer>();
-  
-  protected boolean preemptionEnabled;
-  protected boolean sizeBasedWeight; // Give larger weights to larger jobs
-  protected WeightAdjuster weightAdjuster; // Can be null for no weight adjuster
-  protected boolean continuousSchedulingEnabled; // Continuous Scheduling enabled or not
-  protected int continuousSchedulingSleepMs; // Sleep time for each pass in continuous scheduling
-  private Comparator nodeAvailableResourceComparator =
-          new NodeAvailableResourceComparator(); // Node available resource comparator
-  protected double nodeLocalityThreshold; // Cluster threshold for node locality
-  protected double rackLocalityThreshold; // Cluster threshold for rack locality
-  protected long nodeLocalityDelayMs; // Delay for node locality
-  protected long rackLocalityDelayMs; // Delay for rack locality
-  private FairSchedulerEventLog eventLog; // Machine-readable event log
-  protected boolean assignMultiple; // Allocate multiple containers per
-                                    // heartbeat
-  protected int maxAssign; // Max containers to assign per heartbeat
-
-  @VisibleForTesting
-  final MaxRunningAppsEnforcer maxRunningEnforcer;
-
-  private AllocationFileLoaderService allocsLoader;
-  @VisibleForTesting
-  AllocationConfiguration allocConf;
-  
-  public FairScheduler() {
-    clock = new SystemClock();
-    allocsLoader = new AllocationFileLoaderService();
-    queueMgr = new QueueManager(this);
-    maxRunningEnforcer = new MaxRunningAppsEnforcer(this);
-  }
-
-  private void validateConf(Configuration conf) {
-    // validate scheduler memory allocation setting
-    int minMem = conf.getInt(
-      YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB,
-      YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB);
-    int maxMem = conf.getInt(
-      YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
-      YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB);
-
-    if (minMem < 0 || minMem > maxMem) {
-      throw new YarnRuntimeException("Invalid resource scheduler memory"
-        + " allocation configuration"
-        + ", " + YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB
-        + "=" + minMem
-        + ", " + YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_MB
-        + "=" + maxMem + ", min should equal greater than 0"
-        + ", max should be no smaller than min.");
-    }
-
-    // validate scheduler vcores allocation setting
-    int minVcores = conf.getInt(
-      YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES,
-      YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
-    int maxVcores = conf.getInt(
-      YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES,
-      YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES);
-
-    if (minVcores < 0 || minVcores > maxVcores) {
-      throw new YarnRuntimeException("Invalid resource scheduler vcores"
-        + " allocation configuration"
-        + ", " + YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES
-        + "=" + minVcores
-        + ", " + YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES
-        + "=" + maxVcores + ", min should equal greater than 0"
-        + ", max should be no smaller than min.");
-    }
-  }
-
-  public FairSchedulerConfiguration getConf() {
-    return conf;
-  }
-
-  public QueueManager getQueueManager() {
-    return queueMgr;
-  }
-
-  @Override
-  public RMContainer getRMContainer(ContainerId containerId) {
-    FSSchedulerApp attempt = getCurrentAttemptForContainer(containerId);
-    return (attempt == null) ? null : attempt.getRMContainer(containerId);
-  }
-
-  private FSSchedulerApp getCurrentAttemptForContainer(
-      ContainerId containerId) {
-    SchedulerApplication app =
-        applications.get(containerId.getApplicationAttemptId()
-          .getApplicationId());
-    if (app != null) {
-      return (FSSchedulerApp) app.getCurrentAppAttempt();
-    }
-    return null;
-  }
-
-  /**
-   * A runnable which calls {@link FairScheduler#update()} every
-   * <code>UPDATE_INTERVAL</code> milliseconds.
-   */
-  private class UpdateThread implements Runnable {
-    public void run() {
-      while (true) {
-        try {
-          Thread.sleep(UPDATE_INTERVAL);
-          update();
-          preemptTasksIfNecessary();
-        } catch (Exception e) {
-          LOG.error("Exception in fair scheduler UpdateThread", e);
-        }
-      }
-    }
-  }
-
-  /**
-   * Recompute the internal variables used by the scheduler - per-job weights,
-   * fair shares, deficits, minimum slot allocations, and amount of used and
-   * required resources per job.
-   */
-  protected synchronized void update() {
-    updatePreemptionVariables(); // Determine if any queues merit preemption
-
-    FSQueue rootQueue = queueMgr.getRootQueue();
-
-    // Recursively update demands for all queues
-    rootQueue.updateDemand();
-
-    rootQueue.setFairShare(clusterCapacity);
-    // Recursively compute fair shares for all queues
-    // and update metrics
-    rootQueue.recomputeShares();
-  }
-
-  /**
-   * Update the preemption fields for all QueueScheduables, i.e. the times since
-   * each queue last was at its guaranteed share and at > 1/2 of its fair share
-   * for each type of task.
-   */
-  private void updatePreemptionVariables() {
-    long now = clock.getTime();
-    lastPreemptionUpdateTime = now;
-    for (FSLeafQueue sched : queueMgr.getLeafQueues()) {
-      if (!isStarvedForMinShare(sched)) {
-        sched.setLastTimeAtMinShare(now);
-      }
-      if (!isStarvedForFairShare(sched)) {
-        sched.setLastTimeAtHalfFairShare(now);
-      }
-    }
-  }
-
-  /**
-   * Is a queue below its min share for the given task type?
-   */
-  boolean isStarvedForMinShare(FSLeafQueue sched) {
-    Resource desiredShare = Resources.min(RESOURCE_CALCULATOR, clusterCapacity,
-      sched.getMinShare(), sched.getDemand());
-    return Resources.lessThan(RESOURCE_CALCULATOR, clusterCapacity,
-        sched.getResourceUsage(), desiredShare);
-  }
-
-  /**
-   * Is a queue being starved for fair share for the given task type? This is
-   * defined as being below half its fair share.
-   */
-  boolean isStarvedForFairShare(FSLeafQueue sched) {
-    Resource desiredFairShare = Resources.min(RESOURCE_CALCULATOR, clusterCapacity,
-        Resources.multiply(sched.getFairShare(), .5), sched.getDemand());
-    return Resources.lessThan(RESOURCE_CALCULATOR, clusterCapacity,
-        sched.getResourceUsage(), desiredFairShare);
-  }
-
-  /**
-   * Check for queues that need tasks preempted, either because they have been
-   * below their guaranteed share for minSharePreemptionTimeout or they have
-   * been below half their fair share for the fairSharePreemptionTimeout. If
-   * such queues exist, compute how many tasks of each type need to be preempted
-   * and then select the right ones using preemptTasks.
-   */
-  protected synchronized void preemptTasksIfNecessary() {
-    if (!preemptionEnabled) {
-      return;
-    }
-
-    long curTime = clock.getTime();
-    if (curTime - lastPreemptCheckTime < preemptionInterval) {
-      return;
-    }
-    lastPreemptCheckTime = curTime;
-
-    Resource resToPreempt = Resources.none();
-
-    for (FSLeafQueue sched : queueMgr.getLeafQueues()) {
-      resToPreempt = Resources.add(resToPreempt, resToPreempt(sched, curTime));
-    }
-    if (Resources.greaterThan(RESOURCE_CALCULATOR, clusterCapacity, resToPreempt,
-        Resources.none())) {
-      preemptResources(queueMgr.getLeafQueues(), resToPreempt);
-    }
-  }
-
-  /**
-   * Preempt a quantity of resources from a list of QueueSchedulables. The
-   * policy for this is to pick apps from queues that are over their fair share,
-   * but make sure that no queue is placed below its fair share in the process.
-   * We further prioritize preemption by choosing containers with lowest
-   * priority to preempt.
-   */
-  protected void preemptResources(Collection<FSLeafQueue> scheds,
-      Resource toPreempt) {
-    if (scheds.isEmpty() || Resources.equals(toPreempt, Resources.none())) {
-      return;
-    }
-
-    Map<RMContainer, FSSchedulerApp> apps = 
-        new HashMap<RMContainer, FSSchedulerApp>();
-    Map<RMContainer, FSLeafQueue> queues = 
-        new HashMap<RMContainer, FSLeafQueue>();
-
-    // Collect running containers from over-scheduled queues
-    List<RMContainer> runningContainers = new ArrayList<RMContainer>();
-    for (FSLeafQueue sched : scheds) {
-      if (Resources.greaterThan(RESOURCE_CALCULATOR, clusterCapacity,
-          sched.getResourceUsage(), sched.getFairShare())) {
-        for (AppSchedulable as : sched.getRunnableAppSchedulables()) {
-          for (RMContainer c : as.getApp().getLiveContainers()) {
-            runningContainers.add(c);
-            apps.put(c, as.getApp());
-            queues.put(c, sched);
-          }
-        }
-      }
-    }
-
-    // Sort containers into reverse order of priority
-    Collections.sort(runningContainers, new Comparator<RMContainer>() {
-      public int compare(RMContainer c1, RMContainer c2) {
-        int ret = c1.getContainer().getPriority().compareTo(
-            c2.getContainer().getPriority());
-        if (ret == 0) {
-          return c2.getContainerId().compareTo(c1.getContainerId());
-        }
-        return ret;
-      }
-    });
-    
-    // Scan down the list of containers we've already warned and kill them
-    // if we need to.  Remove any containers from the list that we don't need
-    // or that are no longer running.
-    Iterator<RMContainer> warnedIter = warnedContainers.iterator();
-    Set<RMContainer> preemptedThisRound = new HashSet<RMContainer>();
-    while (warnedIter.hasNext()) {
-      RMContainer container = warnedIter.next();
-      if (container.getState() == RMContainerState.RUNNING &&
-          Resources.greaterThan(RESOURCE_CALCULATOR, clusterCapacity,
-              toPreempt, Resources.none())) {
-        warnOrKillContainer(container, apps.get(container), queues.get(container));
-        preemptedThisRound.add(container);
-        Resources.subtractFrom(toPreempt, container.getContainer().getResource());
-      } else {
-        warnedIter.remove();
-      }
-    }
-
-    // Scan down the rest of the containers until we've preempted enough, making
-    // sure we don't preempt too many from any queue
-    Iterator<RMContainer> runningIter = runningContainers.iterator();
-    while (runningIter.hasNext() &&
-        Resources.greaterThan(RESOURCE_CALCULATOR, clusterCapacity,
-            toPreempt, Resources.none())) {
-      RMContainer container = runningIter.next();
-      FSLeafQueue sched = queues.get(container);
-      if (!preemptedThisRound.contains(container) &&
-          Resources.greaterThan(RESOURCE_CALCULATOR, clusterCapacity,
-              sched.getResourceUsage(), sched.getFairShare())) {
-        warnOrKillContainer(container, apps.get(container), sched);
-        
-        warnedContainers.add(container);
-        Resources.subtractFrom(toPreempt, container.getContainer().getResource());
-      }
-    }
-  }
-  
-  private void warnOrKillContainer(RMContainer container, FSSchedulerApp app,
-      FSLeafQueue queue) {
-    LOG.info("Preempting container (prio=" + container.getContainer().getPriority() +
-        "res=" + container.getContainer().getResource() +
-        ") from queue " + queue.getName());
-    
-    Long time = app.getContainerPreemptionTime(container);
-
-    if (time != null) {
-      // if we asked for preemption more than maxWaitTimeBeforeKill ms ago,
-      // proceed with kill
-      if (time + waitTimeBeforeKill < clock.getTime()) {
-        ContainerStatus status =
-          SchedulerUtils.createPreemptedContainerStatus(
-            container.getContainerId(), SchedulerUtils.PREEMPTED_CONTAINER);
-
-        // TODO: Not sure if this ever actually adds this to the list of cleanup
-        // containers on the RMNode (see SchedulerNode.releaseContainer()).
-        completedContainer(container, status, RMContainerEventType.KILL);
-        LOG.info("Killing container" + container +
-            " (after waiting for premption for " +
-            (clock.getTime() - time) + "ms)");
-      }
-    } else {
-      // track the request in the FSSchedulerApp itself
-      app.addPreemption(container, clock.getTime());
-    }
-  }
-
-  /**
-   * Return the resource amount that this queue is allowed to preempt, if any.
-   * If the queue has been below its min share for at least its preemption
-   * timeout, it should preempt the difference between its current share and
-   * this min share. If it has been below half its fair share for at least the
-   * fairSharePreemptionTimeout, it should preempt enough tasks to get up to its
-   * full fair share. If both conditions hold, we preempt the max of the two
-   * amounts (this shouldn't happen unless someone sets the timeouts to be
-   * identical for some reason).
-   */
-  protected Resource resToPreempt(FSLeafQueue sched, long curTime) {
-    String queue = sched.getName();
-    long minShareTimeout = allocConf.getMinSharePreemptionTimeout(queue);
-    long fairShareTimeout = allocConf.getFairSharePreemptionTimeout();
-    Resource resDueToMinShare = Resources.none();
-    Resource resDueToFairShare = Resources.none();
-    if (curTime - sched.getLastTimeAtMinShare() > minShareTimeout) {
-      Resource target = Resources.min(RESOURCE_CALCULATOR, clusterCapacity,
-          sched.getMinShare(), sched.getDemand());
-      resDueToMinShare = Resources.max(RESOURCE_CALCULATOR, clusterCapacity,
-          Resources.none(), Resources.subtract(target, sched.getResourceUsage()));
-    }
-    if (curTime - sched.getLastTimeAtHalfFairShare() > fairShareTimeout) {
-      Resource target = Resources.min(RESOURCE_CALCULATOR, clusterCapacity,
-          sched.getFairShare(), sched.getDemand());
-      resDueToFairShare = Resources.max(RESOURCE_CALCULATOR, clusterCapacity,
-          Resources.none(), Resources.subtract(target, sched.getResourceUsage()));
-    }
-    Resource resToPreempt = Resources.max(RESOURCE_CALCULATOR, clusterCapacity,
-        resDueToMinShare, resDueToFairShare);
-    if (Resources.greaterThan(RESOURCE_CALCULATOR, clusterCapacity,
-        resToPreempt, Resources.none())) {
-      String message = "Should preempt " + resToPreempt + " res for queue "
-          + sched.getName() + ": resDueToMinShare = " + resDueToMinShare
-          + ", resDueToFairShare = " + resDueToFairShare;
-      LOG.info(message);
-    }
-    return resToPreempt;
-  }
-
-  public RMContainerTokenSecretManager getContainerTokenSecretManager() {
-    return rmContext.getContainerTokenSecretManager();
-  }
-
-  // synchronized for sizeBasedWeight
-  public synchronized ResourceWeights getAppWeight(AppSchedulable app) {
-    double weight = 1.0;
-    if (sizeBasedWeight) {
-      // Set weight based on current memory demand
-      weight = Math.log1p(app.getDemand().getMemory()) / Math.log(2);
-    }
-    weight *= app.getPriority().getPriority();
-    if (weightAdjuster != null) {
-      // Run weight through the user-supplied weightAdjuster
-      weight = weightAdjuster.adjustWeight(app, weight);
-    }
-    return new ResourceWeights((float)weight);
-  }
-
-  @Override
-  public Resource getMinimumResourceCapability() {
-    return minimumAllocation;
-  }
-
-  public Resource getIncrementResourceCapability() {
-    return incrAllocation;
-  }
-
-  @Override
-  public Resource getMaximumResourceCapability() {
-    return maximumAllocation;
-  }
-
-  public double getNodeLocalityThreshold() {
-    return nodeLocalityThreshold;
-  }
-
-  public double getRackLocalityThreshold() {
-    return rackLocalityThreshold;
-  }
-
-  public long getNodeLocalityDelayMs() {
-    return nodeLocalityDelayMs;
-  }
-
-  public long getRackLocalityDelayMs() {
-    return rackLocalityDelayMs;
-  }
-
-  public boolean isContinuousSchedulingEnabled() {
-    return continuousSchedulingEnabled;
-  }
-
-  public synchronized int getContinuousSchedulingSleepMs() {
-    return continuousSchedulingSleepMs;
-  }
-
-  public Resource getClusterCapacity() {
-    return clusterCapacity;
-  }
-
-  public synchronized Clock getClock() {
-    return clock;
-  }
-
-  protected synchronized void setClock(Clock clock) {
-    this.clock = clock;
-  }
-
-  public FairSchedulerEventLog getEventLog() {
-    return eventLog;
-  }
-
-  /**
-   * Add a new application to the scheduler, with a given id, queue name, and
-   * user. This will accept a new app even if the user or queue is above
-   * configured limits, but the app will not be marked as runnable.
-   */
-  protected synchronized void addApplication(ApplicationId applicationId,
-      String queueName, String user) {
-    if (queueName == null || queueName.isEmpty()) {
-      String message = "Reject application " + applicationId +
-              " submitted by user " + user + " with an empty queue name.";
-      LOG.info(message);
-      rmContext.getDispatcher().getEventHandler()
-          .handle(new RMAppRejectedEvent(applicationId, message));
-      return;
-    }
-
-    RMApp rmApp = rmContext.getRMApps().get(applicationId);
-    FSLeafQueue queue = assignToQueue(rmApp, queueName, user);
-    if (queue == null) {
-      rmContext.getDispatcher().getEventHandler().handle(
-          new RMAppRejectedEvent(applicationId,
-              "Application rejected by queue placement policy"));
-      return;
-    }
-
-    // Enforce ACLs
-    UserGroupInformation userUgi = UserGroupInformation.createRemoteUser(user);
-
-    if (!queue.hasAccess(QueueACL.SUBMIT_APPLICATIONS, userUgi)
-        && !queue.hasAccess(QueueACL.ADMINISTER_QUEUE, userUgi)) {
-      String msg = "User " + userUgi.getUserName() +
-              " cannot submit applications to queue " + queue.getName();
-      LOG.info(msg);
-      rmContext.getDispatcher().getEventHandler()
-          .handle(new RMAppRejectedEvent(applicationId, msg));
-      return;
-    }
-  
-    SchedulerApplication application =
-        new SchedulerApplication(queue, user);
-    applications.put(applicationId, application);
-    queue.getMetrics().submitApp(user);
-
-    LOG.info("Accepted application " + applicationId + " from user: " + user
-        + ", in queue: " + queueName + ", currently num of applications: "
-        + applications.size());
-    rmContext.getDispatcher().getEventHandler()
-        .handle(new RMAppEvent(applicationId, RMAppEventType.APP_ACCEPTED));
-  }
-
-  /**
-   * Add a new application attempt to the scheduler.
-   */
-  protected synchronized void addApplicationAttempt(
-      ApplicationAttemptId applicationAttemptId,
-      boolean transferStateFromPreviousAttempt) {
-    SchedulerApplication application =
-        applications.get(applicationAttemptId.getApplicationId());
-    String user = application.getUser();
-    FSLeafQueue queue = (FSLeafQueue) application.getQueue();
-
-    FSSchedulerApp attempt =
-        new FSSchedulerApp(applicationAttemptId, user,
-            queue, new ActiveUsersManager(getRootQueueMetrics()),
-            rmContext);
-    if (transferStateFromPreviousAttempt) {
-      attempt.transferStateFromPreviousAttempt(application
-        .getCurrentAppAttempt());
-    }
-    application.setCurrentAppAttempt(attempt);
-
-    boolean runnable = maxRunningEnforcer.canAppBeRunnable(queue, user);
-    queue.addApp(attempt, runnable);
-    if (runnable) {
-      maxRunningEnforcer.trackRunnableApp(attempt);
-    } else {
-      maxRunningEnforcer.trackNonRunnableApp(attempt);
-    }
-    
-    queue.getMetrics().submitAppAttempt(user);
-
-    LOG.info("Added Application Attempt " + applicationAttemptId
-        + " to scheduler from user: " + user);
-    rmContext.getDispatcher().getEventHandler().handle(
-        new RMAppAttemptEvent(applicationAttemptId,
-            RMAppAttemptEventType.ATTEMPT_ADDED));
-  }
-  
-  @VisibleForTesting
-  FSLeafQueue assignToQueue(RMApp rmApp, String queueName, String user) {
-    FSLeafQueue queue = null;
-    try {
-      QueuePlacementPolicy placementPolicy = allocConf.getPlacementPolicy();
-      queueName = placementPolicy.assignAppToQueue(queueName, user);
-      if (queueName == null) {
-        return null;
-      }
-      queue = queueMgr.getLeafQueue(queueName, true);
-    } catch (IOException ex) {
-      LOG.error("Error assigning app to queue, rejecting", ex);
-    }
-    
-    if (rmApp != null) {
-      rmApp.setQueue(queue.getName());
-    } else {
-      LOG.warn("Couldn't find RM app to set queue name on");
-    }
-    
-    return queue;
-  }
-
-  private synchronized void removeApplication(ApplicationId applicationId,
-      RMAppState finalState) {
-    SchedulerApplication application = applications.get(applicationId);
-    if (application == null){
-      LOG.warn("Couldn't find application " + applicationId);
-      return;
-    }
-    application.stop(finalState);
-    applications.remove(applicationId);
-  }
-
-  private synchronized void removeApplicationAttempt(
-      ApplicationAttemptId applicationAttemptId,
-      RMAppAttemptState rmAppAttemptFinalState, boolean keepContainers) {
-    LOG.info("Application " + applicationAttemptId + " is done." +
-        " finalState=" + rmAppAttemptFinalState);
-    SchedulerApplication application =
-        applications.get(applicationAttemptId.getApplicationId());
-    FSSchedulerApp attempt = getSchedulerApp(applicationAttemptId);
-
-    if (attempt == null || application == null) {
-      LOG.info("Unknown application " + applicationAttemptId + " has completed!");
-      return;
-    }
-
-    // Release all the running containers
-    for (RMContainer rmContainer : attempt.getLiveContainers()) {
-      if (keepContainers
-          && rmContainer.getState().equals(RMContainerState.RUNNING)) {
-        // do not kill the running container in the case of work-preserving AM
-        // restart.
-        LOG.info("Skip killing " + rmContainer.getContainerId());
-        continue;
-      }
-      completedContainer(rmContainer,
-          SchedulerUtils.createAbnormalContainerStatus(
-              rmContainer.getContainerId(),
-              SchedulerUtils.COMPLETED_APPLICATION),
-              RMContainerEventType.KILL);
-    }
-
-    // Release all reserved containers
-    for (RMContainer rmContainer : attempt.getReservedContainers()) {
-      completedContainer(rmContainer,
-          SchedulerUtils.createAbnormalContainerStatus(
-              rmContainer.getContainerId(),
-              "Application Complete"),
-              RMContainerEventType.KILL);
-    }
-    // Clean up pending requests, metrics etc.
-    attempt.stop(rmAppAttemptFinalState);
-
-    // Inform the queue
-    FSLeafQueue queue = queueMgr.getLeafQueue(attempt.getQueue()
-        .getQueueName(), false);
-    boolean wasRunnable = queue.removeApp(attempt);
-
-    if (wasRunnable) {
-      maxRunningEnforcer.updateRunnabilityOnAppRemoval(attempt);
-    } else {
-      maxRunningEnforcer.untrackNonRunnableApp(attempt);
-    }
-  }
-
-  /**
-   * Clean up a completed container.
-   */
-  private synchronized void completedContainer(RMContainer rmContainer,
-      ContainerStatus containerStatus, RMContainerEventType event) {
-    if (rmContainer == null) {
-      LOG.info("Null container completed...");
-      return;
-    }
-
-    Container container = rmContainer.getContainer();
-
-    // Get the application for the finished container
-    FSSchedulerApp application =
-        getCurrentAttemptForContainer(container.getId());
-    ApplicationId appId =
-        container.getId().getApplicationAttemptId().getApplicationId();
-    if (application == null) {
-      LOG.info("Container " + container + " of" +
-          " unknown application attempt " + appId +
-          " completed with event " + event);
-      return;
-    }
-
-    // Get the node on which the container was allocated
-    FSSchedulerNode node = nodes.get(container.getNodeId());
-
-    if (rmContainer.getState() == RMContainerState.RESERVED) {
-      application.unreserve(node, rmContainer.getReservedPriority());
-      node.unreserveResource(application);
-    } else {
-      application.containerCompleted(rmContainer, containerStatus, event);
-      node.releaseContainer(container);
-      updateRootQueueMetrics();
-    }
-
-    LOG.info("Application attempt " + application.getApplicationAttemptId()
-        + " released container " + container.getId() + " on node: " + node
-        + " with event: " + event);
-  }
-
-  private synchronized void addNode(RMNode node) {
-    nodes.put(node.getNodeID(), new FSSchedulerNode(node, usePortForNodeName));
-    Resources.addTo(clusterCapacity, node.getTotalCapability());
-    updateRootQueueMetrics();
-
-    LOG.info("Added node " + node.getNodeAddress() +
-        " cluster capacity: " + clusterCapacity);
-  }
-
-  private synchronized void removeNode(RMNode rmNode) {
-    FSSchedulerNode node = nodes.get(rmNode.getNodeID());
-    // This can occur when an UNHEALTHY node reconnects
-    if (node == null) {
-      return;
-    }
-    Resources.subtractFrom(clusterCapacity, rmNode.getTotalCapability());
-    updateRootQueueMetrics();
-
-    // Remove running containers
-    List<RMContainer> runningContainers = node.getRunningContainers();
-    for (RMContainer container : runningContainers) {
-      completedContainer(container,
-          SchedulerUtils.createAbnormalContainerStatus(
-              container.getContainerId(),
-              SchedulerUtils.LOST_CONTAINER),
-          RMContainerEventType.KILL);
-    }
-
-    // Remove reservations, if any
-    RMContainer reservedContainer = node.getReservedContainer();
-    if (reservedContainer != null) {
-      completedContainer(reservedContainer,
-          SchedulerUtils.createAbnormalContainerStatus(
-              reservedContainer.getContainerId(),
-              SchedulerUtils.LOST_CONTAINER),
-          RMContainerEventType.KILL);
-    }
-
-    nodes.remove(rmNode.getNodeID());
-    LOG.info("Removed node " + rmNode.getNodeAddress() +
-        " cluster capacity: " + clusterCapacity);
-  }
-
-  @Override
-  public Allocation allocate(ApplicationAttemptId appAttemptId,
-      List<ResourceRequest> ask, List<ContainerId> release, List<String> blacklistAdditions, List<String> blacklistRemovals) {
-
-    // Make sure this application exists
-    FSSchedulerApp application = getSchedulerApp(appAttemptId);
-    if (application == null) {
-      LOG.info("Calling allocate on removed " +
-          "or non existant application " + appAttemptId);
-      return EMPTY_ALLOCATION;
-    }
-
-    // Sanity check
-    SchedulerUtils.normalizeRequests(ask, new DominantResourceCalculator(),
-        clusterCapacity, minimumAllocation, maximumAllocation, incrAllocation);
-
-    // Release containers
-    for (ContainerId releasedContainerId : release) {
-      RMContainer rmContainer = getRMContainer(releasedContainerId);
-      if (rmContainer == null) {
-        RMAuditLogger.logFailure(application.getUser(),
-            AuditConstants.RELEASE_CONTAINER,
-            "Unauthorized access or invalid container", "FairScheduler",
-            "Trying to release container not owned by app or with invalid id",
-            application.getApplicationId(), releasedContainerId);
-      }
-      completedContainer(rmContainer,
-          SchedulerUtils.createAbnormalContainerStatus(
-              releasedContainerId,
-              SchedulerUtils.RELEASED_CONTAINER),
-          RMContainerEventType.RELEASED);
-    }
-
-    synchronized (application) {
-      if (!ask.isEmpty()) {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("allocate: pre-update" +
-              " applicationAttemptId=" + appAttemptId +
-              " application=" + application.getApplicationId());
-        }
-        application.showRequests();
-
-        // Update application requests
-        application.updateResourceRequests(ask);
-
-        LOG.debug("allocate: post-update");
-        application.showRequests();
-      }
-
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("allocate:" +
-            " applicationAttemptId=" + appAttemptId +
-            " #ask=" + ask.size());
-
-        LOG.debug("Preempting " + application.getPreemptionContainers().size()
-            + " container(s)");
-      }
-      
-      Set<ContainerId> preemptionContainerIds = new HashSet<ContainerId>();
-      for (RMContainer container : application.getPreemptionContainers()) {
-        preemptionContainerIds.add(container.getContainerId());
-      }
-
-      application.updateBlacklist(blacklistAdditions, blacklistRemovals);
-      
-      return new Allocation(application.pullNewlyAllocatedContainers(),
-          application.getHeadroom(), preemptionContainerIds);
-    }
-  }
-
-  /**
-   * Process a container which has launched on a node, as reported by the node.
-   */
-  private void containerLaunchedOnNode(ContainerId containerId, FSSchedulerNode node) {
-    // Get the application for the finished container
-    FSSchedulerApp application = getCurrentAttemptForContainer(containerId);
-    if (application == null) {
-      LOG.info("Unknown application "
-          + containerId.getApplicationAttemptId().getApplicationId()
-          + " launched container " + containerId + " on node: " + node);
-      return;
-    }
-
-    application.containerLaunchedOnNode(containerId, node.getNodeID());
-  }
-
-  /**
-   * Process a heartbeat update from a node.
-   */
-  private synchronized void nodeUpdate(RMNode nm) {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("nodeUpdate: " + nm + " cluster capacity: " + clusterCapacity);
-    }
-    eventLog.log("HEARTBEAT", nm.getHostName());
-    FSSchedulerNode node = nodes.get(nm.getNodeID());
-
-    // Update resource if any change
-    SchedulerUtils.updateResourceIfChanged(node, nm, clusterCapacity, LOG);
-    
-    List<UpdatedContainerInfo> containerInfoList = nm.pullContainerUpdates();
-    List<ContainerStatus> newlyLaunchedContainers = new ArrayList<ContainerStatus>();
-    List<ContainerStatus> completedContainers = new ArrayList<ContainerStatus>();
-    for(UpdatedContainerInfo containerInfo : containerInfoList) {
-      newlyLaunchedContainers.addAll(containerInfo.getNewlyLaunchedContainers());
-      completedContainers.addAll(containerInfo.getCompletedContainers());
-    } 
-    // Processing the newly launched containers
-    for (ContainerStatus launchedContainer : newlyLaunchedContainers) {
-      containerLaunchedOnNode(launchedContainer.getContainerId(), node);
-    }
-
-    // Process completed containers
-    for (ContainerStatus completedContainer : completedContainers) {
-      ContainerId containerId = completedContainer.getContainerId();
-      LOG.debug("Container FINISHED: " + containerId);
-      completedContainer(getRMContainer(containerId),
-          completedContainer, RMContainerEventType.FINISHED);
-    }
-
-    if (continuousSchedulingEnabled) {
-      if (!completedContainers.isEmpty()) {
-        attemptScheduling(node);
-      }
-    } else {
-      attemptScheduling(node);
-    }
-  }
-
-  private void continuousScheduling() {
-    while (true) {
-      List<NodeId> nodeIdList = new ArrayList<NodeId>(nodes.keySet());
-      Collections.sort(nodeIdList, nodeAvailableResourceComparator);
-
-      // iterate all nodes
-      for (NodeId nodeId : nodeIdList) {
-        if (nodes.containsKey(nodeId)) {
-          FSSchedulerNode node = nodes.get(nodeId);
-          try {
-            if (Resources.fitsIn(minimumAllocation,
-                    node.getAvailableResource())) {
-              attemptScheduling(node);
-            }
-          } catch (Throwable ex) {
-            LOG.warn("Error while attempting scheduling for node " + node +
-                    ": " + ex.toString(), ex);
-          }
-        }
-      }
-      try {
-        Thread.sleep(getContinuousSchedulingSleepMs());
-      } catch (InterruptedException e) {
-        LOG.warn("Error while doing sleep in continuous scheduling: " +
-                e.toString(), e);
-      }
-    }
-  }
-
-  /** Sort nodes by available resource */
-  private class NodeAvailableResourceComparator implements Comparator<NodeId> {
-
-    @Override
-    public int compare(NodeId n1, NodeId n2) {
-      return RESOURCE_CALCULATOR.compare(clusterCapacity,
-              nodes.get(n2).getAvailableResource(),
-              nodes.get(n1).getAvailableResource());
-    }
-  }
-  
-  private synchronized void attemptScheduling(FSSchedulerNode node) {
-    // Assign new containers...
-    // 1. Check for reserved applications
-    // 2. Schedule if there are no reservations
-
-    AppSchedulable reservedAppSchedulable = node.getReservedAppSchedulable();
-    if (reservedAppSchedulable != null) {
-      Priority reservedPriority = node.getReservedContainer().getReservedPriority();
-      if (!reservedAppSchedulable.hasContainerForNode(reservedPriority, node)) {
-        // Don't hold the reservation if app can no longer use it
-        LOG.info("Releasing reservation that cannot be satisfied for application "
-            + reservedAppSchedulable.getApp().getApplicationAttemptId()
-            + " on node " + node);
-        reservedAppSchedulable.unreserve(reservedPriority, node);
-        reservedAppSchedulable = null;
-      } else {
-        // Reservation exists; try to fulfill the reservation
-        LOG.info("Trying to fulfill reservation for application "
-            + reservedAppSchedulable.getApp().getApplicationAttemptId()
-            + " on node: " + node);
-
-        node.getReservedAppSchedulable().assignReservedContainer(node);
-      }
-    }
-    if (reservedAppSchedulable == null) {
-      // No reservation, schedule at queue which is farthest below fair share
-      int assignedContainers = 0;
-      while (node.getReservedContainer() == null) {
-        boolean assignedContainer = false;
-        if (Resources.greaterThan(RESOURCE_CALCULATOR, clusterCapacity,
-              queueMgr.getRootQueue().assignContainer(node),
-              Resources.none())) {
-          assignedContainers++;
-          assignedContainer = true;
-        }
-        if (!assignedContainer) { break; }
-        if (!assignMultiple) { break; }
-        if ((assignedContainers >= maxAssign) && (maxAssign > 0)) { break; }
-      }
-    }
-    updateRootQueueMetrics();
-  }
-
-  @Override
-  public SchedulerNodeReport getNodeReport(NodeId nodeId) {
-    FSSchedulerNode node = nodes.get(nodeId);
-    return node == null ? null : new SchedulerNodeReport(node);
-  }
-  
-  public FSSchedulerApp getSchedulerApp(ApplicationAttemptId appAttemptId) {
-    SchedulerApplication app =
-        applications.get(appAttemptId.getApplicationId());
-    if (app != null) {
-      return (FSSchedulerApp) app.getCurrentAppAttempt();
-    }
-    return null;
-  }
-  
-  @Override
-  public SchedulerAppReport getSchedulerAppInfo(
-      ApplicationAttemptId appAttemptId) {
-    FSSchedulerApp attempt = getSchedulerApp(appAttemptId);
-    if (attempt == null) {
-      LOG.error("Request for appInfo of unknown attempt" + appAttemptId);
-      return null;
-    }
-    return new SchedulerAppReport(attempt);
-  }
-  
-  @Override
-  public ApplicationResourceUsageReport getAppResourceUsageReport(
-      ApplicationAttemptId appAttemptId) {
-    FSSchedulerApp attempt = getSchedulerApp(appAttemptId);
-    if (attempt == null) {
-      LOG.error("Request for appInfo of unknown attempt" + appAttemptId);
-      return null;
-    }
-    return attempt.getResourceUsageReport();
-  }
-  
-  /**
-   * Subqueue metrics might be a little out of date because fair shares are
-   * recalculated at the update interval, but the root queue metrics needs to
-   * be updated synchronously with allocations and completions so that cluster
-   * metrics will be consistent.
-   */
-  private void updateRootQueueMetrics() {
-    rootMetrics.setAvailableResourcesToQueue(
-        Resources.subtract(
-            clusterCapacity, rootMetrics.getAllocatedResources()));
-  }
-
-  @Override
-  public QueueMetrics getRootQueueMetrics() {
-    return rootMetrics;
-  }
-
-  @Override
-  public void handle(SchedulerEvent event) {
-    switch (event.getType()) {
-    case NODE_ADDED:
-      if (!(event instanceof NodeAddedSchedulerEvent)) {
-        throw new RuntimeException("Unexpected event type: " + event);
-      }
-      NodeAddedSchedulerEvent nodeAddedEvent = (NodeAddedSchedulerEvent)event;
-      addNode(nodeAddedEvent.getAddedRMNode());
-      break;
-    case NODE_REMOVED:
-      if (!(event instanceof NodeRemovedSchedulerEvent)) {
-        throw new RuntimeException("Unexpected event type: " + event);
-      }
-      NodeRemovedSchedulerEvent nodeRemovedEvent = (NodeRemovedSchedulerEvent)event;
-      removeNode(nodeRemovedEvent.getRemovedRMNode());
-      break;
-    case NODE_UPDATE:
-      if (!(event instanceof NodeUpdateSchedulerEvent)) {
-        throw new RuntimeException("Unexpected event type: " + event);
-      }
-      NodeUpdateSchedulerEvent nodeUpdatedEvent = (NodeUpdateSchedulerEvent)event;
-      nodeUpdate(nodeUpdatedEvent.getRMNode());
-      break;
-    case APP_ADDED:
-      if (!(event instanceof AppAddedSchedulerEvent)) {
-        throw new RuntimeException("Unexpected event type: " + event);
-      }
-      AppAddedSchedulerEvent appAddedEvent = (AppAddedSchedulerEvent) event;
-      addApplication(appAddedEvent.getApplicationId(),
-        appAddedEvent.getQueue(), appAddedEvent.getUser());
-      break;
-    case APP_REMOVED:
-      if (!(event instanceof AppRemovedSchedulerEvent)) {
-        throw new RuntimeException("Unexpected event type: " + event);
-      }
-      AppRemovedSchedulerEvent appRemovedEvent = (AppRemovedSchedulerEvent)event;
-      removeApplication(appRemovedEvent.getApplicationID(),
-        appRemovedEvent.getFinalState());
-      break;
-    case APP_ATTEMPT_ADDED:
-      if (!(event instanceof AppAttemptAddedSchedulerEvent)) {
-        throw new RuntimeException("Unexpected event type: " + event);
-      }
-      AppAttemptAddedSchedulerEvent appAttemptAddedEvent =
-          (AppAttemptAddedSchedulerEvent) event;
-      addApplicationAttempt(appAttemptAddedEvent.getApplicationAttemptId(),
-        appAttemptAddedEvent.getTransferStateFromPreviousAttempt());
-      break;
-    case APP_ATTEMPT_REMOVED:
-      if (!(event instanceof AppAttemptRemovedSchedulerEvent)) {
-        throw new RuntimeException("Unexpected event type: " + event);
-      }
-      AppAttemptRemovedSchedulerEvent appAttemptRemovedEvent =
-          (AppAttemptRemovedSchedulerEvent) event;
-      removeApplicationAttempt(
-          appAttemptRemovedEvent.getApplicationAttemptID(),
-          appAttemptRemovedEvent.getFinalAttemptState(),
-          appAttemptRemovedEvent.getKeepContainersAcrossAppAttempts());
-      break;
-    case CONTAINER_EXPIRED:
-      if (!(event instanceof ContainerExpiredSchedulerEvent)) {
-        throw new RuntimeException("Unexpected event type: " + event);
-      }
-      ContainerExpiredSchedulerEvent containerExpiredEvent =
-          (ContainerExpiredSchedulerEvent)event;
-      ContainerId containerId = containerExpiredEvent.getContainerId();
-      completedContainer(getRMContainer(containerId),
-          SchedulerUtils.createAbnormalContainerStatus(
-              containerId,
-              SchedulerUtils.EXPIRED_CONTAINER),
-          RMContainerEventType.EXPIRE);
-      break;
-    default:
-      LOG.error("Unknown event arrived at FairScheduler: " + event.toString());
-    }
-  }
-
-  @Override
-  public void recover(RMState state) throws Exception {
-    // NOT IMPLEMENTED
-  }
-
-  @Override
-  public synchronized void reinitialize(Configuration conf, RMContext rmContext)
-      throws IOException {
-    if (!initialized) {
-      this.conf = new FairSchedulerConfiguration(conf);
-      validateConf(this.conf);
-      minimumAllocation = this.conf.getMinimumAllocation();
-      maximumAllocation = this.conf.getMaximumAllocation();
-      incrAllocation = this.conf.getIncrementAllocation();
-      continuousSchedulingEnabled = this.conf.isContinuousSchedulingEnabled();
-      continuousSchedulingSleepMs =
-              this.conf.getContinuousSchedulingSleepMs();
-      nodeLocalityThreshold = this.conf.getLocalityThresholdNode();
-      rackLocalityThreshold = this.conf.getLocalityThresholdRack();
-      nodeLocalityDelayMs = this.conf.getLocalityDelayNodeMs();
-      rackLocalityDelayMs = this.conf.getLocalityDelayRackMs();
-      preemptionEnabled = this.conf.getPreemptionEnabled();
-      assignMultiple = this.conf.getAssignMultiple();
-      maxAssign = this.conf.getMaxAssign();
-      sizeBasedWeight = this.conf.getSizeBasedWeight();
-      preemptionInterval = this.conf.getPreemptionInterval();
-      waitTimeBeforeKill = this.conf.getWaitTimeBeforeKill();
-      usePortForNodeName = this.conf.getUsePortForNodeName();
-      
-      rootMetrics = FSQueueMetrics.forQueue("root", null, true, conf);
-      this.rmContext = rmContext;
-      this.eventLog = new FairSchedulerEventLog();
-      eventLog.init(this.conf);
-
-      initialized = true;
-
-      allocConf = new AllocationConfiguration(conf);
-      try {
-        queueMgr.initialize(conf);
-      } catch (Exception e) {
-        throw new IOException("Failed to start FairScheduler", e);
-      }
-
-      Thread updateThread = new Thread(new UpdateThread());
-      updateThread.setName("FairSchedulerUpdateThread");
-      updateThread.setDaemon(true);
-      updateThread.start();
-
-      if (continuousSchedulingEnabled) {
-        // start continuous scheduling thread
-        Thread schedulingThread = new Thread(
-          new Runnable() {
-            @Override
-            public void run() {
-              continuousScheduling();
-            }
-          }
-        );
-        schedulingThread.setName("ContinuousScheduling");
-        schedulingThread.setDaemon(true);
-        schedulingThread.start();
-      }
-      
-      allocsLoader.init(conf);
-      allocsLoader.setReloadListener(new AllocationReloadListener());
-      // If we fail to load allocations file on initialize, we want to fail
-      // immediately.  After a successful load, exceptions on future reloads
-      // will just result in leaving things as they are.
-      try {
-        allocsLoader.reloadAllocations();
-      } catch (Exception e) {
-        throw new IOException("Failed to initialize FairScheduler", e);
-      }
-      allocsLoader.start();
-    } else {
-      try {
-        allocsLoader.reloadAllocations();
-      } catch (Exception e) {
-        LOG.error("Failed to reload allocations file", e);
-      }
-    }
-  }
-
-  @Override
-  public QueueInfo getQueueInfo(String queueName, boolean includeChildQueues,
-      boolean recursive) throws IOException {
-    if (!queueMgr.exists(queueName)) {
-      throw new IOException("queue " + queueName + " does not exist");
-    }
-    return queueMgr.getQueue(queueName).getQueueInfo(includeChildQueues,
-        recursive);
-  }
-
-  @Override
-  public List<QueueUserACLInfo> getQueueUserAclInfo() {
-    UserGroupInformation user = null;
-    try {
-      user = UserGroupInformation.getCurrentUser();
-    } catch (IOException ioe) {
-      return new ArrayList<QueueUserACLInfo>();
-    }
-
-    return queueMgr.getRootQueue().getQueueUserAclInfo(user);
-  }
-
-  @Override
-  public int getNumClusterNodes() {
-    return nodes.size();
-  }
-
-  @Override
-  public synchronized boolean checkAccess(UserGroupInformation callerUGI,
-      QueueACL acl, String queueName) {
-    FSQueue queue = getQueueManager().getQueue(queueName);
-    if (queue == null) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("ACL not found for queue access-type " + acl
-            + " for queue " + queueName);
-      }
-      return false;
-    }
-    return queue.hasAccess(acl, callerUGI);
-  }
-  
-  public AllocationConfiguration getAllocationConfiguration() {
-    return allocConf;
-  }
-  
-  private class AllocationReloadListener implements
-      AllocationFileLoaderService.Listener {
-
-    @Override
-    public void onReload(AllocationConfiguration queueInfo) {
-      // Commit the reload; also create any queue defined in the alloc file
-      // if it does not already exist, so it can be displayed on the web UI.
-      synchronized (FairScheduler.this) {
-        allocConf = queueInfo;
-        allocConf.getDefaultSchedulingPolicy().initialize(clusterCapacity);
-        queueMgr.updateAllocationConfiguration(allocConf);
-      }
-    }
-  }
-
-  @Override
-  public List<ApplicationAttemptId> getAppsInQueue(String queueName) {
-    FSQueue queue = queueMgr.getQueue(queueName);
-    if (queue == null) {
-      return null;
-    }
-    List<ApplicationAttemptId> apps = new ArrayList<ApplicationAttemptId>();
-    queue.collectSchedulerApplications(apps);
-    return apps;
-  }
-
-}

+ 0 - 615
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java.orig

@@ -1,615 +0,0 @@
-/**
-* Licensed to the Apache Software Foundation (ASF) under one
-* or more contributor license agreements.  See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership.  The ASF licenses this file
-* to you under the Apache License, Version 2.0 (the
-* "License"); you may not use this file except in compliance
-* with the License.  You may obtain a copy of the License at
-*
-*     http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing, software
-* distributed under the License is distributed on an "AS IS" BASIS,
-* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-* See the License for the specific language governing permissions and
-* limitations under the License.
-*/
-
-package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-import java.lang.reflect.Method;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-
-import junit.framework.Assert;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.net.NetworkTopology;
-import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.NodeId;
-import org.apache.hadoop.yarn.api.records.Priority;
-import org.apache.hadoop.yarn.api.records.QueueInfo;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.api.records.ResourceOption;
-import org.apache.hadoop.yarn.api.records.ResourceRequest;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.event.AsyncDispatcher;
-import org.apache.hadoop.yarn.event.InlineDispatcher;
-import org.apache.hadoop.yarn.exceptions.YarnException;
-import org.apache.hadoop.yarn.factories.RecordFactory;
-import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
-import org.apache.hadoop.yarn.server.resourcemanager.Application;
-import org.apache.hadoop.yarn.server.resourcemanager.MockNodes;
-import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
-import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
-import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl;
-import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
-import org.apache.hadoop.yarn.server.resourcemanager.Task;
-import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.TestSchedulerUtils;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestCapacityScheduler;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
-import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM;
-import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
-import org.apache.hadoop.yarn.server.utils.BuilderUtils;
-import org.apache.hadoop.yarn.util.resource.Resources;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-public class TestFifoScheduler {
-  private static final Log LOG = LogFactory.getLog(TestFifoScheduler.class);
-  private final int GB = 1024;
-
-  private ResourceManager resourceManager = null;
-  
-  private static final RecordFactory recordFactory = 
-      RecordFactoryProvider.getRecordFactory(null);
-  
-  @Before
-  public void setUp() throws Exception {
-    resourceManager = new ResourceManager();
-    Configuration conf = new Configuration();
-    conf.setClass(YarnConfiguration.RM_SCHEDULER, 
-        FifoScheduler.class, ResourceScheduler.class);
-    resourceManager.init(conf);
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    resourceManager.stop();
-  }
-  
-  private org.apache.hadoop.yarn.server.resourcemanager.NodeManager
-      registerNode(String hostName, int containerManagerPort, int nmHttpPort,
-          String rackName, Resource capability) throws IOException,
-          YarnException {
-    return new org.apache.hadoop.yarn.server.resourcemanager.NodeManager(
-        hostName, containerManagerPort, nmHttpPort, rackName, capability,
-        resourceManager);
-  }
-  
-  private ApplicationAttemptId createAppAttemptId(int appId, int attemptId) {
-    ApplicationId appIdImpl = ApplicationId.newInstance(0, appId);
-    ApplicationAttemptId attId =
-        ApplicationAttemptId.newInstance(appIdImpl, attemptId);
-    return attId;
-  }
-
-  private ResourceRequest createResourceRequest(int memory, String host,
-      int priority, int numContainers) {
-    ResourceRequest request = recordFactory
-        .newRecordInstance(ResourceRequest.class);
-    request.setCapability(Resources.createResource(memory));
-    request.setResourceName(host);
-    request.setNumContainers(numContainers);
-    Priority prio = recordFactory.newRecordInstance(Priority.class);
-    prio.setPriority(priority);
-    request.setPriority(prio);
-    return request;
-  }
-
-  @Test(timeout=5000)
-  public void testFifoSchedulerCapacityWhenNoNMs() {
-    FifoScheduler scheduler = new FifoScheduler();
-    QueueInfo queueInfo = scheduler.getQueueInfo(null, false, false);
-    Assert.assertEquals(0.0f, queueInfo.getCurrentCapacity());
-  }
-  
-  @Test(timeout=5000)
-  public void testAppAttemptMetrics() throws Exception {
-    AsyncDispatcher dispatcher = new InlineDispatcher();
-    RMContext rmContext = new RMContextImpl(dispatcher, null,
-        null, null, null, null, null, null, null);
-
-    FifoScheduler schedular = new FifoScheduler();
-    schedular.reinitialize(new Configuration(), rmContext);
-    QueueMetrics metrics = schedular.getRootQueueMetrics();
-    int beforeAppsSubmitted = metrics.getAppsSubmitted();
-
-    ApplicationId appId = BuilderUtils.newApplicationId(200, 1);
-    ApplicationAttemptId appAttemptId = BuilderUtils.newApplicationAttemptId(
-        appId, 1);
-
-    SchedulerEvent appEvent = new AppAddedSchedulerEvent(appId, "queue", "user");
-    schedular.handle(appEvent);
-    SchedulerEvent attemptEvent =
-        new AppAttemptAddedSchedulerEvent(appAttemptId, false);
-    schedular.handle(attemptEvent);
-
-    appAttemptId = BuilderUtils.newApplicationAttemptId(appId, 2);
-    SchedulerEvent attemptEvent2 =
-        new AppAttemptAddedSchedulerEvent(appAttemptId, false);
-    schedular.handle(attemptEvent2);
-
-    int afterAppsSubmitted = metrics.getAppsSubmitted();
-    Assert.assertEquals(1, afterAppsSubmitted - beforeAppsSubmitted);
-  }
-
-  @Test(timeout=2000)
-  public void testNodeLocalAssignment() throws Exception {
-    AsyncDispatcher dispatcher = new InlineDispatcher();
-    Configuration conf = new Configuration();
-    RMContainerTokenSecretManager containerTokenSecretManager =
-        new RMContainerTokenSecretManager(conf);
-    containerTokenSecretManager.rollMasterKey();
-    NMTokenSecretManagerInRM nmTokenSecretManager =
-        new NMTokenSecretManagerInRM(conf);
-    nmTokenSecretManager.rollMasterKey();
-    RMContext rmContext = new RMContextImpl(dispatcher, null, null, null, null,
-        null, containerTokenSecretManager, nmTokenSecretManager, null);
-
-    FifoScheduler scheduler = new FifoScheduler();
-    scheduler.reinitialize(new Configuration(), rmContext);
-
-    RMNode node0 = MockNodes.newNodeInfo(1,
-        Resources.createResource(1024 * 64), 1, "127.0.0.1");
-    NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node0);
-    scheduler.handle(nodeEvent1);
-
-    int _appId = 1;
-    int _appAttemptId = 1;
-    ApplicationAttemptId appAttemptId = createAppAttemptId(_appId,
-        _appAttemptId);
-    AppAddedSchedulerEvent appEvent =
-        new AppAddedSchedulerEvent(appAttemptId.getApplicationId(), "queue1",
-          "user1");
-    scheduler.handle(appEvent);
-    AppAttemptAddedSchedulerEvent attemptEvent =
-        new AppAttemptAddedSchedulerEvent(appAttemptId, false);
-    scheduler.handle(attemptEvent);
-
-    int memory = 64;
-    int nConts = 3;
-    int priority = 20;
-
-    List<ResourceRequest> ask = new ArrayList<ResourceRequest>();
-    ResourceRequest nodeLocal = createResourceRequest(memory,
-        node0.getHostName(), priority, nConts);
-    ResourceRequest rackLocal = createResourceRequest(memory,
-        node0.getRackName(), priority, nConts);
-    ResourceRequest any = createResourceRequest(memory, ResourceRequest.ANY, priority,
-        nConts);
-    ask.add(nodeLocal);
-    ask.add(rackLocal);
-    ask.add(any);
-    scheduler.allocate(appAttemptId, ask, new ArrayList<ContainerId>(), null, null);
-
-    NodeUpdateSchedulerEvent node0Update = new NodeUpdateSchedulerEvent(node0);
-
-    // Before the node update event, there are 3 local requests outstanding
-    Assert.assertEquals(3, nodeLocal.getNumContainers());
-
-    scheduler.handle(node0Update);
-
-    // After the node update event, check that there are no more local requests
-    // outstanding
-    Assert.assertEquals(0, nodeLocal.getNumContainers());
-    //Also check that the containers were scheduled
-    SchedulerAppReport info = scheduler.getSchedulerAppInfo(appAttemptId);
-    Assert.assertEquals(3, info.getLiveContainers().size());
-  }
-  
-  @Test(timeout=2000)
-  public void testUpdateResourceOnNode() throws Exception {
-    AsyncDispatcher dispatcher = new InlineDispatcher();
-    Configuration conf = new Configuration();
-    RMContainerTokenSecretManager containerTokenSecretManager =
-        new RMContainerTokenSecretManager(conf);
-    containerTokenSecretManager.rollMasterKey();
-    NMTokenSecretManagerInRM nmTokenSecretManager =
-        new NMTokenSecretManagerInRM(conf);
-    nmTokenSecretManager.rollMasterKey();
-    RMContext rmContext = new RMContextImpl(dispatcher, null, null, null, null,
-        null, containerTokenSecretManager, nmTokenSecretManager, null);
-
-    FifoScheduler scheduler = new FifoScheduler(){
-      @SuppressWarnings("unused")
-      public Map<NodeId, FiCaSchedulerNode> getNodes(){
-        return nodes;
-      }
-    };
-    scheduler.reinitialize(new Configuration(), rmContext);
-    RMNode node0 = MockNodes.newNodeInfo(1,
-        Resources.createResource(2048, 4), 1, "127.0.0.1");
-    NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node0);
-    scheduler.handle(nodeEvent1);
-    
-    Method method = scheduler.getClass().getDeclaredMethod("getNodes");
-    @SuppressWarnings("unchecked")
-    Map<NodeId, FiCaSchedulerNode> schedulerNodes = 
-        (Map<NodeId, FiCaSchedulerNode>) method.invoke(scheduler);
-    assertEquals(schedulerNodes.values().size(), 1);
-    
-    // set resource of RMNode to 1024 and verify it works.
-    node0.setResourceOption(ResourceOption.newInstance(
-        Resources.createResource(1024, 4), RMNode.OVER_COMMIT_TIMEOUT_MILLIS_DEFAULT));
-    assertEquals(node0.getTotalCapability().getMemory(), 1024);
-    // verify that SchedulerNode's resource hasn't been changed.
-    assertEquals(schedulerNodes.get(node0.getNodeID()).
-        getAvailableResource().getMemory(), 2048);
-    // now, NM heartbeat comes.
-    NodeUpdateSchedulerEvent node0Update = new NodeUpdateSchedulerEvent(node0);
-    scheduler.handle(node0Update);
-    // SchedulerNode's available resource is changed.
-    assertEquals(schedulerNodes.get(node0.getNodeID()).
-        getAvailableResource().getMemory(), 1024);
-    QueueInfo queueInfo = scheduler.getQueueInfo(null, false, false);
-    Assert.assertEquals(0.0f, queueInfo.getCurrentCapacity());
-    
-    int _appId = 1;
-    int _appAttemptId = 1;
-    ApplicationAttemptId appAttemptId = createAppAttemptId(_appId,
-        _appAttemptId);
-    AppAddedSchedulerEvent appEvent =
-        new AppAddedSchedulerEvent(appAttemptId.getApplicationId(), "queue1",
-          "user1");
-    scheduler.handle(appEvent);
-    AppAttemptAddedSchedulerEvent attemptEvent =
-        new AppAttemptAddedSchedulerEvent(appAttemptId, false);
-    scheduler.handle(attemptEvent);
-
-    int memory = 1024;
-    int priority = 1;
-
-    List<ResourceRequest> ask = new ArrayList<ResourceRequest>();
-    ResourceRequest nodeLocal = createResourceRequest(memory,
-        node0.getHostName(), priority, 1);
-    ResourceRequest rackLocal = createResourceRequest(memory,
-        node0.getRackName(), priority, 1);
-    ResourceRequest any = createResourceRequest(memory, ResourceRequest.ANY, priority,
-        1);
-    ask.add(nodeLocal);
-    ask.add(rackLocal);
-    ask.add(any);
-    scheduler.allocate(appAttemptId, ask, new ArrayList<ContainerId>(), null, null);
-
-    // Before the node update event, there are one local request
-    Assert.assertEquals(1, nodeLocal.getNumContainers());
-
-    // Now schedule.
-    scheduler.handle(node0Update);
-
-    // After the node update event, check no local request
-    Assert.assertEquals(0, nodeLocal.getNumContainers());
-    // Also check that one container was scheduled
-    SchedulerAppReport info = scheduler.getSchedulerAppInfo(appAttemptId);
-    Assert.assertEquals(1, info.getLiveContainers().size());
-    // And check the default Queue now is full.
-    queueInfo = scheduler.getQueueInfo(null, false, false);
-    Assert.assertEquals(1.0f, queueInfo.getCurrentCapacity());
-  }
-  
-//  @Test
-  public void testFifoScheduler() throws Exception {
-
-    LOG.info("--- START: testFifoScheduler ---");
-        
-    final int GB = 1024;
-    
-    // Register node1
-    String host_0 = "host_0";
-    org.apache.hadoop.yarn.server.resourcemanager.NodeManager nm_0 = 
-      registerNode(host_0, 1234, 2345, NetworkTopology.DEFAULT_RACK, 
-          Resources.createResource(4 * GB, 1));
-    nm_0.heartbeat();
-    
-    // Register node2
-    String host_1 = "host_1";
-    org.apache.hadoop.yarn.server.resourcemanager.NodeManager nm_1 = 
-      registerNode(host_1, 1234, 2345, NetworkTopology.DEFAULT_RACK, 
-          Resources.createResource(2 * GB, 1));
-    nm_1.heartbeat();
-
-    // ResourceRequest priorities
-    Priority priority_0 = 
-      org.apache.hadoop.yarn.server.resourcemanager.resource.Priority.create(0); 
-    Priority priority_1 = 
-      org.apache.hadoop.yarn.server.resourcemanager.resource.Priority.create(1);
-    
-    // Submit an application
-    Application application_0 = new Application("user_0", resourceManager);
-    application_0.submit();
-    
-    application_0.addNodeManager(host_0, 1234, nm_0);
-    application_0.addNodeManager(host_1, 1234, nm_1);
-
-    Resource capability_0_0 = Resources.createResource(GB);
-    application_0.addResourceRequestSpec(priority_1, capability_0_0);
-    
-    Resource capability_0_1 = Resources.createResource(2 * GB);
-    application_0.addResourceRequestSpec(priority_0, capability_0_1);
-
-    Task task_0_0 = new Task(application_0, priority_1, 
-        new String[] {host_0, host_1});
-    application_0.addTask(task_0_0);
-       
-    // Submit another application
-    Application application_1 = new Application("user_1", resourceManager);
-    application_1.submit();
-    
-    application_1.addNodeManager(host_0, 1234, nm_0);
-    application_1.addNodeManager(host_1, 1234, nm_1);
-    
-    Resource capability_1_0 = Resources.createResource(3 * GB);
-    application_1.addResourceRequestSpec(priority_1, capability_1_0);
-    
-    Resource capability_1_1 = Resources.createResource(4 * GB);
-    application_1.addResourceRequestSpec(priority_0, capability_1_1);
-
-    Task task_1_0 = new Task(application_1, priority_1, 
-        new String[] {host_0, host_1});
-    application_1.addTask(task_1_0);
-        
-    // Send resource requests to the scheduler
-    LOG.info("Send resource requests to the scheduler");
-    application_0.schedule();
-    application_1.schedule();
-    
-    // Send a heartbeat to kick the tires on the Scheduler
-    LOG.info("Send a heartbeat to kick the tires on the Scheduler... " +
-    		"nm0 -> task_0_0 and task_1_0 allocated, used=4G " +
-    		"nm1 -> nothing allocated");
-    nm_0.heartbeat();             // task_0_0 and task_1_0 allocated, used=4G
-    nm_1.heartbeat();             // nothing allocated
-    
-    // Get allocations from the scheduler
-    application_0.schedule();     // task_0_0 
-    checkApplicationResourceUsage(GB, application_0);
-
-    application_1.schedule();     // task_1_0
-    checkApplicationResourceUsage(3 * GB, application_1);
-    
-    nm_0.heartbeat();
-    nm_1.heartbeat();
-    
-    checkNodeResourceUsage(4*GB, nm_0);  // task_0_0 (1G) and task_1_0 (3G)
-    checkNodeResourceUsage(0*GB, nm_1);  // no tasks, 2G available
-    
-    LOG.info("Adding new tasks...");
-    
-    Task task_1_1 = new Task(application_1, priority_1, 
-        new String[] {ResourceRequest.ANY});
-    application_1.addTask(task_1_1);
-
-    Task task_1_2 = new Task(application_1, priority_1, 
-        new String[] {ResourceRequest.ANY});
-    application_1.addTask(task_1_2);
-
-    Task task_1_3 = new Task(application_1, priority_0, 
-        new String[] {ResourceRequest.ANY});
-    application_1.addTask(task_1_3);
-    
-    application_1.schedule();
-    
-    Task task_0_1 = new Task(application_0, priority_1, 
-        new String[] {host_0, host_1});
-    application_0.addTask(task_0_1);
-
-    Task task_0_2 = new Task(application_0, priority_1, 
-        new String[] {host_0, host_1});
-    application_0.addTask(task_0_2);
-    
-    Task task_0_3 = new Task(application_0, priority_0, 
-        new String[] {ResourceRequest.ANY});
-    application_0.addTask(task_0_3);
-
-    application_0.schedule();
-
-    // Send a heartbeat to kick the tires on the Scheduler
-    LOG.info("Sending hb from " + nm_0.getHostName());
-    nm_0.heartbeat();                   // nothing new, used=4G
-    
-    LOG.info("Sending hb from " + nm_1.getHostName());
-    nm_1.heartbeat();                   // task_0_3, used=2G
-    
-    // Get allocations from the scheduler
-    LOG.info("Trying to allocate...");
-    application_0.schedule();
-    checkApplicationResourceUsage(3 * GB, application_0);
-    application_1.schedule();
-    checkApplicationResourceUsage(3 * GB, application_1);
-    nm_0.heartbeat();
-    nm_1.heartbeat();
-    checkNodeResourceUsage(4*GB, nm_0);
-    checkNodeResourceUsage(2*GB, nm_1);
-    
-    // Complete tasks
-    LOG.info("Finishing up task_0_0");
-    application_0.finishTask(task_0_0); // Now task_0_1
-    application_0.schedule();
-    application_1.schedule();
-    nm_0.heartbeat();
-    nm_1.heartbeat();
-    checkApplicationResourceUsage(3 * GB, application_0);
-    checkApplicationResourceUsage(3 * GB, application_1);
-    checkNodeResourceUsage(4*GB, nm_0);
-    checkNodeResourceUsage(2*GB, nm_1);
-
-    LOG.info("Finishing up task_1_0");
-    application_1.finishTask(task_1_0);  // Now task_0_2
-    application_0.schedule(); // final overcommit for app0 caused here
-    application_1.schedule();
-    nm_0.heartbeat(); // final overcommit for app0 occurs here
-    nm_1.heartbeat();
-    checkApplicationResourceUsage(4 * GB, application_0);
-    checkApplicationResourceUsage(0 * GB, application_1);
-    //checkNodeResourceUsage(1*GB, nm_0);  // final over-commit -> rm.node->1G, test.node=2G
-    checkNodeResourceUsage(2*GB, nm_1);
-
-    LOG.info("Finishing up task_0_3");
-    application_0.finishTask(task_0_3); // No more
-    application_0.schedule();
-    application_1.schedule();
-    nm_0.heartbeat();
-    nm_1.heartbeat();
-    checkApplicationResourceUsage(2 * GB, application_0);
-    checkApplicationResourceUsage(0 * GB, application_1);
-    //checkNodeResourceUsage(2*GB, nm_0);  // final over-commit, rm.node->1G, test.node->2G
-    checkNodeResourceUsage(0*GB, nm_1);
-    
-    LOG.info("Finishing up task_0_1");
-    application_0.finishTask(task_0_1);
-    application_0.schedule();
-    application_1.schedule();
-    nm_0.heartbeat();
-    nm_1.heartbeat();
-    checkApplicationResourceUsage(1 * GB, application_0);
-    checkApplicationResourceUsage(0 * GB, application_1);
-    
-    LOG.info("Finishing up task_0_2");
-    application_0.finishTask(task_0_2); // now task_1_3 can go!
-    application_0.schedule();
-    application_1.schedule();
-    nm_0.heartbeat();
-    nm_1.heartbeat();
-    checkApplicationResourceUsage(0 * GB, application_0);
-    checkApplicationResourceUsage(4 * GB, application_1);
-    
-    LOG.info("Finishing up task_1_3");
-    application_1.finishTask(task_1_3); // now task_1_1
-    application_0.schedule();
-    application_1.schedule();
-    nm_0.heartbeat();
-    nm_1.heartbeat();
-    checkApplicationResourceUsage(0 * GB, application_0);
-    checkApplicationResourceUsage(3 * GB, application_1);
-    
-    LOG.info("Finishing up task_1_1");
-    application_1.finishTask(task_1_1);
-    application_0.schedule();
-    application_1.schedule();
-    nm_0.heartbeat();
-    nm_1.heartbeat();
-    checkApplicationResourceUsage(0 * GB, application_0);
-    checkApplicationResourceUsage(3 * GB, application_1);
-    
-    LOG.info("--- END: testFifoScheduler ---");
-  }
-
-  @SuppressWarnings("resource")
-  @Test
-  public void testBlackListNodes() throws Exception {
-    Configuration conf = new Configuration();
-    conf.setClass(YarnConfiguration.RM_SCHEDULER, FifoScheduler.class,
-        ResourceScheduler.class);
-    MockRM rm = new MockRM(conf);
-    rm.start();
-    FifoScheduler fs = (FifoScheduler) rm.getResourceScheduler();
-
-    String host = "127.0.0.1";
-    RMNode node =
-        MockNodes.newNodeInfo(0, MockNodes.newResource(4 * GB), 1, host);
-    fs.handle(new NodeAddedSchedulerEvent(node));
-
-    ApplicationId appId = BuilderUtils.newApplicationId(100, 1);
-    ApplicationAttemptId appAttemptId = BuilderUtils.newApplicationAttemptId(
-        appId, 1);
-    SchedulerEvent appEvent =
-        new AppAddedSchedulerEvent(appId, "default",
-          "user");
-    fs.handle(appEvent);
-    SchedulerEvent attemptEvent =
-        new AppAttemptAddedSchedulerEvent(appAttemptId, false);
-    fs.handle(attemptEvent);
-
-    // Verify the blacklist can be updated independent of requesting containers
-    fs.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
-        Collections.<ContainerId>emptyList(),
-        Collections.singletonList(host), null);
-    Assert.assertTrue(fs.getApplicationAttempt(appAttemptId).isBlacklisted(host));
-    fs.allocate(appAttemptId, Collections.<ResourceRequest>emptyList(),
-        Collections.<ContainerId>emptyList(), null,
-        Collections.singletonList(host));
-    Assert.assertFalse(fs.getApplicationAttempt(appAttemptId).isBlacklisted(host));
-    rm.stop();
-  }
-  
-  @Test
-  public void testGetAppsInQueue() throws Exception {
-    Application application_0 = new Application("user_0", resourceManager);
-    application_0.submit();
-    
-    Application application_1 = new Application("user_0", resourceManager);
-    application_1.submit();
-    
-    ResourceScheduler scheduler = resourceManager.getResourceScheduler();
-    
-    List<ApplicationAttemptId> appsInDefault = scheduler.getAppsInQueue("default");
-    assertTrue(appsInDefault.contains(application_0.getApplicationAttemptId()));
-    assertTrue(appsInDefault.contains(application_1.getApplicationAttemptId()));
-    assertEquals(2, appsInDefault.size());
-    
-    Assert.assertNull(scheduler.getAppsInQueue("someotherqueue"));
-  }
-
-  @Test
-  public void testAddAndRemoveAppFromFiFoScheduler() throws Exception {
-    Configuration conf = new Configuration();
-    conf.setClass(YarnConfiguration.RM_SCHEDULER, FifoScheduler.class,
-        ResourceScheduler.class);
-    MockRM rm = new MockRM(conf);
-    FifoScheduler fs = (FifoScheduler)rm.getResourceScheduler();
-    TestSchedulerUtils.verifyAppAddedAndRemovedFromScheduler(fs.applications,
-      fs, "queue");
-  }
-
-  private void checkApplicationResourceUsage(int expected, 
-      Application application) {
-    Assert.assertEquals(expected, application.getUsedResources().getMemory());
-  }
-  
-  private void checkNodeResourceUsage(int expected,
-      org.apache.hadoop.yarn.server.resourcemanager.NodeManager node) {
-    Assert.assertEquals(expected, node.getUsed().getMemory());
-    node.checkResourceUsage();
-  }
-
-  public static void main(String[] arg) throws Exception {
-    TestFifoScheduler t = new TestFifoScheduler();
-    t.setUp();
-    t.testFifoScheduler();
-    t.tearDown();
-  }
-}

+ 36 - 1
hadoop-yarn-project/pom.xml

@@ -41,7 +41,42 @@
     <module>hadoop-yarn</module>
     <module>hadoop-yarn</module>
   </modules>
   </modules>
 
 
-  <!-- Do not add dependencies here, add them to the POM of the leaf module -->
+  <!--
+  Do not add 3rd party dependencies here, add them to the POM of the leaf module
+
+  The dependencies in this module are for the assembly plugin, packaging purposes
+  -->
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-api</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-client</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-common</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-server-common</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-server-nodemanager</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-server-resourcemanager</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-server-web-proxy</artifactId>
+    </dependency>
+  </dependencies>
 
 
   <build>
   <build>
     <plugins>
     <plugins>