瀏覽代碼

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 年之前
父節點
當前提交
1025121874
共有 74 個文件被更改,包括 3200 次插入2333 次删除
  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:
 
 * 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)
+* 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
 * 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>
       <!-- Exclude hadoop artifacts. They will be found via HADOOP* env -->
       <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 -->
         <exclude>org.slf4j:slf4j-api</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.
     (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.
     (Binglin Chang via Colin Patrick McCabe)
@@ -110,6 +111,8 @@ Trunk (Unreleased)
 
     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
 
     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-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
     (Jon Eagles via bobby)
@@ -517,6 +520,16 @@ Release 2.4.0 - UNRELEASED
     HADOOP-10223. MiniKdc#main() should close the FileReader it creates. 
     (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
 
   INCOMPATIBLE CHANGES
@@ -605,6 +618,13 @@ Release 2.3.0 - UNRELEASED
     HADOOP-10178. Configuration deprecation always emit "deprecated" warnings
     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
 
   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
     elif [ "$COMMAND" = "jar" ] ; then
       CLASS=org.apache.hadoop.util.RunJar
+    elif [ "$COMMAND" = "key" ] ; then
+      CLASS=org.apache.hadoop.crypto.key.KeyShell
     elif [ "$COMMAND" = "checknative" ] ; then
       CLASS=org.apache.hadoop.util.NativeLibraryChecker
     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%
 
-  goto :eof
+  exit /b %ERRORLEVEL%
 
 :fs 
   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 {
     this.uri = uri;
     path = unnestUri(uri);
-    fs = FileSystem.get(conf);
+    fs = path.getFileSystem(conf);
     // Get the password from the user's environment
     String pw = System.getenv(KEYSTORE_PASSWORD_NAME);
     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);
   }
 
+  /**
+   * 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
    * 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();
   }
 
+  @Override
+  public boolean isTransient() {
+    return true;
+  }
+
   @Override
   public KeyVersion getKeyVersion(String 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, clientId)) {
           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()));
         }
       }

+ 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()) {
             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 error = null;
           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.IOException;
 import java.io.InputStreamReader;
+import java.io.InputStream;
 import java.util.Arrays;
 import java.util.Map;
 import java.util.Timer;
@@ -511,7 +512,17 @@ abstract public class Shell {
       }
       // close the input stream
       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) {
         LOG.warn("Error while closing the input stream", ioe);
       }
@@ -524,7 +535,10 @@ abstract public class Shell {
         LOG.warn("Interrupted while joining errThread");
       }
       try {
-        errReader.close();
+        InputStream stderr = process.getErrorStream();
+        synchronized (stderr) {
+          errReader.close();
+        }
       } catch (IOException 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-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
 
   INCOMPATIBLE CHANGES
@@ -512,6 +521,9 @@ Release 2.4.0 - UNRELEASED
     as a collection of storages (see breakdown of tasks below for features and
     contributors).
 
+    HDFS-5784. reserve space in edit log header and fsimage header for feature
+    flag section (cmccabe)
+
   IMPROVEMENTS
 
     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-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
 
     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
     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
 
     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"
    * because it first reads the data to user buffer and then checks
    * 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;
 

+ 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)
       throws IOException {
-    if (dataBuf == null) return 0;
+    if (dataBuf == null) return -1;
     int oldLimit = dataBuf.limit();
     int nRead = Math.min(dataBuf.remaining(), buf.remaining());
-    if (nRead == 0) return 0;
+    if (nRead == 0) {
+      return (dataBuf.remaining() == 0) ? -1 : 0;
+    }
     try {
       dataBuf.limit(dataBuf.position() + nRead);
       buf.put(dataBuf);
@@ -444,13 +446,11 @@ class BlockReaderLocal implements BlockReader {
     int total = 0;
     while (buf.hasRemaining()) {
       int nRead = dataIn.read(buf, dataPos);
-      if (nRead < 0) {
-        break;
-      }
+      if (nRead <= 0) break;
       dataPos += 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,
         boolean canSkipChecksum) throws IOException {
     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;
-      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
         // bounce buffer.
         int oldLimit = buf.limit();
@@ -540,9 +540,13 @@ class BlockReaderLocal implements BlockReader {
         if (fillDataBuf(canSkipChecksum)) {
           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
@@ -587,8 +591,10 @@ class BlockReaderLocal implements BlockReader {
     int nRead = dataIn.read(ByteBuffer.wrap(arr, off, len), dataPos);
     if (nRead > 0) {
       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,
@@ -599,9 +605,10 @@ class BlockReaderLocal implements BlockReader {
       dataBuf.limit(maxReadaheadLength);
       fillDataBuf(canSkipChecksum);
     }
+    if (dataBuf.remaining() == 0) return -1;
     int toRead = Math.min(dataBuf.remaining(), len);
     dataBuf.get(arr, off, toRead);
-    return toRead == 0 ? -1 : toRead;
+    return toRead;
   }
 
   @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.
           refetchEncryptionKey--;
           dfsClient.clearDataEncryptionKey();
+          continue;
         } else if (refetchToken > 0 && tokenRefetchNeeded(e, targetAddr)) {
           refetchToken--;
           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"),
     EDITLOG_SUPPORT_RETRYCACHE(-47, "Record ClientId and CallId in editlog to " 
         + "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."
         + " 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 ancestorLV;
     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);
   
   private final FSNamesystem namesystem;
+  private final SerializerCompat serializerCompat = new SerializerCompat();
 
   public DelegationTokenSecretManager(long delegationKeyUpdateInterval,
       long delegationTokenMaxLifetime, long delegationTokenRenewInterval,
@@ -150,24 +151,21 @@ public class DelegationTokenSecretManager
       throw new IOException("No delegation token found for this identifier");
     }
   }
-  
+
   /**
    * Load SecretManager state from fsimage.
    * 
    * @param in input stream to read fsimage
    * @throws IOException
    */
-  public synchronized void loadSecretManagerState(DataInput in)
+  public synchronized void loadSecretManagerStateCompat(DataInput in)
       throws IOException {
     if (running) {
       // a safety check
       throw new IOException(
           "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
    * @throws IOException
    */
-  public synchronized void saveSecretManagerState(DataOutputStream out,
+  public synchronized void saveSecretManagerStateCompat(DataOutputStream out,
       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();
   }
 
-  /**
-   * 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.
    */
@@ -420,4 +330,105 @@ public class DelegationTokenSecretManager
     c.addToken(new Text(ugi.getShortUserName()), token);
     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 SerializerCompat serializerCompat = new SerializerCompat();
+
   /**
    * The CacheReplicationMonitor.
    */
@@ -926,11 +928,9 @@ public final class CacheManager {
    * @param sdPath path of the storage directory
    * @throws IOException
    */
-  public void saveState(DataOutputStream out, String sdPath)
+  public void saveStateCompat(DataOutputStream out, String sdPath)
       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
    * @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() {

+ 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.hdfs.DFSConfigKeys;
 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.namenode.TransferFsImage.HttpGetFailedException;
 import org.apache.hadoop.hdfs.web.URLConnectionFactory;
@@ -146,6 +149,14 @@ public class EditLogFileInputStream extends EditLogInputStream {
       } catch (EOFException eofe) {
         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.setMaxOpSize(maxOpSize);
       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.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.LayoutFlags;
 import org.apache.hadoop.io.IOUtils;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -132,6 +133,7 @@ public class EditLogFileOutputStream extends EditLogOutputStream {
   @VisibleForTesting
   public static void writeHeader(DataOutputStream out) throws IOException {
     out.writeInt(HdfsConstants.LAYOUT_VERSION);
+    LayoutFlags.write(out);
   }
 
   @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.
    */

+ 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.HdfsConstants;
 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.Storage.FormatConfirmable;
 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.AddCachePoolOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddOp;
@@ -710,6 +712,17 @@ public class FSEditLog implements LogsPurgeable {
     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) {
     Preconditions.checkArgument(file.isUnderConstruction());
     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.BlockInfoUnderConstruction;
 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.AddCachePoolOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCloseOp;
@@ -417,7 +418,18 @@ public class FSEditLogLoader {
       }
       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: {
       SetReplicationOp setReplicationOp = (SetReplicationOp)op;
       short replication = fsNamesys.getBlockManager().adjustReplication(
@@ -722,6 +734,44 @@ public class FSEditLogLoader {
     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.
    * @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;
 
 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_POOL;
 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_REMOVE_CACHE_POOL, new RemoveCachePoolOp());
 
+      inst.put(OP_ADD_BLOCK, new AddBlockOp());
       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 @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_SET_GENSTAMP_V2            ((byte) 31),
   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.  
   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.LayoutVersion;
 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.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
@@ -261,6 +262,9 @@ public class FSImageFormat {
         }
         boolean supportSnapshot = LayoutVersion.supports(Feature.SNAPSHOT,
             imgVersion);
+        if (LayoutVersion.supports(Feature.ADD_LAYOUT_FLAGS, imgVersion)) {
+          LayoutFlags.read(in);
+        }
 
         // read namespaceID: first appeared in version -2
         in.readInt();
@@ -870,7 +874,7 @@ public class FSImageFormat {
         //This must not happen if security is turned on.
         return; 
       }
-      namesystem.loadSecretManagerState(in);
+      namesystem.loadSecretManagerStateCompat(in);
     }
 
     private void loadCacheManagerState(DataInput in) throws IOException {
@@ -878,7 +882,7 @@ public class FSImageFormat {
       if (!LayoutVersion.supports(Feature.CACHING, imgVersion)) {
         return;
       }
-      namesystem.getCacheManager().loadState(in);
+      namesystem.getCacheManager().loadStateCompat(in);
     }
 
     private int getLayoutVersion() {
@@ -990,6 +994,7 @@ public class FSImageFormat {
       DataOutputStream out = new DataOutputStream(fos);
       try {
         out.writeInt(HdfsConstants.LAYOUT_VERSION);
+        LayoutFlags.write(out);
         // We use the non-locked version of getNamespaceInfo here since
         // the coordinating thread of saveNamespace already has read-locked
         // the namespace for us. If we attempt to take another readlock
@@ -1032,9 +1037,9 @@ public class FSImageFormat {
         sourceNamesystem.saveFilesUnderConstruction(out, snapshotUCMap);
         
         context.checkCancelled();
-        sourceNamesystem.saveSecretManagerState(out, sdPath);
+        sourceNamesystem.saveSecretManagerStateCompat(out, sdPath);
         context.checkCancelled();
-        sourceNamesystem.getCacheManager().saveState(out, sdPath);
+        sourceNamesystem.getCacheManager().saveStateCompat(out, sdPath);
         context.checkCancelled();
         out.flush();
         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();
       saveAllocatedBlock(src, inodesInPath, newBlock, targets);
 
-      dir.persistBlocks(src, pendingFile, false);
+      dir.persistNewBlock(src, pendingFile);
       offset = pendingFile.computeFileSize();
     } finally {
       writeUnlock();
@@ -6239,16 +6239,16 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    * @param out save state of the secret manager
    * @param sdPath String storage directory path
    */
-  void saveSecretManagerState(DataOutputStream out, String sdPath)
+  void saveSecretManagerStateCompat(DataOutputStream out, String sdPath)
       throws IOException {
-    dtSecretManager.saveSecretManagerState(out, sdPath);
+    dtSecretManager.saveSecretManagerStateCompat(out, sdPath);
   }
 
   /**
    * @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.fs.permission.FsPermission;
 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.Feature;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
@@ -126,7 +127,7 @@ class ImageLoaderCurrent implements ImageLoader {
                                       new SimpleDateFormat("yyyy-MM-dd HH:mm");
   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,
-      -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 final Map<Long, Boolean> subtreeMap = new HashMap<Long, Boolean>();
@@ -157,6 +158,9 @@ class ImageLoaderCurrent implements ImageLoader {
       imageVersion = in.readInt();
       if( !canLoadVersion(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.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
   public void testBlockReaderLocalOnFileWithoutChecksum()
       throws IOException {
@@ -631,6 +649,35 @@ public class TestBlockReaderLocal {
     runBlockReaderLocalTest(new TestBlockReaderLocalOnFileWithoutChecksum(),
         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)
   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);
       // OP_ADD to create file
-      // OP_UPDATE_BLOCKS for first block
+      // OP_ADD_BLOCK for first block
       // OP_CLOSE to close file
       // OP_ADD to reopen file
-      // OP_UPDATE_BLOCKS for second block
+      // OP_ADD_BLOCK for second block
       // OP_CLOSE to close file
       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);
 
       Path p2 = new Path("/not-block-boundaries");
       writeAndAppend(fs, p2, BLOCK_SIZE/2, BLOCK_SIZE);
       counts = FSImageTestUtil.countEditLogOpTypes(editLog);
       // OP_ADD to create file
-      // OP_UPDATE_BLOCKS for first block
+      // OP_ADD_BLOCK for first block
       // OP_CLOSE to close file
       // OP_ADD to re-establish the lease
       // 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
-      // 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+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);
       
       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
   public void testAppendWithPipelineRecovery() throws Exception {
     Configuration conf = new Configuration();
     MiniDFSCluster cluster = null;
+    FSDataOutputStream out = null;
     try {
       cluster = new MiniDFSCluster.Builder(conf).manageDataDfsDirs(true)
           .manageNameDfsDirs(true).numDataNodes(4)
@@ -192,11 +194,20 @@ public class TestFileAppendRestart {
 
       DistributedFileSystem fs = cluster.getFileSystem();
       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);
-      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 {
+      IOUtils.closeStream(out);
       if (null != cluster) {
         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));
     byte[] expected = Bytes.concat(
             Ints.toByteArray(HdfsConstants.LAYOUT_VERSION),
+            (new byte[] { 0, 0, 0, 0 }), // layout flags section
             EDITS_DATA);
 
     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");
     SortedMap<Long, Long> offsetToTxId = Maps.newTreeMap();
     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 =
         EditLogFileInputStream.validateEditLog(logFile);
     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.assertTrue;
 
-import java.io.File;
 import java.io.FileOutputStream;
 import java.io.IOException;
 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"?>
 <EDITS>
+<<<<<<< .working
   <EDITS_VERSION>-50</EDITS_VERSION>
+=======
+  <EDITS_VERSION>-51</EDITS_VERSION>
+>>>>>>> .merge-right.r1559304
   <RECORD>
     <OPCODE>OP_START_LOG_SEGMENT</OPCODE>
     <DATA>
@@ -13,8 +17,13 @@
       <TXID>2</TXID>
       <DELEGATION_KEY>
         <KEY_ID>1</KEY_ID>
+<<<<<<< .working
         <EXPIRY_DATE>1389421314720</EXPIRY_DATE>
         <KEY>d2a03d66ebfac521</KEY>
+=======
+        <EXPIRY_DATE>1390519460949</EXPIRY_DATE>
+        <KEY>dc8d30edc97df67d</KEY>
+>>>>>>> .merge-right.r1559304
       </DELEGATION_KEY>
     </DATA>
   </RECORD>
@@ -24,8 +33,13 @@
       <TXID>3</TXID>
       <DELEGATION_KEY>
         <KEY_ID>2</KEY_ID>
+<<<<<<< .working
         <EXPIRY_DATE>1389421314722</EXPIRY_DATE>
         <KEY>ef94532092f55aef</KEY>
+=======
+        <EXPIRY_DATE>1390519460952</EXPIRY_DATE>
+        <KEY>096bc20b6debed03</KEY>
+>>>>>>> .merge-right.r1559304
       </DELEGATION_KEY>
     </DATA>
   </RECORD>
@@ -35,20 +49,38 @@
       <TXID>4</TXID>
       <LENGTH>0</LENGTH>
       <INODEID>16386</INODEID>
-      <PATH>/file_create_u\0001;F431</PATH>
+      <PATH>/file_create</PATH>
       <REPLICATION>1</REPLICATION>
+<<<<<<< .working
       <MTIME>1388730115316</MTIME>
       <ATIME>1388730115316</ATIME>
+=======
+      <MTIME>1389828264873</MTIME>
+      <ATIME>1389828264873</ATIME>
+>>>>>>> .merge-right.r1559304
       <BLOCKSIZE>512</BLOCKSIZE>
+<<<<<<< .working
       <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>
       <PERMISSION_STATUS>
+<<<<<<< .working
         <USERNAME>szetszwo</USERNAME>
+=======
+        <USERNAME>jing</USERNAME>
+>>>>>>> .merge-right.r1559304
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
+<<<<<<< .working
       <RPC_CLIENTID>8205c453-0c7f-4b25-955a-7786e56bce86</RPC_CLIENTID>
       <RPC_CALLID>6</RPC_CALLID>
+=======
+      <RPC_CLIENTID>b5928e80-e373-4807-a688-f94483d08ce5</RPC_CLIENTID>
+      <RPC_CALLID>9</RPC_CALLID>
+>>>>>>> .merge-right.r1559304
     </DATA>
   </RECORD>
   <RECORD>
@@ -57,15 +89,24 @@
       <TXID>5</TXID>
       <LENGTH>0</LENGTH>
       <INODEID>0</INODEID>
-      <PATH>/file_create_u\0001;F431</PATH>
+      <PATH>/file_create</PATH>
       <REPLICATION>1</REPLICATION>
+<<<<<<< .working
       <MTIME>1388730115327</MTIME>
       <ATIME>1388730115316</ATIME>
+=======
+      <MTIME>1389828265699</MTIME>
+      <ATIME>1389828264873</ATIME>
+>>>>>>> .merge-right.r1559304
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_MACHINE></CLIENT_MACHINE>
       <PERMISSION_STATUS>
+<<<<<<< .working
         <USERNAME>szetszwo</USERNAME>
+=======
+        <USERNAME>jing</USERNAME>
+>>>>>>> .merge-right.r1559304
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
@@ -76,11 +117,17 @@
     <DATA>
       <TXID>6</TXID>
       <LENGTH>0</LENGTH>
-      <SRC>/file_create_u\0001;F431</SRC>
+      <SRC>/file_create</SRC>
       <DST>/file_moved</DST>
+<<<<<<< .working
       <TIMESTAMP>1388730115331</TIMESTAMP>
       <RPC_CLIENTID>8205c453-0c7f-4b25-955a-7786e56bce86</RPC_CLIENTID>
       <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>
   </RECORD>
   <RECORD>
@@ -89,9 +136,15 @@
       <TXID>7</TXID>
       <LENGTH>0</LENGTH>
       <PATH>/file_moved</PATH>
+<<<<<<< .working
       <TIMESTAMP>1388730115336</TIMESTAMP>
       <RPC_CLIENTID>8205c453-0c7f-4b25-955a-7786e56bce86</RPC_CLIENTID>
       <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>
   </RECORD>
   <RECORD>
@@ -101,9 +154,17 @@
       <LENGTH>0</LENGTH>
       <INODEID>16387</INODEID>
       <PATH>/directory_mkdir</PATH>
+<<<<<<< .working
       <TIMESTAMP>1388730115342</TIMESTAMP>
+=======
+      <TIMESTAMP>1389828265722</TIMESTAMP>
+>>>>>>> .merge-right.r1559304
       <PERMISSION_STATUS>
+<<<<<<< .working
         <USERNAME>szetszwo</USERNAME>
+=======
+        <USERNAME>jing</USERNAME>
+>>>>>>> .merge-right.r1559304
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>493</MODE>
       </PERMISSION_STATUS>
@@ -136,8 +197,13 @@
       <TXID>12</TXID>
       <SNAPSHOTROOT>/directory_mkdir</SNAPSHOTROOT>
       <SNAPSHOTNAME>snapshot1</SNAPSHOTNAME>
+<<<<<<< .working
       <RPC_CLIENTID>8205c453-0c7f-4b25-955a-7786e56bce86</RPC_CLIENTID>
       <RPC_CALLID>14</RPC_CALLID>
+=======
+      <RPC_CLIENTID>b5928e80-e373-4807-a688-f94483d08ce5</RPC_CLIENTID>
+      <RPC_CALLID>17</RPC_CALLID>
+>>>>>>> .merge-right.r1559304
     </DATA>
   </RECORD>
   <RECORD>
@@ -147,8 +213,13 @@
       <SNAPSHOTROOT>/directory_mkdir</SNAPSHOTROOT>
       <SNAPSHOTOLDNAME>snapshot1</SNAPSHOTOLDNAME>
       <SNAPSHOTNEWNAME>snapshot2</SNAPSHOTNEWNAME>
+<<<<<<< .working
       <RPC_CLIENTID>8205c453-0c7f-4b25-955a-7786e56bce86</RPC_CLIENTID>
       <RPC_CALLID>15</RPC_CALLID>
+=======
+      <RPC_CLIENTID>b5928e80-e373-4807-a688-f94483d08ce5</RPC_CLIENTID>
+      <RPC_CALLID>18</RPC_CALLID>
+>>>>>>> .merge-right.r1559304
     </DATA>
   </RECORD>
   <RECORD>
@@ -157,8 +228,13 @@
       <TXID>14</TXID>
       <SNAPSHOTROOT>/directory_mkdir</SNAPSHOTROOT>
       <SNAPSHOTNAME>snapshot2</SNAPSHOTNAME>
+<<<<<<< .working
       <RPC_CLIENTID>8205c453-0c7f-4b25-955a-7786e56bce86</RPC_CLIENTID>
       <RPC_CALLID>16</RPC_CALLID>
+=======
+      <RPC_CLIENTID>b5928e80-e373-4807-a688-f94483d08ce5</RPC_CLIENTID>
+      <RPC_CALLID>19</RPC_CALLID>
+>>>>>>> .merge-right.r1559304
     </DATA>
   </RECORD>
   <RECORD>
@@ -167,20 +243,38 @@
       <TXID>15</TXID>
       <LENGTH>0</LENGTH>
       <INODEID>16388</INODEID>
-      <PATH>/file_create_u\0001;F431</PATH>
+      <PATH>/file_create</PATH>
       <REPLICATION>1</REPLICATION>
+<<<<<<< .working
       <MTIME>1388730115362</MTIME>
       <ATIME>1388730115362</ATIME>
+=======
+      <MTIME>1389828265757</MTIME>
+      <ATIME>1389828265757</ATIME>
+>>>>>>> .merge-right.r1559304
       <BLOCKSIZE>512</BLOCKSIZE>
+<<<<<<< .working
       <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>
       <PERMISSION_STATUS>
+<<<<<<< .working
         <USERNAME>szetszwo</USERNAME>
+=======
+        <USERNAME>jing</USERNAME>
+>>>>>>> .merge-right.r1559304
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
+<<<<<<< .working
       <RPC_CLIENTID>8205c453-0c7f-4b25-955a-7786e56bce86</RPC_CLIENTID>
       <RPC_CALLID>17</RPC_CALLID>
+=======
+      <RPC_CLIENTID>b5928e80-e373-4807-a688-f94483d08ce5</RPC_CLIENTID>
+      <RPC_CALLID>20</RPC_CALLID>
+>>>>>>> .merge-right.r1559304
     </DATA>
   </RECORD>
   <RECORD>
@@ -189,15 +283,24 @@
       <TXID>16</TXID>
       <LENGTH>0</LENGTH>
       <INODEID>0</INODEID>
-      <PATH>/file_create_u\0001;F431</PATH>
+      <PATH>/file_create</PATH>
       <REPLICATION>1</REPLICATION>
+<<<<<<< .working
       <MTIME>1388730115363</MTIME>
       <ATIME>1388730115362</ATIME>
+=======
+      <MTIME>1389828265759</MTIME>
+      <ATIME>1389828265757</ATIME>
+>>>>>>> .merge-right.r1559304
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_MACHINE></CLIENT_MACHINE>
       <PERMISSION_STATUS>
+<<<<<<< .working
         <USERNAME>szetszwo</USERNAME>
+=======
+        <USERNAME>jing</USERNAME>
+>>>>>>> .merge-right.r1559304
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
@@ -207,7 +310,7 @@
     <OPCODE>OP_SET_REPLICATION</OPCODE>
     <DATA>
       <TXID>17</TXID>
-      <PATH>/file_create_u\0001;F431</PATH>
+      <PATH>/file_create</PATH>
       <REPLICATION>1</REPLICATION>
     </DATA>
   </RECORD>
@@ -215,7 +318,7 @@
     <OPCODE>OP_SET_PERMISSIONS</OPCODE>
     <DATA>
       <TXID>18</TXID>
-      <SRC>/file_create_u\0001;F431</SRC>
+      <SRC>/file_create</SRC>
       <MODE>511</MODE>
     </DATA>
   </RECORD>
@@ -223,7 +326,7 @@
     <OPCODE>OP_SET_OWNER</OPCODE>
     <DATA>
       <TXID>19</TXID>
-      <SRC>/file_create_u\0001;F431</SRC>
+      <SRC>/file_create</SRC>
       <USERNAME>newOwner</USERNAME>
     </DATA>
   </RECORD>
@@ -232,7 +335,7 @@
     <DATA>
       <TXID>20</TXID>
       <LENGTH>0</LENGTH>
-      <PATH>/file_create_u\0001;F431</PATH>
+      <PATH>/file_create</PATH>
       <MTIME>1285195527000</MTIME>
       <ATIME>1285195527000</ATIME>
     </DATA>
@@ -251,12 +354,21 @@
     <DATA>
       <TXID>22</TXID>
       <LENGTH>0</LENGTH>
-      <SRC>/file_create_u\0001;F431</SRC>
+      <SRC>/file_create</SRC>
       <DST>/file_moved</DST>
+<<<<<<< .working
       <TIMESTAMP>1388730115378</TIMESTAMP>
+=======
+      <TIMESTAMP>1389828265782</TIMESTAMP>
+>>>>>>> .merge-right.r1559304
       <OPTIONS>NONE</OPTIONS>
+<<<<<<< .working
       <RPC_CLIENTID>8205c453-0c7f-4b25-955a-7786e56bce86</RPC_CLIENTID>
       <RPC_CALLID>24</RPC_CALLID>
+=======
+      <RPC_CLIENTID>b5928e80-e373-4807-a688-f94483d08ce5</RPC_CLIENTID>
+      <RPC_CALLID>27</RPC_CALLID>
+>>>>>>> .merge-right.r1559304
     </DATA>
   </RECORD>
   <RECORD>
@@ -267,18 +379,36 @@
       <INODEID>16389</INODEID>
       <PATH>/file_concat_target</PATH>
       <REPLICATION>1</REPLICATION>
+<<<<<<< .working
       <MTIME>1388730115382</MTIME>
       <ATIME>1388730115382</ATIME>
+=======
+      <MTIME>1389828265787</MTIME>
+      <ATIME>1389828265787</ATIME>
+>>>>>>> .merge-right.r1559304
       <BLOCKSIZE>512</BLOCKSIZE>
+<<<<<<< .working
       <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>
       <PERMISSION_STATUS>
+<<<<<<< .working
         <USERNAME>szetszwo</USERNAME>
+=======
+        <USERNAME>jing</USERNAME>
+>>>>>>> .merge-right.r1559304
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
+<<<<<<< .working
       <RPC_CLIENTID>8205c453-0c7f-4b25-955a-7786e56bce86</RPC_CLIENTID>
       <RPC_CALLID>26</RPC_CALLID>
+=======
+      <RPC_CLIENTID>b5928e80-e373-4807-a688-f94483d08ce5</RPC_CLIENTID>
+      <RPC_CALLID>29</RPC_CALLID>
+>>>>>>> .merge-right.r1559304
     </DATA>
   </RECORD>
   <RECORD>
@@ -296,7 +426,7 @@
     </DATA>
   </RECORD>
   <RECORD>
-    <OPCODE>OP_UPDATE_BLOCKS</OPCODE>
+    <OPCODE>OP_ADD_BLOCK</OPCODE>
     <DATA>
       <TXID>26</TXID>
       <PATH>/file_concat_target</PATH>
@@ -324,7 +454,7 @@
     </DATA>
   </RECORD>
   <RECORD>
-    <OPCODE>OP_UPDATE_BLOCKS</OPCODE>
+    <OPCODE>OP_ADD_BLOCK</OPCODE>
     <DATA>
       <TXID>29</TXID>
       <PATH>/file_concat_target</PATH>
@@ -357,15 +487,10 @@
     </DATA>
   </RECORD>
   <RECORD>
-    <OPCODE>OP_UPDATE_BLOCKS</OPCODE>
+    <OPCODE>OP_ADD_BLOCK</OPCODE>
     <DATA>
       <TXID>32</TXID>
       <PATH>/file_concat_target</PATH>
-      <BLOCK>
-        <BLOCK_ID>1073741825</BLOCK_ID>
-        <NUM_BYTES>512</NUM_BYTES>
-        <GENSTAMP>1001</GENSTAMP>
-      </BLOCK>
       <BLOCK>
         <BLOCK_ID>1073741826</BLOCK_ID>
         <NUM_BYTES>512</NUM_BYTES>
@@ -388,8 +513,13 @@
       <INODEID>0</INODEID>
       <PATH>/file_concat_target</PATH>
       <REPLICATION>1</REPLICATION>
+<<<<<<< .working
       <MTIME>1388730115461</MTIME>
       <ATIME>1388730115382</ATIME>
+=======
+      <MTIME>1389828266540</MTIME>
+      <ATIME>1389828265787</ATIME>
+>>>>>>> .merge-right.r1559304
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_MACHINE></CLIENT_MACHINE>
@@ -409,7 +539,11 @@
         <GENSTAMP>1003</GENSTAMP>
       </BLOCK>
       <PERMISSION_STATUS>
+<<<<<<< .working
         <USERNAME>szetszwo</USERNAME>
+=======
+        <USERNAME>jing</USERNAME>
+>>>>>>> .merge-right.r1559304
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
@@ -423,18 +557,36 @@
       <INODEID>16390</INODEID>
       <PATH>/file_concat_0</PATH>
       <REPLICATION>1</REPLICATION>
+<<<<<<< .working
       <MTIME>1388730115463</MTIME>
       <ATIME>1388730115463</ATIME>
+=======
+      <MTIME>1389828266544</MTIME>
+      <ATIME>1389828266544</ATIME>
+>>>>>>> .merge-right.r1559304
       <BLOCKSIZE>512</BLOCKSIZE>
+<<<<<<< .working
       <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>
       <PERMISSION_STATUS>
+<<<<<<< .working
         <USERNAME>szetszwo</USERNAME>
+=======
+        <USERNAME>jing</USERNAME>
+>>>>>>> .merge-right.r1559304
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
+<<<<<<< .working
       <RPC_CLIENTID>8205c453-0c7f-4b25-955a-7786e56bce86</RPC_CLIENTID>
       <RPC_CALLID>39</RPC_CALLID>
+=======
+      <RPC_CLIENTID>b5928e80-e373-4807-a688-f94483d08ce5</RPC_CLIENTID>
+      <RPC_CALLID>41</RPC_CALLID>
+>>>>>>> .merge-right.r1559304
     </DATA>
   </RECORD>
   <RECORD>
@@ -452,7 +604,7 @@
     </DATA>
   </RECORD>
   <RECORD>
-    <OPCODE>OP_UPDATE_BLOCKS</OPCODE>
+    <OPCODE>OP_ADD_BLOCK</OPCODE>
     <DATA>
       <TXID>37</TXID>
       <PATH>/file_concat_0</PATH>
@@ -480,7 +632,7 @@
     </DATA>
   </RECORD>
   <RECORD>
-    <OPCODE>OP_UPDATE_BLOCKS</OPCODE>
+    <OPCODE>OP_ADD_BLOCK</OPCODE>
     <DATA>
       <TXID>40</TXID>
       <PATH>/file_concat_0</PATH>
@@ -513,15 +665,10 @@
     </DATA>
   </RECORD>
   <RECORD>
-    <OPCODE>OP_UPDATE_BLOCKS</OPCODE>
+    <OPCODE>OP_ADD_BLOCK</OPCODE>
     <DATA>
       <TXID>43</TXID>
       <PATH>/file_concat_0</PATH>
-      <BLOCK>
-        <BLOCK_ID>1073741828</BLOCK_ID>
-        <NUM_BYTES>512</NUM_BYTES>
-        <GENSTAMP>1004</GENSTAMP>
-      </BLOCK>
       <BLOCK>
         <BLOCK_ID>1073741829</BLOCK_ID>
         <NUM_BYTES>512</NUM_BYTES>
@@ -544,8 +691,13 @@
       <INODEID>0</INODEID>
       <PATH>/file_concat_0</PATH>
       <REPLICATION>1</REPLICATION>
+<<<<<<< .working
       <MTIME>1388730115477</MTIME>
       <ATIME>1388730115463</ATIME>
+=======
+      <MTIME>1389828266569</MTIME>
+      <ATIME>1389828266544</ATIME>
+>>>>>>> .merge-right.r1559304
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_MACHINE></CLIENT_MACHINE>
@@ -565,7 +717,11 @@
         <GENSTAMP>1006</GENSTAMP>
       </BLOCK>
       <PERMISSION_STATUS>
+<<<<<<< .working
         <USERNAME>szetszwo</USERNAME>
+=======
+        <USERNAME>jing</USERNAME>
+>>>>>>> .merge-right.r1559304
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
@@ -579,18 +735,36 @@
       <INODEID>16391</INODEID>
       <PATH>/file_concat_1</PATH>
       <REPLICATION>1</REPLICATION>
+<<<<<<< .working
       <MTIME>1388730115479</MTIME>
       <ATIME>1388730115479</ATIME>
+=======
+      <MTIME>1389828266572</MTIME>
+      <ATIME>1389828266572</ATIME>
+>>>>>>> .merge-right.r1559304
       <BLOCKSIZE>512</BLOCKSIZE>
+<<<<<<< .working
       <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>
       <PERMISSION_STATUS>
+<<<<<<< .working
         <USERNAME>szetszwo</USERNAME>
+=======
+        <USERNAME>jing</USERNAME>
+>>>>>>> .merge-right.r1559304
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
+<<<<<<< .working
       <RPC_CLIENTID>8205c453-0c7f-4b25-955a-7786e56bce86</RPC_CLIENTID>
       <RPC_CALLID>51</RPC_CALLID>
+=======
+      <RPC_CLIENTID>b5928e80-e373-4807-a688-f94483d08ce5</RPC_CLIENTID>
+      <RPC_CALLID>53</RPC_CALLID>
+>>>>>>> .merge-right.r1559304
     </DATA>
   </RECORD>
   <RECORD>
@@ -608,7 +782,7 @@
     </DATA>
   </RECORD>
   <RECORD>
-    <OPCODE>OP_UPDATE_BLOCKS</OPCODE>
+    <OPCODE>OP_ADD_BLOCK</OPCODE>
     <DATA>
       <TXID>48</TXID>
       <PATH>/file_concat_1</PATH>
@@ -636,7 +810,7 @@
     </DATA>
   </RECORD>
   <RECORD>
-    <OPCODE>OP_UPDATE_BLOCKS</OPCODE>
+    <OPCODE>OP_ADD_BLOCK</OPCODE>
     <DATA>
       <TXID>51</TXID>
       <PATH>/file_concat_1</PATH>
@@ -669,15 +843,10 @@
     </DATA>
   </RECORD>
   <RECORD>
-    <OPCODE>OP_UPDATE_BLOCKS</OPCODE>
+    <OPCODE>OP_ADD_BLOCK</OPCODE>
     <DATA>
       <TXID>54</TXID>
       <PATH>/file_concat_1</PATH>
-      <BLOCK>
-        <BLOCK_ID>1073741831</BLOCK_ID>
-        <NUM_BYTES>512</NUM_BYTES>
-        <GENSTAMP>1007</GENSTAMP>
-      </BLOCK>
       <BLOCK>
         <BLOCK_ID>1073741832</BLOCK_ID>
         <NUM_BYTES>512</NUM_BYTES>
@@ -700,8 +869,13 @@
       <INODEID>0</INODEID>
       <PATH>/file_concat_1</PATH>
       <REPLICATION>1</REPLICATION>
+<<<<<<< .working
       <MTIME>1388730115495</MTIME>
       <ATIME>1388730115479</ATIME>
+=======
+      <MTIME>1389828266599</MTIME>
+      <ATIME>1389828266572</ATIME>
+>>>>>>> .merge-right.r1559304
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_MACHINE></CLIENT_MACHINE>
@@ -721,7 +895,11 @@
         <GENSTAMP>1009</GENSTAMP>
       </BLOCK>
       <PERMISSION_STATUS>
+<<<<<<< .working
         <USERNAME>szetszwo</USERNAME>
+=======
+        <USERNAME>jing</USERNAME>
+>>>>>>> .merge-right.r1559304
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
@@ -733,13 +911,22 @@
       <TXID>56</TXID>
       <LENGTH>0</LENGTH>
       <TRG>/file_concat_target</TRG>
+<<<<<<< .working
       <TIMESTAMP>1388730115498</TIMESTAMP>
+=======
+      <TIMESTAMP>1389828266603</TIMESTAMP>
+>>>>>>> .merge-right.r1559304
       <SOURCES>
         <SOURCE1>/file_concat_0</SOURCE1>
         <SOURCE2>/file_concat_1</SOURCE2>
       </SOURCES>
+<<<<<<< .working
       <RPC_CLIENTID>8205c453-0c7f-4b25-955a-7786e56bce86</RPC_CLIENTID>
       <RPC_CALLID>62</RPC_CALLID>
+=======
+      <RPC_CLIENTID>b5928e80-e373-4807-a688-f94483d08ce5</RPC_CLIENTID>
+      <RPC_CALLID>64</RPC_CALLID>
+>>>>>>> .merge-right.r1559304
     </DATA>
   </RECORD>
   <RECORD>
@@ -750,21 +937,36 @@
       <INODEID>16392</INODEID>
       <PATH>/file_symlink</PATH>
       <VALUE>/file_concat_target</VALUE>
+<<<<<<< .working
       <MTIME>1388730115502</MTIME>
       <ATIME>1388730115502</ATIME>
+=======
+      <MTIME>1389828266633</MTIME>
+      <ATIME>1389828266633</ATIME>
+>>>>>>> .merge-right.r1559304
       <PERMISSION_STATUS>
+<<<<<<< .working
         <USERNAME>szetszwo</USERNAME>
+=======
+        <USERNAME>jing</USERNAME>
+>>>>>>> .merge-right.r1559304
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>511</MODE>
       </PERMISSION_STATUS>
+<<<<<<< .working
       <RPC_CLIENTID>8205c453-0c7f-4b25-955a-7786e56bce86</RPC_CLIENTID>
       <RPC_CALLID>63</RPC_CALLID>
+=======
+      <RPC_CLIENTID>b5928e80-e373-4807-a688-f94483d08ce5</RPC_CLIENTID>
+      <RPC_CALLID>66</RPC_CALLID>
+>>>>>>> .merge-right.r1559304
     </DATA>
   </RECORD>
   <RECORD>
-    <OPCODE>OP_GET_DELEGATION_TOKEN</OPCODE>
+    <OPCODE>OP_ADD</OPCODE>
     <DATA>
       <TXID>58</TXID>
+<<<<<<< .working
       <DELEGATION_TOKEN_IDENTIFIER>
         <KIND>HDFS_DELEGATION_TOKEN</KIND>
         <SEQUENCE_NUMBER>1</SEQUENCE_NUMBER>
@@ -883,42 +1085,62 @@
     <OPCODE>OP_ADD</OPCODE>
     <DATA>
       <TXID>67</TXID>
+=======
+>>>>>>> .merge-right.r1559304
       <LENGTH>0</LENGTH>
       <INODEID>16393</INODEID>
       <PATH>/hard-lease-recovery-test</PATH>
       <REPLICATION>1</REPLICATION>
+<<<<<<< .working
       <MTIME>1388730115596</MTIME>
       <ATIME>1388730115596</ATIME>
+=======
+      <MTIME>1389828266637</MTIME>
+      <ATIME>1389828266637</ATIME>
+>>>>>>> .merge-right.r1559304
       <BLOCKSIZE>512</BLOCKSIZE>
+<<<<<<< .working
       <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>
       <PERMISSION_STATUS>
+<<<<<<< .working
         <USERNAME>szetszwo</USERNAME>
+=======
+        <USERNAME>jing</USERNAME>
+>>>>>>> .merge-right.r1559304
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
+<<<<<<< .working
       <RPC_CLIENTID>8205c453-0c7f-4b25-955a-7786e56bce86</RPC_CLIENTID>
       <RPC_CALLID>73</RPC_CALLID>
+=======
+      <RPC_CLIENTID>b5928e80-e373-4807-a688-f94483d08ce5</RPC_CLIENTID>
+      <RPC_CALLID>67</RPC_CALLID>
+>>>>>>> .merge-right.r1559304
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_ALLOCATE_BLOCK_ID</OPCODE>
     <DATA>
-      <TXID>68</TXID>
+      <TXID>59</TXID>
       <BLOCK_ID>1073741834</BLOCK_ID>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_SET_GENSTAMP_V2</OPCODE>
     <DATA>
-      <TXID>69</TXID>
+      <TXID>60</TXID>
       <GENSTAMPV2>1010</GENSTAMPV2>
     </DATA>
   </RECORD>
   <RECORD>
-    <OPCODE>OP_UPDATE_BLOCKS</OPCODE>
+    <OPCODE>OP_ADD_BLOCK</OPCODE>
     <DATA>
-      <TXID>70</TXID>
+      <TXID>61</TXID>
       <PATH>/hard-lease-recovery-test</PATH>
       <BLOCK>
         <BLOCK_ID>1073741834</BLOCK_ID>
@@ -932,7 +1154,7 @@
   <RECORD>
     <OPCODE>OP_UPDATE_BLOCKS</OPCODE>
     <DATA>
-      <TXID>71</TXID>
+      <TXID>62</TXID>
       <PATH>/hard-lease-recovery-test</PATH>
       <BLOCK>
         <BLOCK_ID>1073741834</BLOCK_ID>
@@ -946,29 +1168,45 @@
   <RECORD>
     <OPCODE>OP_SET_GENSTAMP_V2</OPCODE>
     <DATA>
-      <TXID>72</TXID>
+      <TXID>63</TXID>
       <GENSTAMPV2>1011</GENSTAMPV2>
     </DATA>
   </RECORD>
   <RECORD>
     <OPCODE>OP_REASSIGN_LEASE</OPCODE>
     <DATA>
+<<<<<<< .working
       <TXID>73</TXID>
       <LEASEHOLDER>DFSClient_NONMAPREDUCE_381408282_1</LEASEHOLDER>
+=======
+      <TXID>64</TXID>
+      <LEASEHOLDER>DFSClient_NONMAPREDUCE_16108824_1</LEASEHOLDER>
+>>>>>>> .merge-right.r1559304
       <PATH>/hard-lease-recovery-test</PATH>
       <NEWHOLDER>HDFS_NameNode</NEWHOLDER>
     </DATA>
   </RECORD>
   <RECORD>
+<<<<<<< .working
     <OPCODE>OP_CLOSE</OPCODE>
     <DATA>
       <TXID>74</TXID>
+=======
+    <OPCODE>OP_CLOSE</OPCODE>
+    <DATA>
+      <TXID>65</TXID>
+>>>>>>> .merge-right.r1559304
       <LENGTH>0</LENGTH>
       <INODEID>0</INODEID>
       <PATH>/hard-lease-recovery-test</PATH>
       <REPLICATION>1</REPLICATION>
+<<<<<<< .working
       <MTIME>1388730118281</MTIME>
       <ATIME>1388730115596</ATIME>
+=======
+      <MTIME>1389828269751</MTIME>
+      <ATIME>1389828266637</ATIME>
+>>>>>>> .merge-right.r1559304
       <BLOCKSIZE>512</BLOCKSIZE>
       <CLIENT_NAME></CLIENT_NAME>
       <CLIENT_MACHINE></CLIENT_MACHINE>
@@ -978,22 +1216,98 @@
         <GENSTAMP>1011</GENSTAMP>
       </BLOCK>
       <PERMISSION_STATUS>
+<<<<<<< .working
         <USERNAME>szetszwo</USERNAME>
+=======
+        <USERNAME>jing</USERNAME>
+>>>>>>> .merge-right.r1559304
         <GROUPNAME>supergroup</GROUPNAME>
         <MODE>420</MODE>
       </PERMISSION_STATUS>
     </DATA>
   </RECORD>
   <RECORD>
+<<<<<<< .working
     <OPCODE>OP_UPGRADE_MARKER</OPCODE>
     <DATA>
       <TXID>75</TXID>
     </DATA>
   </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>
     <DATA>
+<<<<<<< .working
       <TXID>76</TXID>
+=======
+      <TXID>72</TXID>
+>>>>>>> .merge-right.r1559304
     </DATA>
   </RECORD>
 </EDITS>

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

@@ -146,6 +146,8 @@ Trunk (Unreleased)
     MAPREDUCE-5191. TestQueue#testQueue fails with timeout on Windows. (Ivan
     Mitic via hitesh)
 
+    MAPREDUCE-5717. Task pings are interpreted as task progress (jlowe)
+
 Release 2.4.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -198,6 +200,9 @@ Release 2.4.0 - UNRELEASED
 
     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
 
     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 
     cannot be fulfilled. (lohit via kasha)
 
+    MAPREDUCE-5724. JobHistoryServer does not start if HDFS is not running. 
+    (tucu)
+
 Release 2.3.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -328,6 +336,9 @@ Release 2.3.0 - UNRELEASED
     MAPREDUCE-5674. Missing start and finish time in mapred.JobStatus.
     (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
 
   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,
       Vector<String> vargs,
       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(
@@ -210,19 +212,11 @@ public class MapReduceChildJVM {
     if (conf.getProfileEnabled()) {
       if (conf.getProfileTaskRange(task.isMapTask()
                                    ).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) {
       //We are using statusUpdate only as a simple ping
       LOG.info("Ping from " + taskAttemptID.toString());
-      taskHeartbeatHandler.progressing(yarnAttemptID);
       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 =
     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
    *  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.Apps;
 import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.apache.log4j.RollingFileAppender;
 
 /**
  * Helper class for MR applications
@@ -476,16 +477,24 @@ public class MRApps extends Apps {
    * Add the JVM system properties necessary to configure {@link ContainerLogAppender}.
    * @param logLevel the desired log level (eg INFO/WARN/DEBUG)
    * @param logSize See {@link ContainerLogAppender#setTotalLogFileSize(long)}
+   * @param numBackups See {@link RollingFileAppender#setMaxBackupIndex(int)}
    * @param vargs the argument list to append to
    */
   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("-D" + YarnConfiguration.YARN_APP_CONTAINER_LOG_DIR + "=" +
         ApplicationConstants.LOG_DIR_EXPANSION_VAR);
     vargs.add(
         "-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";
   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*/
   public static final String MR_AM_NUM_PROGRESS_SPLITS = 
     MR_AM_PREFIX+"num-progress-splits";
@@ -692,7 +696,11 @@ public interface MRJobConfig {
       + "$HADOOP_MAPRED_HOME/share/hadoop/mapreduce/lib/*";
 
   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_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>
 </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>
   <name>mapreduce.job.maxtaskfailures.per.tracker</name>
   <value>3</value>
@@ -577,6 +602,31 @@
     </description>
   </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>
     <name>mapreduce.task.skip.start.attempts</name>
     <value>2</value>

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

@@ -33,6 +33,10 @@
   </properties>
 
   <dependencies>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-hdfs</artifactId>
+    </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-mapreduce-client-common</artifactId>
@@ -53,6 +57,12 @@
       <type>test-jar</type>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-hdfs</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 
   <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.IOException;
+import java.net.ConnectException;
 import java.util.ArrayList;
 import java.util.Collection;
 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.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
@@ -464,7 +467,8 @@ public class HistoryFileManager extends AbstractService {
 
   private JobACLsManager aclsMgr;
 
-  private Configuration conf;
+  @VisibleForTesting
+  Configuration conf;
 
   private String serialNumberFormat;
 
@@ -491,36 +495,10 @@ public class HistoryFileManager extends AbstractService {
         + (JobHistoryUtils.SERIAL_NUMBER_DIRECTORY_DIGITS + serialNumberLowDigits)
         + "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);
 
@@ -544,6 +522,107 @@ public class HistoryFileManager extends AbstractService {
     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
   public void serviceStop() throws Exception {
     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));
     String logLevel = jobConf.get(
         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
     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.KillApplicationRequest;
 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.RenewDelegationTokenResponse;
 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.YarnApplicationState;
 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.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
@@ -354,6 +357,12 @@ public class TestClientRedirect {
         CancelDelegationTokenRequest request) throws IOException {
       return null;
     }
+
+    @Override
+    public MoveApplicationAcrossQueuesResponse moveApplicationAcrossQueues(
+        MoveApplicationAcrossQueuesRequest request) throws YarnException, IOException {
+      return null;
+    }
   }
 
   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.FileNotFoundException;
 import java.io.FileOutputStream;
+import java.io.InputStreamReader;
 import java.io.IOException;
 import java.io.StringReader;
 import java.net.URI;
 import java.security.PrivilegedExceptionAction;
+import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.Map;
 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.NullWritable;
 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.Job;
 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.TaskReport;
 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.output.FileOutputCommitter;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
 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.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.util.JarFinder;
 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.server.resourcemanager.rmapp.RMAppState;
+import org.apache.log4j.Level;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
@@ -84,6 +93,9 @@ import org.junit.Test;
 public class TestMRJobs {
 
   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 MiniDFSCluster dfsCluster;
@@ -122,7 +134,8 @@ public class TestMRJobs {
     }
 
     if (mrCluster == null) {
-      mrCluster = new MiniMRYarnCluster(TestMRJobs.class.getName(), 3);
+      mrCluster = new MiniMRYarnCluster(TestMRJobs.class.getName(),
+          NUM_NODE_MGRS);
       Configuration conf = new Configuration();
       conf.set("fs.defaultFS", remoteFs.getUri().toString());   // use HDFS
       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
   }
 
+  @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
       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>
         <exclusions>
           <exclusion>
-            <groupId>javax.xml.stream</groupId>
+            <groupId>stax</groupId>
             <artifactId>stax-api</artifactId>
           </exclusion>
         </exclusions>
@@ -770,6 +770,12 @@
         <groupId>org.codehaus.jettison</groupId>
         <artifactId>jettison</artifactId>
         <version>1.1</version>
+        <exclusions>
+          <exclusion>
+            <groupId>stax</groupId>
+            <artifactId>stax-api</artifactId>
+          </exclusion>
+        </exclusions>
       </dependency>
       <dependency>
         <groupId>com.sun.jersey</groupId>

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

@@ -6,6 +6,9 @@ Trunk - Unreleased
 
   NEW FEATURES
 
+    YARN-1496. Protocol additions to allow moving apps between queues (Sandy
+    Ryza)
+
   IMPROVEMENTS
 
   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-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
 
   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.KillApplicationRequest;
 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.RenewDelegationTokenResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
@@ -339,4 +341,17 @@ public interface ApplicationClientProtocol {
   public CancelDelegationTokenResponse cancelDelegationToken(
       CancelDelegationTokenRequest request) throws YarnException,
       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 =
       YARN_PREFIX + "app.container.log.filesize";
 
+  public static final String YARN_APP_CONTAINER_LOG_BACKUPS =
+      YARN_PREFIX + "app.container.log.backups";
+
   ////////////////////////////////
   // 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 renewDelegationToken(hadoop.common.RenewDelegationTokenRequestProto) returns (hadoop.common.RenewDelegationTokenResponseProto);
   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;
 }
 
+message MoveApplicationAcrossQueuesRequestProto {
+  required ApplicationIdProto application_id = 1;
+  required string target_queue = 2;
+}
+
+message MoveApplicationAcrossQueuesResponseProto {
+}
+
 message GetApplicationsRequestProto {
   repeated string application_types = 1;
   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.KillApplicationRequest;
 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.RenewDelegationTokenResponse;
 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.KillApplicationRequestPBImpl;
 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.RenewDelegationTokenResponsePBImpl;
 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.GetQueueUserAclsInfoRequestProto;
 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 com.google.protobuf.ServiceException;
@@ -291,4 +296,20 @@ public class ApplicationClientProtocolPBClientImpl implements ApplicationClientP
       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.GetQueueUserAclsInfoResponse;
 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.SubmitApplicationResponse;
 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.KillApplicationRequestPBImpl;
 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.RenewDelegationTokenResponsePBImpl;
 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.KillApplicationRequestProto;
 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.SubmitApplicationResponseProto;
 
@@ -278,4 +283,20 @@ public class ApplicationClientProtocolPBServiceImpl implements ApplicationClient
         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
     the specified session timeout period (i.e. no heartbeat).</description>
     <name>yarn.resourcemanager.zk-timeout-ms</name>
-    <value>60000</value>
+    <value>10000</value>
   </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()]);
     ShellCommandExecutor shExec = new ShellCommandExecutor(commandArray);
-    // TODO: DEBUG
-    LOG.info("initApplication: " + Arrays.toString(commandArray));
     if (LOG.isDebugEnabled()) {
       LOG.debug("initApplication: " + Arrays.toString(commandArray));
     }
@@ -274,8 +272,9 @@ public class LinuxContainerExecutor extends ContainerExecutor {
         String[] commandArray = command.toArray(new String[command.size()]);
         shExec = new ShellCommandExecutor(commandArray, null, // NM's cwd
             container.getLaunchContext().getEnvironment()); // sanitized env
-        // DEBUG
-        LOG.info("launchContainer: " + Arrays.toString(commandArray));
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("launchContainer: " + Arrays.toString(commandArray));
+        }
         shExec.execute();
         if (LOG.isDebugEnabled()) {
           logOutput(shExec.getOutput());
@@ -374,7 +373,6 @@ public class LinuxContainerExecutor extends ContainerExecutor {
     }
     String[] commandArray = command.toArray(new String[command.size()]);
     ShellCommandExecutor shExec = new ShellCommandExecutor(commandArray);
-    LOG.info(" -- DEBUG -- deleteAsUser: " + Arrays.toString(commandArray));
     if (LOG.isDebugEnabled()) {
       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.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
 # 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.KillApplicationRequest;
 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.RenewDelegationTokenResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
@@ -683,6 +685,12 @@ public class ClientRMService extends AbstractService implements
       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)
       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)
         && !Resources.fitsIn(minQueueResources.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)));
     }
   }

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