Browse Source

Merge branch 'trunk' into HADOOP-13345

Steve Loughran 7 years ago
parent
commit
67eb6202e6
78 changed files with 1388 additions and 542 deletions
  1. 0 6
      hadoop-client-modules/hadoop-client-minicluster/pom.xml
  2. 0 7
      hadoop-client-modules/hadoop-client-runtime/pom.xml
  3. 0 23
      hadoop-common-project/hadoop-common/src/main/bin/hadoop-functions.sh
  4. 9 10
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileStatus.java
  5. 1 7
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFsFileStatus.java
  6. 0 6
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFsLocatedFileStatus.java
  7. 7 7
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
  8. 0 32
      hadoop-common-project/hadoop-common/src/test/scripts/hadoop_escape_chars.bats
  9. 5 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
  10. 1 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
  11. 3 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/SlowDiskReports.java
  12. 1 1
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java
  13. 4 2
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3Utils.java
  14. 4 4
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java
  15. 26 0
      hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
  16. 0 5
      hadoop-hdfs-project/hadoop-hdfs/pom.xml
  17. 10 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNode.java
  18. 8 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
  19. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
  20. 15 12
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorageRetentionManager.java
  21. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
  22. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
  23. 28 13
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/XmlEditsVisitor.java
  24. 7 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java
  25. 39 0
      hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md
  26. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java
  27. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestStorageMover.java
  28. 26 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeOptionParsing.java
  29. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java
  30. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogTailer.java
  31. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailureToReadEdits.java
  32. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestInitializeSharedEdits.java
  33. BIN
      hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored
  34. 31 31
      hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml
  35. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineFileInputFormat.java
  36. 1 9
      hadoop-mapreduce-project/hadoop-mapreduce-client/pom.xml
  37. 1 9
      hadoop-project-dist/pom.xml
  38. 3 11
      hadoop-project/pom.xml
  39. 4 4
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestOutOfBandAzureBlobOperations.java
  40. 4 12
      hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/snative/SwiftFileStatus.java
  41. 2 2
      hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/snative/SwiftNativeFileSystemStore.java
  42. 2 2
      hadoop-tools/hadoop-openstack/src/test/java/org/apache/hadoop/fs/swift/TestSwiftFileSystemDirectories.java
  43. 1 1
      hadoop-tools/hadoop-openstack/src/test/java/org/apache/hadoop/fs/swift/TestSwiftFileSystemPartitionedUploads.java
  44. 366 29
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java
  45. 3 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestNMClient.java
  46. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml
  47. 82 50
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
  48. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/Container.java
  49. 17 20
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
  50. 0 15
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java
  51. 73 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java
  52. 1 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerSchedulerEventType.java
  53. 85 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/UpdateContainerSchedulerEvent.java
  54. 10 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java
  55. 31 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java
  56. 167 100
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
  57. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java
  58. 96 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/TestContainerSchedulerQueuing.java
  59. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockContainer.java
  60. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
  61. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
  62. 1 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
  63. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
  64. 8 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java
  65. 13 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
  66. 0 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
  67. 0 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java
  68. 0 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationConfiguration.java
  69. 13 2
      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
  70. 13 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java
  71. 28 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/RMContainerTokenSecretManager.java
  72. 23 23
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
  73. 3 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageApps.java
  74. 9 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageEntities.java
  75. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/AppIdKeyConverter.java
  76. 33 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/HBaseTimelineStorageUtils.java
  77. 39 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestCustomApplicationIdConversion.java
  78. 1 9
      hadoop-yarn-project/hadoop-yarn/pom.xml

+ 0 - 6
hadoop-client-modules/hadoop-client-minicluster/pom.xml

@@ -628,12 +628,6 @@
                         <exclude>**/*.class</exclude>
                       </excludes>
                     </filter>
-                    <filter>
-                      <artifact>xerces:xercesImpl</artifact>
-                      <excludes>
-                        <exclude>**/*</exclude>
-                      </excludes>
-                    </filter>
                     <filter>
                       <artifact>org.apache.hadoop:hadoop-mapreduce-client-jobclient:*</artifact>
                       <excludes>

+ 0 - 7
hadoop-client-modules/hadoop-client-runtime/pom.xml

@@ -174,13 +174,6 @@
                         <exclude>org/apache/jasper/compiler/Localizer.class</exclude>
                       </excludes>
                     </filter>
-                    <!-- We only have xerces as a dependency for XML output for the fsimage edits, we don't need anything specific to it for javax xml support -->
-                    <filter>
-                      <artifact>xerces:xercesImpl</artifact>
-                      <excludes>
-                        <exclude>META-INF/services/*</exclude>
-                      </excludes>
-                    </filter>
                     <!-- We rely on jersey for our web interfaces. We want to use its java services stuff only internal to jersey -->
                     <filter>
                       <artifact>com.sun.jersey:*</artifact>

+ 0 - 23
hadoop-common-project/hadoop-common/src/main/bin/hadoop-functions.sh

@@ -2578,29 +2578,6 @@ function hadoop_parse_args
   hadoop_debug "hadoop_parse: asking caller to skip ${HADOOP_PARSE_COUNTER}"
 }
 
-## @description  XML-escapes the characters (&'"<>) in the given parameter.
-## @audience     private
-## @stability    evolving
-## @replaceable  yes
-## @param        string
-## @return       XML-escaped string
-function hadoop_xml_escape
-{
-  sed -e 's/&/\&amp;/g' -e 's/"/\\\&quot;/g' \
-    -e "s/'/\\\\\&apos;/g" -e 's/</\\\&lt;/g' -e 's/>/\\\&gt;/g' <<< "$1"
-}
-
-## @description  sed-escapes the characters (\/&) in the given parameter.
-## @audience     private
-## @stability    evolving
-## @replaceable  yes
-## @param        string
-## @return       sed-escaped string
-function hadoop_sed_escape
-{
-  sed -e 's/[\/&]/\\&/g' <<< "$1"
-}
-
 ## @description Handle subcommands from main program entries
 ## @audience private
 ## @stability evolving

+ 9 - 10
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileStatus.java

@@ -172,7 +172,7 @@ public class FileStatus implements Writable, Comparable<Object>,
    * @return true if this is a file
    */
   public boolean isFile() {
-    return !isdir && !isSymlink();
+    return !isDirectory() && !isSymlink();
   }
 
   /**
@@ -182,20 +182,20 @@ public class FileStatus implements Writable, Comparable<Object>,
   public boolean isDirectory() {
     return isdir;
   }
-  
+
   /**
-   * Old interface, instead use the explicit {@link FileStatus#isFile()}, 
-   * {@link FileStatus#isDirectory()}, and {@link FileStatus#isSymlink()} 
+   * Old interface, instead use the explicit {@link FileStatus#isFile()},
+   * {@link FileStatus#isDirectory()}, and {@link FileStatus#isSymlink()}
    * @return true if this is a directory.
-   * @deprecated Use {@link FileStatus#isFile()},  
-   * {@link FileStatus#isDirectory()}, and {@link FileStatus#isSymlink()} 
+   * @deprecated Use {@link FileStatus#isFile()},
+   * {@link FileStatus#isDirectory()}, and {@link FileStatus#isSymlink()}
    * instead.
    */
   @Deprecated
-  public boolean isDir() {
-    return isdir;
+  public final boolean isDir() {
+    return isDirectory();
   }
-  
+
   /**
    * Is this a symbolic link?
    * @return true if this is a symbolic link
@@ -448,7 +448,6 @@ public class FileStatus implements Writable, Comparable<Object>,
     FileStatus other = PBHelper.convert(proto);
     isdir = other.isDirectory();
     length = other.getLen();
-    isdir = other.isDirectory();
     block_replication = other.getReplication();
     blocksize = other.getBlockSize();
     modification_time = other.getModificationTime();

+ 1 - 7
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFsFileStatus.java

@@ -61,13 +61,7 @@ class ViewFsFileStatus extends FileStatus {
    public boolean isDirectory() {
      return  myFs.isDirectory();
    }
-   
-   @Override
-   @SuppressWarnings("deprecation")
-   public boolean isDir() {
-     return myFs.isDirectory();
-   }
-   
+
    @Override
    public boolean isSymlink() {
      return myFs.isSymlink();

+ 0 - 6
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFsLocatedFileStatus.java

@@ -49,12 +49,6 @@ class ViewFsLocatedFileStatus extends LocatedFileStatus {
     return myFs.isDirectory();
   }
 
-  @Override
-  @SuppressWarnings("deprecation")
-  public boolean isDir() {
-    return myFs.isDirectory();
-  }
-
   @Override
   public boolean isSymlink() {
     return myFs.isSymlink();

+ 7 - 7
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java

@@ -190,7 +190,7 @@ public class ProtobufRpcEngine implements RpcEngine {
         throws ServiceException {
       long startTime = 0;
       if (LOG.isDebugEnabled()) {
-        startTime = Time.now();
+        startTime = Time.monotonicNow();
       }
       
       if (args.length != 2) { // RpcController + Message
@@ -245,7 +245,7 @@ public class ProtobufRpcEngine implements RpcEngine {
       }
 
       if (LOG.isDebugEnabled()) {
-        long callTime = Time.now() - startTime;
+        long callTime = Time.monotonicNow() - startTime;
         LOG.debug("Call: " + method.getName() + " took " + callTime + "ms");
       }
       
@@ -373,19 +373,19 @@ public class ProtobufRpcEngine implements RpcEngine {
         this.server = currentCallInfo.get().server;
         this.call = Server.getCurCall().get();
         this.methodName = currentCallInfo.get().methodName;
-        this.setupTime = Time.now();
+        this.setupTime = Time.monotonicNow();
       }
 
       @Override
       public void setResponse(Message message) {
-        long processingTime = Time.now() - setupTime;
+        long processingTime = Time.monotonicNow() - setupTime;
         call.setDeferredResponse(RpcWritable.wrap(message));
         server.updateDeferredMetrics(methodName, processingTime);
       }
 
       @Override
       public void error(Throwable t) {
-        long processingTime = Time.now() - setupTime;
+        long processingTime = Time.monotonicNow() - setupTime;
         String detailedMetricsName = t.getClass().getSimpleName();
         server.updateDeferredMetrics(detailedMetricsName, processingTime);
         call.setDeferredError(t);
@@ -513,7 +513,7 @@ public class ProtobufRpcEngine implements RpcEngine {
         Message param = request.getValue(prototype);
 
         Message result;
-        long startTime = Time.now();
+        long startTime = Time.monotonicNow();
         int qTime = (int) (startTime - receiveTime);
         Exception exception = null;
         boolean isDeferred = false;
@@ -537,7 +537,7 @@ public class ProtobufRpcEngine implements RpcEngine {
           throw e;
         } finally {
           currentCallInfo.set(null);
-          int processingTime = (int) (Time.now() - startTime);
+          int processingTime = (int) (Time.monotonicNow() - startTime);
           if (LOG.isDebugEnabled()) {
             String msg =
                 "Served: " + methodName + (isDeferred ? ", deferred" : "") +

+ 0 - 32
hadoop-common-project/hadoop-common/src/test/scripts/hadoop_escape_chars.bats

@@ -1,32 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one or more
-# contributor license agreements.  See the NOTICE file distributed with
-# this work for additional information regarding copyright ownership.
-# The ASF licenses this file to You under the Apache License, Version 2.0
-# (the "License"); you may not use this file except in compliance with
-# the License.  You may obtain a copy of the License at
-#
-#     http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
-load hadoop-functions_test_helper
-
-@test "hadoop_escape_sed (positive 1)" {
-  ret="$(hadoop_sed_escape "\pass&&word\0#\$asdf/g  ><'\"~\`!@#$%^&*()_+-=")"
-  expected="\\\\pass\&\&word\\\0#\$asdf\/g  ><'\"~\`!@#$%^\&*()_+-="
-  echo "actual >${ret}<"
-  echo "expected >${expected}<"
-  [ "${ret}" = "${expected}" ]
-}
-
-@test "hadoop_escape_xml (positive 1)" {
-  ret="$(hadoop_xml_escape "\pass&&word\0#\$asdf/g  ><'\"~\`!@#$%^&*()_+-=")"
-  expected="\\pass&amp;&amp;word\0#\$asdf/g  \&gt;\&lt;\&apos;\&quot;~\`!@#\$%^&amp;*()_+-="
-  echo "actual >${ret}<"
-  echo "expected >${expected}<"
-  [ "${ret}" = "${expected}" ]
-}

+ 5 - 2
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -2901,9 +2901,12 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     }
     synchronized (DFSClient.class) {
       if (STRIPED_READ_THREAD_POOL == null) {
-        STRIPED_READ_THREAD_POOL = DFSUtilClient.getThreadPoolExecutor(1,
+        // Only after thread pool is fully constructed then save it to
+        // volatile field.
+        ThreadPoolExecutor threadPool = DFSUtilClient.getThreadPoolExecutor(1,
             numThreads, 60, "StripedRead-", true);
-        STRIPED_READ_THREAD_POOL.allowCoreThreadTimeOut(true);
+        threadPool.allowCoreThreadTimeOut(true);
+        STRIPED_READ_THREAD_POOL = threadPool;
       }
     }
   }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java

@@ -2095,7 +2095,7 @@ public class PBHelperClient {
     if (fs == null)
       return null;
     FileType fType = FileType.IS_FILE;
-    if (fs.isDir()) {
+    if (fs.isDirectory()) {
       fType = FileType.IS_DIR;
     } else if (fs.isSymlink()) {
       fType = FileType.IS_SYMLINK;

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/SlowDiskReports.java

@@ -101,8 +101,9 @@ public final class SlowDiskReports {
     }
 
     boolean areEqual;
-    for (String disk : this.slowDisks.keySet()) {
-      if (!this.slowDisks.get(disk).equals(that.slowDisks.get(disk))) {
+    for (Map.Entry<String, Map<DiskOp, Double>> entry : this.slowDisks
+        .entrySet()) {
+      if (!entry.getValue().equals(that.slowDisks.get(entry.getKey()))) {
         return false;
       }
     }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java

@@ -669,7 +669,7 @@ public class FSOperations {
     /**
      * Creates a list-status executor.
      *
-     * @param path the directory to retrieve the status of its contents.
+     * @param path the directory/file to retrieve the status of its contents.
      * @param filter glob filter to use.
      *
      * @throws IOException thrown if the filter expression is incorrect.

+ 4 - 2
hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3Utils.java

@@ -65,7 +65,9 @@ public class Nfs3Utils {
      * client takes only the lower 32bit of the fileId and treats it as signed
      * int. When the 32th bit is 1, the client considers it invalid.
      */
-    NfsFileType fileType = fs.isDir() ? NfsFileType.NFSDIR : NfsFileType.NFSREG;
+    NfsFileType fileType = fs.isDirectory()
+        ? NfsFileType.NFSDIR
+        : NfsFileType.NFSREG;
     fileType = fs.isSymlink() ? NfsFileType.NFSLNK : fileType;
     int nlink = (fileType == NfsFileType.NFSDIR) ? fs.getChildrenNum() + 2 : 1;
     long size = (fileType == NfsFileType.NFSDIR) ? getDirSize(fs
@@ -98,7 +100,7 @@ public class Nfs3Utils {
       return null;
     }
 
-    long size = fstat.isDir() ? getDirSize(fstat.getChildrenNum()) : fstat
+    long size = fstat.isDirectory() ? getDirSize(fstat.getChildrenNum()) : fstat
         .getLen();
     return new WccAttr(size, new NfsTime(fstat.getModificationTime()),
         new NfsTime(fstat.getModificationTime()));

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java

@@ -1208,7 +1208,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
       if (fstat == null) {
         return new REMOVE3Response(Nfs3Status.NFS3ERR_NOENT, errWcc);
       }
-      if (fstat.isDir()) {
+      if (fstat.isDirectory()) {
         return new REMOVE3Response(Nfs3Status.NFS3ERR_ISDIR, errWcc);
       }
 
@@ -1289,7 +1289,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
       if (fstat == null) {
         return new RMDIR3Response(Nfs3Status.NFS3ERR_NOENT, errWcc);
       }
-      if (!fstat.isDir()) {
+      if (!fstat.isDirectory()) {
         return new RMDIR3Response(Nfs3Status.NFS3ERR_NOTDIR, errWcc);
       }
 
@@ -1565,7 +1565,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
         LOG.info("Can't get path for fileId: " + handle.getFileId());
         return new READDIR3Response(Nfs3Status.NFS3ERR_STALE);
       }
-      if (!dirStatus.isDir()) {
+      if (!dirStatus.isDirectory()) {
         LOG.error("Can't readdir for regular file, fileId: "
             + handle.getFileId());
         return new READDIR3Response(Nfs3Status.NFS3ERR_NOTDIR);
@@ -1732,7 +1732,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
         LOG.info("Can't get path for fileId: " + handle.getFileId());
         return new READDIRPLUS3Response(Nfs3Status.NFS3ERR_STALE);
       }
-      if (!dirStatus.isDir()) {
+      if (!dirStatus.isDirectory()) {
         LOG.error("Can't readdirplus for regular file, fileId: "
             + handle.getFileId());
         return new READDIRPLUS3Response(Nfs3Status.NFS3ERR_NOTDIR);

+ 26 - 0
hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml

@@ -264,4 +264,30 @@
         <Field name="locations" />
         <Bug pattern="SE_TRANSIENT_FIELD_NOT_RESTORED" />
     </Match>
+    <Match>
+       <Class name="org.apache.hadoop.hdfs.server.namenode.NNUpgradeUtil$1" />
+       <Method name="visitFile" />
+       <Bug pattern="NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE" />
+     </Match>
+     <!-- Ignore warnings for not changing the startup option parsing behavior. -->
+     <Match>
+       <Class name="org.apache.hadoop.hdfs.server.common.HdfsServerConstants$StartupOption" />
+       <Method name="setClusterId" />
+       <Bug pattern="ME_ENUM_FIELD_SETTER" />
+     </Match>
+     <Match>
+       <Class name="org.apache.hadoop.hdfs.server.common.HdfsServerConstants$StartupOption" />
+       <Method name="setForce" />
+       <Bug pattern="ME_ENUM_FIELD_SETTER" />
+     </Match>
+     <Match>
+       <Class name="org.apache.hadoop.hdfs.server.common.HdfsServerConstants$StartupOption" />
+       <Method name="setForceFormat" />
+       <Bug pattern="ME_ENUM_FIELD_SETTER" />
+     </Match>
+     <Match>
+      <Class name="org.apache.hadoop.hdfs.server.common.HdfsServerConstants$StartupOption" />
+      <Method name="setInteractiveFormat" />
+      <Bug pattern="ME_ENUM_FIELD_SETTER" />
+    </Match>
  </FindBugsFilter>

+ 0 - 5
hadoop-hdfs-project/hadoop-hdfs/pom.xml

@@ -173,11 +173,6 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
       <artifactId>netty-all</artifactId>
       <scope>compile</scope>
     </dependency>
-    <dependency>
-      <groupId>xerces</groupId>
-      <artifactId>xercesImpl</artifactId>
-      <scope>compile</scope>
-    </dependency>
     <dependency>
       <groupId>org.apache.htrace</groupId>
       <artifactId>htrace-core4</artifactId>

+ 10 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNode.java

@@ -299,14 +299,18 @@ public class JournalNode implements Tool, Configurable, JournalNodeMXBean {
         return file.isDirectory();
       }
     });
-    for (File journalDir : journalDirs) {
-      String jid = journalDir.getName();
-      if (!status.containsKey(jid)) {
-        Map<String, String> jMap = new HashMap<String, String>();
-        jMap.put("Formatted", "true");
-        status.put(jid, jMap);
+
+    if (journalDirs != null) {
+      for (File journalDir : journalDirs) {
+        String jid = journalDir.getName();
+        if (!status.containsKey(jid)) {
+          Map<String, String> jMap = new HashMap<String, String>();
+          jMap.put("Formatted", "true");
+          status.put(jid, jMap);
+        }
       }
     }
+
     return JSON.toString(status);
   }
   

+ 8 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java

@@ -1336,10 +1336,14 @@ public class DataStorage extends Storage {
           return name.startsWith(BLOCK_SUBDIR_PREFIX);
         }
       });
-    for(int i = 0; i < otherNames.length; i++)
-      linkBlocksHelper(new File(from, otherNames[i]),
-          new File(to, otherNames[i]), oldLV, hl, upgradeToIdBasedLayout,
-          blockRoot, idBasedLayoutSingleLinks);
+
+    if (otherNames != null) {
+      for (int i = 0; i < otherNames.length; i++) {
+        linkBlocksHelper(new File(from, otherNames[i]),
+            new File(to, otherNames[i]), oldLV, hl, upgradeToIdBasedLayout,
+            blockRoot, idBasedLayoutSingleLinks);
+      }
+    }
   }
 
   /**

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java

@@ -348,7 +348,7 @@ public class Mover {
     private void processRecursively(String parent, HdfsFileStatus status,
         Result result) {
       String fullPath = status.getFullName(parent);
-      if (status.isDir()) {
+      if (status.isDirectory()) {
         if (!fullPath.endsWith(Path.SEPARATOR)) {
           fullPath = fullPath + Path.SEPARATOR;
         }

+ 15 - 12
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorageRetentionManager.java

@@ -255,24 +255,27 @@ public class NNStorageRetentionManager {
     });
 
     // Check whether there is any work to do.
-    if (filesInStorage.length <= numCheckpointsToRetain) {
+    if (filesInStorage != null
+        && filesInStorage.length <= numCheckpointsToRetain) {
       return;
     }
 
     // Create a sorted list of txids from the file names.
     TreeSet<Long> sortedTxIds = new TreeSet<Long>();
-    for (String fName : filesInStorage) {
-      // Extract the transaction id from the file name.
-      long fTxId;
-      try {
-        fTxId = Long.parseLong(fName.substring(oivImagePrefix.length() + 1));
-      } catch (NumberFormatException nfe) {
-        // This should not happen since we have already filtered it.
-        // Log and continue.
-        LOG.warn("Invalid file name. Skipping " + fName);
-        continue;
+    if (filesInStorage != null) {
+      for (String fName : filesInStorage) {
+        // Extract the transaction id from the file name.
+        long fTxId;
+        try {
+          fTxId = Long.parseLong(fName.substring(oivImagePrefix.length() + 1));
+        } catch (NumberFormatException nfe) {
+          // This should not happen since we have already filtered it.
+          // Log and continue.
+          LOG.warn("Invalid file name. Skipping " + fName);
+          continue;
+        }
+        sortedTxIds.add(Long.valueOf(fTxId));
       }
-      sortedTxIds.add(Long.valueOf(fTxId));
     }
 
     int numFilesToDelete = sortedTxIds.size() - numCheckpointsToRetain;

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

@@ -471,7 +471,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
   void check(String parent, HdfsFileStatus file, Result replRes, Result ecRes)
       throws IOException {
     String path = file.getFullName(parent);
-    if (file.isDir()) {
+    if (file.isDirectory()) {
       checkDir(path, replRes, ecRes);
       return;
     }
@@ -1115,7 +1115,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
       if (lfStatus == null) { // not exists
         lfInitedOk = dfs.mkdirs(lfName, null, true);
         lostFound = lfName;
-      } else if (!lfStatus.isDir()) { // exists but not a directory
+      } else if (!lfStatus.isDirectory()) { // exists but not a directory
         LOG.warn("Cannot use /lost+found : a regular file with this name exists.");
         lfInitedOk = false;
       }  else { // exists and is a directory

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java

@@ -1992,7 +1992,7 @@ public class DFSAdmin extends FsShell {
         return exitCode;
       }
     } else if ("-report".equals(cmd)) {
-      if (argv.length < 1) {
+      if (argv.length > 6) {
         printUsage(cmd);
         return exitCode;
       }
@@ -2022,7 +2022,7 @@ public class DFSAdmin extends FsShell {
         return exitCode;
       }
     } else if (RollingUpgradeCommand.matches(cmd)) {
-      if (argv.length < 1 || argv.length > 2) {
+      if (argv.length > 2) {
         printUsage(cmd);
         return exitCode;
       }
@@ -2097,7 +2097,7 @@ public class DFSAdmin extends FsShell {
         return exitCode;
       }
     } else if ("-triggerBlockReport".equals(cmd)) {
-      if (argv.length < 1) {
+      if ((argv.length != 2) && (argv.length != 3)) {
         printUsage(cmd);
         return exitCode;
       }

+ 28 - 13
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/XmlEditsVisitor.java

@@ -20,17 +20,21 @@ package org.apache.hadoop.hdfs.tools.offlineEditsViewer;
 import java.io.IOException;
 import java.io.OutputStream;
 
+import javax.xml.transform.OutputKeys;
+import javax.xml.transform.TransformerConfigurationException;
+import javax.xml.transform.sax.SAXTransformerFactory;
+import javax.xml.transform.sax.TransformerHandler;
+import javax.xml.transform.stream.StreamResult;
+import org.xml.sax.ContentHandler;
+import org.xml.sax.SAXException;
+import org.xml.sax.helpers.AttributesImpl;
+
 import org.apache.hadoop.hdfs.util.XMLUtils;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.xml.sax.ContentHandler;
-import org.xml.sax.SAXException;
-import org.xml.sax.helpers.AttributesImpl;
 
-import org.apache.xml.serialize.OutputFormat;
-import org.apache.xml.serialize.XMLSerializer;
 
 /**
  * An XmlEditsVisitor walks over an EditLog structure and writes out
@@ -41,26 +45,37 @@ import org.apache.xml.serialize.XMLSerializer;
 public class XmlEditsVisitor implements OfflineEditsVisitor {
   private final OutputStream out;
   private ContentHandler contentHandler;
+  private final SAXTransformerFactory factory;
+  private final static String XML_INDENTATION_PROP ="{http://xml.apache.org/" +
+          "xslt}indent-amount";
+  private final static String XML_INDENTATION_NUM ="2";
 
   /**
    * Create a processor that writes to the file named and may or may not
    * also output to the screen, as specified.
    *
-   * @param filename Name of file to write output to
-   * @param printToScreen Mirror output to screen?
+   * @param out output stream to write
+   * @throws IOException on any error
    */
   public XmlEditsVisitor(OutputStream out)
       throws IOException {
     this.out = out;
-    OutputFormat outFormat = new OutputFormat("XML", "UTF-8", true);
-    outFormat.setIndenting(true);
-    outFormat.setIndent(2);
-    outFormat.setDoctype(null, null);
-    XMLSerializer serializer = new XMLSerializer(out, outFormat);
-    contentHandler = serializer.asContentHandler();
+    factory =(SAXTransformerFactory)SAXTransformerFactory.newInstance();
     try {
+      TransformerHandler handler = factory.newTransformerHandler();
+      handler.getTransformer().setOutputProperty(OutputKeys.METHOD, "xml");
+      handler.getTransformer().setOutputProperty(OutputKeys.ENCODING, "UTF-8");
+      handler.getTransformer().setOutputProperty(OutputKeys.INDENT, "yes");
+      handler.getTransformer().setOutputProperty(XML_INDENTATION_PROP,
+              XML_INDENTATION_NUM);
+      handler.getTransformer().setOutputProperty(OutputKeys.STANDALONE, "yes");
+      handler.setResult(new StreamResult(out));
+      contentHandler = handler;
+      
       contentHandler.startDocument();
       contentHandler.startElement("", "", "EDITS", new AttributesImpl());
+    } catch (TransformerConfigurationException e) {
+      throw new IOException("SAXTransformer error: " + e.getMessage());
     } catch (SAXException e) {
       throw new IOException("SAX error: " + e.getMessage());
     }

+ 7 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java

@@ -722,9 +722,13 @@ class ImageLoaderCurrent implements ImageLoader {
       if (supportSnapshot && supportInodeId) {
         dirNodeMap.put(inodeId, pathName);
       }
-      v.visit(ImageElement.NS_QUOTA, numBlocks == -1 ? in.readLong() : -1);
-      if (NameNodeLayoutVersion.supports(Feature.DISKSPACE_QUOTA, imageVersion))
-        v.visit(ImageElement.DS_QUOTA, numBlocks == -1 ? in.readLong() : -1);
+
+      v.visit(ImageElement.NS_QUOTA, in.readLong());
+      if (NameNodeLayoutVersion.supports(Feature.DISKSPACE_QUOTA,
+          imageVersion)) {
+        v.visit(ImageElement.DS_QUOTA, in.readLong());
+      }
+
       if (supportSnapshot) {
         boolean snapshottable = in.readBoolean();
         if (!snapshottable) {

+ 39 - 0
hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md

@@ -495,6 +495,45 @@ See also: [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).getFileSt
 
 See also: [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).listStatus
 
+### List a File
+
+* Submit a HTTP GET request.
+
+        curl -i  "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?op=LISTSTATUS"
+
+    The client receives a response with a [`FileStatuses` JSON object](#FileStatuses_JSON_Schema):
+
+        HTTP/1.1 200 OK
+        Content-Type: application/json
+        Content-Length: 427
+
+        {
+          "FileStatuses":
+          {
+            "FileStatus":
+            [
+              {
+                "accessTime"      : 1320171722771,
+                "blockSize"       : 33554432,
+                "childrenNum"     : 0,
+                "fileId"          : 16390,
+                "group"           : "supergroup",
+                "length"          : 1366,
+                "modificationTime": 1501770633062,
+                "owner"           : "webuser",
+                "pathSuffix"      : "",
+                "permission"      : "644",
+                "replication"     : 1,
+                "storagePolicy"   : 0,
+                "type"            : "FILE"
+              }
+            ]
+          }
+        }
+
+See also: [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).listStatus
+
+
 ### Iteratively List a Directory
 
 * Submit a HTTP GET request.

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java

@@ -51,7 +51,6 @@ import org.apache.hadoop.util.StringUtils;
 import org.apache.log4j.Logger;
 import org.junit.Test;
 
-import static org.apache.hadoop.hdfs.inotify.Event.CreateEvent;
 import static org.junit.Assert.*;
 
 /**
@@ -572,7 +571,7 @@ public class TestDFSUpgradeFromImage {
       Path path) throws IOException {
     String pathStr = path.toString();
     HdfsFileStatus status = dfs.getFileInfo(pathStr);
-    if (!status.isDir()) {
+    if (!status.isDirectory()) {
       for (int retries = 10; retries > 0; retries--) {
         if (dfs.recoverLease(pathStr)) {
           return;

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestStorageMover.java

@@ -283,7 +283,7 @@ public class TestStorageMover {
 
     private void verifyRecursively(final Path parent,
         final HdfsFileStatus status) throws Exception {
-      if (status.isDir()) {
+      if (status.isDirectory()) {
         Path fullPath = parent == null ?
             new Path("/") : status.getFullPath(parent);
         DirectoryListing children = dfs.getClient().listPaths(

+ 26 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeOptionParsing.java

@@ -138,5 +138,30 @@ public class TestNameNodeOptionParsing {
       }
     }
   }
-    
+
+  @Test
+  public void testFormat() {
+    String[] args = new String[] {"-format"};
+    StartupOption opt = NameNode.parseArguments(args);
+    assertEquals(StartupOption.FORMAT, opt);
+    assertEquals(true, opt.getInteractiveFormat());
+    assertEquals(false, opt.getForceFormat());
+
+    args = new String[] {"-format", "-nonInteractive"};
+    opt = NameNode.parseArguments(args);
+    assertEquals(StartupOption.FORMAT, opt);
+    assertEquals(false, opt.getInteractiveFormat());
+    assertEquals(false, opt.getForceFormat());
+
+    args = new String[] {"-format", "-nonInteractive", "-force"};
+    opt = NameNode.parseArguments(args);
+    assertEquals(StartupOption.FORMAT, opt);
+    assertEquals(false, opt.getInteractiveFormat());
+    assertEquals(true, opt.getForceFormat());
+
+    // test error condition
+    args = new String[] {"-nonInteractive"};
+    opt = NameNode.parseArguments(args);
+    assertNull(opt);
+  }
 }

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

@@ -450,7 +450,7 @@ public class TestStartup {
     namenode.getNamesystem().mkdirs("/test",
         new PermissionStatus("hairong", null, FsPermission.getDefault()), true);
     NamenodeProtocols nnRpc = namenode.getRpcServer();
-    assertTrue(nnRpc.getFileInfo("/test").isDir());
+    assertTrue(nnRpc.getFileInfo("/test").isDirectory());
     nnRpc.setSafeMode(SafeModeAction.SAFEMODE_ENTER, false);
     nnRpc.saveNamespace(0, 0);
     namenode.stop();
@@ -481,7 +481,7 @@ public class TestStartup {
   private void checkNameSpace(Configuration conf) throws IOException {
     NameNode namenode = new NameNode(conf);
     NamenodeProtocols nnRpc = namenode.getRpcServer();
-    assertTrue(nnRpc.getFileInfo("/test").isDir());
+    assertTrue(nnRpc.getFileInfo("/test").isDirectory());
     nnRpc.setSafeMode(SafeModeAction.SAFEMODE_ENTER, false);
     nnRpc.saveNamespace(0, 0);
     namenode.stop();

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogTailer.java

@@ -124,7 +124,7 @@ public class TestEditLogTailer {
       
       for (int i = 0; i < DIRS_TO_MAKE / 2; i++) {
         assertTrue(NameNodeAdapter.getFileInfo(nn2,
-            getDirPath(i), false).isDir());
+            getDirPath(i), false).isDirectory());
       }
       
       for (int i = DIRS_TO_MAKE / 2; i < DIRS_TO_MAKE; i++) {
@@ -137,7 +137,7 @@ public class TestEditLogTailer {
       
       for (int i = DIRS_TO_MAKE / 2; i < DIRS_TO_MAKE; i++) {
         assertTrue(NameNodeAdapter.getFileInfo(nn2,
-            getDirPath(i), false).isDir());
+            getDirPath(i), false).isDirectory());
       }
     } finally {
       cluster.shutdown();

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailureToReadEdits.java

@@ -205,7 +205,7 @@ public class TestFailureToReadEdits {
         TEST_DIR1, false));
     // Should have been successfully created.
     assertTrue(NameNodeAdapter.getFileInfo(nn1,
-        TEST_DIR2, false).isDir());
+        TEST_DIR2, false).isDirectory());
     // Null because it hasn't been created yet.
     assertNull(NameNodeAdapter.getFileInfo(nn1,
         TEST_DIR3, false));
@@ -219,10 +219,10 @@ public class TestFailureToReadEdits {
         TEST_DIR1, false));
     // Should have been successfully created.
     assertTrue(NameNodeAdapter.getFileInfo(nn1,
-        TEST_DIR2, false).isDir());
+        TEST_DIR2, false).isDirectory());
     // Should now have been successfully created.
     assertTrue(NameNodeAdapter.getFileInfo(nn1,
-        TEST_DIR3, false).isDir());
+        TEST_DIR3, false).isDirectory());
   }
   
   /**

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestInitializeSharedEdits.java

@@ -128,7 +128,7 @@ public class TestInitializeSharedEdits {
       HATestUtil.waitForStandbyToCatchUp(cluster.getNameNode(0),
           cluster.getNameNode(1));
       assertTrue(NameNodeAdapter.getFileInfo(cluster.getNameNode(1),
-          newPath.toString(), false).isDir());
+          newPath.toString(), false).isDirectory());
     } finally {
       if (fs != null) {
         fs.close();

BIN
hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored


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

@@ -1,6 +1,6 @@
-<?xml version="1.0" encoding="UTF-8"?>
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
 <EDITS>
-  <EDITS_VERSION>-63</EDITS_VERSION>
+  <EDITS_VERSION>-64</EDITS_VERSION>
   <RECORD>
     <OPCODE>OP_START_LOG_SEGMENT</OPCODE>
     <DATA>
@@ -63,8 +63,8 @@
       <MTIME>1422406380369</MTIME>
       <ATIME>1422406380345</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME></CLIENT_NAME>
-      <CLIENT_MACHINE></CLIENT_MACHINE>
+      <CLIENT_NAME/>
+      <CLIENT_MACHINE/>
       <OVERWRITE>false</OVERWRITE>
       <PERMISSION_STATUS>
         <USERNAME>xyao</USERNAME>
@@ -96,8 +96,8 @@
       <MTIME>1422406380376</MTIME>
       <ATIME>1422406380345</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME></CLIENT_NAME>
-      <CLIENT_MACHINE></CLIENT_MACHINE>
+      <CLIENT_NAME/>
+      <CLIENT_MACHINE/>
       <OVERWRITE>false</OVERWRITE>
       <PERMISSION_STATUS>
         <USERNAME>xyao</USERNAME>
@@ -238,8 +238,8 @@
       <MTIME>1422406380425</MTIME>
       <ATIME>1422406380423</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME></CLIENT_NAME>
-      <CLIENT_MACHINE></CLIENT_MACHINE>
+      <CLIENT_NAME/>
+      <CLIENT_MACHINE/>
       <OVERWRITE>false</OVERWRITE>
       <PERMISSION_STATUS>
         <USERNAME>xyao</USERNAME>
@@ -360,7 +360,7 @@
         <NUM_BYTES>0</NUM_BYTES>
         <GENSTAMP>1001</GENSTAMP>
       </BLOCK>
-      <RPC_CLIENTID></RPC_CLIENTID>
+      <RPC_CLIENTID/>
       <RPC_CALLID>-2</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -393,7 +393,7 @@
         <NUM_BYTES>0</NUM_BYTES>
         <GENSTAMP>1002</GENSTAMP>
       </BLOCK>
-      <RPC_CLIENTID></RPC_CLIENTID>
+      <RPC_CLIENTID/>
       <RPC_CALLID>-2</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -426,7 +426,7 @@
         <NUM_BYTES>0</NUM_BYTES>
         <GENSTAMP>1003</GENSTAMP>
       </BLOCK>
-      <RPC_CLIENTID></RPC_CLIENTID>
+      <RPC_CLIENTID/>
       <RPC_CALLID>-2</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -441,8 +441,8 @@
       <MTIME>1422406380534</MTIME>
       <ATIME>1422406380446</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME></CLIENT_NAME>
-      <CLIENT_MACHINE></CLIENT_MACHINE>
+      <CLIENT_NAME/>
+      <CLIENT_MACHINE/>
       <OVERWRITE>false</OVERWRITE>
       <BLOCK>
         <BLOCK_ID>1073741825</BLOCK_ID>
@@ -513,7 +513,7 @@
         <NUM_BYTES>0</NUM_BYTES>
         <GENSTAMP>1004</GENSTAMP>
       </BLOCK>
-      <RPC_CLIENTID></RPC_CLIENTID>
+      <RPC_CLIENTID/>
       <RPC_CALLID>-2</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -546,7 +546,7 @@
         <NUM_BYTES>0</NUM_BYTES>
         <GENSTAMP>1005</GENSTAMP>
       </BLOCK>
-      <RPC_CLIENTID></RPC_CLIENTID>
+      <RPC_CLIENTID/>
       <RPC_CALLID>-2</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -579,7 +579,7 @@
         <NUM_BYTES>0</NUM_BYTES>
         <GENSTAMP>1006</GENSTAMP>
       </BLOCK>
-      <RPC_CLIENTID></RPC_CLIENTID>
+      <RPC_CLIENTID/>
       <RPC_CALLID>-2</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -594,8 +594,8 @@
       <MTIME>1422406380558</MTIME>
       <ATIME>1422406380537</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME></CLIENT_NAME>
-      <CLIENT_MACHINE></CLIENT_MACHINE>
+      <CLIENT_NAME/>
+      <CLIENT_MACHINE/>
       <OVERWRITE>false</OVERWRITE>
       <BLOCK>
         <BLOCK_ID>1073741828</BLOCK_ID>
@@ -666,7 +666,7 @@
         <NUM_BYTES>0</NUM_BYTES>
         <GENSTAMP>1007</GENSTAMP>
       </BLOCK>
-      <RPC_CLIENTID></RPC_CLIENTID>
+      <RPC_CLIENTID/>
       <RPC_CALLID>-2</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -699,7 +699,7 @@
         <NUM_BYTES>0</NUM_BYTES>
         <GENSTAMP>1008</GENSTAMP>
       </BLOCK>
-      <RPC_CLIENTID></RPC_CLIENTID>
+      <RPC_CLIENTID/>
       <RPC_CALLID>-2</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -732,7 +732,7 @@
         <NUM_BYTES>0</NUM_BYTES>
         <GENSTAMP>1009</GENSTAMP>
       </BLOCK>
-      <RPC_CLIENTID></RPC_CLIENTID>
+      <RPC_CLIENTID/>
       <RPC_CALLID>-2</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -747,8 +747,8 @@
       <MTIME>1422406380579</MTIME>
       <ATIME>1422406380560</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME></CLIENT_NAME>
-      <CLIENT_MACHINE></CLIENT_MACHINE>
+      <CLIENT_NAME/>
+      <CLIENT_MACHINE/>
       <OVERWRITE>false</OVERWRITE>
       <BLOCK>
         <BLOCK_ID>1073741831</BLOCK_ID>
@@ -834,7 +834,7 @@
         <NUM_BYTES>0</NUM_BYTES>
         <GENSTAMP>1010</GENSTAMP>
       </BLOCK>
-      <RPC_CLIENTID></RPC_CLIENTID>
+      <RPC_CLIENTID/>
       <RPC_CALLID>-2</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -867,7 +867,7 @@
         <NUM_BYTES>0</NUM_BYTES>
         <GENSTAMP>1011</GENSTAMP>
       </BLOCK>
-      <RPC_CLIENTID></RPC_CLIENTID>
+      <RPC_CLIENTID/>
       <RPC_CALLID>-2</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -882,8 +882,8 @@
       <MTIME>1422406380599</MTIME>
       <ATIME>1422406380586</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME></CLIENT_NAME>
-      <CLIENT_MACHINE></CLIENT_MACHINE>
+      <CLIENT_NAME/>
+      <CLIENT_MACHINE/>
       <OVERWRITE>false</OVERWRITE>
       <BLOCK>
         <BLOCK_ID>1073741834</BLOCK_ID>
@@ -979,7 +979,7 @@
         <NUM_BYTES>0</NUM_BYTES>
         <GENSTAMP>1012</GENSTAMP>
       </BLOCK>
-      <RPC_CLIENTID></RPC_CLIENTID>
+      <RPC_CLIENTID/>
       <RPC_CALLID>-2</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -993,7 +993,7 @@
         <NUM_BYTES>11</NUM_BYTES>
         <GENSTAMP>1012</GENSTAMP>
       </BLOCK>
-      <RPC_CLIENTID></RPC_CLIENTID>
+      <RPC_CLIENTID/>
       <RPC_CALLID>-2</RPC_CALLID>
     </DATA>
   </RECORD>
@@ -1024,8 +1024,8 @@
       <MTIME>1422406383261</MTIME>
       <ATIME>1422406380608</ATIME>
       <BLOCKSIZE>512</BLOCKSIZE>
-      <CLIENT_NAME></CLIENT_NAME>
-      <CLIENT_MACHINE></CLIENT_MACHINE>
+      <CLIENT_NAME/>
+      <CLIENT_MACHINE/>
       <OVERWRITE>false</OVERWRITE>
       <BLOCK>
         <BLOCK_ID>1073741836</BLOCK_ID>

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineFileInputFormat.java

@@ -154,7 +154,7 @@ public class TestCombineFileInputFormat {
     @Override
     public BlockLocation[] getFileBlockLocations(
         FileStatus stat, long start, long len) throws IOException {
-      if (stat.isDir()) {
+      if (stat.isDirectory()) {
         return null;
       }
       System.out.println("File " + stat.getPath());

+ 1 - 9
hadoop-mapreduce-project/hadoop-mapreduce-client/pom.xml

@@ -238,14 +238,6 @@
                       <outputDirectory>${project.build.directory}</outputDirectory>
                       <destFileName>hadoop-annotations.jar</destFileName>
                     </artifactItem>
-                    <artifactItem>
-                      <groupId>xerces</groupId>
-                      <artifactId>xercesImpl</artifactId>
-                      <version>${xerces.version.jdiff}</version>
-                      <overWrite>false</overWrite>
-                      <outputDirectory>${project.build.directory}</outputDirectory>
-                      <destFileName>xerces.jar</destFileName>
-                    </artifactItem>
                   </artifactItems>
                 </configuration>
               </execution>
@@ -283,7 +275,7 @@
                            sourceFiles="${dev-support.relative.dir}/jdiff/Null.java"
                            maxmemory="${jdiff.javadoc.maxmemory}">
                     <doclet name="org.apache.hadoop.classification.tools.IncludePublicAnnotationsJDiffDoclet"
-                            path="${project.build.directory}/hadoop-annotations.jar:${project.build.directory}/jdiff.jar:${project.build.directory}/xerces.jar">
+                            path="${project.build.directory}/hadoop-annotations.jar:${project.build.directory}/jdiff.jar">
                       <param name="-oldapi" value="${project.name} ${jdiff.stable.api}"/>
                       <param name="-newapi" value="${project.name} ${project.version}"/>
                       <param name="-oldapidir" value="${basedir}/${dev-support.relative.dir}/jdiff"/>

+ 1 - 9
hadoop-project-dist/pom.xml

@@ -194,14 +194,6 @@
                       <outputDirectory>${project.build.directory}</outputDirectory>
                       <destFileName>hadoop-annotations.jar</destFileName>
                     </artifactItem>
-                    <artifactItem>
-                      <groupId>xerces</groupId>
-                      <artifactId>xercesImpl</artifactId>
-                      <version>${xerces.jdiff.version}</version>
-                      <overWrite>false</overWrite>
-                      <outputDirectory>${project.build.directory}</outputDirectory>
-                      <destFileName>xerces.jar</destFileName>
-                    </artifactItem>
                   </artifactItems>
                 </configuration>
               </execution>
@@ -267,7 +259,7 @@
                              sourceFiles="${basedir}/dev-support/jdiff/Null.java"
                              maxmemory="${jdiff.javadoc.maxmemory}">
                       <doclet name="org.apache.hadoop.classification.tools.IncludePublicAnnotationsJDiffDoclet"
-                              path="${project.build.directory}/hadoop-annotations.jar:${project.build.directory}/jdiff.jar:${project.build.directory}/xerces.jar">
+                              path="${project.build.directory}/hadoop-annotations.jar:${project.build.directory}/jdiff.jar">
                         <param name="-oldapi" value="${project.name} ${jdiff.stable.api}"/>
                         <param name="-newapi" value="${project.name} ${project.version}"/>
                         <param name="-oldapidir" value="${basedir}/dev-support/jdiff"/>

+ 3 - 11
hadoop-project/pom.xml

@@ -45,8 +45,6 @@
     <!-- These 2 versions are defined here because they are used -->
     <!-- JDIFF generation from embedded ant in the antrun plugin -->
     <jdiff.version>1.0.9</jdiff.version>
-    <!-- Version number for xerces used by JDiff -->
-    <xerces.jdiff.version>2.11.0</xerces.jdiff.version>
 
     <kafka.version>0.8.2.1</kafka.version>
     <hbase.version>1.2.6</hbase.version>
@@ -98,7 +96,7 @@
     <apacheds.version>2.0.0-M21</apacheds.version>
     <ldap-api.version>1.0.0-M33</ldap-api.version>
 
-    <jcache.version>1.0.0</jcache.version>
+    <jcache.version>1.0-alpha-1</jcache.version>
     <ehcache.version>3.3.1</ehcache.version>
     <hikari.version>2.4.12</hikari.version>
     <mssql.version>6.2.1.jre7</mssql.version>
@@ -1166,12 +1164,6 @@
         </exclusions>
      </dependency>
 
-     <dependency>
-       <groupId>xerces</groupId>
-       <artifactId>xercesImpl</artifactId>
-       <version>2.9.1</version>
-     </dependency>
-
      <dependency>
        <groupId>org.apache.curator</groupId>
        <artifactId>curator-recipes</artifactId>
@@ -1295,8 +1287,8 @@
           <version>1.0.0</version>
         </dependency>
         <dependency>
-          <groupId>javax.cache</groupId>
-          <artifactId>cache-api</artifactId>
+          <groupId>org.apache.geronimo.specs</groupId>
+          <artifactId>geronimo-jcache_1.0_spec</artifactId>
           <version>${jcache.version}</version>
         </dependency>
         <dependency>

+ 4 - 4
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestOutOfBandAzureBlobOperations.java

@@ -73,20 +73,20 @@ public class TestOutOfBandAzureBlobOperations {
     FileStatus[] obtained = fs.listStatus(new Path("/root/b"));
     assertNotNull(obtained);
     assertEquals(1, obtained.length);
-    assertFalse(obtained[0].isDir());
+    assertFalse(obtained[0].isDirectory());
     assertEquals("/root/b", obtained[0].getPath().toUri().getPath());
 
     // List the directory
     obtained = fs.listStatus(new Path("/root"));
     assertNotNull(obtained);
     assertEquals(1, obtained.length);
-    assertFalse(obtained[0].isDir());
+    assertFalse(obtained[0].isDirectory());
     assertEquals("/root/b", obtained[0].getPath().toUri().getPath());
 
     // Get the directory's file status
     FileStatus dirStatus = fs.getFileStatus(new Path("/root"));
     assertNotNull(dirStatus);
-    assertTrue(dirStatus.isDir());
+    assertTrue(dirStatus.isDirectory());
     assertEquals("/root", dirStatus.getPath().toUri().getPath());
   }
 
@@ -114,7 +114,7 @@ public class TestOutOfBandAzureBlobOperations {
     FileStatus[] listResult = fs.listStatus(new Path("/root/b"));
     // File should win.
     assertEquals(1, listResult.length);
-    assertFalse(listResult[0].isDir());
+    assertFalse(listResult[0].isDirectory());
     try {
       // Trying to delete root/b/c would cause a dilemma for WASB, so
       // it should throw.

+ 4 - 12
hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/snative/SwiftFileStatus.java

@@ -71,7 +71,7 @@ public class SwiftFileStatus extends FileStatus {
    * @return true if the status is considered to be a file
    */
   @Override
-  public boolean isDir() {
+  public boolean isDirectory() {
     return super.isDirectory() || getLen() == 0;
   }
 
@@ -79,19 +79,11 @@ public class SwiftFileStatus extends FileStatus {
    * A entry is a file if it is not a directory.
    * By implementing it <i>and not marking as an override</i> this
    * subclass builds and runs in both Hadoop versions.
-   * @return the opposite value to {@link #isDir()}
+   * @return the opposite value to {@link #isDirectory()}
    */
   @Override
   public boolean isFile() {
-    return !isDir();
-  }
-
-  /**
-   * Directory test
-   * @return true if the file is considered to be a directory
-   */
-  public boolean isDirectory() {
-    return isDir();
+    return !this.isDirectory();
   }
 
   @Override
@@ -100,7 +92,7 @@ public class SwiftFileStatus extends FileStatus {
     sb.append(getClass().getSimpleName());
     sb.append("{ ");
     sb.append("path=").append(getPath());
-    sb.append("; isDirectory=").append(isDir());
+    sb.append("; isDirectory=").append(isDirectory());
     sb.append("; length=").append(getLen());
     sb.append("; blocksize=").append(getBlockSize());
     sb.append("; modification_time=").append(getModificationTime());

+ 2 - 2
hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/snative/SwiftNativeFileSystemStore.java

@@ -578,7 +578,7 @@ public class SwiftNativeFileSystemStore {
 
     //enum the child entries and everything underneath
     List<FileStatus> childStats = listDirectory(srcObject, true, true);
-    boolean srcIsFile = !srcMetadata.isDir();
+    boolean srcIsFile = !srcMetadata.isDirectory();
     if (srcIsFile) {
 
       //source is a simple file OR a partitioned file
@@ -945,7 +945,7 @@ public class SwiftNativeFileSystemStore {
     //>1 entry implies directory with children. Run through them,
     // but first check for the recursive flag and reject it *unless it looks
     // like a partitioned file (len > 0 && has children)
-    if (!fileStatus.isDir()) {
+    if (!fileStatus.isDirectory()) {
       LOG.debug("Multiple child entries but entry has data: assume partitioned");
     } else if (!recursive) {
       //if there are children, unless this is a recursive operation, fail immediately

+ 2 - 2
hadoop-tools/hadoop-openstack/src/test/java/org/apache/hadoop/fs/swift/TestSwiftFileSystemDirectories.java

@@ -87,7 +87,7 @@ public class TestSwiftFileSystemDirectories extends SwiftFileSystemBaseTest {
     assertEquals("Wrong number of elements in file status " + statusString, 1,
                  statuses.length);
     SwiftFileStatus stat = (SwiftFileStatus) statuses[0];
-    assertTrue("isDir(): Not a directory: " + stat, stat.isDir());
+    assertTrue("isDir(): Not a directory: " + stat, stat.isDirectory());
     extraStatusAssertions(stat);
   }
 
@@ -135,7 +135,7 @@ public class TestSwiftFileSystemDirectories extends SwiftFileSystemBaseTest {
     SwiftTestUtils.writeTextFile(fs, src, "testMultiByteFilesAreFiles", false);
     assertIsFile(src);
     FileStatus status = fs.getFileStatus(src);
-    assertFalse(status.isDir());
+    assertFalse(status.isDirectory());
   }
 
 }

+ 1 - 1
hadoop-tools/hadoop-openstack/src/test/java/org/apache/hadoop/fs/swift/TestSwiftFileSystemPartitionedUploads.java

@@ -228,7 +228,7 @@ public class TestSwiftFileSystemPartitionedUploads extends
                  status.getLen());
     String fileInfo = qualifiedPath + "  " + status;
     assertFalse("File claims to be a directory " + fileInfo,
-                status.isDir());
+                status.isDirectory());
 
     FileStatus listedFileStat = resolveChild(parentDirListing, qualifiedPath);
     assertNotNull("Did not find " + path + " in " + parentDirLS,

+ 366 - 29
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClient.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.yarn.client.api.impl;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 import static org.mockito.Matchers.any;
@@ -36,6 +37,7 @@ 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.TreeSet;
 
@@ -142,6 +144,10 @@ public class TestAMRMClient {
     // set the minimum allocation so that resource decrease can go under 1024
     conf.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 512);
     conf.setLong(YarnConfiguration.NM_LOG_RETAIN_SECONDS, 1);
+    conf.setBoolean(
+        YarnConfiguration.OPPORTUNISTIC_CONTAINER_ALLOCATION_ENABLED, true);
+    conf.setInt(
+        YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH, 10);
     yarnCluster = new MiniYARNCluster(TestAMRMClient.class.getName(), nodeCount, 1, 1);
     yarnCluster.init(conf);
     yarnCluster.start();
@@ -924,8 +930,8 @@ public class TestAMRMClient {
     // add exp=x to ANY
     client.addContainerRequest(new ContainerRequest(Resource.newInstance(1024,
         1), null, null, Priority.UNDEFINED, true, "x"));
-    Assert.assertEquals(1, client.ask.size());
-    Assert.assertEquals("x", client.ask.iterator().next()
+    assertEquals(1, client.ask.size());
+    assertEquals("x", client.ask.iterator().next()
         .getNodeLabelExpression());
 
     // add exp=x then add exp=a to ANY in same priority, only exp=a should kept
@@ -933,8 +939,8 @@ public class TestAMRMClient {
         1), null, null, Priority.UNDEFINED, true, "x"));
     client.addContainerRequest(new ContainerRequest(Resource.newInstance(1024,
         1), null, null, Priority.UNDEFINED, true, "a"));
-    Assert.assertEquals(1, client.ask.size());
-    Assert.assertEquals("a", client.ask.iterator().next()
+    assertEquals(1, client.ask.size());
+    assertEquals("a", client.ask.iterator().next()
         .getNodeLabelExpression());
     
     // add exp=x to ANY, rack and node, only resource request has ANY resource
@@ -943,10 +949,10 @@ public class TestAMRMClient {
     client.addContainerRequest(new ContainerRequest(Resource.newInstance(1024,
         1), null, null, Priority.UNDEFINED, true,
         "y"));
-    Assert.assertEquals(1, client.ask.size());
+    assertEquals(1, client.ask.size());
     for (ResourceRequest req : client.ask) {
       if (ResourceRequest.ANY.equals(req.getResourceName())) {
-        Assert.assertEquals("y", req.getNodeLabelExpression());
+        assertEquals("y", req.getNodeLabelExpression());
       } else {
         Assert.assertNull(req.getNodeLabelExpression());
       }
@@ -957,7 +963,7 @@ public class TestAMRMClient {
         new String[] { "node1", "node2" }, Priority.UNDEFINED, true, "y"));
     for (ResourceRequest req : client.ask) {
       if (ResourceRequest.ANY.equals(req.getResourceName())) {
-        Assert.assertEquals("y", req.getNodeLabelExpression());
+        assertEquals("y", req.getNodeLabelExpression());
       } else {
         Assert.assertNull(req.getNodeLabelExpression());
       }
@@ -971,7 +977,7 @@ public class TestAMRMClient {
     } catch (InvalidContainerRequestException e) {
       return;
     }
-    Assert.fail();
+    fail();
   }
   
   @Test(timeout=30000)
@@ -1042,7 +1048,8 @@ public class TestAMRMClient {
     // get allocations
     AllocateResponse allocResponse = amClient.allocate(0.1f);
     List<Container> containers = allocResponse.getAllocatedContainers();
-    Assert.assertEquals(num, containers.size());
+    assertEquals(num, containers.size());
+
     // build container launch context
     Credentials ts = new Credentials();
     DataOutputBuffer dob = new DataOutputBuffer();
@@ -1083,14 +1090,14 @@ public class TestAMRMClient {
   private void doContainerResourceChange(
       final AMRMClient<ContainerRequest> amClient, List<Container> containers)
       throws YarnException, IOException {
-    Assert.assertEquals(3, containers.size());
+    assertEquals(3, containers.size());
     // remember the container IDs
     Container container1 = containers.get(0);
     Container container2 = containers.get(1);
     Container container3 = containers.get(2);
     AMRMClientImpl<ContainerRequest> amClientImpl =
         (AMRMClientImpl<ContainerRequest>) amClient;
-    Assert.assertEquals(0, amClientImpl.change.size());
+    assertEquals(0, amClientImpl.change.size());
     // verify newer request overwrites older request for the container1
     amClientImpl.requestContainerUpdate(container1,
         UpdateContainerRequest.newInstance(container1.getVersion(),
@@ -1100,21 +1107,21 @@ public class TestAMRMClient {
         UpdateContainerRequest.newInstance(container1.getVersion(),
             container1.getId(), ContainerUpdateType.INCREASE_RESOURCE,
             Resource.newInstance(4096, 1), null));
-    Assert.assertEquals(Resource.newInstance(4096, 1),
+    assertEquals(Resource.newInstance(4096, 1),
         amClientImpl.change.get(container1.getId()).getValue().getCapability());
     // verify new decrease request cancels old increase request for container1
     amClientImpl.requestContainerUpdate(container1,
         UpdateContainerRequest.newInstance(container1.getVersion(),
             container1.getId(), ContainerUpdateType.DECREASE_RESOURCE,
             Resource.newInstance(512, 1), null));
-    Assert.assertEquals(Resource.newInstance(512, 1),
+    assertEquals(Resource.newInstance(512, 1),
         amClientImpl.change.get(container1.getId()).getValue().getCapability());
     // request resource increase for container2
     amClientImpl.requestContainerUpdate(container2,
         UpdateContainerRequest.newInstance(container2.getVersion(),
             container2.getId(), ContainerUpdateType.INCREASE_RESOURCE,
             Resource.newInstance(2048, 1), null));
-    Assert.assertEquals(Resource.newInstance(2048, 1),
+    assertEquals(Resource.newInstance(2048, 1),
         amClientImpl.change.get(container2.getId()).getValue().getCapability());
     // verify release request will cancel pending change requests for the same
     // container
@@ -1122,27 +1129,357 @@ public class TestAMRMClient {
         UpdateContainerRequest.newInstance(container3.getVersion(),
             container3.getId(), ContainerUpdateType.INCREASE_RESOURCE,
             Resource.newInstance(2048, 1), null));
-    Assert.assertEquals(3, amClientImpl.pendingChange.size());
+    assertEquals(3, amClientImpl.pendingChange.size());
     amClientImpl.releaseAssignedContainer(container3.getId());
-    Assert.assertEquals(2, amClientImpl.pendingChange.size());
+    assertEquals(2, amClientImpl.pendingChange.size());
     // as of now: container1 asks to decrease to (512, 1)
     //            container2 asks to increase to (2048, 1)
     // send allocation requests
     AllocateResponse allocResponse = amClient.allocate(0.1f);
-    Assert.assertEquals(0, amClientImpl.change.size());
+    assertEquals(0, amClientImpl.change.size());
     // we should get decrease confirmation right away
     List<UpdatedContainer> updatedContainers =
         allocResponse.getUpdatedContainers();
-    Assert.assertEquals(1, updatedContainers.size());
+    assertEquals(1, updatedContainers.size());
     // we should get increase allocation after the next NM's heartbeat to RM
     triggerSchedulingWithNMHeartBeat();
     // get allocations
     allocResponse = amClient.allocate(0.1f);
     updatedContainers =
         allocResponse.getUpdatedContainers();
-    Assert.assertEquals(1, updatedContainers.size());
+    assertEquals(1, updatedContainers.size());
+  }
+
+  @Test(timeout=60000)
+  public void testAMRMClientWithContainerPromotion()
+      throws YarnException, IOException {
+    AMRMClientImpl<AMRMClient.ContainerRequest> amClient =
+        (AMRMClientImpl<AMRMClient.ContainerRequest>) AMRMClient
+            .createAMRMClient();
+    //asserting we are not using the singleton instance cache
+    Assert.assertSame(NMTokenCache.getSingleton(),
+        amClient.getNMTokenCache());
+    amClient.init(conf);
+    amClient.start();
+
+    // start am nm client
+    NMClientImpl nmClient = (NMClientImpl) NMClient.createNMClient();
+    Assert.assertNotNull(nmClient);
+    // asserting we are using the singleton instance cache
+    Assert.assertSame(
+        NMTokenCache.getSingleton(), nmClient.getNMTokenCache());
+    nmClient.init(conf);
+    nmClient.start();
+    assertEquals(STATE.STARTED, nmClient.getServiceState());
+
+    amClient.registerApplicationMaster("Host", 10000, "");
+    // setup container request
+    assertEquals(0, amClient.ask.size());
+    assertEquals(0, amClient.release.size());
+
+    // START OPPORTUNISTIC Container, Send allocation request to RM
+    amClient.addContainerRequest(
+        new AMRMClient.ContainerRequest(capability, null, null, priority2, 0,
+            true, null, ExecutionTypeRequest
+            .newInstance(ExecutionType.OPPORTUNISTIC, true)));
+
+    int oppContainersRequestedAny =
+        amClient.getTable(0).get(priority2, ResourceRequest.ANY,
+            ExecutionType.OPPORTUNISTIC, capability).remoteRequest
+            .getNumContainers();
+
+    assertEquals(1, oppContainersRequestedAny);
+    assertEquals(1, amClient.ask.size());
+    assertEquals(0, amClient.release.size());
+
+    // RM should allocate container within 2 calls to allocate()
+    int allocatedContainerCount = 0;
+    Map<ContainerId, Container> allocatedOpportContainers = new HashMap<>();
+    int iterationsLeft = 50;
+
+    amClient.getNMTokenCache().clearCache();
+    assertEquals(0,
+        amClient.getNMTokenCache().numberOfTokensInCache());
+
+    AllocateResponse allocResponse = null;
+    while (allocatedContainerCount < oppContainersRequestedAny
+        && iterationsLeft-- > 0) {
+      allocResponse = amClient.allocate(0.1f);
+      // let NM heartbeat to RM and trigger allocations
+      //triggerSchedulingWithNMHeartBeat();
+      assertEquals(0, amClient.ask.size());
+      assertEquals(0, amClient.release.size());
+
+      allocatedContainerCount +=
+          allocResponse.getAllocatedContainers().size();
+      for (Container container : allocResponse.getAllocatedContainers()) {
+        if (container.getExecutionType() == ExecutionType.OPPORTUNISTIC) {
+          allocatedOpportContainers.put(container.getId(), container);
+        }
+      }
+      if (allocatedContainerCount < oppContainersRequestedAny) {
+        // sleep to let NM's heartbeat to RM and trigger allocations
+        sleep(100);
+      }
+    }
+
+    assertEquals(oppContainersRequestedAny, allocatedContainerCount);
+    assertEquals(oppContainersRequestedAny, allocatedOpportContainers.size());
+
+    startContainer(allocResponse, nmClient);
+
+    // SEND PROMOTION REQUEST TO RM
+    try {
+      Container c = allocatedOpportContainers.values().iterator().next();
+      amClient.requestContainerUpdate(
+          c, UpdateContainerRequest.newInstance(c.getVersion(),
+              c.getId(), ContainerUpdateType.PROMOTE_EXECUTION_TYPE,
+              null, ExecutionType.OPPORTUNISTIC));
+      fail("Should throw Exception..");
+    } catch (IllegalArgumentException e) {
+      System.out.println("## " + e.getMessage());
+      assertTrue(e.getMessage().contains(
+          "target should be GUARANTEED and original should be OPPORTUNISTIC"));
+    }
+
+    Container c = allocatedOpportContainers.values().iterator().next();
+    amClient.requestContainerUpdate(
+        c, UpdateContainerRequest.newInstance(c.getVersion(),
+            c.getId(), ContainerUpdateType.PROMOTE_EXECUTION_TYPE,
+            null, ExecutionType.GUARANTEED));
+    iterationsLeft = 120;
+    Map<ContainerId, UpdatedContainer> updatedContainers = new HashMap<>();
+    // do a few iterations to ensure RM is not going to send new containers
+    while (iterationsLeft-- > 0 && updatedContainers.isEmpty()) {
+      // inform RM of rejection
+      allocResponse = amClient.allocate(0.1f);
+      // RM did not send new containers because AM does not need any
+      if (allocResponse.getUpdatedContainers() != null) {
+        for (UpdatedContainer updatedContainer : allocResponse
+            .getUpdatedContainers()) {
+          System.out.println("Got update..");
+          updatedContainers.put(updatedContainer.getContainer().getId(),
+              updatedContainer);
+        }
+      }
+      if (iterationsLeft > 0) {
+        // sleep to make sure NM's heartbeat
+        sleep(100);
+      }
+    }
+    assertEquals(1, updatedContainers.size());
+
+    for (ContainerId cId : allocatedOpportContainers.keySet()) {
+      Container orig = allocatedOpportContainers.get(cId);
+      UpdatedContainer updatedContainer = updatedContainers.get(cId);
+      assertNotNull(updatedContainer);
+      assertEquals(ExecutionType.GUARANTEED,
+          updatedContainer.getContainer().getExecutionType());
+      assertEquals(orig.getResource(),
+          updatedContainer.getContainer().getResource());
+      assertEquals(orig.getNodeId(),
+          updatedContainer.getContainer().getNodeId());
+      assertEquals(orig.getVersion() + 1,
+          updatedContainer.getContainer().getVersion());
+    }
+    assertEquals(0, amClient.ask.size());
+    assertEquals(0, amClient.release.size());
+
+    // SEND UPDATE EXECTYPE UPDATE TO NM
+    updateContainerExecType(allocResponse, ExecutionType.GUARANTEED, nmClient);
+
+    amClient.ask.clear();
+  }
+
+  @Test(timeout=60000)
+  public void testAMRMClientWithContainerDemotion()
+      throws YarnException, IOException {
+    AMRMClientImpl<AMRMClient.ContainerRequest> amClient =
+        (AMRMClientImpl<AMRMClient.ContainerRequest>) AMRMClient
+            .createAMRMClient();
+    //asserting we are not using the singleton instance cache
+    Assert.assertSame(NMTokenCache.getSingleton(),
+        amClient.getNMTokenCache());
+    amClient.init(conf);
+    amClient.start();
+
+    NMClientImpl nmClient = (NMClientImpl) NMClient.createNMClient();
+    Assert.assertNotNull(nmClient);
+    // asserting we are using the singleton instance cache
+    Assert.assertSame(
+        NMTokenCache.getSingleton(), nmClient.getNMTokenCache());
+    nmClient.init(conf);
+    nmClient.start();
+    assertEquals(STATE.STARTED, nmClient.getServiceState());
+
+    amClient.registerApplicationMaster("Host", 10000, "");
+    assertEquals(0, amClient.ask.size());
+    assertEquals(0, amClient.release.size());
+
+    // START OPPORTUNISTIC Container, Send allocation request to RM
+    amClient.addContainerRequest(
+        new AMRMClient.ContainerRequest(capability, null, null, priority2, 0,
+            true, null, ExecutionTypeRequest
+            .newInstance(ExecutionType.GUARANTEED, true)));
+
+    int oppContainersRequestedAny =
+        amClient.getTable(0).get(priority2, ResourceRequest.ANY,
+            ExecutionType.GUARANTEED, capability).remoteRequest
+            .getNumContainers();
+
+    assertEquals(1, oppContainersRequestedAny);
+    assertEquals(1, amClient.ask.size());
+    assertEquals(0, amClient.release.size());
+
+    // RM should allocate container within 2 calls to allocate()
+    int allocatedContainerCount = 0;
+    Map<ContainerId, Container> allocatedGuaranteedContainers = new HashMap<>();
+    int iterationsLeft = 50;
+
+    amClient.getNMTokenCache().clearCache();
+    assertEquals(0,
+        amClient.getNMTokenCache().numberOfTokensInCache());
+
+    AllocateResponse allocResponse = null;
+    while (allocatedContainerCount < oppContainersRequestedAny
+        && iterationsLeft-- > 0) {
+      allocResponse = amClient.allocate(0.1f);
+      // let NM heartbeat to RM and trigger allocations
+      //triggerSchedulingWithNMHeartBeat();
+      assertEquals(0, amClient.ask.size());
+      assertEquals(0, amClient.release.size());
+
+      allocatedContainerCount +=
+          allocResponse.getAllocatedContainers().size();
+      for (Container container : allocResponse.getAllocatedContainers()) {
+        if (container.getExecutionType() == ExecutionType.GUARANTEED) {
+          allocatedGuaranteedContainers.put(container.getId(), container);
+        }
+      }
+      if (allocatedContainerCount < oppContainersRequestedAny) {
+        // sleep to let NM's heartbeat to RM and trigger allocations
+        sleep(100);
+      }
+    }
+    assertEquals(oppContainersRequestedAny, allocatedContainerCount);
+    assertEquals(oppContainersRequestedAny,
+        allocatedGuaranteedContainers.size());
+    startContainer(allocResponse, nmClient);
+
+    // SEND DEMOTION REQUEST TO RM
+    try {
+      Container c = allocatedGuaranteedContainers.values().iterator().next();
+      amClient.requestContainerUpdate(
+          c, UpdateContainerRequest.newInstance(c.getVersion(),
+              c.getId(), ContainerUpdateType.DEMOTE_EXECUTION_TYPE,
+              null, ExecutionType.GUARANTEED));
+      fail("Should throw Exception..");
+    } catch (IllegalArgumentException e) {
+      System.out.println("## " + e.getMessage());
+      assertTrue(e.getMessage().contains(
+          "target should be OPPORTUNISTIC and original should be GUARANTEED"));
+    }
+
+    Container c = allocatedGuaranteedContainers.values().iterator().next();
+    amClient.requestContainerUpdate(
+        c, UpdateContainerRequest.newInstance(c.getVersion(),
+            c.getId(), ContainerUpdateType.DEMOTE_EXECUTION_TYPE,
+            null, ExecutionType.OPPORTUNISTIC));
+    iterationsLeft = 120;
+    Map<ContainerId, UpdatedContainer> updatedContainers = new HashMap<>();
+    // do a few iterations to ensure RM is not going to send new containers
+    while (iterationsLeft-- > 0 && updatedContainers.isEmpty()) {
+      // inform RM of rejection
+      allocResponse = amClient.allocate(0.1f);
+      // RM did not send new containers because AM does not need any
+      if (allocResponse.getUpdatedContainers() != null) {
+        for (UpdatedContainer updatedContainer : allocResponse
+            .getUpdatedContainers()) {
+          System.out.println("Got update..");
+          updatedContainers.put(updatedContainer.getContainer().getId(),
+              updatedContainer);
+        }
+      }
+      if (iterationsLeft > 0) {
+        // sleep to make sure NM's heartbeat
+        sleep(100);
+      }
+    }
+    assertEquals(1, updatedContainers.size());
+
+    for (ContainerId cId : allocatedGuaranteedContainers.keySet()) {
+      Container orig = allocatedGuaranteedContainers.get(cId);
+      UpdatedContainer updatedContainer = updatedContainers.get(cId);
+      assertNotNull(updatedContainer);
+      assertEquals(ExecutionType.OPPORTUNISTIC,
+          updatedContainer.getContainer().getExecutionType());
+      assertEquals(orig.getResource(),
+          updatedContainer.getContainer().getResource());
+      assertEquals(orig.getNodeId(),
+          updatedContainer.getContainer().getNodeId());
+      assertEquals(orig.getVersion() + 1,
+          updatedContainer.getContainer().getVersion());
+    }
+    assertEquals(0, amClient.ask.size());
+    assertEquals(0, amClient.release.size());
+
+    updateContainerExecType(allocResponse, ExecutionType.OPPORTUNISTIC,
+        nmClient);
+    amClient.ask.clear();
+  }
+
+  private void updateContainerExecType(AllocateResponse allocResponse,
+      ExecutionType expectedExecType, NMClientImpl nmClient)
+      throws IOException, YarnException {
+    for (UpdatedContainer updatedContainer : allocResponse
+        .getUpdatedContainers()) {
+      Container container = updatedContainer.getContainer();
+      nmClient.increaseContainerResource(container);
+      // NodeManager may still need some time to get the stable
+      // container status
+      while (true) {
+        ContainerStatus status = nmClient
+            .getContainerStatus(container.getId(), container.getNodeId());
+        if (status.getExecutionType() == expectedExecType) {
+          break;
+        }
+        sleep(10);
+      }
+    }
+  }
+
+  private void startContainer(AllocateResponse allocResponse,
+      NMClientImpl nmClient) throws IOException, YarnException {
+    // START THE CONTAINER IN NM
+    // build container launch context
+    Credentials ts = new Credentials();
+    DataOutputBuffer dob = new DataOutputBuffer();
+    ts.writeTokenStorageToStream(dob);
+    ByteBuffer securityTokens =
+        ByteBuffer.wrap(dob.getData(), 0, dob.getLength());
+    // start a process long enough for increase/decrease action to take effect
+    ContainerLaunchContext clc = BuilderUtils.newContainerLaunchContext(
+        Collections.<String, LocalResource>emptyMap(),
+        new HashMap<String, String>(), Arrays.asList("sleep", "100"),
+        new HashMap<String, ByteBuffer>(), securityTokens,
+        new HashMap<ApplicationAccessType, String>());
+    // start the containers and make sure they are in RUNNING state
+    for (Container container : allocResponse.getAllocatedContainers()) {
+      nmClient.startContainer(container, clc);
+      // NodeManager may still need some time to get the stable
+      // container status
+      while (true) {
+        ContainerStatus status = nmClient
+            .getContainerStatus(container.getId(), container.getNodeId());
+        if (status.getState() == ContainerState.RUNNING) {
+          break;
+        }
+        sleep(10);
+      }
+    }
   }
 
+
   private void testAllocation(final AMRMClientImpl<ContainerRequest> amClient)
       throws YarnException, IOException {
     // setup container request
@@ -1172,7 +1509,7 @@ public class TestAMRMClient {
     Set<ContainerId> releases = new TreeSet<ContainerId>();
     
     amClient.getNMTokenCache().clearCache();
-    Assert.assertEquals(0, amClient.getNMTokenCache().numberOfTokensInCache());
+    assertEquals(0, amClient.getNMTokenCache().numberOfTokensInCache());
     HashMap<String, Token> receivedNMTokens = new HashMap<String, Token>();
     
     while (allocatedContainerCount < containersRequestedAny
@@ -1192,7 +1529,7 @@ public class TestAMRMClient {
       for (NMToken token : allocResponse.getNMTokens()) {
         String nodeID = token.getNodeId().toString();
         if (receivedNMTokens.containsKey(nodeID)) {
-          Assert.fail("Received token again for : " + nodeID);          
+          fail("Received token again for : " + nodeID);
         }
         receivedNMTokens.put(nodeID, token.getToken());
       }
@@ -1204,7 +1541,7 @@ public class TestAMRMClient {
     }
     
     // Should receive atleast 1 token
-    Assert.assertTrue(receivedNMTokens.size() > 0
+    assertTrue(receivedNMTokens.size() > 0
         && receivedNMTokens.size() <= nodeCount);
     
     assertEquals(allocatedContainerCount, containersRequestedAny);
@@ -1444,7 +1781,7 @@ public class TestAMRMClient {
       org.apache.hadoop.security.token.Token<AMRMTokenIdentifier> amrmToken_1 =
           getAMRMToken();
       Assert.assertNotNull(amrmToken_1);
-      Assert.assertEquals(amrmToken_1.decodeIdentifier().getKeyId(),
+      assertEquals(amrmToken_1.decodeIdentifier().getKeyId(),
         amrmTokenSecretManager.getMasterKey().getMasterKey().getKeyId());
 
       // Wait for enough time and make sure the roll_over happens
@@ -1459,7 +1796,7 @@ public class TestAMRMClient {
       org.apache.hadoop.security.token.Token<AMRMTokenIdentifier> amrmToken_2 =
           getAMRMToken();
       Assert.assertNotNull(amrmToken_2);
-      Assert.assertEquals(amrmToken_2.decodeIdentifier().getKeyId(),
+      assertEquals(amrmToken_2.decodeIdentifier().getKeyId(),
         amrmTokenSecretManager.getMasterKey().getMasterKey().getKeyId());
 
       Assert.assertNotEquals(amrmToken_1, amrmToken_2);
@@ -1474,7 +1811,7 @@ public class TestAMRMClient {
       AMRMTokenIdentifierForTest newVersionTokenIdentifier = 
           new AMRMTokenIdentifierForTest(amrmToken_2.decodeIdentifier(), "message");
       
-      Assert.assertEquals("Message is changed after set to newVersionTokenIdentifier",
+      assertEquals("Message is changed after set to newVersionTokenIdentifier",
           "message", newVersionTokenIdentifier.getMessage());
       org.apache.hadoop.security.token.Token<AMRMTokenIdentifier> newVersionToken = 
           new org.apache.hadoop.security.token.Token<AMRMTokenIdentifier> (
@@ -1530,10 +1867,10 @@ public class TestAMRMClient {
                 .getBindAddress(), conf);
           }
         }).allocate(Records.newRecord(AllocateRequest.class));
-        Assert.fail("The old Token should not work");
+        fail("The old Token should not work");
       } catch (Exception ex) {
-        Assert.assertTrue(ex instanceof InvalidToken);
-        Assert.assertTrue(ex.getMessage().contains(
+        assertTrue(ex instanceof InvalidToken);
+        assertTrue(ex.getMessage().contains(
           "Invalid AMRMToken from "
               + amrmToken_2.decodeIdentifier().getApplicationAttemptId()));
       }
@@ -1560,7 +1897,7 @@ public class TestAMRMClient {
       org.apache.hadoop.security.token.Token<?> token = iter.next();
       if (token.getKind().equals(AMRMTokenIdentifier.KIND_NAME)) {
         if (result != null) {
-          Assert.fail("credentials has more than one AMRM token."
+          fail("credentials has more than one AMRM token."
               + " token1: " + result + " token2: " + token);
         }
         result = (org.apache.hadoop.security.token.Token<AMRMTokenIdentifier>)

+ 3 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestNMClient.java

@@ -301,7 +301,6 @@ public class TestNMClient {
         assertTrue("The thrown exception is not expected",
             e.getMessage().contains("is not handled by this NodeManager"));
       }
-
       // increaseContainerResource shouldn't be called before startContainer,
       // otherwise, NodeManager cannot find the container
       try {
@@ -475,10 +474,10 @@ public class TestNMClient {
     try {
       nmClient.increaseContainerResource(container);
     } catch (YarnException e) {
-      // NM container will only be in SCHEDULED state, so expect the increase
-      // action to fail.
+      // NM container increase container resource should fail without a version
+      // increase action to fail.
       if (!e.getMessage().contains(
-          "can only be changed when a container is in RUNNING state")) {
+          container.getId() + " has update version ")) {
         throw (AssertionError)
             (new AssertionError("Exception is not expected: " + e)
                 .initCause(e));

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml

@@ -103,8 +103,8 @@
       <artifactId>leveldbjni-all</artifactId>
     </dependency>
     <dependency>
-      <groupId>javax.cache</groupId>
-      <artifactId>cache-api</artifactId>
+      <groupId>org.apache.geronimo.specs</groupId>
+      <artifactId>geronimo-jcache_1.0_spec</artifactId>
     </dependency>
     <dependency>
       <groupId>org.ehcache</groupId>

+ 82 - 50
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java

@@ -66,6 +66,7 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.ExecutionType;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
 import org.apache.hadoop.yarn.api.records.LogAggregationContext;
@@ -136,13 +137,14 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.LogHandler;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.NonAggregatingLogHandler;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.event.LogHandlerEventType;
-import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ChangeMonitoringContainerResourceEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitorEventType;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitorImpl;
 
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler.ContainerScheduler;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler.ContainerSchedulerEventType;
+
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler.UpdateContainerSchedulerEvent;
 import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService;
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService.RecoveredApplicationsState;
@@ -410,8 +412,24 @@ public class ContainerManagerImpl extends CompositeService implements
       throws IOException {
     StartContainerRequest req = rcs.getStartRequest();
     ContainerLaunchContext launchContext = req.getContainerLaunchContext();
-    ContainerTokenIdentifier token =
-        BuilderUtils.newContainerTokenIdentifier(req.getContainerToken());
+    ContainerTokenIdentifier token = null;
+    if(rcs.getCapability() != null) {
+      ContainerTokenIdentifier originalToken =
+          BuilderUtils.newContainerTokenIdentifier(req.getContainerToken());
+      token = new ContainerTokenIdentifier(originalToken.getContainerID(),
+          originalToken.getVersion(), originalToken.getNmHostAddress(),
+          originalToken.getApplicationSubmitter(), rcs.getCapability(),
+          originalToken.getExpiryTimeStamp(), originalToken.getMasterKeyId(),
+          originalToken.getRMIdentifier(), originalToken.getPriority(),
+          originalToken.getCreationTime(),
+          originalToken.getLogAggregationContext(),
+          originalToken.getNodeLabelExpression(),
+          originalToken.getContainerType(), originalToken.getExecutionType());
+
+    } else {
+      token = BuilderUtils.newContainerTokenIdentifier(req.getContainerToken());
+    }
+
     ContainerId containerId = token.getContainerID();
     ApplicationId appId =
         containerId.getApplicationAttemptId().getApplicationId();
@@ -1183,9 +1201,7 @@ public class ContainerManagerImpl extends CompositeService implements
           // as container resource increase request will have come with
           // an updated NMToken.
           updateNMTokenIdentifier(nmTokenIdentifier);
-          Resource resource = containerTokenIdentifier.getResource();
-          changeContainerResourceInternal(containerId,
-              containerTokenIdentifier.getVersion(), resource, true);
+          updateContainerInternal(containerId, containerTokenIdentifier);
           successfullyUpdatedContainers.add(containerId);
         } catch (YarnException | InvalidToken e) {
           failedContainers.put(containerId, SerializedException.newInstance(e));
@@ -1199,9 +1215,9 @@ public class ContainerManagerImpl extends CompositeService implements
   }
 
   @SuppressWarnings("unchecked")
-  private void changeContainerResourceInternal(ContainerId containerId,
-      int containerVersion, Resource targetResource, boolean increase)
-          throws YarnException, IOException {
+  private void updateContainerInternal(ContainerId containerId,
+      ContainerTokenIdentifier containerTokenIdentifier)
+      throws YarnException, IOException {
     Container container = context.getContainers().get(containerId);
     // Check container existence
     if (container == null) {
@@ -1213,64 +1229,77 @@ public class ContainerManagerImpl extends CompositeService implements
             + " is not handled by this NodeManager");
       }
     }
+    // Check container version.
+    int currentVersion = container.getContainerTokenIdentifier().getVersion();
+    if (containerTokenIdentifier.getVersion() <= currentVersion) {
+      throw RPCUtil.getRemoteException("Container " + containerId.toString()
+          + " has update version [" + currentVersion + "] >= requested version"
+          + " [" + containerTokenIdentifier.getVersion() + "]");
+    }
+
     // Check container state
     org.apache.hadoop.yarn.server.nodemanager.
         containermanager.container.ContainerState currentState =
         container.getContainerState();
     if (currentState != org.apache.hadoop.yarn.server.
-        nodemanager.containermanager.container.ContainerState.RUNNING) {
+            nodemanager.containermanager.container.ContainerState.RUNNING &&
+        currentState != org.apache.hadoop.yarn.server.
+            nodemanager.containermanager.container.ContainerState.SCHEDULED) {
       throw RPCUtil.getRemoteException("Container " + containerId.toString()
           + " is in " + currentState.name() + " state."
           + " Resource can only be changed when a container is in"
-          + " RUNNING state");
+          + " RUNNING or SCHEDULED state");
     }
+
     // Check validity of the target resource.
     Resource currentResource = container.getResource();
-    if (currentResource.equals(targetResource)) {
-      LOG.warn("Unable to change resource for container "
-          + containerId.toString()
-          + ". The target resource "
-          + targetResource.toString()
-          + " is the same as the current resource");
-      return;
-    }
-    if (increase && !Resources.fitsIn(currentResource, targetResource)) {
-      throw RPCUtil.getRemoteException("Unable to increase resource for "
-          + "container " + containerId.toString()
-          + ". The target resource "
-          + targetResource.toString()
-          + " is smaller than the current resource "
-          + currentResource.toString());
-    }
-    if (!increase &&
-        (!Resources.fitsIn(Resources.none(), targetResource)
-            || !Resources.fitsIn(targetResource, currentResource))) {
-      throw RPCUtil.getRemoteException("Unable to decrease resource for "
-          + "container " + containerId.toString()
-          + ". The target resource "
-          + targetResource.toString()
-          + " is not smaller than the current resource "
-          + currentResource.toString());
-    }
-    if (increase) {
-      org.apache.hadoop.yarn.api.records.Container increasedContainer =
-          org.apache.hadoop.yarn.api.records.Container.newInstance(
-              containerId, null, null, targetResource, null, null);
+    ExecutionType currentExecType =
+        container.getContainerTokenIdentifier().getExecutionType();
+    boolean isResourceChange = false;
+    boolean isExecTypeUpdate = false;
+    Resource targetResource = containerTokenIdentifier.getResource();
+    ExecutionType targetExecType = containerTokenIdentifier.getExecutionType();
+
+    // Is true if either the resources has increased or execution type
+    // updated from opportunistic to guaranteed
+    boolean isIncrease = false;
+    if (!currentResource.equals(targetResource)) {
+      isResourceChange = true;
+      isIncrease = Resources.fitsIn(currentResource, targetResource)
+          && !Resources.fitsIn(targetResource, currentResource);
+    } else if (!currentExecType.equals(targetExecType)) {
+      isExecTypeUpdate = true;
+      isIncrease = currentExecType == ExecutionType.OPPORTUNISTIC &&
+          targetExecType == ExecutionType.GUARANTEED;
+    }
+    if (isIncrease) {
+      org.apache.hadoop.yarn.api.records.Container increasedContainer = null;
+      if (isResourceChange) {
+        increasedContainer =
+            org.apache.hadoop.yarn.api.records.Container.newInstance(
+                containerId, null, null, targetResource, null, null,
+                currentExecType);
+      } else {
+        increasedContainer =
+            org.apache.hadoop.yarn.api.records.Container.newInstance(
+                containerId, null, null, currentResource, null, null,
+                targetExecType);
+      }
       if (context.getIncreasedContainers().putIfAbsent(containerId,
           increasedContainer) != null){
         throw RPCUtil.getRemoteException("Container " + containerId.toString()
-            + " resource is being increased.");
+            + " resource is being increased -or- " +
+            "is undergoing ExecutionType promoted.");
       }
     }
     this.readLock.lock();
     try {
       if (!serviceStopped) {
-        // Persist container resource change for recovery
-        this.context.getNMStateStore().storeContainerResourceChanged(
-            containerId, containerVersion, targetResource);
-        getContainersMonitor().handle(
-            new ChangeMonitoringContainerResourceEvent(
-                containerId, targetResource));
+        // Dispatch message to ContainerScheduler to actually
+        // make the change.
+        dispatcher.getEventHandler().handle(new UpdateContainerSchedulerEvent(
+            container, containerTokenIdentifier, isResourceChange,
+            isExecTypeUpdate, isIncrease));
       } else {
         throw new YarnException(
             "Unable to change container resource as the NodeManager is "
@@ -1571,8 +1600,11 @@ public class ContainerManagerImpl extends CompositeService implements
       for (org.apache.hadoop.yarn.api.records.Container container
           : containersDecreasedEvent.getContainersToDecrease()) {
         try {
-          changeContainerResourceInternal(container.getId(),
-              container.getVersion(), container.getResource(), false);
+          ContainerTokenIdentifier containerTokenIdentifier =
+              BuilderUtils.newContainerTokenIdentifier(
+                  container.getContainerToken());
+          updateContainerInternal(container.getId(),
+              containerTokenIdentifier);
         } catch (YarnException e) {
           LOG.error("Unable to decrease container resource", e);
         } catch (IOException e) {

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/Container.java

@@ -39,10 +39,10 @@ public interface Container extends EventHandler<ContainerEvent> {
 
   Resource getResource();
 
-  void setResource(Resource targetResource);
-
   ContainerTokenIdentifier getContainerTokenIdentifier();
 
+  void setContainerTokenIdentifier(ContainerTokenIdentifier token);
+
   String getUser();
   
   ContainerState getContainerState();

+ 17 - 20
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java

@@ -148,9 +148,8 @@ public class ContainerImpl implements Container {
   private final Credentials credentials;
   private final NodeManagerMetrics metrics;
   private volatile ContainerLaunchContext launchContext;
-  private final ContainerTokenIdentifier containerTokenIdentifier;
+  private volatile ContainerTokenIdentifier containerTokenIdentifier;
   private final ContainerId containerId;
-  private volatile Resource resource;
   private final String user;
   private int version;
   private int exitCode = ContainerExitStatus.INVALID;
@@ -201,7 +200,6 @@ public class ContainerImpl implements Container {
         YarnConfiguration.DEFAULT_NM_CONTAINER_DIAGNOSTICS_MAXIMUM_SIZE);
     this.containerTokenIdentifier = containerTokenIdentifier;
     this.containerId = containerTokenIdentifier.getContainerID();
-    this.resource = containerTokenIdentifier.getResource();
     this.diagnostics = new StringBuilder();
     this.credentials = creds;
     this.metrics = metrics;
@@ -269,13 +267,6 @@ public class ContainerImpl implements Container {
     this.exitCode = rcs.getExitCode();
     this.recoveredAsKilled = rcs.getKilled();
     this.diagnostics.append(rcs.getDiagnostics());
-    Resource recoveredCapability = rcs.getCapability();
-    if (recoveredCapability != null
-        && !this.resource.equals(recoveredCapability)) {
-      // resource capability had been updated before NM was down
-      this.resource = Resource.newInstance(recoveredCapability.getMemorySize(),
-          recoveredCapability.getVirtualCores());
-    }
     this.version = rcs.getVersion();
     this.remainingRetryAttempts = rcs.getRemainingRetryAttempts();
     this.workDir = rcs.getWorkDir();
@@ -640,14 +631,8 @@ public class ContainerImpl implements Container {
 
   @Override
   public Resource getResource() {
-    return Resources.clone(this.resource);
-  }
-
-  @Override
-  public void setResource(Resource targetResource) {
-    Resource currentResource = getResource();
-    this.resource = Resources.clone(targetResource);
-    this.metrics.changeContainer(currentResource, targetResource);
+    return Resources.clone(
+        this.containerTokenIdentifier.getResource());
   }
 
   @Override
@@ -660,6 +645,16 @@ public class ContainerImpl implements Container {
     }
   }
 
+  @Override
+  public void setContainerTokenIdentifier(ContainerTokenIdentifier token) {
+    this.writeLock.lock();
+    try {
+      this.containerTokenIdentifier = token;
+    } finally {
+      this.writeLock.unlock();
+    }
+  }
+
   @Override
   public String getWorkDir() {
     return workDir;
@@ -833,7 +828,8 @@ public class ContainerImpl implements Container {
             AuditConstants.FINISH_KILLED_CONTAINER, "ContainerImpl",
             container.containerId.getApplicationAttemptId().getApplicationId(),
             container.containerId);
-        container.metrics.releaseContainer(container.resource);
+        container.metrics.releaseContainer(
+            container.containerTokenIdentifier.getResource());
         container.sendFinishedEvents();
         return ContainerState.DONE;
       }
@@ -1517,7 +1513,8 @@ public class ContainerImpl implements Container {
     @Override
     @SuppressWarnings("unchecked")
     public void transition(ContainerImpl container, ContainerEvent event) {
-      container.metrics.releaseContainer(container.resource);
+      container.metrics.releaseContainer(
+          container.containerTokenIdentifier.getResource());
       if (container.containerMetrics != null) {
         container.containerMetrics
             .recordFinishTimeAndExitCode(clock.getTime(), container.exitCode);

+ 0 - 15
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java

@@ -741,19 +741,6 @@ public class ContainersMonitorImpl extends AbstractService implements
     }
   }
 
-  private void changeContainerResource(
-      ContainerId containerId, Resource resource) {
-    Container container = context.getContainers().get(containerId);
-    // Check container existence
-    if (container == null) {
-      LOG.warn("Container " + containerId.toString() + "does not exist");
-      return;
-    }
-    // YARN-5860: Route this through the ContainerScheduler to
-    //       fix containerAllocation
-    container.setResource(resource);
-  }
-
   private void updateContainerMetrics(ContainersMonitorEvent monitoringEvent) {
     if (!containerMetricsEnabled || monitoringEvent == null) {
       return;
@@ -902,8 +889,6 @@ public class ContainersMonitorImpl extends AbstractService implements
       int cpuVcores = changeEvent.getResource().getVirtualCores();
       processTreeInfo.setResourceLimit(pmemLimit, vmemLimit, cpuVcores);
     }
-
-    changeContainerResource(containerId, changeEvent.getResource());
   }
 
   private void onStopMonitoringContainer(

+ 73 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java

@@ -31,6 +31,9 @@ import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit;
 import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus;
 import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
+
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor
+    .ChangeMonitoringContainerResourceEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor;
 
 
@@ -136,6 +139,13 @@ public class ContainerScheduler extends AbstractService implements
     case CONTAINER_COMPLETED:
       onContainerCompleted(event.getContainer());
       break;
+    case UPDATE_CONTAINER:
+      if (event instanceof UpdateContainerSchedulerEvent) {
+        onUpdateContainer((UpdateContainerSchedulerEvent) event);
+      } else {
+        LOG.error("Unknown event type on UpdateCOntainer: " + event.getType());
+      }
+      break;
     case SHED_QUEUED_CONTAINERS:
       shedQueuedOpportunisticContainers();
       break;
@@ -145,6 +155,69 @@ public class ContainerScheduler extends AbstractService implements
     }
   }
 
+  /**
+   * We assume that the ContainerManager has already figured out what kind
+   * of update this is.
+   */
+  private void onUpdateContainer(UpdateContainerSchedulerEvent updateEvent) {
+    ContainerId containerId = updateEvent.getContainer().getContainerId();
+    if (updateEvent.isResourceChange()) {
+      if (runningContainers.containsKey(containerId)) {
+        this.utilizationTracker.subtractContainerResource(
+            updateEvent.getContainer());
+        updateEvent.getContainer().setContainerTokenIdentifier(
+            updateEvent.getUpdatedToken());
+        this.utilizationTracker.addContainerResources(
+            updateEvent.getContainer());
+        getContainersMonitor().handle(
+            new ChangeMonitoringContainerResourceEvent(containerId,
+                updateEvent.getUpdatedToken().getResource()));
+      } else {
+        updateEvent.getContainer().setContainerTokenIdentifier(
+            updateEvent.getUpdatedToken());
+      }
+      try {
+        // Persist change in the state store.
+        this.context.getNMStateStore().storeContainerResourceChanged(
+            containerId,
+            updateEvent.getUpdatedToken().getVersion(),
+            updateEvent.getUpdatedToken().getResource());
+      } catch (IOException e) {
+        LOG.warn("Could not store container [" + containerId + "] resource " +
+            "change..", e);
+      }
+    }
+
+    if (updateEvent.isExecTypeUpdate()) {
+      updateEvent.getContainer().setContainerTokenIdentifier(
+          updateEvent.getUpdatedToken());
+      // If this is a running container.. just change the execution type
+      // and be done with it.
+      if (!runningContainers.containsKey(containerId)) {
+        // Promotion or not (Increase signifies either a promotion
+        // or container size increase)
+        if (updateEvent.isIncrease()) {
+          // Promotion of queued container..
+          if (queuedOpportunisticContainers.remove(containerId) != null) {
+            queuedGuaranteedContainers.put(containerId,
+                updateEvent.getContainer());
+          }
+          //Kill opportunistic containers if any to make room for
+          // promotion request
+          killOpportunisticContainers(updateEvent.getContainer());
+        } else {
+          // Demotion of queued container.. Should not happen too often
+          // since you should not find too many queued guaranteed
+          // containers
+          if (queuedGuaranteedContainers.remove(containerId) != null) {
+            queuedOpportunisticContainers.put(containerId,
+                updateEvent.getContainer());
+          }
+        }
+      }
+    }
+  }
+
   /**
    * Return number of queued containers.
    * @return Number of queued containers.

+ 1 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerSchedulerEventType.java

@@ -24,6 +24,7 @@ package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler;
 public enum ContainerSchedulerEventType {
   SCHEDULE_CONTAINER,
   CONTAINER_COMPLETED,
+  UPDATE_CONTAINER,
   // Producer: Node HB response - RM has asked to shed the queue
   SHED_QUEUED_CONTAINERS,
 }

+ 85 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/UpdateContainerSchedulerEvent.java

@@ -0,0 +1,85 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler;
+
+import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container
+    .Container;
+/**
+ * Update Event consumed by the {@link ContainerScheduler}.
+ */
+public class UpdateContainerSchedulerEvent extends ContainerSchedulerEvent {
+
+  private ContainerTokenIdentifier updatedToken;
+  private boolean isResourceChange;
+  private boolean isExecTypeUpdate;
+  private boolean isIncrease;
+
+  /**
+   * Create instance of Event.
+   *
+   * @param originalContainer Original Container.
+   * @param updatedToken Updated Container Token.
+   * @param isResourceChange is this a Resource Change.
+   * @param isExecTypeUpdate is this an ExecTypeUpdate.
+   * @param isIncrease is this a Container Increase.
+   */
+  public UpdateContainerSchedulerEvent(Container originalContainer,
+      ContainerTokenIdentifier updatedToken, boolean isResourceChange,
+      boolean isExecTypeUpdate, boolean isIncrease) {
+    super(originalContainer, ContainerSchedulerEventType.UPDATE_CONTAINER);
+    this.updatedToken = updatedToken;
+    this.isResourceChange = isResourceChange;
+    this.isExecTypeUpdate = isExecTypeUpdate;
+    this.isIncrease = isIncrease;
+  }
+
+  /**
+   * Update Container Token.
+   *
+   * @return Container Token.
+   */
+  public ContainerTokenIdentifier getUpdatedToken() {
+    return updatedToken;
+  }
+
+  /**
+   * isResourceChange.
+   * @return isResourceChange.
+   */
+  public boolean isResourceChange() {
+    return isResourceChange;
+  }
+
+  /**
+   * isExecTypeUpdate.
+   * @return isExecTypeUpdate.
+   */
+  public boolean isExecTypeUpdate() {
+    return isExecTypeUpdate;
+  }
+
+  /**
+   * isIncrease.
+   * @return isIncrease.
+   */
+  public boolean isIncrease() {
+    return isIncrease;
+  }
+}

+ 10 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java

@@ -682,7 +682,7 @@ public class TestNodeManagerResync {
         try{
           try {
             updateBarrier.await();
-            increaseTokens.add(getContainerToken(targetResource));
+            increaseTokens.add(getContainerToken(targetResource, 1));
             ContainerUpdateRequest updateRequest =
                 ContainerUpdateRequest.newInstance(increaseTokens);
             ContainerUpdateResponse updateResponse =
@@ -710,6 +710,15 @@ public class TestNodeManagerResync {
           getNMContext().getNodeId(), user, resource,
           getNMContext().getContainerTokenSecretManager(), null);
     }
+
+    private Token getContainerToken(Resource resource, int version)
+        throws IOException {
+      ContainerId cId = TestContainerManager.createContainerId(0);
+      return TestContainerManager.createContainerToken(
+          cId, version, DUMMY_RM_IDENTIFIER,
+          getNMContext().getNodeId(), user, resource,
+          getNMContext().getContainerTokenSecretManager(), null);
+    }
   }
 
   public static NMContainerStatus createNMContainerStatus(int id,

+ 31 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java

@@ -421,6 +421,20 @@ public abstract class BaseContainerManagerTest {
             containerTokenIdentifier);
   }
 
+  public static Token createContainerToken(ContainerId cId, int version,
+      long rmIdentifier, NodeId nodeId, String user, Resource resource,
+      NMContainerTokenSecretManager containerTokenSecretManager,
+      LogAggregationContext logAggregationContext) throws IOException {
+    ContainerTokenIdentifier containerTokenIdentifier =
+        new ContainerTokenIdentifier(cId, version, nodeId.toString(), user,
+            resource, System.currentTimeMillis() + 100000L, 123, rmIdentifier,
+            Priority.newInstance(0), 0, logAggregationContext, null,
+            ContainerType.TASK, ExecutionType.GUARANTEED);
+    return BuilderUtils.newContainerToken(nodeId,
+        containerTokenSecretManager.retrievePassword(containerTokenIdentifier),
+        containerTokenIdentifier);
+  }
+
   public static Token createContainerToken(ContainerId cId, long rmIdentifier,
       NodeId nodeId, String user, Resource resource,
       NMContainerTokenSecretManager containerTokenSecretManager,
@@ -431,8 +445,23 @@ public abstract class BaseContainerManagerTest {
             System.currentTimeMillis() + 100000L, 123, rmIdentifier,
             Priority.newInstance(0), 0, logAggregationContext, null,
             ContainerType.TASK, executionType);
-    return BuilderUtils.newContainerToken(nodeId, containerTokenSecretManager
-            .retrievePassword(containerTokenIdentifier),
+    return BuilderUtils.newContainerToken(nodeId,
+        containerTokenSecretManager.retrievePassword(containerTokenIdentifier),
+        containerTokenIdentifier);
+  }
+
+  public static Token createContainerToken(ContainerId cId, int version,
+      long rmIdentifier, NodeId nodeId, String user, Resource resource,
+      NMContainerTokenSecretManager containerTokenSecretManager,
+      LogAggregationContext logAggregationContext, ExecutionType executionType)
+      throws IOException {
+    ContainerTokenIdentifier containerTokenIdentifier =
+        new ContainerTokenIdentifier(cId, version, nodeId.toString(), user,
+            resource, System.currentTimeMillis() + 100000L, 123, rmIdentifier,
+            Priority.newInstance(0), 0, logAggregationContext, null,
+            ContainerType.TASK, executionType);
+    return BuilderUtils.newContainerToken(nodeId,
+        containerTokenSecretManager.retrievePassword(containerTokenIdentifier),
         containerTokenIdentifier);
   }
 

+ 167 - 100
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.server.nodemanager.containermanager;
 
 import static org.junit.Assert.assertEquals;
 import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.timeout;
 import static org.mockito.Mockito.verify;
 
@@ -70,6 +71,7 @@ import org.apache.hadoop.yarn.api.records.ContainerRetryContext;
 import org.apache.hadoop.yarn.api.records.ContainerRetryPolicy;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.ExecutionType;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LocalResourceType;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
@@ -80,14 +82,15 @@ import org.apache.hadoop.yarn.api.records.SignalContainerCommand;
 import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.api.records.URL;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.ConfigurationException;
 import org.apache.hadoop.yarn.exceptions.InvalidContainerException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
 import org.apache.hadoop.yarn.security.NMTokenIdentifier;
 import org.apache.hadoop.yarn.server.api.ResourceManagerConstants;
 import org.apache.hadoop.yarn.server.nodemanager.CMgrCompletedAppsEvent;
-import org.apache.hadoop.yarn.server.nodemanager.CMgrDecreaseContainersResourceEvent;
 import org.apache.hadoop.yarn.server.nodemanager.CMgrSignalContainersEvent;
+import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
 import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.Signal;
 import org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor;
 import org.apache.hadoop.yarn.server.nodemanager.DeletionService;
@@ -100,6 +103,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.Conta
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ContainerLocalizer;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService;
 import org.apache.hadoop.yarn.server.nodemanager.executor.ContainerSignalContext;
+import org.apache.hadoop.yarn.server.nodemanager.executor.ContainerStartContext;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.junit.Assert;
 import org.junit.Before;
@@ -116,10 +120,34 @@ public class TestContainerManager extends BaseContainerManagerTest {
   static {
     LOG = LogFactory.getLog(TestContainerManager.class);
   }
-  
+
+  private boolean delayContainers = false;
+
+  @Override
+  protected ContainerExecutor createContainerExecutor() {
+    DefaultContainerExecutor exec = new DefaultContainerExecutor() {
+      @Override
+      public int launchContainer(ContainerStartContext ctx)
+          throws IOException, ConfigurationException {
+        if (delayContainers) {
+          try {
+            Thread.sleep(10000);
+          } catch (InterruptedException e) {
+            // Nothing..
+          }
+        }
+        return super.launchContainer(ctx);
+      }
+    };
+    exec.setConf(conf);
+    return spy(exec);
+  }
+
   @Override
   @Before
   public void setup() throws IOException {
+    conf.setInt(
+        YarnConfiguration.NM_OPPORTUNISTIC_CONTAINERS_MAX_QUEUE_LENGTH, 10);
     super.setup();
   }
   
@@ -1468,7 +1496,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
     Assert.assertEquals(strExceptionMsg,
         ContainerManagerImpl.INVALID_NMTOKEN_MSG);
 
-    ContainerManagerImpl spyContainerMgr = Mockito.spy(cMgrImpl);
+    ContainerManagerImpl spyContainerMgr = spy(cMgrImpl);
     UserGroupInformation ugInfo = UserGroupInformation.createRemoteUser("a");
     Mockito.when(spyContainerMgr.getRemoteUgi()).thenReturn(ugInfo);
     Mockito.when(spyContainerMgr.
@@ -1543,7 +1571,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
     // container will have exited, and won't be in RUNNING state
     ContainerId cId0 = createContainerId(0);
     Token containerToken =
-        createContainerToken(cId0, DUMMY_RM_IDENTIFIER,
+        createContainerToken(cId0, 1, DUMMY_RM_IDENTIFIER,
             context.getNodeId(), user,
                 Resource.newInstance(1234, 3),
                     context.getContainerTokenSecretManager(), null);
@@ -1572,7 +1600,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
       if (cId0.equals(entry.getKey())) {
         Assert.assertTrue(entry.getValue().getMessage()
           .contains("Resource can only be changed when a "
-              + "container is in RUNNING state"));
+              + "container is in RUNNING or SCHEDULED state"));
       } else if (cId7.equals(entry.getKey())) {
         Assert.assertTrue(entry.getValue().getMessage()
             .contains("Container " + cId7.toString()
@@ -1584,89 +1612,6 @@ public class TestContainerManager extends BaseContainerManagerTest {
     }
   }
 
-  @Test
-  public void testIncreaseContainerResourceWithInvalidResource() throws Exception {
-    containerManager.start();
-    File scriptFile = Shell.appendScriptExtension(tmpDir, "scriptFile");
-    PrintWriter fileWriter = new PrintWriter(scriptFile);
-    // Construct the Container-id
-    ContainerId cId = createContainerId(0);
-    if (Shell.WINDOWS) {
-      fileWriter.println("@ping -n 100 127.0.0.1 >nul");
-    } else {
-      fileWriter.write("\numask 0");
-      fileWriter.write("\nexec sleep 100");
-    }
-    fileWriter.close();
-    ContainerLaunchContext containerLaunchContext =
-        recordFactory.newRecordInstance(ContainerLaunchContext.class);
-    URL resource_alpha =
-        URL.fromPath(localFS
-            .makeQualified(new Path(scriptFile.getAbsolutePath())));
-    LocalResource rsrc_alpha =
-        recordFactory.newRecordInstance(LocalResource.class);
-    rsrc_alpha.setResource(resource_alpha);
-    rsrc_alpha.setSize(-1);
-    rsrc_alpha.setVisibility(LocalResourceVisibility.APPLICATION);
-    rsrc_alpha.setType(LocalResourceType.FILE);
-    rsrc_alpha.setTimestamp(scriptFile.lastModified());
-    String destinationFile = "dest_file";
-    Map<String, LocalResource> localResources =
-        new HashMap<String, LocalResource>();
-    localResources.put(destinationFile, rsrc_alpha);
-    containerLaunchContext.setLocalResources(localResources);
-    List<String> commands =
-        Arrays.asList(Shell.getRunScriptCommand(scriptFile));
-    containerLaunchContext.setCommands(commands);
-
-    StartContainerRequest scRequest =
-        StartContainerRequest.newInstance(
-            containerLaunchContext,
-            createContainerToken(cId, DUMMY_RM_IDENTIFIER, context.getNodeId(),
-            user, context.getContainerTokenSecretManager()));
-    List<StartContainerRequest> list = new ArrayList<>();
-    list.add(scRequest);
-    StartContainersRequest allRequests =
-        StartContainersRequest.newInstance(list);
-    containerManager.startContainers(allRequests);
-    // Make sure the container reaches RUNNING state
-    BaseContainerManagerTest.waitForNMContainerState(containerManager, cId,
-        org.apache.hadoop.yarn.server.nodemanager.
-            containermanager.container.ContainerState.RUNNING);
-    // Construct container resource increase request,
-    List<Token> increaseTokens = new ArrayList<>();
-    // Add increase request. The increase request should fail
-    // as the current resource does not fit in the target resource
-    Token containerToken =
-        createContainerToken(cId, DUMMY_RM_IDENTIFIER,
-            context.getNodeId(), user,
-            Resource.newInstance(512, 1),
-            context.getContainerTokenSecretManager(), null);
-    increaseTokens.add(containerToken);
-    ContainerUpdateRequest updateRequest =
-        ContainerUpdateRequest.newInstance(increaseTokens);
-    ContainerUpdateResponse updateResponse =
-        containerManager.updateContainer(updateRequest);
-    // Check response
-    Assert.assertEquals(
-        0, updateResponse.getSuccessfullyUpdatedContainers().size());
-    Assert.assertEquals(1, updateResponse.getFailedRequests().size());
-    for (Map.Entry<ContainerId, SerializedException> entry : updateResponse
-        .getFailedRequests().entrySet()) {
-      if (cId.equals(entry.getKey())) {
-        Assert.assertNotNull("Failed message", entry.getValue().getMessage());
-        Assert.assertTrue(entry.getValue().getMessage()
-            .contains("The target resource "
-                + Resource.newInstance(512, 1).toString()
-                + " is smaller than the current resource "
-                + Resource.newInstance(1024, 1)));
-      } else {
-        throw new YarnException("Received failed request from wrong"
-            + " container: " + entry.getKey().toString());
-      }
-    }
-  }
-
   @Test
   public void testChangeContainerResource() throws Exception {
     containerManager.start();
@@ -1720,7 +1665,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
     List<Token> increaseTokens = new ArrayList<>();
     // Add increase request.
     Resource targetResource = Resource.newInstance(4096, 2);
-    Token containerToken = createContainerToken(cId, DUMMY_RM_IDENTIFIER,
+    Token containerToken = createContainerToken(cId, 1, DUMMY_RM_IDENTIFIER,
         context.getNodeId(), user, targetResource,
             context.getContainerTokenSecretManager(), null);
     increaseTokens.add(containerToken);
@@ -1741,15 +1686,19 @@ public class TestContainerManager extends BaseContainerManagerTest {
     // Check status immediately as resource increase is blocking
     assertEquals(targetResource, containerStatus.getCapability());
     // Simulate a decrease request
-    List<org.apache.hadoop.yarn.api.records.Container> containersToDecrease
-        = new ArrayList<>();
+    List<Token> decreaseTokens = new ArrayList<>();
     targetResource = Resource.newInstance(2048, 2);
-    org.apache.hadoop.yarn.api.records.Container decreasedContainer =
-        org.apache.hadoop.yarn.api.records.Container
-            .newInstance(cId, null, null, targetResource, null, null);
-    containersToDecrease.add(decreasedContainer);
-    containerManager.handle(
-        new CMgrDecreaseContainersResourceEvent(containersToDecrease));
+    Token token = createContainerToken(cId, 2, DUMMY_RM_IDENTIFIER,
+        context.getNodeId(), user, targetResource,
+        context.getContainerTokenSecretManager(), null);
+    decreaseTokens.add(token);
+    updateRequest = ContainerUpdateRequest.newInstance(decreaseTokens);
+    updateResponse = containerManager.updateContainer(updateRequest);
+
+    Assert.assertEquals(
+        1, updateResponse.getSuccessfullyUpdatedContainers().size());
+    Assert.assertTrue(updateResponse.getFailedRequests().isEmpty());
+
     // Check status with retry
     containerStatus = containerManager
         .getContainerStatuses(gcsRequest).getContainerStatuses().get(0);
@@ -1879,7 +1828,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
     ContainerLaunchContext containerLaunchContext =
         recordFactory.newRecordInstance(ContainerLaunchContext.class);
     ContainerLaunchContext spyContainerLaunchContext =
-        Mockito.spy(containerLaunchContext);
+        spy(containerLaunchContext);
     Mockito.when(spyContainerLaunchContext.getLocalResources())
         .thenReturn(localResources);
 
@@ -1924,7 +1873,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
     ContainerLaunchContext containerLaunchContext =
         recordFactory.newRecordInstance(ContainerLaunchContext.class);
     ContainerLaunchContext spyContainerLaunchContext =
-        Mockito.spy(containerLaunchContext);
+        spy(containerLaunchContext);
     Mockito.when(spyContainerLaunchContext.getLocalResources())
         .thenReturn(localResources);
 
@@ -1969,7 +1918,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
     ContainerLaunchContext containerLaunchContext =
         recordFactory.newRecordInstance(ContainerLaunchContext.class);
     ContainerLaunchContext spyContainerLaunchContext =
-        Mockito.spy(containerLaunchContext);
+        spy(containerLaunchContext);
     Mockito.when(spyContainerLaunchContext.getLocalResources())
         .thenReturn(localResources);
 
@@ -1996,4 +1945,122 @@ public class TestContainerManager extends BaseContainerManagerTest {
     Assert.assertTrue(response.getFailedRequests().get(cId).getMessage()
         .contains("Null resource visibility for local resource"));
   }
+
+  @Test
+  public void testContainerUpdateExecTypeOpportunisticToGuaranteed()
+      throws IOException, YarnException, InterruptedException {
+    delayContainers = true;
+    containerManager.start();
+    // Construct the Container-id
+    ContainerId cId = createContainerId(0);
+    ContainerLaunchContext containerLaunchContext =
+        recordFactory.newRecordInstance(ContainerLaunchContext.class);
+
+    StartContainerRequest scRequest =
+        StartContainerRequest.newInstance(
+            containerLaunchContext,
+            createContainerToken(cId, DUMMY_RM_IDENTIFIER,
+                context.getNodeId(), user, BuilderUtils.newResource(512, 1),
+                context.getContainerTokenSecretManager(), null,
+                ExecutionType.OPPORTUNISTIC));
+    List<StartContainerRequest> list = new ArrayList<>();
+    list.add(scRequest);
+    StartContainersRequest allRequests =
+        StartContainersRequest.newInstance(list);
+    containerManager.startContainers(allRequests);
+    // Make sure the container reaches RUNNING state
+    BaseContainerManagerTest.waitForNMContainerState(containerManager, cId,
+        org.apache.hadoop.yarn.server.nodemanager.
+            containermanager.container.ContainerState.RUNNING);
+    // Construct container resource increase request,
+    List<Token> updateTokens = new ArrayList<>();
+    Token containerToken =
+        createContainerToken(cId, 1, DUMMY_RM_IDENTIFIER, context.getNodeId(),
+            user, BuilderUtils.newResource(512, 1),
+            context.getContainerTokenSecretManager(), null,
+            ExecutionType.GUARANTEED);
+    updateTokens.add(containerToken);
+    ContainerUpdateRequest updateRequest =
+        ContainerUpdateRequest.newInstance(updateTokens);
+    ContainerUpdateResponse updateResponse =
+        containerManager.updateContainer(updateRequest);
+
+    Assert.assertEquals(
+        1, updateResponse.getSuccessfullyUpdatedContainers().size());
+    Assert.assertTrue(updateResponse.getFailedRequests().isEmpty());
+
+    //Make sure the container is running
+    List<ContainerId> statList = new ArrayList<ContainerId>();
+    statList.add(cId);
+    GetContainerStatusesRequest statRequest =
+        GetContainerStatusesRequest.newInstance(statList);
+    List<ContainerStatus> containerStatuses = containerManager
+        .getContainerStatuses(statRequest).getContainerStatuses();
+    Assert.assertEquals(1, containerStatuses.size());
+    for (ContainerStatus status : containerStatuses) {
+      Assert.assertEquals(
+          org.apache.hadoop.yarn.api.records.ContainerState.RUNNING,
+          status.getState());
+      Assert.assertEquals(ExecutionType.GUARANTEED, status.getExecutionType());
+    }
+  }
+
+  @Test
+  public void testContainerUpdateExecTypeGuaranteedToOpportunistic()
+      throws IOException, YarnException, InterruptedException {
+    delayContainers = true;
+    containerManager.start();
+    // Construct the Container-id
+    ContainerId cId = createContainerId(0);
+    ContainerLaunchContext containerLaunchContext =
+        recordFactory.newRecordInstance(ContainerLaunchContext.class);
+
+    StartContainerRequest scRequest =
+        StartContainerRequest.newInstance(
+            containerLaunchContext,
+            createContainerToken(cId, DUMMY_RM_IDENTIFIER,
+                context.getNodeId(), user, BuilderUtils.newResource(512, 1),
+                context.getContainerTokenSecretManager(), null));
+    List<StartContainerRequest> list = new ArrayList<>();
+    list.add(scRequest);
+    StartContainersRequest allRequests =
+        StartContainersRequest.newInstance(list);
+    containerManager.startContainers(allRequests);
+    // Make sure the container reaches RUNNING state
+    BaseContainerManagerTest.waitForNMContainerState(containerManager, cId,
+        org.apache.hadoop.yarn.server.nodemanager.
+            containermanager.container.ContainerState.RUNNING);
+    // Construct container resource increase request,
+    List<Token> updateTokens = new ArrayList<>();
+    Token containerToken =
+        createContainerToken(cId, 1, DUMMY_RM_IDENTIFIER, context.getNodeId(),
+            user, BuilderUtils.newResource(512, 1),
+            context.getContainerTokenSecretManager(), null,
+            ExecutionType.OPPORTUNISTIC);
+    updateTokens.add(containerToken);
+    ContainerUpdateRequest updateRequest =
+        ContainerUpdateRequest.newInstance(updateTokens);
+    ContainerUpdateResponse updateResponse =
+        containerManager.updateContainer(updateRequest);
+
+    Assert.assertEquals(
+        1, updateResponse.getSuccessfullyUpdatedContainers().size());
+    Assert.assertTrue(updateResponse.getFailedRequests().isEmpty());
+
+    //Make sure the container is running
+    List<ContainerId> statList = new ArrayList<ContainerId>();
+    statList.add(cId);
+    GetContainerStatusesRequest statRequest =
+        GetContainerStatusesRequest.newInstance(statList);
+    List<ContainerStatus> containerStatuses = containerManager
+        .getContainerStatuses(statRequest).getContainerStatuses();
+    Assert.assertEquals(1, containerStatuses.size());
+    for (ContainerStatus status : containerStatuses) {
+      Assert.assertEquals(
+          org.apache.hadoop.yarn.api.records.ContainerState.RUNNING,
+          status.getState());
+      Assert
+          .assertEquals(ExecutionType.OPPORTUNISTIC, status.getExecutionType());
+    }
+  }
 }

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java

@@ -652,7 +652,7 @@ public class TestContainerManagerRecovery extends BaseContainerManagerTest {
     final List<Token> increaseTokens = new ArrayList<Token>();
     // add increase request
     Token containerToken = TestContainerManager.createContainerToken(
-        cid, 0, context.getNodeId(), user.getShortUserName(),
+        cid, 1, 0, context.getNodeId(), user.getShortUserName(),
         capability, context.getContainerTokenSecretManager(), null);
     increaseTokens.add(containerToken);
     final ContainerUpdateRequest updateRequest =

+ 96 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/TestContainerSchedulerQueuing.java

@@ -27,6 +27,8 @@ import java.util.List;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.UnsupportedFileSystemException;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.protocolrecords.ContainerUpdateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ContainerUpdateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
@@ -37,6 +39,7 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.ExecutionType;
+import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.ConfigurationException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
@@ -951,4 +954,97 @@ public class TestContainerSchedulerQueuing extends BaseContainerManagerTest {
         map.get(org.apache.hadoop.yarn.api.records.ContainerState.SCHEDULED)
             .getContainerId());
   }
+
+  /**
+   * Starts one OPPORTUNISTIC container that takes up the whole node's
+   * resources, and submit one more that will be queued. Now promote the
+   * queued OPPORTUNISTIC container, which should kill the current running
+   * OPPORTUNISTIC container to make room for the promoted request.
+   * @throws Exception
+   */
+  @Test
+  public void testPromotionOfOpportunisticContainers() throws Exception {
+    containerManager.start();
+
+    ContainerLaunchContext containerLaunchContext =
+        recordFactory.newRecordInstance(ContainerLaunchContext.class);
+
+    List<StartContainerRequest> list = new ArrayList<>();
+    list.add(StartContainerRequest.newInstance(
+        containerLaunchContext,
+        createContainerToken(createContainerId(0), DUMMY_RM_IDENTIFIER,
+            context.getNodeId(),
+            user, BuilderUtils.newResource(2048, 1),
+            context.getContainerTokenSecretManager(), null,
+            ExecutionType.OPPORTUNISTIC)));
+    list.add(StartContainerRequest.newInstance(
+        containerLaunchContext,
+        createContainerToken(createContainerId(1), DUMMY_RM_IDENTIFIER,
+            context.getNodeId(),
+            user, BuilderUtils.newResource(1024, 1),
+            context.getContainerTokenSecretManager(), null,
+            ExecutionType.OPPORTUNISTIC)));
+
+    StartContainersRequest allRequests =
+        StartContainersRequest.newInstance(list);
+    containerManager.startContainers(allRequests);
+
+    Thread.sleep(5000);
+
+    // Ensure first container is running and others are queued.
+    List<ContainerId> statList = new ArrayList<ContainerId>();
+    for (int i = 0; i < 3; i++) {
+      statList.add(createContainerId(i));
+    }
+    GetContainerStatusesRequest statRequest = GetContainerStatusesRequest
+        .newInstance(Arrays.asList(createContainerId(0)));
+    List<ContainerStatus> containerStatuses = containerManager
+        .getContainerStatuses(statRequest).getContainerStatuses();
+    for (ContainerStatus status : containerStatuses) {
+      if (status.getContainerId().equals(createContainerId(0))) {
+        Assert.assertEquals(
+            org.apache.hadoop.yarn.api.records.ContainerState.RUNNING,
+            status.getState());
+      } else {
+        Assert.assertEquals(
+            org.apache.hadoop.yarn.api.records.ContainerState.SCHEDULED,
+            status.getState());
+      }
+    }
+
+    ContainerScheduler containerScheduler =
+        containerManager.getContainerScheduler();
+    // Ensure two containers are properly queued.
+    Assert.assertEquals(1, containerScheduler.getNumQueuedContainers());
+    Assert.assertEquals(0,
+        containerScheduler.getNumQueuedGuaranteedContainers());
+    Assert.assertEquals(1,
+        containerScheduler.getNumQueuedOpportunisticContainers());
+
+    // Promote Queued Opportunistic Container
+    Token updateToken =
+        createContainerToken(createContainerId(1), 1, DUMMY_RM_IDENTIFIER,
+            context.getNodeId(), user, BuilderUtils.newResource(1024, 1),
+            context.getContainerTokenSecretManager(), null,
+            ExecutionType.GUARANTEED);
+    List<Token> updateTokens = new ArrayList<Token>();
+    updateTokens.add(updateToken);
+    ContainerUpdateRequest updateRequest =
+        ContainerUpdateRequest.newInstance(updateTokens);
+    ContainerUpdateResponse updateResponse =
+        containerManager.updateContainer(updateRequest);
+
+    Assert.assertEquals(1,
+        updateResponse.getSuccessfullyUpdatedContainers().size());
+    Assert.assertEquals(0, updateResponse.getFailedRequests().size());
+
+    waitForContainerState(containerManager, createContainerId(0),
+        org.apache.hadoop.yarn.api.records.ContainerState.COMPLETE);
+
+    waitForContainerState(containerManager, createContainerId(1),
+        org.apache.hadoop.yarn.api.records.ContainerState.RUNNING);
+
+    // Ensure no containers are queued.
+    Assert.assertEquals(0, containerScheduler.getNumQueuedContainers());
+  }
 }

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockContainer.java

@@ -140,7 +140,7 @@ public class MockContainer implements Container {
   }
 
   @Override
-  public void setResource(Resource targetResource) {
+  public void setContainerTokenIdentifier(ContainerTokenIdentifier token) {
   }
 
   @Override

+ 2 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java

@@ -888,7 +888,8 @@ public class RMAppImpl implements RMApp, Recoverable {
         /* keep the master in sync with the state machine */
         this.stateMachine.doTransition(event.getType(), event);
       } catch (InvalidStateTransitionException e) {
-        LOG.error("Can't handle this event at current state", e);
+        LOG.error("App: " + appID
+            + " can't handle this event at current state", e);
         /* TODO fail the application on the failed transition */
       }
 

+ 2 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java

@@ -911,7 +911,8 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
         /* keep the master in sync with the state machine */
         this.stateMachine.doTransition(event.getType(), event);
       } catch (InvalidStateTransitionException e) {
-        LOG.error("Can't handle this event at current state", e);
+        LOG.error("App attempt: " + appAttemptID
+            + " can't handle this event at current state", e);
         /* TODO fail the application on the failed transition */
       }
 

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

@@ -323,6 +323,7 @@ public abstract class AbstractYarnScheduler
 
   }
 
+  // TODO: Rename it to getCurrentApplicationAttempt
   public T getApplicationAttempt(ApplicationAttemptId applicationAttemptId) {
     SchedulerApplication<T> app = applications.get(
         applicationAttemptId.getApplicationId());

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

@@ -655,7 +655,7 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
               container.getNodeId(), getUser(), container.getResource(),
               container.getPriority(), rmContainer.getCreationTime(),
               this.logAggregationContext, rmContainer.getNodeLabelExpression(),
-              containerType));
+              containerType, container.getExecutionType()));
       updateNMToken(container);
     } catch (IllegalArgumentException e) {
       // DNS might be down, skip returning this container.

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

@@ -76,6 +76,7 @@ public abstract class AbstractCSQueue implements CSQueue {
   private static final Log LOG = LogFactory.getLog(AbstractCSQueue.class);  
   volatile CSQueue parent;
   final String queueName;
+  private final String queuePath;
   volatile int numContainers;
   
   final Resource minimumAllocation;
@@ -119,6 +120,8 @@ public abstract class AbstractCSQueue implements CSQueue {
     this.labelManager = cs.getRMContext().getNodeLabelManager();
     this.parent = parent;
     this.queueName = queueName;
+    this.queuePath =
+      ((parent == null) ? "" : (parent.getQueuePath() + ".")) + this.queueName;
     this.resourceCalculator = cs.getResourceCalculator();
     this.activitiesManager = cs.getActivitiesManager();
     
@@ -150,6 +153,11 @@ public abstract class AbstractCSQueue implements CSQueue {
         queueCapacities,
         parent == null ? null : parent.getQueueCapacities());
   }
+
+  @Override
+  public String getQueuePath() {
+    return queuePath;
+  }
   
   @Override
   public float getCapacity() {

+ 13 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java

@@ -903,6 +903,19 @@ public class CapacityScheduler extends
       ContainerUpdates updateRequests) {
     FiCaSchedulerApp application = getApplicationAttempt(applicationAttemptId);
     if (application == null) {
+      LOG.error("Calling allocate on removed or non existent application " +
+          applicationAttemptId.getApplicationId());
+      return EMPTY_ALLOCATION;
+    }
+
+    // The allocate may be the leftover from previous attempt, and it will
+    // impact current attempt, such as confuse the request and allocation for
+    // current attempt's AM container.
+    // Note outside precondition check for the attempt id may be
+    // outdated here, so double check it here is necessary.
+    if (!application.getApplicationAttemptId().equals(applicationAttemptId)) {
+      LOG.error("Calling allocate on previous or removed " +
+          "or non existent application attempt " + applicationAttemptId);
       return EMPTY_ALLOCATION;
     }
 

+ 0 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java

@@ -299,11 +299,6 @@ public class LeafQueue extends AbstractCSQueue {
     }
   }
 
-  @Override
-  public String getQueuePath() {
-    return getParent().getQueuePath() + "." + getQueueName();
-  }
-
   /**
    * Used only by tests.
    */

+ 0 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java

@@ -198,12 +198,6 @@ public class ParentQueue extends AbstractCSQueue {
     }
   }
 
-  @Override
-  public String getQueuePath() {
-    String parentPath = ((parent == null) ? "" : (parent.getQueuePath() + "."));
-    return parentPath + getQueueName();
-  }
-
   @Override
   public QueueInfo getQueueInfo(
       boolean includeChildQueues, boolean recursive) {

+ 0 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationConfiguration.java

@@ -23,8 +23,6 @@ import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.yarn.api.records.QueueACL;
@@ -41,7 +39,6 @@ import org.apache.hadoop.yarn.util.resource.Resources;
 import com.google.common.annotations.VisibleForTesting;
 
 public class AllocationConfiguration extends ReservationSchedulerConfiguration {
-  private static final Log LOG = LogFactory.getLog(FSQueue.class.getName());
   private static final AccessControlList EVERYBODY_ACL = new AccessControlList("*");
   private static final AccessControlList NOBODY_ACL = new AccessControlList(" ");
   private static final ResourceCalculator RESOURCE_CALCULATOR =

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

@@ -835,8 +835,19 @@ public class FairScheduler extends
     // Make sure this application exists
     FSAppAttempt application = getSchedulerApp(appAttemptId);
     if (application == null) {
-      LOG.info("Calling allocate on removed " +
-          "or non existent application " + appAttemptId);
+      LOG.error("Calling allocate on removed or non existent application " +
+          appAttemptId.getApplicationId());
+      return EMPTY_ALLOCATION;
+    }
+
+    // The allocate may be the leftover from previous attempt, and it will
+    // impact current attempt, such as confuse the request and allocation for
+    // current attempt's AM container.
+    // Note outside precondition check for the attempt id may be
+    // outdated here, so double check it here is necessary.
+    if (!application.getApplicationAttemptId().equals(appAttemptId)) {
+      LOG.error("Calling allocate on previous or removed " +
+          "or non existent application attempt " + appAttemptId);
       return EMPTY_ALLOCATION;
     }
 

+ 13 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java

@@ -329,8 +329,19 @@ public class FifoScheduler extends
       ContainerUpdates updateRequests) {
     FifoAppAttempt application = getApplicationAttempt(applicationAttemptId);
     if (application == null) {
-      LOG.error("Calling allocate on removed " +
-          "or non-existent application " + applicationAttemptId);
+      LOG.error("Calling allocate on removed or non existent application " +
+          applicationAttemptId.getApplicationId());
+      return EMPTY_ALLOCATION;
+    }
+
+    // The allocate may be the leftover from previous attempt, and it will
+    // impact current attempt, such as confuse the request and allocation for
+    // current attempt's AM container.
+    // Note outside precondition check for the attempt id may be
+    // outdated here, so double check it here is necessary.
+    if (!application.getApplicationAttemptId().equals(applicationAttemptId)) {
+      LOG.error("Calling allocate on previous or removed " +
+          "or non existent application attempt " + applicationAttemptId);
       return EMPTY_ALLOCATION;
     }
 

+ 28 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/RMContainerTokenSecretManager.java

@@ -186,6 +186,31 @@ public class RMContainerTokenSecretManager extends
         null, null, ContainerType.TASK);
   }
 
+  /**
+   * Helper function for creating ContainerTokens.
+   *
+   * @param containerId containerId.
+   * @param containerVersion containerVersion.
+   * @param nodeId nodeId.
+   * @param appSubmitter appSubmitter.
+   * @param capability capability.
+   * @param priority priority.
+   * @param createTime createTime.
+   * @param logAggregationContext logAggregationContext.
+   * @param nodeLabelExpression nodeLabelExpression.
+   * @param containerType containerType.
+   * @return the container-token.
+   */
+  public Token createContainerToken(ContainerId containerId,
+      int containerVersion, NodeId nodeId, String appSubmitter,
+      Resource capability, Priority priority, long createTime,
+      LogAggregationContext logAggregationContext, String nodeLabelExpression,
+      ContainerType containerType) {
+    return createContainerToken(containerId, containerVersion, nodeId,
+        appSubmitter, capability, priority, createTime, null, null,
+        ContainerType.TASK, ExecutionType.GUARANTEED);
+  }
+
   /**
    * Helper function for creating ContainerTokens
    *
@@ -199,13 +224,14 @@ public class RMContainerTokenSecretManager extends
    * @param logAggregationContext Log Aggregation Context
    * @param nodeLabelExpression Node Label Expression
    * @param containerType Container Type
+   * @param execType Execution Type
    * @return the container-token
    */
   public Token createContainerToken(ContainerId containerId,
       int containerVersion, NodeId nodeId, String appSubmitter,
       Resource capability, Priority priority, long createTime,
       LogAggregationContext logAggregationContext, String nodeLabelExpression,
-      ContainerType containerType) {
+      ContainerType containerType, ExecutionType execType) {
     byte[] password;
     ContainerTokenIdentifier tokenIdentifier;
     long expiryTimeStamp =
@@ -220,7 +246,7 @@ public class RMContainerTokenSecretManager extends
               this.currentMasterKey.getMasterKey().getKeyId(),
               ResourceManager.getClusterTimeStamp(), priority, createTime,
               logAggregationContext, nodeLabelExpression, containerType,
-              ExecutionType.GUARANTEED);
+              execType);
       password = this.createPassword(tokenIdentifier);
 
     } finally {

+ 23 - 23
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java

@@ -2107,49 +2107,49 @@ public class TestFairScheduler extends FairSchedulerTestBase {
     scheduler.init(conf);
     scheduler.start();
     scheduler.reinitialize(conf, resourceManager.getRMContext());
+    int minReqSize =
+        FairSchedulerConfiguration.DEFAULT_RM_SCHEDULER_INCREMENT_ALLOCATION_MB;
 
+    // First ask, queue1 requests 1 large (minReqSize * 2).
     ApplicationAttemptId id11 = createAppAttemptId(1, 1);
     createMockRMApp(id11);
-    scheduler.addApplication(id11.getApplicationId(), "root.queue1", "user1", false);
+    scheduler.addApplication(id11.getApplicationId(),
+        "root.queue1", "user1", false);
     scheduler.addApplicationAttempt(id11, false, false);
-    ApplicationAttemptId id21 = createAppAttemptId(2, 1);
-    createMockRMApp(id21);
-    scheduler.addApplication(id21.getApplicationId(), "root.queue2", "user1", false);
-    scheduler.addApplicationAttempt(id21, false, false);
-    ApplicationAttemptId id22 = createAppAttemptId(2, 2);
-    createMockRMApp(id22);
-
-    scheduler.addApplication(id22.getApplicationId(), "root.queue2", "user1", false);
-    scheduler.addApplicationAttempt(id22, false, false);
-
-    int minReqSize = 
-        FairSchedulerConfiguration.DEFAULT_RM_SCHEDULER_INCREMENT_ALLOCATION_MB;
-    
-    // First ask, queue1 requests 1 large (minReqSize * 2).
     List<ResourceRequest> ask1 = new ArrayList<ResourceRequest>();
-    ResourceRequest request1 =
-        createResourceRequest(minReqSize * 2, ResourceRequest.ANY, 1, 1, true);
+    ResourceRequest request1 = createResourceRequest(minReqSize * 2,
+        ResourceRequest.ANY, 1, 1, true);
     ask1.add(request1);
     scheduler.allocate(id11, ask1, new ArrayList<ContainerId>(),
         null, null, NULL_UPDATE_REQUESTS);
 
     // Second ask, queue2 requests 1 large.
+    ApplicationAttemptId id21 = createAppAttemptId(2, 1);
+    createMockRMApp(id21);
+    scheduler.addApplication(id21.getApplicationId(),
+        "root.queue2", "user1", false);
+    scheduler.addApplicationAttempt(id21, false, false);
     List<ResourceRequest> ask2 = new ArrayList<ResourceRequest>();
-    ResourceRequest request2 = createResourceRequest(2 * minReqSize, "foo", 1, 1,
-        false);
+    ResourceRequest request2 = createResourceRequest(2 * minReqSize,
+        "foo", 1, 1, false);
     ResourceRequest request3 = createResourceRequest(2 * minReqSize,
-            ResourceRequest.ANY, 1, 1, false);
+        ResourceRequest.ANY, 1, 1, false);
     ask2.add(request2);
     ask2.add(request3);
     scheduler.allocate(id21, ask2, new ArrayList<ContainerId>(),
         null, null, NULL_UPDATE_REQUESTS);
 
     // Third ask, queue2 requests 2 small (minReqSize).
+    ApplicationAttemptId id22 = createAppAttemptId(2, 2);
+    createMockRMApp(id22);
+    scheduler.addApplication(id22.getApplicationId(),
+        "root.queue2", "user1", false);
+    scheduler.addApplicationAttempt(id22, false, false);
     List<ResourceRequest> ask3 = new ArrayList<ResourceRequest>();
-    ResourceRequest request4 = createResourceRequest(minReqSize, "bar", 2, 2,
-            true);
+    ResourceRequest request4 = createResourceRequest(minReqSize,
+        "bar", 2, 2, true);
     ResourceRequest request5 = createResourceRequest(minReqSize,
-            ResourceRequest.ANY, 2, 2, true);
+        ResourceRequest.ANY, 2, 2, true);
     ask3.add(request4);
     ask3.add(request5);
     scheduler.allocate(id22, ask3, new ArrayList<ContainerId>(),

+ 3 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageApps.java

@@ -69,6 +69,7 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.application.Applica
 import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationTable;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.EventColumnName;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.EventColumnNameConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.StringKeyConverter;
@@ -493,7 +494,8 @@ public class TestHBaseTimelineStorageApps {
     event.addInfo(expKey, expVal);
 
     final TimelineEntity entity = new ApplicationEntity();
-    entity.setId(ApplicationId.newInstance(0, 1).toString());
+    entity.setId(HBaseTimelineStorageUtils.convertApplicationIdToString(
+        ApplicationId.newInstance(0, 1)));
     entity.addEvent(event);
 
     TimelineEntities entities = new TimelineEntities();

+ 9 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageEntities.java

@@ -62,6 +62,7 @@ import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelinePrefi
 import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.EventColumnName;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.EventColumnNameConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.StringKeyConverter;
@@ -202,8 +203,9 @@ public class TestHBaseTimelineStorageEntities {
       String flow = "some_flow_name";
       String flowVersion = "AB7822C10F1111";
       long runid = 1002345678919L;
-      String appName = ApplicationId.newInstance(System.currentTimeMillis() +
-          9000000L, 1).toString();
+      String appName = HBaseTimelineStorageUtils.convertApplicationIdToString(
+          ApplicationId.newInstance(System.currentTimeMillis() + 9000000L, 1)
+      );
       hbi.write(cluster, user, flow, flowVersion, runid, appName, te);
       hbi.stop();
 
@@ -399,8 +401,8 @@ public class TestHBaseTimelineStorageEntities {
       String flow = "other_flow_name";
       String flowVersion = "1111F01C2287BA";
       long runid = 1009876543218L;
-      String appName = ApplicationId.newInstance(System.currentTimeMillis() +
-          9000000L, 1).toString();
+      String appName = HBaseTimelineStorageUtils.convertApplicationIdToString(
+          ApplicationId.newInstance(System.currentTimeMillis() + 9000000L, 1));
       byte[] startRow =
           new EntityRowKeyPrefix(cluster, user, flow, runid, appName)
               .getRowKeyPrefix();
@@ -487,7 +489,9 @@ public class TestHBaseTimelineStorageEntities {
     event.addInfo(expKey, expVal);
 
     final TimelineEntity entity = new ApplicationEntity();
-    entity.setId(ApplicationId.newInstance(0, 1).toString());
+    entity.setId(
+        HBaseTimelineStorageUtils.convertApplicationIdToString(
+            ApplicationId.newInstance(0, 1)));
     entity.addEvent(event);
 
     TimelineEntities entities = new TimelineEntities();

+ 2 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/AppIdKeyConverter.java

@@ -82,7 +82,8 @@ public final class AppIdKeyConverter implements KeyConverter<String> {
         Bytes.toLong(appIdBytes, 0, Bytes.SIZEOF_LONG));
     int seqId = HBaseTimelineStorageUtils.invertInt(
         Bytes.toInt(appIdBytes, Bytes.SIZEOF_LONG, Bytes.SIZEOF_INT));
-    return ApplicationId.newInstance(clusterTs, seqId).toString();
+    return HBaseTimelineStorageUtils.convertApplicationIdToString(
+        ApplicationId.newInstance(clusterTs, seqId));
   }
 
   /**

+ 33 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/HBaseTimelineStorageUtils.java

@@ -24,6 +24,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.AggregationCompactionDimension;
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.AggregationOperation;
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute;
@@ -32,6 +33,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.text.NumberFormat;
 import java.util.List;
 import java.util.Map;
 
@@ -240,4 +242,35 @@ public final class HBaseTimelineStorageUtils {
     long dayTimestamp = ts - (ts % MILLIS_ONE_DAY);
     return dayTimestamp;
   }
+
+  private static final ThreadLocal<NumberFormat> APP_ID_FORMAT =
+      new ThreadLocal<NumberFormat>() {
+        @Override
+        public NumberFormat initialValue() {
+          NumberFormat fmt = NumberFormat.getInstance();
+          fmt.setGroupingUsed(false);
+          fmt.setMinimumIntegerDigits(4);
+          return fmt;
+        }
+      };
+
+  /**
+   * A utility method that converts ApplicationId to string without using
+   * FastNumberFormat in order to avoid the incompatibility issue caused
+   * by mixing hadoop-common 2.5.1 and hadoop-yarn-api 3.0 in this module.
+   * This is a work-around implementation as discussed in YARN-6905.
+   *
+   * @param appId application id
+   * @return the string representation of the given application id
+   *
+   */
+  public static String convertApplicationIdToString(ApplicationId appId) {
+    StringBuilder sb = new StringBuilder(64);
+    sb.append(ApplicationId.appIdStrPrefix);
+    sb.append("_");
+    sb.append(appId.getClusterTimestamp());
+    sb.append('_');
+    sb.append(APP_ID_FORMAT.get().format(appId.getId()));
+    return sb.toString();
+  }
 }

+ 39 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestCustomApplicationIdConversion.java

@@ -0,0 +1,39 @@
+/**
+ * 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.timelineservice.storage.common;
+
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test for HBaseTimelineStorageUtils.convertApplicationIdToString(),
+ * a custom conversion from ApplicationId to String that avoids the
+ * incompatibility issue caused by mixing hadoop-common 2.5.1 and
+ * hadoop-yarn-api 3.0. See YARN-6905.
+ */
+public class TestCustomApplicationIdConversion {
+  @Test
+  public void testConvertAplicationIdToString() {
+    ApplicationId applicationId = ApplicationId.newInstance(0, 1);
+    String applicationIdStr =
+        HBaseTimelineStorageUtils.convertApplicationIdToString(applicationId);
+    Assert.assertEquals(applicationId,
+        ApplicationId.fromString(applicationIdStr));
+  }
+}

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

@@ -155,14 +155,6 @@
                       <outputDirectory>${project.build.directory}</outputDirectory>
                       <destFileName>hadoop-annotations.jar</destFileName>
                     </artifactItem>
-                    <artifactItem>
-                      <groupId>xerces</groupId>
-                      <artifactId>xercesImpl</artifactId>
-                      <version>${xerces.version.jdiff}</version>
-                      <overWrite>false</overWrite>
-                      <outputDirectory>${project.build.directory}</outputDirectory>
-                      <destFileName>xerces.jar</destFileName>
-                    </artifactItem>
                   </artifactItems>
                 </configuration>
               </execution>
@@ -201,7 +193,7 @@
                       sourceFiles="${dev-support.relative.dir}/jdiff/Null.java"
                              maxmemory="${jdiff.javadoc.maxmemory}">
                       <doclet name="org.apache.hadoop.classification.tools.IncludePublicAnnotationsJDiffDoclet"
-                              path="${project.build.directory}/hadoop-annotations.jar:${project.build.directory}/jdiff.jar:${project.build.directory}/xerces.jar">
+                              path="${project.build.directory}/hadoop-annotations.jar:${project.build.directory}/jdiff.jar">
                         <param name="-oldapi" value="${project.name} ${jdiff.stable.api}"/>
                         <param name="-newapi" value="${project.name} ${project.version}"/>
                         <param name="-oldapidir" value="${basedir}/${dev-support.relative.dir}/jdiff"/>