Explorar el Código

Merge branch 'trunk' into HDFS-7240

Anu Engineer hace 7 años
padre
commit
a20a3fefbe
Se han modificado 100 ficheros con 4024 adiciones y 1376 borrados
  1. 3 0
      .gitignore
  2. 4 4
      BUILDING.txt
  3. 3 0
      hadoop-client-modules/hadoop-client-integration-tests/pom.xml
  4. 4 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/ValueQueue.java
  5. 1 10
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocatedFileStatus.java
  6. 21 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryPolicies.java
  7. 22 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/retry/TestRetryProxy.java
  8. 10 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/retry/UnreliableImplementation.java
  9. 5 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/retry/UnreliableInterface.java
  10. 3 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/fs/Hdfs.java
  11. 3 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
  12. 20 60
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java
  13. 110 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java
  14. 28 37
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
  15. 56 25
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/RequestHedgingProxyProvider.java
  16. 58 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRequestHedgingProxyProvider.java
  17. 4 3
      hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
  18. 11 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
  19. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
  20. 34 27
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
  21. 11 12
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
  22. 7 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java
  23. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
  24. 5 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
  25. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java
  26. 5 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestStorageMover.java
  27. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
  28. 5 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogAutoroll.java
  29. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogRace.java
  30. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogTailer.java
  31. 117 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestOpenFilesWithSnapshot.java
  32. 3 2
      hadoop-project/pom.xml
  33. 4 0
      hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystemStore.java
  34. 7 0
      hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/Constants.java
  35. 1 1
      hadoop-tools/hadoop-aliyun/src/site/markdown/tools/hadoop-aliyun/index.md
  36. 2 0
      hadoop-tools/hadoop-aws/pom.xml
  37. 3 0
      hadoop-tools/hadoop-azure/pom.xml
  38. 1 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceRequest.java
  39. 14 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
  40. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/test/java/org/apache/hadoop/yarn/service/TestApiServer.java
  41. 23 27
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceScheduler.java
  42. 33 18
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Component.java
  43. 12 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ReadinessCheck.java
  44. 9 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Service.java
  45. 5 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/ServiceClient.java
  46. 22 36
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/Component.java
  47. 11 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/ComponentEvent.java
  48. 44 39
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/instance/ComponentInstance.java
  49. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/conf/RestApiConstants.java
  50. 5 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/conf/YarnServiceConf.java
  51. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/ContainerLaunchService.java
  52. 5 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/JavaCommandLineBuilder.java
  53. 3 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/AbstractProviderService.java
  54. 3 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/ProviderService.java
  55. 3 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/ProviderUtils.java
  56. 3 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/timelineservice/ServiceTimelinePublisher.java
  57. 52 14
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/MockServiceAM.java
  58. 4 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/ServiceTestUtils.java
  59. 109 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestServiceAM.java
  60. 18 24
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestYarnNativeServices.java
  61. 12 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/monitor/TestServiceMonitor.java
  62. 30 425
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java
  63. 521 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClientWithReservation.java
  64. 8 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebApps.java
  65. 21 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
  66. 47 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestResourceRequest.java
  67. 5 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml
  68. 338 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationRegistryClient.java
  69. 109 32
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/uam/UnmanagedAMPoolManager.java
  70. 128 84
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/uam/UnmanagedApplicationManager.java
  71. 14 16
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/AMRMClientUtils.java
  72. 56 47
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java
  73. 90 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/TestFederationRegistryClient.java
  74. 85 15
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/uam/TestUnmanagedApplicationManager.java
  75. 16 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/AMRMProxyApplicationContext.java
  76. 29 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/AMRMProxyApplicationContextImpl.java
  77. 65 18
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/AMRMProxyService.java
  78. 204 17
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/FederationInterceptor.java
  79. 5 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
  80. 12 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/BaseAMRMProxyTest.java
  81. 10 11
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestAMRMProxyService.java
  82. 110 16
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestFederationInterceptor.java
  83. 24 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestableFederationInterceptor.java
  84. 1 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
  85. 12 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
  86. 6 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/FifoIntraQueuePreemptionPlugin.java
  87. 52 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/placement/ApplicationPlacementContext.java
  88. 15 19
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/placement/PlacementManager.java
  89. 4 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/placement/PlacementRule.java
  90. 271 13
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/placement/UserGroupMappingPlacementRule.java
  91. 106 39
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
  92. 1 1
      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
  93. 120 76
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractManagedParentQueue.java
  94. 16 11
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AutoCreatedLeafQueue.java
  95. 114 43
      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
  96. 153 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java
  97. 69 34
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerQueueManager.java
  98. 158 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ManagedParentQueue.java
  99. 0 13
      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
  100. 21 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/PlanQueue.java

+ 3 - 0
.gitignore

@@ -8,6 +8,7 @@
 *.sdf
 *.suo
 *.vcxproj.user
+*.patch
 .idea
 .svn
 .classpath
@@ -45,3 +46,5 @@ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/tmp
 yarnregistry.pdf
 patchprocess/
 .history/
+hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/package-lock.json
+hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/yarn-error.log

+ 4 - 4
BUILDING.txt

@@ -4,7 +4,7 @@ Build instructions for Hadoop
 Requirements:
 
 * Unix System
-* JDK 1.8+
+* JDK 1.8
 * Maven 3.3 or later
 * ProtocolBuffer 2.5.0
 * CMake 3.1 or newer (if compiling native code)
@@ -331,10 +331,10 @@ If the build process fails with an out of memory error, you should be able to fi
 it by increasing the memory used by maven which can be done via the environment
 variable MAVEN_OPTS.
 
-Here is an example setting to allocate between 256 and 512 MB of heap space to
+Here is an example setting to allocate between 256 MB and 1 GB of heap space to
 Maven
 
-export MAVEN_OPTS="-Xms256m -Xmx512m"
+export MAVEN_OPTS="-Xms256m -Xmx1g"
 
 ----------------------------------------------------------------------------------
 
@@ -344,7 +344,7 @@ Building on Windows
 Requirements:
 
 * Windows System
-* JDK 1.8+
+* JDK 1.8
 * Maven 3.0 or later
 * ProtocolBuffer 2.5.0
 * CMake 3.1 or newer

+ 3 - 0
hadoop-client-modules/hadoop-client-integration-tests/pom.xml

@@ -128,6 +128,9 @@
                   <goal>integration-test</goal>
                   <goal>verify</goal>
                 </goals>
+                <configuration>
+                  <trimStackTrace>false</trimStackTrace>
+                </configuration>
               </execution>
             </executions>
           </plugin>

+ 4 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/ValueQueue.java

@@ -224,6 +224,9 @@ public class ValueQueue <E> {
     Preconditions.checkArgument(numValues > 0, "\"numValues\" must be > 0");
     Preconditions.checkArgument(((lowWatermark > 0)&&(lowWatermark <= 1)),
         "\"lowWatermark\" must be > 0 and <= 1");
+    final int watermarkValue = (int) (numValues * lowWatermark);
+    Preconditions.checkArgument(watermarkValue > 0,
+        "(int) (\"numValues\" * \"lowWatermark\") must be > 0");
     Preconditions.checkArgument(expiry > 0, "\"expiry\" must be > 0");
     Preconditions.checkArgument(numFillerThreads > 0,
         "\"numFillerThreads\" must be > 0");
@@ -243,8 +246,7 @@ public class ValueQueue <E> {
                       throws Exception {
                     LinkedBlockingQueue<E> keyQueue =
                         new LinkedBlockingQueue<E>();
-                    refiller.fillQueueForKey(keyName, keyQueue,
-                        (int)(lowWatermark * numValues));
+                    refiller.fillQueueForKey(keyName, keyQueue, watermarkValue);
                     return keyQueue;
                   }
                 });

+ 1 - 10
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocatedFileStatus.java

@@ -135,16 +135,7 @@ public class LocatedFileStatus extends FileStatus {
   public BlockLocation[] getBlockLocations() {
     return locations;
   }
-
-  /**
-   * Hook for subclasses to lazily set block locations. The {@link #locations}
-   * field should be null before this is called.
-   * @param locations Block locations for this instance.
-   */
-  protected void setBlockLocations(BlockLocation[] locations) {
-    this.locations = locations;
-  }
-
+  
   /**
    * Compare this FileStatus to another FileStatus
    * @param   o the FileStatus to be compared.

+ 21 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryPolicies.java

@@ -32,11 +32,14 @@ import java.util.Map.Entry;
 import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.TimeUnit;
 
+import javax.security.sasl.SaslException;
+
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.RetriableException;
 import org.apache.hadoop.ipc.StandbyException;
 import org.apache.hadoop.net.ConnectTimeoutException;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
+import org.ietf.jgss.GSSException;
 
 import com.google.common.annotations.VisibleForTesting;
 import org.slf4j.Logger;
@@ -663,6 +666,11 @@ public class RetryPolicies {
             + retries + ") exceeded maximum allowed (" + maxRetries + ")");
       }
 
+      if (isSaslFailure(e)) {
+          return new RetryAction(RetryAction.RetryDecision.FAIL, 0,
+                  "SASL failure");
+      }
+
       if (e instanceof ConnectException ||
           e instanceof EOFException ||
           e instanceof NoRouteToHostException ||
@@ -716,7 +724,7 @@ public class RetryPolicies {
   private static long calculateExponentialTime(long time, int retries) {
     return calculateExponentialTime(time, retries, Long.MAX_VALUE);
   }
-  
+
   private static boolean isWrappedStandbyException(Exception e) {
     if (!(e instanceof RemoteException)) {
       return false;
@@ -725,6 +733,18 @@ public class RetryPolicies {
         StandbyException.class);
     return unwrapped instanceof StandbyException;
   }
+
+  private static boolean isSaslFailure(Exception e) {
+      Throwable current = e;
+      do {
+          if (current instanceof SaslException) {
+            return true;
+          }
+          current = current.getCause();
+      } while (current != null);
+
+      return false;
+  }
   
   static RetriableException getWrappedRetriableException(Exception e) {
     if (!(e instanceof RemoteException)) {

+ 22 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/retry/TestRetryProxy.java

@@ -39,6 +39,8 @@ import java.util.concurrent.*;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
 
+import javax.security.sasl.SaslException;
+
 import static org.apache.hadoop.io.retry.RetryPolicies.*;
 import static org.junit.Assert.*;
 import static org.mockito.Matchers.any;
@@ -326,4 +328,24 @@ public class TestRetryProxy {
     assertEquals(InterruptedException.class, e.getCause().getClass());
     assertEquals("sleep interrupted", e.getCause().getMessage());
   }
+
+  @Test
+  public void testNoRetryOnSaslError() throws Exception {
+    RetryPolicy policy = mock(RetryPolicy.class);
+    RetryPolicy realPolicy = RetryPolicies.failoverOnNetworkException(5);
+    setupMockPolicy(policy, realPolicy);
+
+    UnreliableInterface unreliable = (UnreliableInterface) RetryProxy.create(
+        UnreliableInterface.class, unreliableImpl, policy);
+
+    try {
+      unreliable.failsWithSASLExceptionTenTimes();
+      fail("Should fail");
+    } catch (SaslException e) {
+      // expected
+      verify(policy, times(1)).shouldRetry(any(Exception.class), anyInt(),
+          anyInt(), anyBoolean());
+      assertEquals(RetryDecision.FAIL, caughtRetryAction.action);
+    }
+  }
 }

+ 10 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/retry/UnreliableImplementation.java

@@ -19,6 +19,8 @@ package org.apache.hadoop.io.retry;
 
 import java.io.IOException;
 
+import javax.security.sasl.SaslException;
+
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.StandbyException;
 
@@ -29,6 +31,7 @@ class UnreliableImplementation implements UnreliableInterface {
     failsOnceIOExceptionInvocationCount,
     failsOnceRemoteExceptionInvocationCount,
     failsTenTimesInvocationCount,
+    failsWithSASLExceptionTenTimesInvocationCount,
     succeedsOnceThenFailsCount,
     succeedsOnceThenFailsIdempotentCount,
     succeedsTenTimesThenFailsCount;
@@ -113,6 +116,13 @@ class UnreliableImplementation implements UnreliableInterface {
     }
   }
 
+  @Override
+  public void failsWithSASLExceptionTenTimes() throws SaslException {
+    if (failsWithSASLExceptionTenTimesInvocationCount ++ < 10) {
+      throw new SaslException();
+    }
+  }
+
   @Override
   public String succeedsOnceThenFailsReturningString()
       throws UnreliableException, IOException, StandbyException {

+ 5 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/retry/UnreliableInterface.java

@@ -20,6 +20,8 @@ package org.apache.hadoop.io.retry;
 
 import java.io.IOException;
 
+import javax.security.sasl.SaslException;
+
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.StandbyException;
 
@@ -61,7 +63,9 @@ public interface UnreliableInterface {
   boolean failsOnceThenSucceedsWithReturnValue() throws UnreliableException;
 
   void failsTenTimesThenSucceeds() throws UnreliableException;
-  
+
+  void failsWithSASLExceptionTenTimes() throws SaslException;
+
   public String succeedsOnceThenFailsReturningString()
       throws UnreliableException, StandbyException, IOException;
   @Idempotent

+ 3 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/fs/Hdfs.java

@@ -48,6 +48,7 @@ import org.apache.hadoop.hdfs.client.impl.CorruptFileBlockIterator;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.AccessControlException;
@@ -187,7 +188,8 @@ public class Hdfs extends AbstractFileSystem {
 
       @Override
       public LocatedFileStatus next() throws IOException {
-        return getNext().makeQualifiedLocated(getUri(), p);
+        return ((HdfsLocatedFileStatus)getNext()).makeQualifiedLocated(
+            getUri(), p);
       }
     };
   }

+ 3 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java

@@ -85,6 +85,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsPathHandle;
+import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
 import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
@@ -1211,7 +1212,8 @@ public class DistributedFileSystem extends FileSystem {
         T next;
         HdfsFileStatus fileStat = thisListing.getPartialListing()[i++];
         if (needLocation) {
-          next = (T)fileStat.makeQualifiedLocated(getUri(), p);
+          next = (T)((HdfsLocatedFileStatus)fileStat)
+              .makeQualifiedLocated(getUri(), p);
         } else {
           next = (T)fileStat.makeQualified(getUri(), p);
         }

+ 20 - 60
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java

@@ -26,17 +26,15 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.FileEncryptionInfo;
 import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSUtilClient;
 
-/**
- * HDFS metadata for an entity in the filesystem.
+/** Interface that represents the over the wire information for a file.
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
-public final class HdfsFileStatus extends LocatedFileStatus {
+public class HdfsFileStatus extends FileStatus {
 
   private static final long serialVersionUID = 0x126eb82a;
 
@@ -51,12 +49,11 @@ public final class HdfsFileStatus extends LocatedFileStatus {
   private final int childrenNum;
   private final byte storagePolicy;
 
-  // BlockLocations[] is the user-facing type
-  private transient LocatedBlocks hdfsloc;
-
   public static final byte[] EMPTY_NAME = new byte[0];
 
-  /** Set of features potentially active on an instance. */
+  /**
+   * Set of features potentially active on an instance.
+   */
   public enum Flags {
     HAS_ACL,
     HAS_CRYPT,
@@ -84,19 +81,18 @@ public final class HdfsFileStatus extends LocatedFileStatus {
    * @param storagePolicy ID which specifies storage policy
    * @param ecPolicy the erasure coding policy
    */
-  private HdfsFileStatus(long length, boolean isdir, int replication,
+  protected HdfsFileStatus(long length, boolean isdir, int replication,
                          long blocksize, long mtime, long atime,
                          FsPermission permission, EnumSet<Flags> flags,
                          String owner, String group,
                          byte[] symlink, byte[] path, long fileId,
                          int childrenNum, FileEncryptionInfo feInfo,
-                         byte storagePolicy, ErasureCodingPolicy ecPolicy,
-                         LocatedBlocks hdfsloc) {
+                         byte storagePolicy, ErasureCodingPolicy ecPolicy) {
     super(length, isdir, replication, blocksize, mtime,
         atime, convert(isdir, symlink != null, permission, flags),
         owner, group, null, null,
         flags.contains(Flags.HAS_ACL), flags.contains(Flags.HAS_CRYPT),
-        flags.contains(Flags.HAS_EC), null);
+        flags.contains(Flags.HAS_EC));
     this.flags = flags;
     this.uSymlink = symlink;
     this.uPath = path;
@@ -105,7 +101,6 @@ public final class HdfsFileStatus extends LocatedFileStatus {
     this.feInfo = feInfo;
     this.storagePolicy = storagePolicy;
     this.ecPolicy = ecPolicy;
-    this.hdfsloc = hdfsloc;
   }
 
   /**
@@ -157,7 +152,7 @@ public final class HdfsFileStatus extends LocatedFileStatus {
    * Check if the local name is empty.
    * @return true if the name is empty
    */
-  public boolean isEmptyLocalName() {
+  public final boolean isEmptyLocalName() {
     return uPath.length == 0;
   }
 
@@ -165,7 +160,7 @@ public final class HdfsFileStatus extends LocatedFileStatus {
    * Get the string representation of the local name.
    * @return the local name in string
    */
-  public String getLocalName() {
+  public final String getLocalName() {
     return DFSUtilClient.bytes2String(uPath);
   }
 
@@ -173,7 +168,7 @@ public final class HdfsFileStatus extends LocatedFileStatus {
    * Get the Java UTF8 representation of the local name.
    * @return the local name in java UTF8
    */
-  public byte[] getLocalNameInBytes() {
+  public final byte[] getLocalNameInBytes() {
     return uPath;
   }
 
@@ -182,7 +177,7 @@ public final class HdfsFileStatus extends LocatedFileStatus {
    * @param parent the parent path
    * @return the full path in string
    */
-  public String getFullName(String parent) {
+  public final String getFullName(final String parent) {
     if (isEmptyLocalName()) {
       return parent;
     }
@@ -200,7 +195,7 @@ public final class HdfsFileStatus extends LocatedFileStatus {
    * @param parent the parent path
    * @return the full path
    */
-  public Path getFullPath(Path parent) {
+  public final Path getFullPath(final Path parent) {
     if (isEmptyLocalName()) {
       return parent;
     }
@@ -224,15 +219,15 @@ public final class HdfsFileStatus extends LocatedFileStatus {
   /**
    * Opaque referant for the symlink, to be resolved at the client.
    */
-  public byte[] getSymlinkInBytes() {
+  public final byte[] getSymlinkInBytes() {
     return uSymlink;
   }
 
-  public long getFileId() {
+  public final long getFileId() {
     return fileId;
   }
 
-  public FileEncryptionInfo getFileEncryptionInfo() {
+  public final FileEncryptionInfo getFileEncryptionInfo() {
     return feInfo;
   }
 
@@ -244,12 +239,12 @@ public final class HdfsFileStatus extends LocatedFileStatus {
     return ecPolicy;
   }
 
-  public int getChildrenNum() {
+  public final int getChildrenNum() {
     return childrenNum;
   }
 
   /** @return the storage policy id */
-  public byte getStoragePolicy() {
+  public final byte getStoragePolicy() {
     return storagePolicy;
   }
 
@@ -262,10 +257,6 @@ public final class HdfsFileStatus extends LocatedFileStatus {
     return flags.contains(Flags.SNAPSHOT_ENABLED);
   }
 
-  public LocatedBlocks getLocatedBlocks() {
-    return hdfsloc;
-  }
-
   @Override
   public boolean equals(Object o) {
     // satisfy findbugs
@@ -286,30 +277,11 @@ public final class HdfsFileStatus extends LocatedFileStatus {
    * @param parent Parent path of this element.
    * @return Reference to this instance.
    */
-  public FileStatus makeQualified(URI defaultUri, Path parent) {
+  public final FileStatus makeQualified(URI defaultUri, Path parent) {
     // fully-qualify path
     setPath(getFullPath(parent).makeQualified(defaultUri, null));
     return this; // API compatibility
-  }
 
-  /**
-   * This function is used to transform the underlying HDFS LocatedBlocks to
-   * BlockLocations. This method must be invoked before
-   * {@link #getBlockLocations()}.
-   *
-   * The returned BlockLocation will have different formats for replicated
-   * and erasure coded file.
-   * Please refer to
-   * {@link org.apache.hadoop.fs.FileSystem#getFileBlockLocations
-   * (FileStatus, long, long)}
-   * for examples.
-   */
-  public LocatedFileStatus makeQualifiedLocated(URI defaultUri,
-                                                      Path path) {
-    makeQualified(defaultUri, path);
-    setBlockLocations(
-        DFSUtilClient.locatedBlocks2Locations(getLocatedBlocks()));
-    return this;
   }
 
   /**
@@ -339,7 +311,6 @@ public final class HdfsFileStatus extends LocatedFileStatus {
     private byte storagePolicy             =
         HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
     private ErasureCodingPolicy ecPolicy   = null;
-    private LocatedBlocks locations        = null;
 
     /**
      * Set the length of the entity (default = 0).
@@ -518,24 +489,13 @@ public final class HdfsFileStatus extends LocatedFileStatus {
       return this;
     }
 
-    /**
-     * Set the block locations for this entity (default = null).
-     * @param locations HDFS locations
-     *                  (see {@link #makeQualifiedLocated(URI, Path)})
-     * @return This Builder instance
-     */
-    public Builder locations(LocatedBlocks locations) {
-      this.locations = locations;
-      return this;
-    }
-
     /**
      * @return An {@link HdfsFileStatus} instance from these parameters.
      */
     public HdfsFileStatus build() {
       return new HdfsFileStatus(length, isdir, replication, blocksize,
           mtime, atime, permission, flags, owner, group, symlink, path, fileId,
-          childrenNum, feInfo, storagePolicy, ecPolicy, locations);
+          childrenNum, feInfo, storagePolicy, ecPolicy);
     }
   }
 

+ 110 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java

@@ -0,0 +1,110 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.protocol;
+
+import java.net.URI;
+import java.util.EnumSet;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.FileEncryptionInfo;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DFSUtilClient;
+
+/**
+ * Interface that represents the over the wire information
+ * including block locations for a file.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class HdfsLocatedFileStatus extends HdfsFileStatus {
+
+  private static final long serialVersionUID = 0x23c73328;
+
+  /**
+   * Left transient, because {@link #makeQualifiedLocated(URI,Path)}
+   * is the user-facing type.
+   */
+  private transient LocatedBlocks locations;
+
+  /**
+   * Constructor
+   *
+   * @param length size
+   * @param isdir if this is directory
+   * @param block_replication the file's replication factor
+   * @param blocksize the file's block size
+   * @param modification_time most recent modification time
+   * @param access_time most recent access time
+   * @param permission permission
+   * @param owner owner
+   * @param group group
+   * @param symlink symbolic link
+   * @param path local path name in java UTF8 format
+   * @param fileId the file id
+   * @param locations block locations
+   * @param feInfo file encryption info
+   */
+  public HdfsLocatedFileStatus(long length, boolean isdir,
+      int block_replication, long blocksize, long modification_time,
+      long access_time, FsPermission permission, EnumSet<Flags> flags,
+      String owner, String group, byte[] symlink, byte[] path, long fileId,
+      LocatedBlocks locations, int childrenNum, FileEncryptionInfo feInfo,
+      byte storagePolicy, ErasureCodingPolicy ecPolicy) {
+    super(length, isdir, block_replication, blocksize, modification_time,
+        access_time, permission, flags, owner, group, symlink, path, fileId,
+        childrenNum, feInfo, storagePolicy, ecPolicy);
+    this.locations = locations;
+  }
+
+  public LocatedBlocks getBlockLocations() {
+    return locations;
+  }
+
+  /**
+   * This function is used to transform the underlying HDFS LocatedBlocks to
+   * BlockLocations.
+   *
+   * The returned BlockLocation will have different formats for replicated
+   * and erasure coded file.
+   * Please refer to
+   * {@link org.apache.hadoop.fs.FileSystem#getFileBlockLocations
+   * (FileStatus, long, long)}
+   * for examples.
+   */
+  public final LocatedFileStatus makeQualifiedLocated(URI defaultUri,
+      Path path) {
+    makeQualified(defaultUri, path);
+    return new LocatedFileStatus(this,
+        DFSUtilClient.locatedBlocks2Locations(getBlockLocations()));
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    // satisfy findbugs
+    return super.equals(o);
+  }
+
+  @Override
+  public int hashCode() {
+    // satisfy findbugs
+    return super.hashCode();
+  }
+}

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

@@ -90,6 +90,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsPathHandle;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
@@ -1584,36 +1585,23 @@ public class PBHelperClient {
     EnumSet<HdfsFileStatus.Flags> flags = fs.hasFlags()
         ? convertFlags(fs.getFlags())
         : convertFlags(fs.getPermission());
-    return new HdfsFileStatus.Builder()
-        .length(fs.getLength())
-        .isdir(fs.getFileType().equals(FileType.IS_DIR))
-        .replication(fs.getBlockReplication())
-        .blocksize(fs.getBlocksize())
-        .mtime(fs.getModificationTime())
-        .atime(fs.getAccessTime())
-        .perm(convert(fs.getPermission()))
-        .flags(flags)
-        .owner(fs.getOwner())
-        .group(fs.getGroup())
-        .symlink(FileType.IS_SYMLINK.equals(fs.getFileType())
-            ? fs.getSymlink().toByteArray()
-            : null)
-        .path(fs.getPath().toByteArray())
-        .fileId(fs.hasFileId()
-            ? fs.getFileId()
-            : HdfsConstants.GRANDFATHER_INODE_ID)
-        .locations(fs.hasLocations() ? convert(fs.getLocations()) : null)
-        .children(fs.hasChildrenNum() ? fs.getChildrenNum() : -1)
-        .feInfo(fs.hasFileEncryptionInfo()
-            ? convert(fs.getFileEncryptionInfo())
-            : null)
-        .storagePolicy(fs.hasStoragePolicy()
-            ? (byte) fs.getStoragePolicy()
-            : HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED)
-        .ecPolicy(fs.hasEcPolicy()
-            ? convertErasureCodingPolicy(fs.getEcPolicy())
-            : null)
-        .build();
+    return new HdfsLocatedFileStatus(
+        fs.getLength(), fs.getFileType().equals(FileType.IS_DIR),
+        fs.getBlockReplication(), fs.getBlocksize(),
+        fs.getModificationTime(), fs.getAccessTime(),
+        convert(fs.getPermission()),
+        flags,
+        fs.getOwner(), fs.getGroup(),
+        fs.getFileType().equals(FileType.IS_SYMLINK) ?
+            fs.getSymlink().toByteArray() : null,
+        fs.getPath().toByteArray(),
+        fs.hasFileId()? fs.getFileId(): HdfsConstants.GRANDFATHER_INODE_ID,
+        fs.hasLocations() ? convert(fs.getLocations()) : null,
+        fs.hasChildrenNum() ? fs.getChildrenNum() : -1,
+        fs.hasFileEncryptionInfo() ? convert(fs.getFileEncryptionInfo()) : null,
+        fs.hasStoragePolicy() ? (byte) fs.getStoragePolicy()
+            : HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED,
+        fs.hasEcPolicy() ? convertErasureCodingPolicy(fs.getEcPolicy()) : null);
   }
 
   private static EnumSet<HdfsFileStatus.Flags> convertFlags(int flags) {
@@ -1876,10 +1864,10 @@ public class PBHelperClient {
     if (dl == null)
       return null;
     List<HdfsFileStatusProto> partList =  dl.getPartialListingList();
-    return new DirectoryListing(partList.isEmpty()
-        ? new HdfsFileStatus[0]
-        : convert(partList.toArray(new HdfsFileStatusProto[partList.size()])),
-                  dl.getRemainingEntries());
+    return new DirectoryListing(partList.isEmpty() ?
+        new HdfsLocatedFileStatus[0] :
+        convert(partList.toArray(new HdfsFileStatusProto[partList.size()])),
+        dl.getRemainingEntries());
   }
 
   public static HdfsFileStatus[] convert(HdfsFileStatusProto[] fs) {
@@ -2173,9 +2161,12 @@ public class PBHelperClient {
     if (fs.getFileEncryptionInfo() != null) {
       builder.setFileEncryptionInfo(convert(fs.getFileEncryptionInfo()));
     }
-    LocatedBlocks locations = fs.getLocatedBlocks();
-    if (locations != null) {
-      builder.setLocations(convert(locations));
+    if (fs instanceof HdfsLocatedFileStatus) {
+      final HdfsLocatedFileStatus lfs = (HdfsLocatedFileStatus) fs;
+      LocatedBlocks locations = lfs.getBlockLocations();
+      if (locations != null) {
+        builder.setLocations(convert(locations));
+      }
     }
     if(fs.getErasureCodingPolicy() != null) {
       builder.setEcPolicy(convertErasureCodingPolicy(

+ 56 - 25
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/RequestHedgingProxyProvider.java

@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.server.namenode.ha;
 
 import java.lang.reflect.InvocationHandler;
+import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 import java.lang.reflect.Proxy;
 import java.net.URI;
@@ -29,6 +30,7 @@ import java.util.concurrent.ExecutorCompletionService;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
+import java.util.concurrent.ExecutionException;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.ipc.RemoteException;
@@ -87,9 +89,19 @@ public class RequestHedgingProxyProvider<T> extends
         targetProxies.remove(toIgnore);
         if (targetProxies.size() == 1) {
           ProxyInfo<T> proxyInfo = targetProxies.values().iterator().next();
-          Object retVal = method.invoke(proxyInfo.proxy, args);
-          successfulProxy = proxyInfo;
-          return retVal;
+          try {
+            currentUsedProxy = proxyInfo;
+            Object retVal = method.invoke(proxyInfo.proxy, args);
+            LOG.debug("Invocation successful on [{}]",
+                currentUsedProxy.proxyInfo);
+            return retVal;
+          } catch (InvocationTargetException ex) {
+            Exception unwrappedException = unwrapInvocationTargetException(ex);
+            logProxyException(unwrappedException, currentUsedProxy.proxyInfo);
+            LOG.trace("Unsuccessful invocation on [{}]",
+                currentUsedProxy.proxyInfo);
+            throw unwrappedException;
+          }
         }
         executor = Executors.newFixedThreadPool(proxies.size());
         completionService = new ExecutorCompletionService<>(executor);
@@ -112,15 +124,16 @@ public class RequestHedgingProxyProvider<T> extends
           Future<Object> callResultFuture = completionService.take();
           Object retVal;
           try {
+            currentUsedProxy = proxyMap.get(callResultFuture);
             retVal = callResultFuture.get();
-            successfulProxy = proxyMap.get(callResultFuture);
             LOG.debug("Invocation successful on [{}]",
-                successfulProxy.proxyInfo);
+                currentUsedProxy.proxyInfo);
             return retVal;
-          } catch (Exception ex) {
+          } catch (ExecutionException ex) {
+            Exception unwrappedException = unwrapExecutionException(ex);
             ProxyInfo<T> tProxyInfo = proxyMap.get(callResultFuture);
-            logProxyException(ex, tProxyInfo.proxyInfo);
-            badResults.put(tProxyInfo.proxyInfo, unwrapException(ex));
+            logProxyException(unwrappedException, tProxyInfo.proxyInfo);
+            badResults.put(tProxyInfo.proxyInfo, unwrappedException);
             LOG.trace("Unsuccessful invocation on [{}]", tProxyInfo.proxyInfo);
             numAttempts--;
           }
@@ -143,7 +156,7 @@ public class RequestHedgingProxyProvider<T> extends
   }
 
 
-  private volatile ProxyInfo<T> successfulProxy = null;
+  private volatile ProxyInfo<T> currentUsedProxy = null;
   private volatile String toIgnore = null;
 
   public RequestHedgingProxyProvider(Configuration conf, URI uri,
@@ -154,8 +167,8 @@ public class RequestHedgingProxyProvider<T> extends
   @SuppressWarnings("unchecked")
   @Override
   public synchronized ProxyInfo<T> getProxy() {
-    if (successfulProxy != null) {
-      return successfulProxy;
+    if (currentUsedProxy != null) {
+      return currentUsedProxy;
     }
     Map<String, ProxyInfo<T>> targetProxyInfos = new HashMap<>();
     StringBuilder combinedInfo = new StringBuilder("[");
@@ -175,8 +188,8 @@ public class RequestHedgingProxyProvider<T> extends
 
   @Override
   public synchronized void performFailover(T currentProxy) {
-    toIgnore = successfulProxy.proxyInfo;
-    successfulProxy = null;
+    toIgnore = this.currentUsedProxy.proxyInfo;
+    this.currentUsedProxy = null;
   }
 
   /**
@@ -187,19 +200,18 @@ public class RequestHedgingProxyProvider<T> extends
    */
   private void logProxyException(Exception ex, String proxyInfo) {
     if (isStandbyException(ex)) {
-      LOG.debug("Invocation returned standby exception on [{}]", proxyInfo);
+      LOG.debug("Invocation returned standby exception on [{}]", proxyInfo, ex);
     } else {
-      LOG.warn("Invocation returned exception on [{}]", proxyInfo);
+      LOG.warn("Invocation returned exception on [{}]", proxyInfo, ex);
     }
   }
 
   /**
    * Check if the returned exception is caused by an standby namenode.
-   * @param ex Exception to check.
+   * @param exception Exception to check.
    * @return If the exception is caused by an standby namenode.
    */
-  private boolean isStandbyException(Exception ex) {
-    Exception exception = unwrapException(ex);
+  private boolean isStandbyException(Exception exception) {
     if (exception instanceof RemoteException) {
       return ((RemoteException) exception).unwrapRemoteException()
           instanceof StandbyException;
@@ -208,24 +220,43 @@ public class RequestHedgingProxyProvider<T> extends
   }
 
   /**
-   * Unwraps the exception. <p>
+   * Unwraps the ExecutionException. <p>
    * Example:
    * <blockquote><pre>
    * if ex is
-   * ExecutionException(InvocationTargetExeption(SomeException))
+   * ExecutionException(InvocationTargetException(SomeException))
    * returns SomeException
    * </pre></blockquote>
    *
    * @return unwrapped exception
    */
-  private Exception unwrapException(Exception ex) {
+  private Exception unwrapExecutionException(ExecutionException ex) {
+    if (ex != null) {
+      Throwable cause = ex.getCause();
+      if (cause instanceof InvocationTargetException) {
+        return
+            unwrapInvocationTargetException((InvocationTargetException)cause);
+      }
+    }
+    return ex;
+
+  }
+
+  /**
+   * Unwraps the InvocationTargetException. <p>
+   * Example:
+   * <blockquote><pre>
+   * if ex is InvocationTargetException(SomeException)
+   * returns SomeException
+   * </pre></blockquote>
+   *
+   * @return unwrapped exception
+   */
+  private Exception unwrapInvocationTargetException(
+      InvocationTargetException ex) {
     if (ex != null) {
       Throwable cause = ex.getCause();
       if (cause instanceof Exception) {
-        Throwable innerCause = cause.getCause();
-        if (innerCause instanceof Exception) {
-          return (Exception) innerCause;
-        }
         return (Exception) cause;
       }
     }

+ 58 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRequestHedgingProxyProvider.java

@@ -231,6 +231,64 @@ public class TestRequestHedgingProxyProvider {
     Assert.assertEquals(1, stats[0]);
   }
 
+  @Test
+  public void testFileNotFoundExceptionWithSingleProxy() throws Exception {
+    ClientProtocol active = Mockito.mock(ClientProtocol.class);
+    Mockito
+        .when(active.getBlockLocations(Matchers.anyString(),
+            Matchers.anyLong(), Matchers.anyLong()))
+        .thenThrow(new RemoteException("java.io.FileNotFoundException",
+            "File does not exist!"));
+
+    ClientProtocol standby = Mockito.mock(ClientProtocol.class);
+    Mockito
+        .when(standby.getBlockLocations(Matchers.anyString(),
+            Matchers.anyLong(), Matchers.anyLong()))
+        .thenThrow(
+            new RemoteException("org.apache.hadoop.ipc.StandbyException",
+                "Standby NameNode"));
+
+    RequestHedgingProxyProvider<ClientProtocol> provider =
+        new RequestHedgingProxyProvider<>(conf, nnUri,
+            ClientProtocol.class, createFactory(standby, active));
+    try {
+      provider.getProxy().proxy.getBlockLocations("/tmp/test.file", 0L, 20L);
+      Assert.fail("Should fail since the active namenode throws"
+          + " FileNotFoundException!");
+    } catch (MultiException me) {
+      for (Exception ex : me.getExceptions().values()) {
+        Exception rEx = ((RemoteException) ex).unwrapRemoteException();
+        if (rEx instanceof StandbyException) {
+          continue;
+        }
+        Assert.assertTrue(rEx instanceof FileNotFoundException);
+      }
+    }
+    //Perform failover now, there will only be one active proxy now
+    provider.performFailover(active);
+    try {
+      provider.getProxy().proxy.getBlockLocations("/tmp/test.file", 0L, 20L);
+      Assert.fail("Should fail since the active namenode throws"
+          + " FileNotFoundException!");
+    } catch (RemoteException ex) {
+      Exception rEx = ex.unwrapRemoteException();
+      if (rEx instanceof StandbyException) {
+        Mockito.verify(active).getBlockLocations(Matchers.anyString(),
+            Matchers.anyLong(), Matchers.anyLong());
+        Mockito.verify(standby, Mockito.times(2))
+            .getBlockLocations(Matchers.anyString(),
+            Matchers.anyLong(), Matchers.anyLong());
+      } else {
+        Assert.assertTrue(rEx instanceof FileNotFoundException);
+        Mockito.verify(active, Mockito.times(2))
+            .getBlockLocations(Matchers.anyString(),
+            Matchers.anyLong(), Matchers.anyLong());
+        Mockito.verify(standby).getBlockLocations(Matchers.anyString(),
+            Matchers.anyLong(), Matchers.anyLong());
+      }
+    }
+  }
+
   @Test
   public void testPerformFailoverWith3Proxies() throws Exception {
     conf.set(HdfsClientConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX + "." + ns,

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml

@@ -269,10 +269,11 @@
         <Method name="visitFile" />
         <Bug pattern="NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE" />
     </Match>
-    <!-- BlockLocations are user-facing, but LocatedBlocks are not. -->
+    <!-- HdfsFileStatus is user-facing, but HdfsLocatedFileStatus is not.
+         Defensible compatibility choices over time create odd corners. -->
     <Match>
-        <Class name="org.apache.hadoop.hdfs.protocol.HdfsFileStatus" />
-        <Field name="hdfsloc" />
+        <Class name="org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus" />
+        <Field name="locations" />
         <Bug pattern="SE_TRANSIENT_FIELD_NOT_RESTORED" />
     </Match>
     <Match>

+ 11 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java

@@ -434,7 +434,8 @@ class BlockReceiver implements Closeable {
     if (duration > datanodeSlowLogThresholdMs && LOG.isWarnEnabled()) {
       LOG.warn("Slow flushOrSync took " + duration + "ms (threshold="
           + datanodeSlowLogThresholdMs + "ms), isSync:" + isSync + ", flushTotalNanos="
-          + flushTotalNanos + "ns, volume=" + getVolumeBaseUri());
+          + flushTotalNanos + "ns, volume=" + getVolumeBaseUri()
+          + ", blockId=" + replicaInfo.getBlockId());
     }
   }
 
@@ -591,7 +592,8 @@ class BlockReceiver implements Closeable {
         if (duration > datanodeSlowLogThresholdMs && LOG.isWarnEnabled()) {
           LOG.warn("Slow BlockReceiver write packet to mirror took " + duration
               + "ms (threshold=" + datanodeSlowLogThresholdMs + "ms), "
-              + "downstream DNs=" + Arrays.toString(downstreamDNs));
+              + "downstream DNs=" + Arrays.toString(downstreamDNs)
+              + ", blockId=" + replicaInfo.getBlockId());
         }
       } catch (IOException e) {
         handleMirrorOutError(e);
@@ -725,7 +727,8 @@ class BlockReceiver implements Closeable {
           if (duration > datanodeSlowLogThresholdMs && LOG.isWarnEnabled()) {
             LOG.warn("Slow BlockReceiver write data to disk cost:" + duration
                 + "ms (threshold=" + datanodeSlowLogThresholdMs + "ms), "
-                + "volume=" + getVolumeBaseUri());
+                + "volume=" + getVolumeBaseUri()
+                + ", blockId=" + replicaInfo.getBlockId());
           }
 
           if (duration > maxWriteToDiskMs) {
@@ -917,7 +920,8 @@ class BlockReceiver implements Closeable {
         if (duration > datanodeSlowLogThresholdMs && LOG.isWarnEnabled()) {
           LOG.warn("Slow manageWriterOsCache took " + duration
               + "ms (threshold=" + datanodeSlowLogThresholdMs
-              + "ms), volume=" + getVolumeBaseUri());
+              + "ms), volume=" + getVolumeBaseUri()
+              + ", blockId=" + replicaInfo.getBlockId());
         }
       }
     } catch (Throwable t) {
@@ -1629,7 +1633,9 @@ class BlockReceiver implements Closeable {
       if (duration > datanodeSlowLogThresholdMs) {
         LOG.warn("Slow PacketResponder send ack to upstream took " + duration
             + "ms (threshold=" + datanodeSlowLogThresholdMs + "ms), " + myString
-            + ", replyAck=" + replyAck);
+            + ", replyAck=" + replyAck
+            + ", downstream DNs=" + Arrays.toString(downstreamDNs)
+            + ", blockId=" + replicaInfo.getBlockId());
       } else if (LOG.isDebugEnabled()) {
         LOG.debug(myString + ", replyAck=" + replyAck);
       }

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

@@ -364,7 +364,7 @@ public class Mover {
           if (!isSnapshotPathInCurrent(fullPath)) {
             // the full path is a snapshot path but it is also included in the
             // current directory tree, thus ignore it.
-            processFile(fullPath, status, result);
+            processFile(fullPath, (HdfsLocatedFileStatus) status, result);
           }
         } catch (IOException e) {
           LOG.warn("Failed to check the status of " + parent
@@ -374,7 +374,7 @@ public class Mover {
     }
 
     /** @return true if it is necessary to run another round of migration */
-    private void processFile(String fullPath, HdfsFileStatus status,
+    private void processFile(String fullPath, HdfsLocatedFileStatus status,
         Result result) {
       byte policyId = status.getStoragePolicy();
       if (policyId == HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED) {
@@ -395,7 +395,7 @@ public class Mover {
           status.getReplication());
 
       final ErasureCodingPolicy ecPolicy = status.getErasureCodingPolicy();
-      final LocatedBlocks locatedBlocks = status.getLocatedBlocks();
+      final LocatedBlocks locatedBlocks = status.getBlockLocations();
       final boolean lastBlkComplete = locatedBlocks.isLastBlockComplete();
       List<LocatedBlock> lbs = locatedBlocks.getLocatedBlocks();
       for (int i = 0; i < lbs.size(); i++) {

+ 34 - 27
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java

@@ -33,6 +33,7 @@ import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.SnapshotException;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
@@ -255,13 +256,14 @@ class FSDirStatAndListingOp {
         listing[i] =
             createFileStatus(fsd, iip, child, childStoragePolicy, needLocation);
         listingCnt++;
-        LocatedBlocks blks = listing[i].getLocatedBlocks();
-        if (blks != null) {
-          // Once we  hit lsLimit locations, stop.
-          // This helps to prevent excessively large response payloads.
-          // Approximate #locations with locatedBlockCount() * repl_factor
-          locationBudget -=
-              blks.locatedBlockCount() * listing[i].getReplication();
+        if (listing[i] instanceof HdfsLocatedFileStatus) {
+            // Once we  hit lsLimit locations, stop.
+            // This helps to prevent excessively large response payloads.
+            // Approximate #locations with locatedBlockCount() * repl_factor
+            LocatedBlocks blks =
+                ((HdfsLocatedFileStatus)listing[i]).getBlockLocations();
+            locationBudget -= (blks == null) ? 0 :
+               blks.locatedBlockCount() * listing[i].getReplication();
         }
       }
       // truncate return array if necessary
@@ -484,26 +486,31 @@ class FSDirStatAndListingOp {
       String owner, String group, byte[] symlink, byte[] path, long fileId,
       int childrenNum, FileEncryptionInfo feInfo, byte storagePolicy,
       ErasureCodingPolicy ecPolicy, LocatedBlocks locations) {
-    return new HdfsFileStatus.Builder()
-        .length(length)
-        .isdir(isdir)
-        .replication(replication)
-        .blocksize(blocksize)
-        .mtime(mtime)
-        .atime(atime)
-        .perm(permission)
-        .flags(flags)
-        .owner(owner)
-        .group(group)
-        .symlink(symlink)
-        .path(path)
-        .fileId(fileId)
-        .children(childrenNum)
-        .feInfo(feInfo)
-        .storagePolicy(storagePolicy)
-        .ecPolicy(ecPolicy)
-        .locations(locations)
-        .build();
+    if (locations == null) {
+      return new HdfsFileStatus.Builder()
+          .length(length)
+          .isdir(isdir)
+          .replication(replication)
+          .blocksize(blocksize)
+          .mtime(mtime)
+          .atime(atime)
+          .perm(permission)
+          .flags(flags)
+          .owner(owner)
+          .group(group)
+          .symlink(symlink)
+          .path(path)
+          .fileId(fileId)
+          .children(childrenNum)
+          .feInfo(feInfo)
+          .storagePolicy(storagePolicy)
+          .ecPolicy(ecPolicy)
+          .build();
+    } else {
+      return new HdfsLocatedFileStatus(length, isdir, replication, blocksize,
+          mtime, atime, permission, flags, owner, group, symlink, path,
+          fileId, locations, childrenNum, feInfo, storagePolicy, ecPolicy);
+    }
   }
 
   private static ContentSummary getContentSummaryInt(FSDirectory fsd,

+ 11 - 12
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java

@@ -29,8 +29,6 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicLong;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
@@ -114,6 +112,8 @@ import org.apache.hadoop.security.token.delegation.DelegationKey;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * FSEditLog maintains a log of the namespace modifications.
@@ -122,9 +122,7 @@ import com.google.common.collect.Lists;
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 public class FSEditLog implements LogsPurgeable {
-
-  public static final Log LOG = LogFactory.getLog(FSEditLog.class);
-
+  public static final Logger LOG = LoggerFactory.getLogger(FSEditLog.class);
   /**
    * State machine for edit log.
    * 
@@ -329,7 +327,8 @@ public class FSEditLog implements LogsPurgeable {
       String error = String.format("Cannot start writing at txid %s " +
         "when there is a stream available for read: %s",
         segmentTxId, streams.get(0));
-      IOUtils.cleanup(LOG, streams.toArray(new EditLogInputStream[0]));
+      IOUtils.cleanupWithLogger(LOG,
+          streams.toArray(new EditLogInputStream[0]));
       throw new IllegalStateException(error);
     }
     
@@ -689,9 +688,9 @@ public class FSEditLog implements LogsPurgeable {
                 "Could not sync enough journals to persistent storage " +
                 "due to " + e.getMessage() + ". " +
                 "Unsynced transactions: " + (txid - synctxid);
-            LOG.fatal(msg, new Exception());
+            LOG.error(msg, new Exception());
             synchronized(journalSetLock) {
-              IOUtils.cleanup(LOG, journalSet);
+              IOUtils.cleanupWithLogger(LOG, journalSet);
             }
             terminate(1, msg);
           }
@@ -715,9 +714,9 @@ public class FSEditLog implements LogsPurgeable {
           final String msg =
               "Could not sync enough journals to persistent storage. "
               + "Unsynced transactions: " + (txid - synctxid);
-          LOG.fatal(msg, new Exception());
+          LOG.error(msg, new Exception());
           synchronized(journalSetLock) {
-            IOUtils.cleanup(LOG, journalSet);
+            IOUtils.cleanupWithLogger(LOG, journalSet);
           }
           terminate(1, msg);
         }
@@ -772,7 +771,7 @@ public class FSEditLog implements LogsPurgeable {
     buf.append(editLogStream.getNumSync());
     buf.append(" SyncTimes(ms): ");
     buf.append(journalSet.getSyncTimes());
-    LOG.info(buf);
+    LOG.info(buf.toString());
   }
 
   /** Record the RPC IDs if necessary */
@@ -1711,7 +1710,7 @@ public class FSEditLog implements LogsPurgeable {
       if (recovery != null) {
         // If recovery mode is enabled, continue loading even if we know we
         // can't load up to toAtLeastTxId.
-        LOG.error(e);
+        LOG.error("Exception while selecting input streams", e);
       } else {
         closeAllStreams(streams);
         throw e;

+ 7 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java

@@ -35,6 +35,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 
 import javax.management.ObjectName;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -75,7 +76,7 @@ public class SnapshotManager implements SnapshotStatsMXBean {
   public static final Log LOG = LogFactory.getLog(SnapshotManager.class);
 
   private final FSDirectory fsdir;
-  private final boolean captureOpenFiles;
+  private boolean captureOpenFiles;
   /**
    * If skipCaptureAccessTimeOnlyChange is set to true, if accessTime
    * of a file changed but there is no other modification made to the file,
@@ -121,6 +122,11 @@ public class SnapshotManager implements SnapshotStatsMXBean {
         + snapshotDiffAllowSnapRootDescendant);
   }
 
+  @VisibleForTesting
+  void setCaptureOpenFiles(boolean captureOpenFiles) {
+    this.captureOpenFiles = captureOpenFiles;
+  }
+
   /**
    * @return skipCaptureAccessTimeOnlyChange
    */

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

@@ -4883,7 +4883,7 @@
 
   <property>
     <name>dfs.federation.router.file.resolver.client.class</name>
-    <value>org.apache.hadoop.hdfs.server.federation.MockResolver</value>
+    <value>org.apache.hadoop.hdfs.server.federation.resolver.MountTableResolver</value>
     <description>
       Class to resolve files to subclusters.
     </description>

+ 5 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java

@@ -1092,11 +1092,11 @@ public class TestBlockStoragePolicy {
     return types;
   }
 
-  private void checkLocatedBlocks(HdfsFileStatus status, int blockNum,
+  private void checkLocatedBlocks(HdfsLocatedFileStatus status, int blockNum,
                                   int replicaNum, StorageType... types) {
     List<StorageType> typeList = Lists.newArrayList();
     Collections.addAll(typeList, types);
-    LocatedBlocks lbs = status.getLocatedBlocks();
+    LocatedBlocks lbs = status.getBlockLocations();
     Assert.assertEquals(blockNum, lbs.getLocatedBlocks().size());
     for (LocatedBlock lb : lbs.getLocatedBlocks()) {
       Assert.assertEquals(replicaNum, lb.getStorageTypes().length);
@@ -1127,7 +1127,7 @@ public class TestBlockStoragePolicy {
       HdfsFileStatus[] status = fs.getClient().listPaths(foo.toString(),
           HdfsFileStatus.EMPTY_NAME, true).getPartialListing();
       checkDirectoryListing(status, policyId);
-      HdfsFileStatus fooStatus = status[0];
+      HdfsLocatedFileStatus fooStatus = (HdfsLocatedFileStatus) status[0];
       checkLocatedBlocks(fooStatus, 1, 3, before);
 
       // change the replication factor to 5
@@ -1140,7 +1140,7 @@ public class TestBlockStoragePolicy {
       status = fs.getClient().listPaths(foo.toString(),
           HdfsFileStatus.EMPTY_NAME, true).getPartialListing();
       checkDirectoryListing(status, policyId);
-      fooStatus = status[0];
+      fooStatus = (HdfsLocatedFileStatus) status[0];
       checkLocatedBlocks(fooStatus, 1, numDataNodes, after);
 
       // change the replication factor back to 3
@@ -1157,7 +1157,7 @@ public class TestBlockStoragePolicy {
       status = fs.getClient().listPaths(foo.toString(),
           HdfsFileStatus.EMPTY_NAME, true).getPartialListing();
       checkDirectoryListing(status, policyId);
-      fooStatus = status[0];
+      fooStatus = (HdfsLocatedFileStatus) status[0];
       checkLocatedBlocks(fooStatus, 1, REPLICATION, before);
     } finally {
       cluster.shutdown();

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

@@ -274,7 +274,7 @@ public class TestDFSOutputStream {
     when(client.getTracer()).thenReturn(FsTracer.get(new Configuration()));
     client.clientRunning = true;
     DataStreamer stream = new DataStreamer(
-        new HdfsFileStatus.Builder().build(),
+        mock(HdfsFileStatus.class),
         mock(ExtendedBlock.class),
         client,
         "foo", null, null, null, null, null, null);

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

@@ -44,6 +44,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtocol;
@@ -311,17 +312,18 @@ public class TestStorageMover {
 
     private void verifyFile(final Path parent, final HdfsFileStatus status,
         final Byte expectedPolicyId) throws Exception {
-      byte policyId = status.getStoragePolicy();
+      HdfsLocatedFileStatus fileStatus = (HdfsLocatedFileStatus) status;
+      byte policyId = fileStatus.getStoragePolicy();
       BlockStoragePolicy policy = policies.getPolicy(policyId);
       if (expectedPolicyId != null) {
         Assert.assertEquals((byte)expectedPolicyId, policy.getId());
       }
       final List<StorageType> types = policy.chooseStorageTypes(
           status.getReplication());
-      for(LocatedBlock lb : status.getLocatedBlocks().getLocatedBlocks()) {
+      for(LocatedBlock lb : fileStatus.getBlockLocations().getLocatedBlocks()) {
         final Mover.StorageTypeDiff diff = new Mover.StorageTypeDiff(types,
             lb.getStorageTypes());
-        Assert.assertTrue(status.getFullName(parent.toString())
+        Assert.assertTrue(fileStatus.getFullName(parent.toString())
             + " with policy " + policy + " has non-empty overlap: " + diff
             + ", the corresponding block is " + lb.getBlock().getLocalBlock(),
             diff.removeOverlap(true));

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

@@ -130,7 +130,7 @@ public class TestEditLog {
 
   /**
    * A garbage mkdir op which is used for testing
-   * {@link EditLogFileInputStream#scanEditLog(File)}
+   * {@link EditLogFileInputStream#scanEditLog(File, long, boolean)}
    */
   public static class GarbageMkdirOp extends FSEditLogOp {
     public GarbageMkdirOp() {
@@ -1141,7 +1141,7 @@ public class TestEditLog {
     /**
      * Construct the failure specification. 
      * @param roll number to fail after. e.g. 1 to fail after the first roll
-     * @param loginfo index of journal to fail. 
+     * @param logindex index of journal to fail.
      */
     AbortSpec(int roll, int logindex) {
       this.roll = roll;

+ 5 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogAutoroll.java

@@ -27,8 +27,6 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_TXNS_
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_EDIT_LOG_AUTOROLL_CHECK_INTERVAL_MS;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_EDIT_LOG_AUTOROLL_MULTIPLIER_THRESHOLD;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -38,7 +36,9 @@ import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem.NameNodeEditLogRoller;
 import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.apache.log4j.Level;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.slf4j.event.Level;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -51,7 +51,7 @@ import com.google.common.base.Supplier;
 @RunWith(Parameterized.class)
 public class TestEditLogAutoroll {
   static {
-    GenericTestUtils.setLogLevel(FSEditLog.LOG, Level.ALL);
+    GenericTestUtils.setLogLevel(FSEditLog.LOG, Level.DEBUG);
   }
 
   @Parameters
@@ -74,7 +74,7 @@ public class TestEditLogAutoroll {
   private FSEditLog editLog;
   private final Random random = new Random();
 
-  private static final Log LOG = LogFactory.getLog(TestEditLog.class);
+  public static final Logger LOG = LoggerFactory.getLogger(FSEditLog.class);
 
   @Before
   public void setUp() throws Exception {

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

@@ -52,7 +52,7 @@ import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.Time;
-import org.apache.log4j.Level;
+import org.slf4j.event.Level;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
@@ -67,7 +67,7 @@ import org.mockito.stubbing.Answer;
 @RunWith(Parameterized.class)
 public class TestEditLogRace {
   static {
-    GenericTestUtils.setLogLevel(FSEditLog.LOG, Level.ALL);
+    GenericTestUtils.setLogLevel(FSEditLog.LOG, Level.DEBUG);
   }
 
   @Parameters

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

@@ -60,7 +60,7 @@ import org.mockito.Mockito;
 @RunWith(Parameterized.class)
 public class TestEditLogTailer {
   static {
-    GenericTestUtils.setLogLevel(FSEditLog.LOG, Level.ALL);
+    GenericTestUtils.setLogLevel(FSEditLog.LOG, org.slf4j.event.Level.DEBUG);
   }
 
   @Parameters
@@ -82,9 +82,9 @@ public class TestEditLogTailer {
   static final long NN_LAG_TIMEOUT = 10 * 1000;
   
   static {
-    GenericTestUtils.setLogLevel(FSImage.LOG, Level.ALL);
-    GenericTestUtils.setLogLevel(FSEditLog.LOG, Level.ALL);
-    GenericTestUtils.setLogLevel(EditLogTailer.LOG, Level.ALL);
+    GenericTestUtils.setLogLevel(FSImage.LOG, Level.DEBUG);
+    GenericTestUtils.setLogLevel(FSEditLog.LOG, org.slf4j.event.Level.DEBUG);
+    GenericTestUtils.setLogLevel(EditLogTailer.LOG, Level.DEBUG);
   }
 
   private static Configuration getConf() {

+ 117 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestOpenFilesWithSnapshot.java

@@ -843,6 +843,123 @@ public class TestOpenFilesWithSnapshot {
         hbaseFileCksumBeforeTruncate, hbaseFileCksumS3);
   }
 
+  private Path createSnapshot(Path snapRootDir, String snapName,
+      String fileName) throws Exception {
+    final Path snap1Dir = SnapshotTestHelper.createSnapshot(
+        fs, snapRootDir, snapName);
+    return new Path(snap1Dir, fileName);
+  }
+
+  private void verifyFileSize(long fileSize, Path... filePaths) throws
+      IOException {
+    for (Path filePath : filePaths) {
+      Assert.assertEquals(fileSize, fs.getFileStatus(filePath).getLen());
+    }
+  }
+
+  /**
+   * Verify open files captured in the snapshots across config disable
+   * and enable.
+   */
+  @Test
+  public void testOpenFilesWithMixedConfig() throws Exception {
+    final Path snapRootDir = new Path("/level_0_A");
+    final String flumeFileName = "flume.log";
+    final String snap1Name = "s1";
+    final String snap2Name = "s2";
+    final String snap3Name = "s3";
+    final String snap4Name = "s4";
+    final String snap5Name = "s5";
+
+    // Create files and open streams
+    final Path flumeFile = new Path(snapRootDir, flumeFileName);
+    createFile(flumeFile);
+    FSDataOutputStream flumeOutputStream = fs.append(flumeFile);
+
+    // 1. Disable capture open files
+    cluster.getNameNode().getNamesystem()
+        .getSnapshotManager().setCaptureOpenFiles(false);
+
+    // Create Snapshot S1
+    final Path flumeS1Path = createSnapshot(snapRootDir,
+        snap1Name, flumeFileName);
+
+    // Verify if Snap S1 file length is same as the the current versions
+    verifyFileSize(FILELEN, flumeS1Path);
+
+    // Write more data to files
+    long flumeFileWrittenDataLength = FILELEN;
+    int newWriteLength = (int) (BLOCKSIZE * 1.5);
+    byte[] buf = new byte[newWriteLength];
+    Random random = new Random();
+    random.nextBytes(buf);
+    flumeFileWrittenDataLength += writeToStream(flumeOutputStream, buf);
+
+    // Create Snapshot S2
+    final Path flumeS2Path = createSnapshot(snapRootDir,
+        snap2Name, flumeFileName);
+
+    // Since capture open files was disabled, all snapshots paths
+    // and the current version should have same file lengths.
+    verifyFileSize(flumeFileWrittenDataLength,
+        flumeFile, flumeS2Path, flumeS1Path);
+
+    // 2. Enable capture open files
+    cluster.getNameNode().getNamesystem()
+        .getSnapshotManager() .setCaptureOpenFiles(true);
+
+    // Write more data to files
+    flumeFileWrittenDataLength += writeToStream(flumeOutputStream, buf);
+    long flumeFileLengthAfterS3 = flumeFileWrittenDataLength;
+
+    // Create Snapshot S3
+    final Path flumeS3Path = createSnapshot(snapRootDir,
+        snap3Name, flumeFileName);
+
+    // Since open files captured in the previous snapshots were with config
+    // disabled, their file lengths are now same as the current version.
+    // With the config turned on, any new data written to the open files
+    // will no more reflect in the current version or old snapshot paths.
+    verifyFileSize(flumeFileWrittenDataLength, flumeFile, flumeS3Path,
+        flumeS2Path, flumeS1Path);
+
+    // Write more data to files
+    flumeFileWrittenDataLength += writeToStream(flumeOutputStream, buf);
+
+    // Create Snapshot S4
+    final Path flumeS4Path = createSnapshot(snapRootDir,
+        snap4Name, flumeFileName);
+
+    // Verify S4 has the latest data
+    verifyFileSize(flumeFileWrittenDataLength, flumeFile, flumeS4Path);
+
+    // But, open files captured as of Snapshot S3 and before should
+    // have their old file lengths intact.
+    verifyFileSize(flumeFileLengthAfterS3, flumeS3Path,
+        flumeS2Path, flumeS1Path);
+
+    long flumeFileLengthAfterS4 =  flumeFileWrittenDataLength;
+
+    // 3. Disable capture open files
+    cluster.getNameNode().getNamesystem()
+        .getSnapshotManager() .setCaptureOpenFiles(false);
+
+    // Create Snapshot S5
+    final Path flumeS5Path = createSnapshot(snapRootDir,
+        snap5Name, flumeFileName);
+
+    flumeFileWrittenDataLength += writeToStream(flumeOutputStream, buf);
+
+    // Since capture open files was disabled, any snapshots taken after the
+    // config change and the current version should have same file lengths
+    // for the open files.
+    verifyFileSize(flumeFileWrittenDataLength, flumeFile, flumeS5Path);
+
+    // But, the old snapshots taken before the config disable should
+    // continue to be consistent.
+    verifyFileSize(flumeFileLengthAfterS4, flumeS4Path);
+  }
+
   private void restartNameNode() throws Exception {
     cluster.triggerBlockReports();
     NameNode nameNode = cluster.getNameNode();

+ 3 - 2
hadoop-project/pom.xml

@@ -120,7 +120,7 @@
 
     <!-- Plugin versions and config -->
     <maven-surefire-plugin.argLine>-Xmx2048m -XX:+HeapDumpOnOutOfMemoryError</maven-surefire-plugin.argLine>
-    <maven-surefire-plugin.version>2.17</maven-surefire-plugin.version>
+    <maven-surefire-plugin.version>2.20.1</maven-surefire-plugin.version>
     <maven-surefire-report-plugin.version>${maven-surefire-plugin.version}</maven-surefire-report-plugin.version>
     <maven-failsafe-plugin.version>${maven-surefire-plugin.version}</maven-failsafe-plugin.version>
 
@@ -1647,6 +1647,7 @@
             <DYLD_LIBRARY_PATH>${env.DYLD_LIBRARY_PATH}:${project.build.directory}/native/target/usr/local/lib:${hadoop.common.build.dir}/native/target/usr/local/lib</DYLD_LIBRARY_PATH>
             <MALLOC_ARENA_MAX>4</MALLOC_ARENA_MAX>
           </environmentVariables>
+          <trimStackTrace>false</trimStackTrace>
           <systemPropertyVariables>
 
             <hadoop.log.dir>${project.build.directory}/log</hadoop.log.dir>
@@ -1657,7 +1658,7 @@
             <test.build.data>${test.build.data}</test.build.data>
             <test.build.webapps>${test.build.webapps}</test.build.webapps>
             <test.cache.data>${test.cache.data}</test.cache.data>
-            <test.build.classes>${test.build.classes}</test.build.classes>
+            <test.build.classes>${project.build.directory}/test-classes</test.build.classes>
 
             <java.net.preferIPv4Stack>true</java.net.preferIPv4Stack>
             <java.security.krb5.conf>${project.build.directory}/test-classes/krb5.conf</java.security.krb5.conf>

+ 4 - 0
hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystemStore.java

@@ -53,6 +53,7 @@ import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.util.VersionInfo;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -101,6 +102,9 @@ public class AliyunOSSFileSystemStore {
         ESTABLISH_TIMEOUT_DEFAULT));
     clientConf.setSocketTimeout(conf.getInt(SOCKET_TIMEOUT_KEY,
         SOCKET_TIMEOUT_DEFAULT));
+    clientConf.setUserAgent(
+        conf.get(USER_AGENT_PREFIX, USER_AGENT_PREFIX_DEFAULT) + ", Hadoop/"
+            + VersionInfo.getVersion());
 
     String proxyHost = conf.getTrimmed(PROXY_HOST_KEY, "");
     int proxyPort = conf.getInt(PROXY_PORT_KEY, -1);

+ 7 - 0
hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/Constants.java

@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.fs.aliyun.oss;
 
+import com.aliyun.oss.common.utils.VersionInfoUtils;
+
 /**
  * ALL configuration constants for OSS filesystem.
  */
@@ -26,6 +28,11 @@ public final class Constants {
   private Constants() {
   }
 
+  // User agent
+  public static final String USER_AGENT_PREFIX = "fs.oss.user.agent.prefix";
+  public static final String USER_AGENT_PREFIX_DEFAULT =
+          VersionInfoUtils.getDefaultUserAgent();
+
   // Class of credential provider
   public static final String ALIYUN_OSS_CREDENTIALS_PROVIDER_KEY =
       "fs.oss.credentials.provider";

+ 1 - 1
hadoop-tools/hadoop-aliyun/src/site/markdown/tools/hadoop-aliyun/index.md

@@ -274,7 +274,7 @@ XInclude inclusion. Here is an example of `contract-test-options.xml`:
 
       <property>
         <name>fs.oss.impl</name>
-        <value>org.apache.hadoop.fs.aliyun.AliyunOSSFileSystem</value>
+        <value>org.apache.hadoop.fs.aliyun.oss.AliyunOSSFileSystem</value>
       </property>
 
       <property>

+ 2 - 0
hadoop-tools/hadoop-aws/pom.xml

@@ -153,6 +153,7 @@
                   <reuseForks>false</reuseForks>
                   <argLine>${maven-surefire-plugin.argLine} -DminiClusterDedicatedDirs=true</argLine>
                   <forkedProcessTimeoutInSeconds>${fs.s3a.scale.test.timeout}</forkedProcessTimeoutInSeconds>
+                  <trimStackTrace>false</trimStackTrace>
                   <systemPropertyVariables>
                     <!-- Tell tests that they are being executed in parallel -->
                     <test.parallel.execution>true</test.parallel.execution>
@@ -207,6 +208,7 @@
                 </goals>
                 <configuration>
                   <forkedProcessTimeoutInSeconds>${fs.s3a.scale.test.timeout}</forkedProcessTimeoutInSeconds>
+                  <trimStackTrace>false</trimStackTrace>
                   <systemPropertyVariables>
                     <!-- Tell tests that they are being executed sequentially -->
                     <test.parallel.execution>false</test.parallel.execution>

+ 3 - 0
hadoop-tools/hadoop-azure/pom.xml

@@ -349,6 +349,7 @@
                   <reuseForks>false</reuseForks>
                   <argLine>${maven-surefire-plugin.argLine} -DminiClusterDedicatedDirs=true</argLine>
                   <forkedProcessTimeoutInSeconds>${fs.azure.scale.test.timeout}</forkedProcessTimeoutInSeconds>
+                  <trimStackTrace>false</trimStackTrace>
                   <systemPropertyVariables>
                     <!-- Tell tests that they are being executed in parallel -->
                     <test.parallel.execution>true</test.parallel.execution>
@@ -404,6 +405,7 @@
                 </goals>
                 <configuration>
                   <forkedProcessTimeoutInSeconds>${fs.azure.scale.test.timeout}</forkedProcessTimeoutInSeconds>
+                  <trimStackTrace>false</trimStackTrace>
                   <systemPropertyVariables>
                     <test.parallel.execution>false</test.parallel.execution>
                     <fs.azure.scale.test.enabled>${fs.azure.scale.test.enabled}</fs.azure.scale.test.enabled>
@@ -454,6 +456,7 @@
                     <fs.azure.scale.test.timeout>${fs.azure.scale.test.timeout}</fs.azure.scale.test.timeout>
                   </systemPropertyVariables>
                   <forkedProcessTimeoutInSeconds>${fs.azure.scale.test.timeout}</forkedProcessTimeoutInSeconds>
+                  <trimStackTrace>false</trimStackTrace>
                 </configuration>
               </execution>
             </executions>

+ 1 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ResourceRequest.java

@@ -630,8 +630,7 @@ public abstract class ResourceRequest implements Comparable<ResourceRequest> {
       if (other.getExecutionTypeRequest() != null) {
         return false;
       }
-    } else if (!execTypeRequest.getExecutionType()
-        .equals(other.getExecutionTypeRequest().getExecutionType())) {
+    } else if (!execTypeRequest.equals(other.getExecutionTypeRequest())) {
       return false;
     }
 

+ 14 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java

@@ -1735,7 +1735,7 @@ public class YarnConfiguration extends Configuration {
       DOCKER_CONTAINER_RUNTIME_PREFIX + "enable-userremapping.allowed";
 
   /** Set enable user remapping as false by default. */
-  public static final boolean DEFAULT_NM_DOCKER_ENABLE_USER_REMAPPING = false;
+  public static final boolean DEFAULT_NM_DOCKER_ENABLE_USER_REMAPPING = true;
 
   /** lower limit for acceptable uids of user remapped user. */
   public static final String NM_DOCKER_USER_REMAPPING_UID_THRESHOLD =
@@ -2096,6 +2096,9 @@ public class YarnConfiguration extends Configuration {
   public static final String DEFAULT_AMRM_PROXY_INTERCEPTOR_CLASS_PIPELINE =
       "org.apache.hadoop.yarn.server.nodemanager.amrmproxy."
           + "DefaultRequestInterceptor";
+  public static final String AMRM_PROXY_HA_ENABLED = NM_PREFIX
+      + "amrmproxy.ha.enable";
+  public static final boolean DEFAULT_AMRM_PROXY_HA_ENABLED = false;
 
   /**
    * Default platform-agnostic CLASSPATH for YARN applications. A
@@ -2930,6 +2933,11 @@ public class YarnConfiguration extends Configuration {
   public static final String FEDERATION_CACHE_TIME_TO_LIVE_SECS =
       FEDERATION_PREFIX + "cache-ttl.secs";
 
+  public static final String FEDERATION_REGISTRY_BASE_KEY =
+      FEDERATION_PREFIX + "registry.base-dir";
+  public static final String DEFAULT_FEDERATION_REGISTRY_BASE_KEY =
+      "yarnfederation/";
+
   // 5 minutes
   public static final int DEFAULT_FEDERATION_CACHE_TIME_TO_LIVE_SECS = 5 * 60;
 
@@ -3087,6 +3095,11 @@ public class YarnConfiguration extends Configuration {
   // Other Configs
   ////////////////////////////////
 
+  public static final String YARN_REGISTRY_CLASS =
+      YARN_PREFIX + "registry.class";
+  public static final String DEFAULT_YARN_REGISTRY_CLASS =
+      "org.apache.hadoop.registry.client.impl.FSRegistryOperationsService";
+
   /**
    * Use YARN_CLIENT_APPLICATION_CLIENT_PROTOCOL_POLL_INTERVAL_MS instead.
    * The interval of the yarn client's querying application state after

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/test/java/org/apache/hadoop/yarn/service/TestApiServer.java

@@ -62,8 +62,8 @@ public class TestApiServer {
         this.apiServer.getClass().isAnnotationPresent(Path.class));
     final Path path = this.apiServer.getClass()
         .getAnnotation(Path.class);
-    assertEquals("The path has /ws/v1 annotation", path.value(),
-        "/ws/v1");
+    assertEquals("The path has /v1 annotation", path.value(),
+        "/v1");
   }
 
   @Test

+ 23 - 27
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceScheduler.java

@@ -132,7 +132,6 @@ public class ServiceScheduler extends CompositeService {
   private AMRMClientAsync<AMRMClient.ContainerRequest> amRMClient;
   private NMClientAsync nmClient;
   private AsyncDispatcher dispatcher;
-  AsyncDispatcher compInstanceDispatcher;
   private YarnRegistryViewForProviders yarnRegistryOperations;
   private ServiceContext context;
   private ContainerLaunchService containerLaunchService;
@@ -152,7 +151,7 @@ public class ServiceScheduler extends CompositeService {
     yarnRegistryOperations =
         createYarnRegistryOperations(context, registryClient);
 
-    // register metrics
+    // register metrics,
     serviceMetrics = ServiceMetrics
         .register(app.getName(), "Metrics for service");
     serviceMetrics.tag("type", "Metrics type [component or service]", "service");
@@ -167,14 +166,11 @@ public class ServiceScheduler extends CompositeService {
     dispatcher = new AsyncDispatcher("Component  dispatcher");
     dispatcher.register(ComponentEventType.class,
         new ComponentEventHandler());
+    dispatcher.register(ComponentInstanceEventType.class,
+        new ComponentInstanceEventHandler());
     dispatcher.setDrainEventsOnStop();
     addIfService(dispatcher);
 
-    compInstanceDispatcher =
-        new AsyncDispatcher("CompInstance dispatcher");
-    compInstanceDispatcher.register(ComponentInstanceEventType.class,
-        new ComponentInstanceEventHandler());
-    addIfService(compInstanceDispatcher);
     containerLaunchService = new ContainerLaunchService(context.fs);
     addService(containerLaunchService);
 
@@ -277,10 +273,10 @@ public class ServiceScheduler extends CompositeService {
   }
 
   private void recoverComponents(RegisterApplicationMasterResponse response) {
-    List<Container> recoveredContainers = response
+    List<Container> containersFromPrevAttempt = response
         .getContainersFromPreviousAttempts();
     LOG.info("Received {} containers from previous attempt.",
-        recoveredContainers.size());
+        containersFromPrevAttempt.size());
     Map<String, ServiceRecord> existingRecords = new HashMap<>();
     List<String> existingComps = null;
     try {
@@ -302,9 +298,8 @@ public class ServiceScheduler extends CompositeService {
         }
       }
     }
-    for (Container container : recoveredContainers) {
-      LOG.info("Handling container {} from previous attempt",
-          container.getId());
+    for (Container container : containersFromPrevAttempt) {
+      LOG.info("Handling {} from previous attempt", container.getId());
       ServiceRecord record = existingRecords.get(RegistryPathUtils
           .encodeYarnID(container.getId().toString()));
       if (record != null) {
@@ -487,16 +482,21 @@ public class ServiceScheduler extends CompositeService {
             new ComponentEvent(comp.getName(), CONTAINER_ALLOCATED)
                 .setContainer(container);
         dispatcher.getEventHandler().handle(event);
-        Collection<AMRMClient.ContainerRequest> requests = amRMClient
-            .getMatchingRequests(container.getAllocationRequestId());
-        LOG.info("[COMPONENT {}]: {} outstanding container requests.",
-            comp.getName(), requests.size());
-        // remove the corresponding request
-        if (requests.iterator().hasNext()) {
-          LOG.info("[COMPONENT {}]: removing one container request.", comp
-              .getName());
-          AMRMClient.ContainerRequest request = requests.iterator().next();
-          amRMClient.removeContainerRequest(request);
+        try {
+          Collection<AMRMClient.ContainerRequest> requests = amRMClient
+              .getMatchingRequests(container.getAllocationRequestId());
+          LOG.info("[COMPONENT {}]: remove {} outstanding container requests " +
+                  "for allocateId " + container.getAllocationRequestId(),
+              comp.getName(), requests.size());
+          // remove the corresponding request
+          if (requests.iterator().hasNext()) {
+            AMRMClient.ContainerRequest request = requests.iterator().next();
+            amRMClient.removeContainerRequest(request);
+          }
+        } catch(Exception e) {
+          //TODO Due to YARN-7490, exception may be thrown, catch and ignore for
+          //now.
+          LOG.error("Exception when removing the matching requests. ", e);
         }
       }
     }
@@ -569,7 +569,7 @@ public class ServiceScheduler extends CompositeService {
       }
       ComponentEvent event =
           new ComponentEvent(instance.getCompName(), CONTAINER_STARTED)
-              .setInstance(instance);
+              .setInstance(instance).setContainerId(containerId);
       dispatcher.getEventHandler().handle(event);
     }
 
@@ -649,10 +649,6 @@ public class ServiceScheduler extends CompositeService {
     liveInstances.remove(containerId);
   }
 
-  public AsyncDispatcher getCompInstanceDispatcher() {
-    return compInstanceDispatcher;
-  }
-
   public YarnRegistryViewForProviders getYarnRegistryOperations() {
     return yarnRegistryOperations;
   }

+ 33 - 18
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Component.java

@@ -26,6 +26,8 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Objects;
 
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlElement;
 import javax.xml.bind.annotation.XmlRootElement;
 
@@ -46,22 +48,53 @@ import org.apache.hadoop.classification.InterfaceStability;
 @ApiModel(description = "One or more components of the service. If the service is HBase say, then the component can be a simple role like master or regionserver. If the service is a complex business webapp then a component can be other services say Kafka or Storm. Thereby it opens up the support for complex and nested services.")
 @javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00")
 @XmlRootElement
+@XmlAccessorType(XmlAccessType.FIELD)
 @JsonInclude(JsonInclude.Include.NON_NULL)
 public class Component implements Serializable {
   private static final long serialVersionUID = -8430058381509087805L;
 
+  @JsonProperty("name")
   private String name = null;
+
+  @JsonProperty("dependencies")
   private List<String> dependencies = new ArrayList<String>();
+
+  @JsonProperty("readiness_check")
+  @XmlElement(name = "readiness_check")
   private ReadinessCheck readinessCheck = null;
+
+  @JsonProperty("artifact")
   private Artifact artifact = null;
+
+  @JsonProperty("launch_command")
+  @XmlElement(name = "launch_command")
   private String launchCommand = null;
+
+  @JsonProperty("resource")
   private Resource resource = null;
+
+  @JsonProperty("number_of_containers")
+  @XmlElement(name = "number_of_containers")
   private Long numberOfContainers = null;
+
+  @JsonProperty("run_privileged_container")
+  @XmlElement(name = "run_privileged_container")
   private Boolean runPrivilegedContainer = false;
+
+  @JsonProperty("placement_policy")
+  @XmlElement(name = "placement_policy")
   private PlacementPolicy placementPolicy = null;
+
+  @JsonProperty("state")
   private ComponentState state = ComponentState.FLEXING;
+
+  @JsonProperty("configuration")
   private Configuration configuration = new Configuration();
+
+  @JsonProperty("quicklinks")
   private List<String> quicklinks = new ArrayList<String>();
+
+  @JsonProperty("containers")
   private List<Container> containers =
       Collections.synchronizedList(new ArrayList<Container>());
 
@@ -74,7 +107,6 @@ public class Component implements Serializable {
   }
 
   @ApiModelProperty(example = "null", required = true, value = "Name of the service component (mandatory).")
-  @JsonProperty("name")
   public String getName() {
     return name;
   }
@@ -95,7 +127,6 @@ public class Component implements Serializable {
   }
 
   @ApiModelProperty(example = "null", value = "An array of service components which should be in READY state (as defined by readiness check), before this component can be started. The dependencies across all components of an service should be represented as a DAG.")
-  @JsonProperty("dependencies")
   public List<String> getDependencies() {
     return dependencies;
   }
@@ -113,12 +144,10 @@ public class Component implements Serializable {
   }
 
   @ApiModelProperty(example = "null", value = "Readiness check for this component.")
-  @JsonProperty("readiness_check")
   public ReadinessCheck getReadinessCheck() {
     return readinessCheck;
   }
 
-  @XmlElement(name = "readiness_check")
   public void setReadinessCheck(ReadinessCheck readinessCheck) {
     this.readinessCheck = readinessCheck;
   }
@@ -133,7 +162,6 @@ public class Component implements Serializable {
   }
 
   @ApiModelProperty(example = "null", value = "Artifact of the component (optional). If not specified, the service level global artifact takes effect.")
-  @JsonProperty("artifact")
   public Artifact getArtifact() {
     return artifact;
   }
@@ -153,12 +181,10 @@ public class Component implements Serializable {
   }
 
   @ApiModelProperty(example = "null", value = "The custom launch command of this component (optional). When specified at the component level, it overrides the value specified at the global level (if any).")
-  @JsonProperty("launch_command")
   public String getLaunchCommand() {
     return launchCommand;
   }
 
-  @XmlElement(name = "launch_command")
   public void setLaunchCommand(String launchCommand) {
     this.launchCommand = launchCommand;
   }
@@ -173,7 +199,6 @@ public class Component implements Serializable {
   }
 
   @ApiModelProperty(example = "null", value = "Resource of this component (optional). If not specified, the service level global resource takes effect.")
-  @JsonProperty("resource")
   public Resource getResource() {
     return resource;
   }
@@ -192,18 +217,15 @@ public class Component implements Serializable {
   }
 
   @ApiModelProperty(example = "null", value = "Number of containers for this component (optional). If not specified, the service level global number_of_containers takes effect.")
-  @JsonProperty("number_of_containers")
   public Long getNumberOfContainers() {
     return numberOfContainers;
   }
 
-  @XmlElement(name = "number_of_containers")
   public void setNumberOfContainers(Long numberOfContainers) {
     this.numberOfContainers = numberOfContainers;
   }
 
   @ApiModelProperty(example = "null", value = "Containers of a started component. Specifying a value for this attribute for the POST payload raises a validation error. This blob is available only in the GET response of a started service.")
-  @JsonProperty("containers")
   public List<Container> getContainers() {
     return containers;
   }
@@ -237,12 +259,10 @@ public class Component implements Serializable {
   }
 
   @ApiModelProperty(example = "null", value = "Run all containers of this component in privileged mode (YARN-4262).")
-  @JsonProperty("run_privileged_container")
   public Boolean getRunPrivilegedContainer() {
     return runPrivilegedContainer;
   }
 
-  @XmlElement(name = "run_privileged_container")
   public void setRunPrivilegedContainer(Boolean runPrivilegedContainer) {
     this.runPrivilegedContainer = runPrivilegedContainer;
   }
@@ -259,12 +279,10 @@ public class Component implements Serializable {
   }
 
   @ApiModelProperty(example = "null", value = "Advanced scheduling and placement policies for all containers of this component (optional). If not specified, the service level placement_policy takes effect. Refer to the description at the global level for more details.")
-  @JsonProperty("placement_policy")
   public PlacementPolicy getPlacementPolicy() {
     return placementPolicy;
   }
 
-  @XmlElement(name = "placement_policy")
   public void setPlacementPolicy(PlacementPolicy placementPolicy) {
     this.placementPolicy = placementPolicy;
   }
@@ -278,7 +296,6 @@ public class Component implements Serializable {
   }
 
   @ApiModelProperty(example = "null", value = "Config properties for this component.")
-  @JsonProperty("configuration")
   public Configuration getConfiguration() {
     return configuration;
   }
@@ -297,7 +314,6 @@ public class Component implements Serializable {
   }
 
   @ApiModelProperty(example = "null", value = "A list of quicklink keys defined at the service level, and to be resolved by this component.")
-  @JsonProperty("quicklinks")
   public List<String> getQuicklinks() {
     return quicklinks;
   }
@@ -312,7 +328,6 @@ public class Component implements Serializable {
   }
 
   @ApiModelProperty(example = "null", value = "State of the component.")
-  @JsonProperty("state")
   public ComponentState getState() {
     return state;
   }

+ 12 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ReadinessCheck.java

@@ -25,7 +25,11 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.Objects;
 
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
 import javax.xml.bind.annotation.XmlEnum;
+import javax.xml.bind.annotation.XmlRootElement;
 import javax.xml.bind.annotation.XmlType;
 
 import com.fasterxml.jackson.annotation.JsonProperty;
@@ -43,6 +47,8 @@ import org.apache.hadoop.classification.InterfaceStability;
 @InterfaceStability.Unstable
 @ApiModel(description = "A custom command or a pluggable helper container to determine the readiness of a container of a component. Readiness for every service is different. Hence the need for a simple interface, with scope to support advanced usecases.")
 @javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00")
+@XmlRootElement
+@XmlAccessorType(XmlAccessType.FIELD)
 public class ReadinessCheck implements Serializable {
   private static final long serialVersionUID = -3836839816887186801L;
 
@@ -68,8 +74,14 @@ public class ReadinessCheck implements Serializable {
     }
   }
 
+  @JsonProperty("type")
+  @XmlElement(name = "type")
   private TypeEnum type = null;
+  @JsonProperty("properties")
+  @XmlElement(name = "properties")
   private Map<String, String> properties = new HashMap<String, String>();
+  @JsonProperty("artifact")
+  @XmlElement(name = "artifact")
   private Artifact artifact = null;
 
   /**
@@ -82,7 +94,6 @@ public class ReadinessCheck implements Serializable {
   }
 
   @ApiModelProperty(example = "null", value = "E.g. HTTP (YARN will perform a simple REST call at a regular interval and expect a 204 No content).")
-  @JsonProperty("type")
   public TypeEnum getType() {
     return type;
   }
@@ -129,7 +140,6 @@ public class ReadinessCheck implements Serializable {
   }
 
   @ApiModelProperty(example = "null", value = "Artifact of the pluggable readiness check helper container (optional). If specified, this helper container typically hosts the http uri and encapsulates the complex scripts required to perform actual container readiness check. At the end it is expected to respond a 204 No content just like the simplified use case. This pluggable framework benefits service owners who can run services without any packaging modifications. Note, artifacts of type docker only is supported for now.")
-  @JsonProperty("artifact")
   public Artifact getArtifact() {
     return artifact;
   }

+ 9 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Service.java

@@ -26,6 +26,8 @@ import io.swagger.annotations.ApiModelProperty;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlElement;
 import javax.xml.bind.annotation.XmlRootElement;
 import java.util.ArrayList;
@@ -43,6 +45,7 @@ import java.util.Objects;
 @ApiModel(description = "An Service resource has the following attributes.")
 @javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00")
 @XmlRootElement
+@XmlAccessorType(XmlAccessType.FIELD)
 @JsonInclude(JsonInclude.Include.NON_NULL)
 @JsonPropertyOrder({ "name", "state", "resource", "number_of_containers",
     "lifetime", "containers" })
@@ -53,9 +56,15 @@ public class Service extends BaseResource {
   private String id = null;
   private Artifact artifact = null;
   private Resource resource = null;
+  @JsonProperty("launch_time")
+  @XmlElement(name = "launch_time")
   private Date launchTime = null;
+  @JsonProperty("number_of_running_containers")
+  @XmlElement(name = "number_of_running_containers")
   private Long numberOfRunningContainers = null;
   private Long lifetime = null;
+  @JsonProperty("placement_policy")
+  @XmlElement(name = "placement_policy")
   private PlacementPolicy placementPolicy = null;
   private List<Component> components = new ArrayList<>();
   private Configuration configuration = new Configuration();
@@ -148,12 +157,10 @@ public class Service extends BaseResource {
   }
 
   @ApiModelProperty(example = "null", value = "The time when the service was created, e.g. 2016-03-16T01:01:49.000Z.")
-  @JsonProperty("launch_time")
   public Date getLaunchTime() {
     return launchTime == null ? null : (Date) launchTime.clone();
   }
 
-  @XmlElement(name = "launch_time")
   public void setLaunchTime(Date launchTime) {
     this.launchTime = launchTime == null ? null : (Date) launchTime.clone();
   }
@@ -171,12 +178,10 @@ public class Service extends BaseResource {
   }
 
   @ApiModelProperty(example = "null", value = "In get response this provides the total number of running containers for this service (across all components) at the time of request. Note, a subsequent request can return a different number as and when more containers get allocated until it reaches the total number of containers or if a flex request has been made between the two requests.")
-  @JsonProperty("number_of_running_containers")
   public Long getNumberOfRunningContainers() {
     return numberOfRunningContainers;
   }
 
-  @XmlElement(name = "number_of_running_containers")
   public void setNumberOfRunningContainers(Long numberOfRunningContainers) {
     this.numberOfRunningContainers = numberOfRunningContainers;
   }
@@ -215,12 +220,10 @@ public class Service extends BaseResource {
   }
 
   @ApiModelProperty(example = "null", value = "Advanced scheduling and placement policies (optional). If not specified, it defaults to the default placement policy of the service owner. The design of placement policies are in the works. It is not very clear at this point, how policies in conjunction with labels be exposed to service owners. This is a placeholder for now. The advanced structure of this attribute will be determined by YARN-4902.")
-  @JsonProperty("placement_policy")
   public PlacementPolicy getPlacementPolicy() {
     return placementPolicy;
   }
 
-  @XmlElement(name = "placement_policy")
   public void setPlacementPolicy(PlacementPolicy placementPolicy) {
     this.placementPolicy = placementPolicy;
   }

+ 5 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/ServiceClient.java

@@ -559,7 +559,7 @@ public class ServiceClient extends AppAdminClient implements SliderExitCodes,
     Map<String, String> env = addAMEnv();
 
     // create AM CLI
-    String cmdStr = buildCommandLine(serviceName, conf, appRootDir, hasAMLog4j);
+    String cmdStr = buildCommandLine(app, conf, appRootDir, hasAMLog4j);
     submissionContext.setResource(Resource.newInstance(YarnServiceConf
         .getLong(YarnServiceConf.AM_RESOURCE_MEM,
             YarnServiceConf.DEFAULT_KEY_AM_RESOURCE_MEM, app.getConfiguration(),
@@ -624,12 +624,12 @@ public class ServiceClient extends AppAdminClient implements SliderExitCodes,
     LOG.debug(builder.toString());
   }
 
-  private String buildCommandLine(String serviceName, Configuration conf,
+  private String buildCommandLine(Service app, Configuration conf,
       Path appRootDir, boolean hasSliderAMLog4j) throws BadConfigException {
     JavaCommandLineBuilder CLI = new JavaCommandLineBuilder();
     CLI.forceIPv4().headless();
-    //TODO CLI.setJVMHeap
-    //TODO CLI.addJVMOPTS
+    CLI.setJVMOpts(YarnServiceConf.get(YarnServiceConf.JVM_OPTS, null,
+        app.getConfiguration(), conf));
     if (hasSliderAMLog4j) {
       CLI.sysprop(SYSPROP_LOG4J_CONFIGURATION, YARN_SERVICE_LOG4J_FILENAME);
       CLI.sysprop(SYSPROP_LOG_DIR, ApplicationConstants.LOG_DIR_EXPANSION_VAR);
@@ -637,7 +637,7 @@ public class ServiceClient extends AppAdminClient implements SliderExitCodes,
     CLI.add(ServiceMaster.class.getCanonicalName());
     //TODO debugAM CLI.add(Arguments.ARG_DEBUG)
     CLI.add("-" + ServiceMaster.YARNFILE_OPTION, new Path(appRootDir,
-        serviceName + ".json"));
+        app.getName() + ".json"));
     // pass the registry binding
     CLI.addConfOptionToCLI(conf, RegistryConstants.KEY_REGISTRY_ZK_ROOT,
         RegistryConstants.DEFAULT_ZK_REGISTRY_ROOT);

+ 22 - 36
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/Component.java

@@ -82,7 +82,8 @@ public class Component implements EventHandler<ComponentEvent> {
   private Map<String, ComponentInstance> compInstances =
       new ConcurrentHashMap<>();
   // component instances to be assigned with a container
-  private List<ComponentInstance> pendingInstances = new LinkedList<>();
+  private List<ComponentInstance> pendingInstances =
+      Collections.synchronizedList(new LinkedList<>());
   private ContainerFailureTracker failureTracker;
   private Probe probe;
   private final ReentrantReadWriteLock.ReadLock readLock;
@@ -94,7 +95,7 @@ public class Component implements EventHandler<ComponentEvent> {
 
   private StateMachine<ComponentState, ComponentEventType, ComponentEvent>
       stateMachine;
-  private AsyncDispatcher compInstanceDispatcher;
+  private AsyncDispatcher dispatcher;
   private static final StateMachineFactory<Component, ComponentState, ComponentEventType, ComponentEvent>
       stateMachineFactory =
       new StateMachineFactory<Component, ComponentState, ComponentEventType, ComponentEvent>(
@@ -149,7 +150,7 @@ public class Component implements EventHandler<ComponentEvent> {
     this.readLock = lock.readLock();
     this.writeLock = lock.writeLock();
     this.stateMachine = stateMachineFactory.make(this);
-    compInstanceDispatcher = scheduler.getCompInstanceDispatcher();
+    dispatcher = scheduler.getDispatcher();
     failureTracker =
         new ContainerFailureTracker(context, this);
     probe = MonitorUtils.getProbe(componentSpec.getReadinessCheck());
@@ -256,30 +257,18 @@ public class Component implements EventHandler<ComponentEvent> {
         component.releaseContainer(container);
         return;
       }
-      if (instance.hasContainer()) {
-        LOG.info(
-            "[COMPONENT {}]: Instance {} already has container, release " +
-                "surplus container {}",
-            instance.getCompName(), instance.getCompInstanceId(), container
-                .getId());
-        component.releaseContainer(container);
-        return;
-      }
+
       component.pendingInstances.remove(instance);
-      LOG.info("[COMPONENT {}]: Recovered {} for component instance {} on " +
-              "host {}, num pending component instances reduced to {} ",
-          component.getName(), container.getId(), instance
-              .getCompInstanceName(), container.getNodeId(), component
-              .pendingInstances.size());
       instance.setContainer(container);
       ProviderUtils.initCompInstanceDir(component.getContext().fs, instance);
       component.getScheduler().addLiveCompInstance(container.getId(), instance);
-      LOG.info("[COMPONENT {}]: Marking {} as started for component " +
-          "instance {}", component.getName(), event.getContainer().getId(),
-          instance.getCompInstanceId());
-      component.compInstanceDispatcher.getEventHandler().handle(
-          new ComponentInstanceEvent(instance.getContainerId(),
-              START));
+      LOG.info("[COMPONENT {}]: Recovered {} for component instance {} on " +
+              "host {}, num pending component instances reduced to {} ",
+          component.getName(), container.getId(),
+          instance.getCompInstanceName(), container.getNodeId(),
+          component.pendingInstances.size());
+      component.dispatcher.getEventHandler().handle(
+          new ComponentInstanceEvent(container.getId(), START));
     }
   }
 
@@ -288,9 +277,8 @@ public class Component implements EventHandler<ComponentEvent> {
 
     @Override public ComponentState transition(Component component,
         ComponentEvent event) {
-      component.compInstanceDispatcher.getEventHandler().handle(
-          new ComponentInstanceEvent(event.getInstance().getContainerId(),
-              START));
+      component.dispatcher.getEventHandler().handle(
+          new ComponentInstanceEvent(event.getContainerId(), START));
       return checkIfStable(component);
     }
   }
@@ -313,14 +301,7 @@ public class Component implements EventHandler<ComponentEvent> {
     @Override
     public void transition(Component component, ComponentEvent event) {
       component.updateMetrics(event.getStatus());
-
-      // add back to pending list
-      component.pendingInstances.add(event.getInstance());
-      LOG.info(
-          "[COMPONENT {}]: {} completed, num pending comp instances increased to {}.",
-          component.getName(), event.getStatus().getContainerId(),
-          component.pendingInstances.size());
-      component.compInstanceDispatcher.getEventHandler().handle(
+      component.dispatcher.getEventHandler().handle(
           new ComponentInstanceEvent(event.getStatus().getContainerId(),
               STOP).setStatus(event.getStatus()));
       component.componentSpec.setState(
@@ -328,8 +309,8 @@ public class Component implements EventHandler<ComponentEvent> {
     }
   }
 
-  public ServiceMetrics getCompMetrics () {
-    return componentMetrics;
+  public void reInsertPendingInstance(ComponentInstance instance) {
+    pendingInstances.add(instance);
   }
 
   private void releaseContainer(Container container) {
@@ -581,4 +562,9 @@ public class Component implements EventHandler<ComponentEvent> {
   public ServiceContext getContext() {
     return context;
   }
+
+  // Only for testing
+  public List<ComponentInstance> getPendingInstances() {
+    return pendingInstances;
+  }
 }

+ 11 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/ComponentEvent.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.yarn.service.component;
 
 import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.event.AbstractEvent;
 import org.apache.hadoop.yarn.service.component.instance.ComponentInstance;
@@ -30,6 +31,16 @@ public class ComponentEvent extends AbstractEvent<ComponentEventType> {
   private Container container;
   private ComponentInstance instance;
   private ContainerStatus status;
+  private ContainerId containerId;
+
+  public ContainerId getContainerId() {
+    return containerId;
+  }
+
+  public ComponentEvent setContainerId(ContainerId containerId) {
+    this.containerId = containerId;
+    return this;
+  }
 
   public ComponentEvent(String name, ComponentEventType type) {
     super(type);

+ 44 - 39
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/instance/ComponentInstance.java

@@ -146,7 +146,7 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
       compInstance.containerStatusFuture =
           compInstance.scheduler.executorService.scheduleAtFixedRate(
               new ContainerStatusRetriever(compInstance.scheduler,
-                  compInstance.getContainerId(), compInstance), 0, 1,
+                  event.getContainerId(), compInstance), 0, 1,
               TimeUnit.SECONDS);
       compInstance.component.incRunningContainers();
       long containerStartTime = System.currentTimeMillis();
@@ -160,10 +160,10 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
       }
       org.apache.hadoop.yarn.service.api.records.Container container =
           new org.apache.hadoop.yarn.service.api.records.Container();
-      container.setId(compInstance.getContainerId().toString());
+      container.setId(event.getContainerId().toString());
       container.setLaunchTime(new Date(containerStartTime));
       container.setState(ContainerState.RUNNING_BUT_UNREADY);
-      container.setBareHost(compInstance.container.getNodeId().getHost());
+      container.setBareHost(compInstance.getNodeId().getHost());
       container.setComponentInstanceName(compInstance.getCompInstanceName());
       if (compInstance.containerSpec != null) {
         // remove the previous container.
@@ -219,15 +219,11 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
       // re-ask the failed container.
       Component comp = compInstance.component;
       comp.requestContainers(1);
-      LOG.info(compInstance.getCompInstanceId()
-              + ": Container completed. Requested a new container." + System
-              .lineSeparator() + " exitStatus={}, diagnostics={}.",
-          event.getStatus().getExitStatus(),
-          event.getStatus().getDiagnostics());
       String containerDiag =
           compInstance.getCompInstanceId() + ": " + event.getStatus()
               .getDiagnostics();
       compInstance.diagnostics.append(containerDiag + System.lineSeparator());
+      compInstance.cancelContainerStatusRetriever();
 
       if (compInstance.getState().equals(READY)) {
         compInstance.component.decContainersReady();
@@ -255,11 +251,13 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
         // hdfs dir content will be overwritten when a new container gets started,
         // so no need remove.
         compInstance.scheduler.executorService
-            .submit(compInstance::cleanupRegistry);
+            .submit(() -> compInstance.cleanupRegistry(event.getContainerId()));
+
         if (compInstance.timelineServiceEnabled) {
           // record in ATS
-          compInstance.serviceTimelinePublisher.componentInstanceFinished
-              (compInstance, event.getStatus().getExitStatus(), containerDiag);
+          compInstance.serviceTimelinePublisher
+              .componentInstanceFinished(event.getContainerId(),
+                  event.getStatus().getExitStatus(), containerDiag);
         }
         compInstance.containerSpec.setState(ContainerState.STOPPED);
       }
@@ -267,6 +265,14 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
       // remove the failed ContainerId -> CompInstance mapping
       comp.getScheduler().removeLiveCompInstance(event.getContainerId());
 
+      comp.reInsertPendingInstance(compInstance);
+
+      LOG.info(compInstance.getCompInstanceId()
+              + ": {} completed. Reinsert back to pending list and requested " +
+              "a new container." + System.lineSeparator() +
+              " exitStatus={}, diagnostics={}.",
+          event.getContainerId(), event.getStatus().getExitStatus(),
+          event.getStatus().getDiagnostics());
       if (shouldExit) {
         // Sleep for 5 seconds in hope that the state can be recorded in ATS.
         // in case there's a client polling the comp state, it can be notified.
@@ -277,8 +283,6 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
         }
         ExitUtil.terminate(-1);
       }
-
-      compInstance.removeContainer();
     }
   }
 
@@ -312,15 +316,6 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
     }
   }
 
-  public boolean hasContainer() {
-    return this.container != null;
-  }
-
-  public void removeContainer() {
-    this.container = null;
-    this.compInstanceId.setContainerId(null);
-  }
-
   public void setContainer(Container container) {
     this.container = container;
     this.compInstanceId.setContainerId(container.getId());
@@ -337,7 +332,7 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
   public void updateContainerStatus(ContainerStatus status) {
     this.status = status;
     org.apache.hadoop.yarn.service.api.records.Container container =
-        getCompSpec().getContainer(getContainerId().toString());
+        getCompSpec().getContainer(status.getContainerId().toString());
     if (container != null) {
       container.setIp(StringUtils.join(",", status.getIPs()));
       container.setHostname(status.getHost());
@@ -348,10 +343,6 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
     updateServiceRecord(yarnRegistryOperations, status);
   }
 
-  public ContainerId getContainerId() {
-    return container.getId();
-  }
-
   public String getCompName() {
     return compInstanceId.getCompName();
   }
@@ -423,12 +414,7 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
   public void destroy() {
     LOG.info(getCompInstanceId() + ": Flexed down by user, destroying.");
     diagnostics.append(getCompInstanceId() + ": Flexed down by user");
-    if (container != null) {
-      scheduler.removeLiveCompInstance(container.getId());
-      component.getScheduler().getAmRMClient()
-          .releaseAssignedContainer(container.getId());
-      getCompSpec().removeContainer(containerSpec);
-    }
+
     // update metrics
     if (getState() == STARTED) {
       component.decRunningContainers();
@@ -437,16 +423,29 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
       component.decContainersReady();
       component.decRunningContainers();
     }
+    getCompSpec().removeContainer(containerSpec);
+
+    if (container == null) {
+      LOG.info(getCompInstanceId() + " no container is assigned when " +
+          "destroying");
+      return;
+    }
+
+    ContainerId containerId = container.getId();
+    scheduler.removeLiveCompInstance(containerId);
+    component.getScheduler().getAmRMClient()
+        .releaseAssignedContainer(containerId);
 
     if (timelineServiceEnabled) {
-      serviceTimelinePublisher.componentInstanceFinished(this,
+      serviceTimelinePublisher.componentInstanceFinished(containerId,
           KILLED_BY_APPMASTER, diagnostics.toString());
     }
-    scheduler.executorService.submit(this::cleanupRegistryAndCompHdfsDir);
+    cancelContainerStatusRetriever();
+    scheduler.executorService.submit(() ->
+        cleanupRegistryAndCompHdfsDir(containerId));
   }
 
-  private void cleanupRegistry() {
-    ContainerId containerId = getContainerId();
+  private void cleanupRegistry(ContainerId containerId) {
     String cid = RegistryPathUtils.encodeYarnID(containerId.toString());
     try {
        yarnRegistryOperations.deleteComponent(getCompInstanceId(), cid);
@@ -456,8 +455,8 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
   }
 
   //TODO Maybe have a dedicated cleanup service.
-  public void cleanupRegistryAndCompHdfsDir() {
-    cleanupRegistry();
+  public void cleanupRegistryAndCompHdfsDir(ContainerId containerId) {
+    cleanupRegistry(containerId);
     try {
       if (compInstanceDir != null && fs.exists(compInstanceDir)) {
         boolean deleted = fs.delete(compInstanceDir, true);
@@ -515,6 +514,12 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
     }
   }
 
+  private void cancelContainerStatusRetriever() {
+    if (containerStatusFuture != null && !containerStatusFuture.isDone()) {
+      containerStatusFuture.cancel(true);
+    }
+  }
+
   @Override
   public int compareTo(ComponentInstance to) {
     long delta = containerStartedTime - to.containerStartedTime;

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/conf/RestApiConstants.java

@@ -20,7 +20,7 @@ package org.apache.hadoop.yarn.service.conf;
 public interface RestApiConstants {
 
   // Rest endpoints
-  String CONTEXT_ROOT = "/ws/v1";
+  String CONTEXT_ROOT = "/v1";
   String VERSION = "/services/version";
   String SERVICE_ROOT_PATH = "/services";
   String SERVICE_PATH = "/services/{service_name}";

+ 5 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/conf/YarnServiceConf.java

@@ -85,6 +85,11 @@ public class YarnServiceConf {
   public static final String READINESS_CHECK_INTERVAL = "yarn.service.readiness-check-interval.seconds";
   public static final int DEFAULT_READINESS_CHECK_INTERVAL = 30; // seconds
 
+  /**
+   * JVM opts.
+   */
+  public static final String JVM_OPTS = "yarn.service.am.java.opts";
+
   /**
    * Get long value for the property. First get from the userConf, if not
    * present, get from systemConf.

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/ContainerLaunchService.java

@@ -87,7 +87,7 @@ public class ContainerLaunchService extends AbstractService{
       AbstractLauncher launcher = new AbstractLauncher(fs, null);
       try {
         provider.buildContainerLaunchContext(launcher, service,
-            instance, fs, getConfig());
+            instance, fs, getConfig(), container);
         instance.getComponent().getScheduler().getNmClient()
             .startContainerAsync(container,
                 launcher.completeContainerLaunch());

+ 5 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/containerlaunch/JavaCommandLineBuilder.java

@@ -48,13 +48,12 @@ public class JavaCommandLineBuilder extends CommandLineBuilder {
   }
 
   /**
-   * Set the size of the heap if a non-empty heap is passed in. 
-   * @param heap empty string or something like "128M" ,"1G" etc. The value is
-   * trimmed.
+   * Set JVM opts.
+   * @param jvmOpts JVM opts
    */
-  public void setJVMHeap(String heap) {
-    if (ServiceUtils.isSet(heap)) {
-      add("-Xmx" + heap.trim());
+  public void setJVMOpts(String jvmOpts) {
+    if (ServiceUtils.isSet(jvmOpts)) {
+      add(jvmOpts);
     }
   }
 

+ 3 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/AbstractProviderService.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.service.provider;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
+import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.service.api.records.Service;
 import org.apache.hadoop.yarn.service.conf.YarnServiceConf;
 import org.apache.hadoop.yarn.service.api.records.Component;
@@ -55,7 +56,7 @@ public abstract class AbstractProviderService implements ProviderService,
 
   public void buildContainerLaunchContext(AbstractLauncher launcher,
       Service service, ComponentInstance instance,
-      SliderFileSystem fileSystem, Configuration yarnConf)
+      SliderFileSystem fileSystem, Configuration yarnConf, Container container)
       throws IOException, SliderException {
     Component component = instance.getComponent().getComponentSpec();;
     processArtifact(launcher, instance, fileSystem, service);
@@ -67,7 +68,7 @@ public abstract class AbstractProviderService implements ProviderService,
     Map<String, String> globalTokens =
         instance.getComponent().getScheduler().globalTokens;
     Map<String, String> tokensForSubstitution = ProviderUtils
-        .initCompTokensForSubstitute(instance);
+        .initCompTokensForSubstitute(instance, container);
     tokensForSubstitution.putAll(globalTokens);
     // Set the environment variables in launcher
     launcher.putEnv(ServiceUtils

+ 3 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/ProviderService.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.yarn.service.provider;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.service.api.records.Service;
 import org.apache.hadoop.yarn.service.utils.SliderFileSystem;
 import org.apache.hadoop.yarn.service.exceptions.SliderException;
@@ -34,6 +35,6 @@ public interface ProviderService {
    */
   void buildContainerLaunchContext(AbstractLauncher containerLauncher,
       Service service, ComponentInstance instance,
-      SliderFileSystem sliderFileSystem, Configuration yarnConf)
-      throws IOException, SliderException;
+      SliderFileSystem sliderFileSystem, Configuration yarnConf, Container
+      container) throws IOException, SliderException;
 }

+ 3 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/provider/ProviderUtils.java

@@ -24,6 +24,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LocalResourceType;
 import org.apache.hadoop.yarn.service.ServiceContext;
@@ -393,13 +394,13 @@ public class ProviderUtils implements YarnServiceConstants {
    * @return tokens to replace
    */
   public static Map<String, String> initCompTokensForSubstitute(
-      ComponentInstance instance) {
+      ComponentInstance instance, Container container) {
     Map<String, String> tokens = new HashMap<>();
     tokens.put(COMPONENT_NAME, instance.getCompSpec().getName());
     tokens
         .put(COMPONENT_NAME_LC, instance.getCompSpec().getName().toLowerCase());
     tokens.put(COMPONENT_INSTANCE_NAME, instance.getCompInstanceName());
-    tokens.put(CONTAINER_ID, instance.getContainer().getId().toString());
+    tokens.put(CONTAINER_ID, container.getId().toString());
     tokens.put(COMPONENT_ID,
         String.valueOf(instance.getCompInstanceId().getId()));
     tokens.putAll(instance.getComponent().getDependencyHostIpTokens());

+ 3 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/timelineservice/ServiceTimelinePublisher.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.service.timelineservice;
 
 import org.apache.hadoop.metrics2.AbstractMetric;
 import org.apache.hadoop.service.CompositeService;
+import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
@@ -178,10 +179,10 @@ public class ServiceTimelinePublisher extends CompositeService {
     putEntity(entity);
   }
 
-  public void componentInstanceFinished(ComponentInstance instance,
+  public void componentInstanceFinished(ContainerId containerId,
       int exitCode, String diagnostics) {
     TimelineEntity entity = createComponentInstanceEntity(
-        instance.getContainer().getId().toString());
+        containerId.toString());
 
     // create info keys
     Map<String, Object> entityInfos = new HashMap<String, Object>();

+ 52 - 14
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/MockServiceAM.java

@@ -24,14 +24,8 @@ import org.apache.hadoop.registry.client.api.RegistryOperations;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
-import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.Container;
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
-import org.apache.hadoop.yarn.api.records.NodeId;
-import org.apache.hadoop.yarn.api.records.Priority;
-import org.apache.hadoop.yarn.api.records.Resource;
+
+import org.apache.hadoop.yarn.api.records.*;
 import org.apache.hadoop.yarn.client.api.AMRMClient;
 import org.apache.hadoop.yarn.client.api.NMClient;
 import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync;
@@ -42,15 +36,15 @@ import org.apache.hadoop.yarn.proto.ClientAMProtocol;
 import org.apache.hadoop.yarn.service.api.records.Service;
 import org.apache.hadoop.yarn.service.component.Component;
 import org.apache.hadoop.yarn.service.component.ComponentState;
+import org.apache.hadoop.yarn.service.component.instance.ComponentInstance;
+import org.apache.hadoop.yarn.service.component.instance.ComponentInstanceState;
 import org.apache.hadoop.yarn.service.exceptions.BadClusterStateException;
 import org.apache.hadoop.yarn.service.registry.YarnRegistryViewForProviders;
 import org.apache.hadoop.yarn.service.utils.SliderFileSystem;
+import org.apache.hadoop.yarn.util.Records;
 
 import java.io.IOException;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
+import java.util.*;
 import java.util.concurrent.TimeoutException;
 
 import static org.mockito.Mockito.mock;
@@ -63,6 +57,8 @@ public class MockServiceAM extends ServiceMaster {
   final List<Container> feedContainers =
       Collections.synchronizedList(new LinkedList<>());
 
+  final List<ContainerStatus> failedContainers =
+      Collections.synchronizedList(new LinkedList<>());
   public MockServiceAM(Service service) {
     super(service.getName());
     this.service = service;
@@ -102,10 +98,10 @@ public class MockServiceAM extends ServiceMaster {
 
             AllocateResponse.AllocateResponseBuilder builder =
                 AllocateResponse.newBuilder();
+            // add new containers if any
             synchronized (feedContainers) {
               if (feedContainers.isEmpty()) {
                 System.out.println("Allocating........ no containers");
-                return builder.build();
               } else {
                 // The AMRMClient will return containers for compoenent that are
                 // at FLEXING state
@@ -121,9 +117,20 @@ public class MockServiceAM extends ServiceMaster {
                     itor.remove();
                   }
                 }
-                return builder.allocatedContainers(allocatedContainers).build();
+                builder.allocatedContainers(allocatedContainers);
+              }
+            }
+
+            // add failed containers if any
+            synchronized (failedContainers) {
+              if (!failedContainers.isEmpty()) {
+                List<ContainerStatus> failed =
+                    new LinkedList<>(failedContainers);
+                failedContainers.clear();
+                builder.completedContainersStatuses(failed);
               }
             }
+            return builder.build();
           }
 
           @Override
@@ -184,6 +191,19 @@ public class MockServiceAM extends ServiceMaster {
     return container;
   }
 
+  public void feedFailedContainerToComp(Service service, int id, String
+      compName) {
+    ApplicationId applicationId = ApplicationId.fromString(service.getId());
+    ContainerId containerId = ContainerId
+        .newContainerId(ApplicationAttemptId.newInstance(applicationId, 1), id);
+    ContainerStatus containerStatus = Records.newRecord(ContainerStatus.class);
+    containerStatus.setContainerId(containerId);
+    synchronized (failedContainers) {
+      failedContainers.add(containerStatus);
+    }
+  }
+
+
   public void flexComponent(String compName, long numberOfContainers)
       throws IOException {
     ClientAMProtocol.ComponentCountProto componentCountProto =
@@ -218,4 +238,22 @@ public class MockServiceAM extends ServiceMaster {
       }
     }, 1000, 20000);
   }
+
+
+  public ComponentInstance getCompInstance(String compName, String
+      instanceName) {
+    return context.scheduler.getAllComponents().get(compName)
+        .getComponentInstance(instanceName);
+  }
+
+  public void waitForCompInstanceState(ComponentInstance instance,
+      ComponentInstanceState state)
+      throws TimeoutException, InterruptedException {
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        return instance.getState().equals(state);
+      }
+    }, 1000, 20000);
+  }
 }

+ 4 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/ServiceTestUtils.java

@@ -65,6 +65,7 @@ public class ServiceTestUtils {
 
   private MiniYARNCluster yarnCluster = null;
   private MiniDFSCluster hdfsCluster = null;
+  TestingCluster zkCluster;
   private FileSystem fs = null;
   private Configuration conf = null;
   public static final int NUM_NMS = 1;
@@ -165,7 +166,6 @@ public class ServiceTestUtils {
     conf.setBoolean(NM_VMEM_CHECK_ENABLED, false);
     conf.setBoolean(NM_PMEM_CHECK_ENABLED, false);
     // setup zk cluster
-    TestingCluster zkCluster;
     zkCluster = new TestingCluster(1);
     zkCluster.start();
     conf.set(YarnConfiguration.RM_ZK_ADDRESS, zkCluster.getConnectString());
@@ -239,6 +239,9 @@ public class ServiceTestUtils {
         hdfsCluster = null;
       }
     }
+    if (zkCluster != null) {
+      zkCluster.stop();
+    }
     if (basedir != null) {
       FileUtils.deleteDirectory(basedir);
     }

+ 109 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestServiceAM.java

@@ -0,0 +1,109 @@
+/*
+ * 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.service;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.curator.test.TestingCluster;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.service.api.records.Service;
+import org.apache.hadoop.yarn.service.component.instance.ComponentInstance;
+import org.apache.hadoop.yarn.service.component.instance.ComponentInstanceState;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.hadoop.registry.client.api.RegistryConstants
+    .KEY_REGISTRY_ZK_QUORUM;
+
+public class TestServiceAM extends ServiceTestUtils{
+
+  private File basedir;
+  YarnConfiguration conf = new YarnConfiguration();
+  TestingCluster zkCluster;
+
+  @Before
+  public void setup() throws Exception {
+    basedir = new File("target", "apps");
+    if (basedir.exists()) {
+      FileUtils.deleteDirectory(basedir);
+    } else {
+      basedir.mkdirs();
+    }
+    zkCluster = new TestingCluster(1);
+    zkCluster.start();
+    conf.set(KEY_REGISTRY_ZK_QUORUM, zkCluster.getConnectString());
+    System.out.println("ZK cluster: " +  zkCluster.getConnectString());
+  }
+
+  @After
+  public void tearDown() throws IOException {
+    if (basedir != null) {
+      FileUtils.deleteDirectory(basedir);
+    }
+    if (zkCluster != null) {
+      zkCluster.stop();
+    }
+  }
+
+  // Race condition YARN-7486
+  // 1. Allocate 1 container to compa and wait it to be started
+  // 2. Fail this container, and in the meanwhile allocate the 2nd container.
+  // 3. The 2nd container should not be assigned to compa-0 instance, because
+  //   the compa-0 instance is not stopped yet.
+  // 4. check compa still has the instance in the pending list.
+  @Test
+  public void testContainerCompleted() throws TimeoutException,
+      InterruptedException {
+    ApplicationId applicationId = ApplicationId.newInstance(123456, 1);
+    Service exampleApp = new Service();
+    exampleApp.setId(applicationId.toString());
+    exampleApp.setName("testContainerCompleted");
+    exampleApp.addComponent(createComponent("compa", 1, "pwd"));
+
+    MockServiceAM am = new MockServiceAM(exampleApp);
+    am.init(conf);
+    am.start();
+
+    ComponentInstance compa0 = am.getCompInstance("compa", "compa-0");
+    // allocate a container
+    am.feedContainerToComp(exampleApp, 1, "compa");
+    am.waitForCompInstanceState(compa0, ComponentInstanceState.STARTED);
+
+    System.out.println("Fail the container 1");
+    // fail the container
+    am.feedFailedContainerToComp(exampleApp, 1, "compa");
+
+    // allocate the second container immediately, this container will not be
+    // assigned to comp instance
+    // because the instance is not yet added to the pending list.
+    am.feedContainerToComp(exampleApp, 2, "compa");
+
+    am.waitForCompInstanceState(compa0, ComponentInstanceState.INIT);
+    // still 1 pending instance
+    Assert.assertEquals(1,
+        am.getComponent("compa").getPendingInstances().size());
+    am.stop();
+  }
+}

+ 18 - 24
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestYarnNativeServices.java

@@ -176,7 +176,8 @@ public class TestYarnNativeServices extends ServiceTestUtils {
     ServiceClient client = createClient();
     Service exampleApp = createExampleApplication();
     client.actionCreate(exampleApp);
-    waitForAllCompToBeReady(client, exampleApp);
+    Multimap<String, String> containersBeforeFailure =
+        waitForAllCompToBeReady(client, exampleApp);
 
     LOG.info("Restart the resource manager");
     getYarnCluster().restartResourceManager(
@@ -191,9 +192,6 @@ public class TestYarnNativeServices extends ServiceTestUtils {
     ApplicationAttemptId applicationAttemptId = client.getYarnClient()
         .getApplicationReport(exampleAppId).getCurrentApplicationAttemptId();
 
-    Multimap<String, String> containersBeforeFailure = getContainersForAllComp(
-        client, exampleApp);
-
     LOG.info("Fail the application attempt {}", applicationAttemptId);
     client.getYarnClient().failApplicationAttempt(applicationAttemptId);
     //wait until attempt 2 is running
@@ -208,7 +206,7 @@ public class TestYarnNativeServices extends ServiceTestUtils {
       }
     }, 2000, 200000);
 
-    Multimap<String, String> containersAfterFailure = getContainersForAllComp(
+    Multimap<String, String> containersAfterFailure = waitForAllCompToBeReady(
         client, exampleApp);
     Assert.assertEquals("component container affected by restart",
         containersBeforeFailure, containersAfterFailure);
@@ -318,14 +316,26 @@ public class TestYarnNativeServices extends ServiceTestUtils {
     }, 2000, 200000);
   }
 
-  // wait until all the containers for all components become ready state
-  private void waitForAllCompToBeReady(ServiceClient client,
+  /**
+   * Wait until all the containers for all components become ready state.
+   *
+   * @param client
+   * @param exampleApp
+   * @return all ready containers of a service.
+   * @throws TimeoutException
+   * @throws InterruptedException
+   */
+  private Multimap<String, String> waitForAllCompToBeReady(ServiceClient client,
       Service exampleApp) throws TimeoutException, InterruptedException {
     int expectedTotalContainers = countTotalContainers(exampleApp);
+
+    Multimap<String, String> allContainers = HashMultimap.create();
+
     GenericTestUtils.waitFor(() -> {
       try {
         Service retrievedApp = client.getStatus(exampleApp.getName());
         int totalReadyContainers = 0;
+        allContainers.clear();
         LOG.info("Num Components " + retrievedApp.getComponents().size());
         for (Component component : retrievedApp.getComponents()) {
           LOG.info("looking for  " + component.getName());
@@ -339,6 +349,7 @@ public class TestYarnNativeServices extends ServiceTestUtils {
                         + component.getName());
                 if (container.getState() == ContainerState.READY) {
                   totalReadyContainers++;
+                  allContainers.put(component.getName(), container.getId());
                   LOG.info("Found 1 ready container " + container.getId());
                 }
               }
@@ -358,23 +369,6 @@ public class TestYarnNativeServices extends ServiceTestUtils {
         return false;
       }
     }, 2000, 200000);
-  }
-
-  /**
-   * Get all containers of a service.
-   */
-  private Multimap<String, String> getContainersForAllComp(ServiceClient client,
-      Service example) throws IOException, YarnException {
-
-    Multimap<String, String> allContainers = HashMultimap.create();
-    Service retrievedApp = client.getStatus(example.getName());
-    retrievedApp.getComponents().forEach(component -> {
-      if (component.getContainers() != null) {
-        component.getContainers().forEach(container -> {
-          allContainers.put(component.getName(), container.getId());
-        });
-      }
-    });
     return allContainers;
   }
 

+ 12 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/monitor/TestServiceMonitor.java

@@ -20,6 +20,7 @@
 package org.apache.hadoop.yarn.service.monitor;
 
 import org.apache.commons.io.FileUtils;
+import org.apache.curator.test.TestingCluster;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.service.MockServiceAM;
@@ -37,10 +38,14 @@ import java.io.File;
 import java.io.IOException;
 import java.util.Collections;
 
+import static org.apache.hadoop.registry.client.api.RegistryConstants
+    .KEY_REGISTRY_ZK_QUORUM;
+
 public class TestServiceMonitor extends ServiceTestUtils {
 
   private File basedir;
   YarnConfiguration conf = new YarnConfiguration();
+  TestingCluster zkCluster;
 
   @Before
   public void setup() throws Exception {
@@ -51,6 +56,10 @@ public class TestServiceMonitor extends ServiceTestUtils {
       basedir.mkdirs();
     }
     conf.setLong(YarnServiceConf.READINESS_CHECK_INTERVAL, 2);
+    zkCluster = new TestingCluster(1);
+    zkCluster.start();
+    conf.set(KEY_REGISTRY_ZK_QUORUM, zkCluster.getConnectString());
+    System.out.println("ZK cluster: " +  zkCluster.getConnectString());
   }
 
   @After
@@ -58,6 +67,9 @@ public class TestServiceMonitor extends ServiceTestUtils {
     if (basedir != null) {
       FileUtils.deleteDirectory(basedir);
     }
+    if (zkCluster != null) {
+      zkCluster.stop();
+    }
   }
 
   // Create compa with 1 container

+ 30 - 425
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java

@@ -33,7 +33,6 @@ import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
-import java.util.Collections;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -42,7 +41,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
-import com.google.common.base.Supplier;
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.DataInputByteBuffer;
@@ -75,14 +73,6 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.ReservationListRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.ReservationListResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.SignalContainerRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
@@ -96,13 +86,7 @@ import org.apache.hadoop.yarn.api.records.ContainerReport;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
-import org.apache.hadoop.yarn.api.records.NodeLabel;
 import org.apache.hadoop.yarn.api.records.Priority;
-import org.apache.hadoop.yarn.api.records.ReservationDefinition;
-import org.apache.hadoop.yarn.api.records.ReservationId;
-import org.apache.hadoop.yarn.api.records.ReservationRequest;
-import org.apache.hadoop.yarn.api.records.ReservationRequestInterpreter;
-import org.apache.hadoop.yarn.api.records.ReservationRequests;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.SignalContainerCommand;
 import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState;
@@ -119,23 +103,28 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
 import org.apache.hadoop.yarn.server.MiniYARNCluster;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
+import org.apache.hadoop.yarn.server.resourcemanager.ParameterizedSchedulerTestBase;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
-import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSystemTestUtil;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
-import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.Records;
-import org.apache.hadoop.yarn.util.UTCClock;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 import org.mockito.ArgumentCaptor;
 import org.slf4j.event.Level;
 
-public class TestYarnClient {
+/**
+ * This class is to test class {@link YarnClient) and {@link YarnClientImpl}.
+ */
+public class TestYarnClient extends ParameterizedSchedulerTestBase {
+
+  public TestYarnClient(SchedulerType type) throws IOException {
+    super(type);
+  }
+
+  protected void configureFairScheduler(YarnConfiguration conf) {}
 
   @Before
   public void setup() {
@@ -145,7 +134,7 @@ public class TestYarnClient {
 
   @Test
   public void testClientStop() {
-    Configuration conf = new Configuration();
+    Configuration conf = getConf();
     ResourceManager rm = new ResourceManager();
     rm.init(conf);
     rm.start();
@@ -159,7 +148,7 @@ public class TestYarnClient {
 
   @Test
   public void testStartWithTimelineV15() throws Exception {
-    Configuration conf = new Configuration();
+    Configuration conf = getConf();
     conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
     conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 1.5f);
     YarnClientImpl client = (YarnClientImpl) YarnClient.createYarnClient();
@@ -254,7 +243,7 @@ public class TestYarnClient {
   @SuppressWarnings("deprecation")
   @Test (timeout = 30000)
   public void testSubmitApplication() throws Exception {
-    Configuration conf = new Configuration();
+    Configuration conf = getConf();
     conf.setLong(YarnConfiguration.YARN_CLIENT_APP_SUBMISSION_POLL_INTERVAL_MS,
         100); // speed up tests
     final YarnClient client = new MockYarnClient();
@@ -301,7 +290,7 @@ public class TestYarnClient {
   @SuppressWarnings("deprecation")
   @Test (timeout = 20000)
   public void testSubmitApplicationInterrupted() throws IOException {
-    Configuration conf = new Configuration();
+    Configuration conf = getConf();
     int pollIntervalMs = 1000;
     conf.setLong(YarnConfiguration.YARN_CLIENT_APP_SUBMISSION_POLL_INTERVAL_MS,
         pollIntervalMs);
@@ -406,10 +395,9 @@ public class TestYarnClient {
     rm.start();
     RMApp app = rm.submitApp(2000);
 
-    Configuration conf = new Configuration();
     @SuppressWarnings("resource")
     final YarnClient client = new MockYarnClient();
-    client.init(conf);
+    client.init(getConf());
     client.start();
 
     client.killApplication(app.getApplicationId());
@@ -447,9 +435,8 @@ public class TestYarnClient {
 
   @Test (timeout = 10000)
   public void testGetApplications() throws YarnException, IOException {
-    Configuration conf = new Configuration();
     final YarnClient client = new MockYarnClient();
-    client.init(conf);
+    client.init(getConf());
     client.start();
 
     List<ApplicationReport> expectedReports = ((MockYarnClient)client).getReports();
@@ -500,9 +487,8 @@ public class TestYarnClient {
 
   @Test(timeout = 10000)
   public void testGetApplicationAttempts() throws YarnException, IOException {
-    Configuration conf = new Configuration();
     final YarnClient client = new MockYarnClient();
-    client.init(conf);
+    client.init(getConf());
     client.start();
 
     ApplicationId applicationId = ApplicationId.newInstance(1234, 5);
@@ -539,7 +525,7 @@ public class TestYarnClient {
 
   @Test(timeout = 10000)
   public void testGetContainers() throws YarnException, IOException {
-    Configuration conf = new Configuration();
+    Configuration conf = getConf();
     conf.setBoolean(YarnConfiguration.APPLICATION_HISTORY_ENABLED,
         true);
     
@@ -572,7 +558,7 @@ public class TestYarnClient {
 
   @Test(timeout = 10000)
   public void testGetContainerReport() throws YarnException, IOException {
-    Configuration conf = new Configuration();
+    Configuration conf = getConf();
     conf.setBoolean(YarnConfiguration.APPLICATION_HISTORY_ENABLED,
         true);
     final YarnClient client = new MockYarnClient();
@@ -603,9 +589,8 @@ public class TestYarnClient {
 
   @Test (timeout = 10000)
   public void testGetLabelsToNodes() throws YarnException, IOException {
-    Configuration conf = new Configuration();
     final YarnClient client = new MockYarnClient();
-    client.init(conf);
+    client.init(getConf());
     client.start();
 
     // Get labels to nodes mapping
@@ -629,9 +614,8 @@ public class TestYarnClient {
 
   @Test (timeout = 10000)
   public void testGetNodesToLabels() throws YarnException, IOException {
-    Configuration conf = new Configuration();
     final YarnClient client = new MockYarnClient();
-    client.init(conf);
+    client.init(getConf());
     client.start();
 
     // Get labels to nodes mapping
@@ -1025,7 +1009,7 @@ public class TestYarnClient {
     MiniYARNCluster cluster = new MiniYARNCluster("testMRAMTokens", 1, 1, 1);
     YarnClient rmClient = null;
     try {
-      cluster.init(new YarnConfiguration());
+      cluster.init(getConf());
       cluster.start();
       final Configuration yarnConf = cluster.getConfig();
       rmClient = YarnClient.createYarnClient();
@@ -1146,7 +1130,7 @@ public class TestYarnClient {
       boolean expectedTimeoutEnforcement) {
     YarnClientImpl client = new YarnClientImpl();
     try {
-      Configuration conf = new Configuration();
+      Configuration conf = getConf();
       if (valueForTimeout != null) {
         conf.setLong(
             YarnConfiguration.YARN_CLIENT_APPLICATION_CLIENT_PROTOCOL_POLL_TIMEOUT_MS,
@@ -1165,7 +1149,7 @@ public class TestYarnClient {
   @Test
   public void testBestEffortTimelineDelegationToken()
       throws Exception {
-    Configuration conf = new YarnConfiguration();
+    Configuration conf = getConf();
     conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
     SecurityUtil.setAuthenticationMethod(AuthenticationMethod.KERBEROS, conf);
 
@@ -1199,7 +1183,7 @@ public class TestYarnClient {
   @Test
   public void testAutomaticTimelineDelegationTokenLoading()
       throws Exception {
-    Configuration conf = new YarnConfiguration();
+    Configuration conf = getConf();
     conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
     SecurityUtil.setAuthenticationMethod(AuthenticationMethod.KERBEROS, conf);
     TimelineDelegationTokenIdentifier timelineDT =
@@ -1289,7 +1273,7 @@ public class TestYarnClient {
   public void testParseTimelineDelegationTokenRenewer() throws Exception {
     // Client side
     YarnClientImpl client = (YarnClientImpl) YarnClient.createYarnClient();
-    Configuration conf = new YarnConfiguration();
+    Configuration conf = getConf();
     conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
     conf.set(YarnConfiguration.RM_PRINCIPAL, "rm/_HOST@EXAMPLE.COM");
     conf.set(
@@ -1303,387 +1287,9 @@ public class TestYarnClient {
     }
   }
 
-  private MiniYARNCluster setupMiniYARNCluster() throws Exception {
-    CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
-    ReservationSystemTestUtil.setupQueueConfiguration(conf);
-    conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
-        ResourceScheduler.class);
-    conf.setBoolean(YarnConfiguration.RM_RESERVATION_SYSTEM_ENABLE, true);
-    MiniYARNCluster cluster =
-        new MiniYARNCluster("testReservationAPIs", 2, 1, 1);
-
-    cluster.init(conf);
-    cluster.start();
-
-    GenericTestUtils.waitFor(new Supplier<Boolean>() {
-      @Override
-      public Boolean get() {
-        return cluster.getResourceManager().getRMContext()
-            .getReservationSystem()
-            .getPlan(ReservationSystemTestUtil.reservationQ)
-            .getTotalCapacity().getMemorySize() > 6000;
-      }
-    }, 10, 10000);
-
-    return cluster;
-  }
-
-  private YarnClient setupYarnClient(MiniYARNCluster cluster) {
-    final Configuration yarnConf = cluster.getConfig();
-    YarnClient client = YarnClient.createYarnClient();
-    client.init(yarnConf);
-    client.start();
-    return client;
-  }
-
-  private ReservationSubmissionRequest submitReservationTestHelper(
-      YarnClient client, long arrival, long deadline, long duration)
-      throws IOException, YarnException {
-    ReservationId reservationID = client.createReservation().getReservationId();
-    ReservationSubmissionRequest sRequest = createSimpleReservationRequest(
-        reservationID, 4, arrival, deadline, duration);
-    ReservationSubmissionResponse sResponse =
-        client.submitReservation(sRequest);
-    Assert.assertNotNull(sResponse);
-    Assert.assertNotNull(reservationID);
-    System.out.println("Submit reservation response: " + reservationID);
-
-    return sRequest;
-  }
-
-  @Test
-  public void testCreateReservation() throws Exception {
-    MiniYARNCluster cluster = setupMiniYARNCluster();
-    YarnClient client = setupYarnClient(cluster);
-    try {
-      Clock clock = new UTCClock();
-      long arrival = clock.getTime();
-      long duration = 60000;
-      long deadline = (long) (arrival + 1.05 * duration);
-      ReservationSubmissionRequest sRequest =
-          submitReservationTestHelper(client, arrival, deadline, duration);
-
-      // Submit the reservation again with the same request and make sure it
-      // passes.
-      client.submitReservation(sRequest);
-
-      // Submit the reservation with the same reservation id but different
-      // reservation definition, and ensure YarnException is thrown.
-      arrival = clock.getTime();
-      ReservationDefinition rDef = sRequest.getReservationDefinition();
-      rDef.setArrival(arrival + duration);
-      sRequest.setReservationDefinition(rDef);
-      try {
-        client.submitReservation(sRequest);
-        Assert.fail("Reservation submission should fail if a duplicate "
-            + "reservation id is used, but the reservation definition has been "
-            + "updated.");
-      } catch (Exception e) {
-        Assert.assertTrue(e instanceof YarnException);
-      }
-    } finally {
-      // clean-up
-      if (client != null) {
-        client.stop();
-      }
-      cluster.stop();
-    }
-  }
-
-  @Test
-  public void testUpdateReservation() throws Exception {
-    MiniYARNCluster cluster = setupMiniYARNCluster();
-    YarnClient client = setupYarnClient(cluster);
-    try {
-      Clock clock = new UTCClock();
-      long arrival = clock.getTime();
-      long duration = 60000;
-      long deadline = (long) (arrival + 1.05 * duration);
-      ReservationSubmissionRequest sRequest =
-          submitReservationTestHelper(client, arrival, deadline, duration);
-
-      ReservationDefinition rDef = sRequest.getReservationDefinition();
-      ReservationRequest rr =
-          rDef.getReservationRequests().getReservationResources().get(0);
-      ReservationId reservationID = sRequest.getReservationId();
-      rr.setNumContainers(5);
-      arrival = clock.getTime();
-      duration = 30000;
-      deadline = (long) (arrival + 1.05 * duration);
-      rr.setDuration(duration);
-      rDef.setArrival(arrival);
-      rDef.setDeadline(deadline);
-      ReservationUpdateRequest uRequest =
-          ReservationUpdateRequest.newInstance(rDef, reservationID);
-      ReservationUpdateResponse uResponse = client.updateReservation(uRequest);
-      Assert.assertNotNull(uResponse);
-      System.out.println("Update reservation response: " + uResponse);
-    } finally {
-      // clean-up
-      if (client != null) {
-        client.stop();
-      }
-      cluster.stop();
-    }
-  }
-
-  @Test
-  public void testListReservationsByReservationId() throws Exception{
-    MiniYARNCluster cluster = setupMiniYARNCluster();
-    YarnClient client = setupYarnClient(cluster);
-    try {
-      Clock clock = new UTCClock();
-      long arrival = clock.getTime();
-      long duration = 60000;
-      long deadline = (long) (arrival + 1.05 * duration);
-      ReservationSubmissionRequest sRequest =
-          submitReservationTestHelper(client, arrival, deadline, duration);
-
-      ReservationId reservationID = sRequest.getReservationId();
-      ReservationListRequest request = ReservationListRequest.newInstance(
-          ReservationSystemTestUtil.reservationQ, reservationID.toString(), -1,
-          -1, false);
-      ReservationListResponse response = client.listReservations(request);
-      Assert.assertNotNull(response);
-      Assert.assertEquals(1, response.getReservationAllocationState().size());
-      Assert.assertEquals(response.getReservationAllocationState().get(0)
-          .getReservationId().getId(), reservationID.getId());
-      Assert.assertEquals(response.getReservationAllocationState().get(0)
-          .getResourceAllocationRequests().size(), 0);
-    } finally {
-      // clean-up
-      if (client != null) {
-        client.stop();
-      }
-      cluster.stop();
-    }
-  }
-
-  @Test
-  public void testListReservationsByTimeInterval() throws Exception {
-    MiniYARNCluster cluster = setupMiniYARNCluster();
-    YarnClient client = setupYarnClient(cluster);
-    try {
-      Clock clock = new UTCClock();
-      long arrival = clock.getTime();
-      long duration = 60000;
-      long deadline = (long) (arrival + 1.05 * duration);
-      ReservationSubmissionRequest sRequest =
-          submitReservationTestHelper(client, arrival, deadline, duration);
-
-      // List reservations, search by a point in time within the reservation
-      // range.
-      arrival = clock.getTime();
-      ReservationId reservationID = sRequest.getReservationId();
-      ReservationListRequest request = ReservationListRequest.newInstance(
-          ReservationSystemTestUtil.reservationQ, "", arrival + duration / 2,
-          arrival + duration / 2, true);
-
-      ReservationListResponse response = client.listReservations(request);
-      Assert.assertNotNull(response);
-      Assert.assertEquals(1, response.getReservationAllocationState().size());
-      Assert.assertEquals(response.getReservationAllocationState().get(0)
-          .getReservationId().getId(), reservationID.getId());
-      // List reservations, search by time within reservation interval.
-      request = ReservationListRequest.newInstance(
-          ReservationSystemTestUtil.reservationQ, "", 1, Long.MAX_VALUE, true);
-
-      response = client.listReservations(request);
-      Assert.assertNotNull(response);
-      Assert.assertEquals(1, response.getReservationAllocationState().size());
-      Assert.assertEquals(response.getReservationAllocationState().get(0)
-          .getReservationId().getId(), reservationID.getId());
-      // Verify that the full resource allocations exist.
-      Assert.assertTrue(response.getReservationAllocationState().get(0)
-          .getResourceAllocationRequests().size() > 0);
-
-      // Verify that the full RDL is returned.
-      ReservationRequests reservationRequests =
-          response.getReservationAllocationState().get(0)
-              .getReservationDefinition().getReservationRequests();
-      Assert.assertEquals("R_ALL",
-          reservationRequests.getInterpreter().toString());
-      Assert.assertTrue(reservationRequests.getReservationResources().get(0)
-          .getDuration() == duration);
-    } finally {
-      // clean-up
-      if (client != null) {
-        client.stop();
-      }
-      cluster.stop();
-    }
-  }
-
-  @Test
-  public void testListReservationsByInvalidTimeInterval() throws Exception {
-    MiniYARNCluster cluster = setupMiniYARNCluster();
-    YarnClient client = setupYarnClient(cluster);
-    try {
-      Clock clock = new UTCClock();
-      long arrival = clock.getTime();
-      long duration = 60000;
-      long deadline = (long) (arrival + 1.05 * duration);
-      ReservationSubmissionRequest sRequest =
-          submitReservationTestHelper(client, arrival, deadline, duration);
-
-      // List reservations, search by invalid end time == -1.
-      ReservationListRequest request = ReservationListRequest
-          .newInstance(ReservationSystemTestUtil.reservationQ, "", 1, -1, true);
-
-      ReservationListResponse response = client.listReservations(request);
-      Assert.assertNotNull(response);
-      Assert.assertEquals(1, response.getReservationAllocationState().size());
-      Assert.assertEquals(response.getReservationAllocationState().get(0)
-          .getReservationId().getId(), sRequest.getReservationId().getId());
-
-      // List reservations, search by invalid end time < -1.
-      request = ReservationListRequest.newInstance(
-          ReservationSystemTestUtil.reservationQ, "", 1, -10, true);
-
-      response = client.listReservations(request);
-      Assert.assertNotNull(response);
-      Assert.assertEquals(1, response.getReservationAllocationState().size());
-      Assert.assertEquals(response.getReservationAllocationState().get(0)
-          .getReservationId().getId(), sRequest.getReservationId().getId());
-    } finally {
-      // clean-up
-      if (client != null) {
-        client.stop();
-      }
-      cluster.stop();
-    }
-  }
-
-  @Test
-  public void testListReservationsByTimeIntervalContainingNoReservations()
-      throws Exception {
-    MiniYARNCluster cluster = setupMiniYARNCluster();
-    YarnClient client = setupYarnClient(cluster);
-    try {
-      Clock clock = new UTCClock();
-      long arrival = clock.getTime();
-      long duration = 60000;
-      long deadline = (long) (arrival + 1.05 * duration);
-      ReservationSubmissionRequest sRequest =
-          submitReservationTestHelper(client, arrival, deadline, duration);
-
-      // List reservations, search by very large start time.
-      ReservationListRequest request = ReservationListRequest.newInstance(
-          ReservationSystemTestUtil.reservationQ, "", Long.MAX_VALUE, -1,
-          false);
-
-      ReservationListResponse response = client.listReservations(request);
-
-      // Ensure all reservations are filtered out.
-      Assert.assertNotNull(response);
-      Assert.assertEquals(response.getReservationAllocationState().size(), 0);
-
-      duration = 30000;
-      deadline = sRequest.getReservationDefinition().getDeadline();
-
-      // List reservations, search by start time after the reservation
-      // end time.
-      request = ReservationListRequest.newInstance(
-          ReservationSystemTestUtil.reservationQ, "", deadline + duration,
-          deadline + 2 * duration, false);
-
-      response = client.listReservations(request);
-
-      // Ensure all reservations are filtered out.
-      Assert.assertNotNull(response);
-      Assert.assertEquals(response.getReservationAllocationState().size(), 0);
-
-      arrival = clock.getTime();
-      // List reservations, search by end time before the reservation start
-      // time.
-      request = ReservationListRequest.newInstance(
-          ReservationSystemTestUtil.reservationQ, "", 0, arrival - duration,
-          false);
-
-      response = client.listReservations(request);
-
-      // Ensure all reservations are filtered out.
-      Assert.assertNotNull(response);
-      Assert.assertEquals(response.getReservationAllocationState().size(), 0);
-
-      // List reservations, search by very small end time.
-      request = ReservationListRequest
-          .newInstance(ReservationSystemTestUtil.reservationQ, "", 0, 1, false);
-
-      response = client.listReservations(request);
-
-      // Ensure all reservations are filtered out.
-      Assert.assertNotNull(response);
-      Assert.assertEquals(response.getReservationAllocationState().size(), 0);
-
-    } finally {
-      // clean-up
-      if (client != null) {
-        client.stop();
-      }
-      cluster.stop();
-    }
-  }
-
-  @Test
-  public void testReservationDelete() throws Exception {
-    MiniYARNCluster cluster = setupMiniYARNCluster();
-    YarnClient client = setupYarnClient(cluster);
-    try {
-      Clock clock = new UTCClock();
-      long arrival = clock.getTime();
-      long duration = 60000;
-      long deadline = (long) (arrival + 1.05 * duration);
-      ReservationSubmissionRequest sRequest =
-          submitReservationTestHelper(client, arrival, deadline, duration);
-
-      ReservationId reservationID = sRequest.getReservationId();
-      // Delete the reservation
-      ReservationDeleteRequest dRequest =
-          ReservationDeleteRequest.newInstance(reservationID);
-      ReservationDeleteResponse dResponse = client.deleteReservation(dRequest);
-      Assert.assertNotNull(dResponse);
-      System.out.println("Delete reservation response: " + dResponse);
-
-      // List reservations, search by non-existent reservationID
-      ReservationListRequest request = ReservationListRequest.newInstance(
-          ReservationSystemTestUtil.reservationQ, reservationID.toString(), -1,
-          -1, false);
-
-      ReservationListResponse response =  client.listReservations(request);
-      Assert.assertNotNull(response);
-      Assert.assertEquals(0, response.getReservationAllocationState().size());
-    } finally {
-      // clean-up
-      if (client != null) {
-        client.stop();
-      }
-      cluster.stop();
-    }
-  }
-
-  private ReservationSubmissionRequest createSimpleReservationRequest(
-      ReservationId reservationId, int numContainers, long arrival,
-      long deadline, long duration) {
-    // create a request with a single atomic ask
-    ReservationRequest r =
-        ReservationRequest.newInstance(Resource.newInstance(1024, 1),
-            numContainers, 1, duration);
-    ReservationRequests reqs =
-        ReservationRequests.newInstance(Collections.singletonList(r),
-            ReservationRequestInterpreter.R_ALL);
-    ReservationDefinition rDef =
-        ReservationDefinition.newInstance(arrival, deadline, reqs,
-            "testYarnClient#reservation");
-    ReservationSubmissionRequest request =
-        ReservationSubmissionRequest.newInstance(rDef,
-            ReservationSystemTestUtil.reservationQ, reservationId);
-    return request;
-  }
-
   @Test(timeout = 30000, expected = ApplicationNotFoundException.class)
   public void testShouldNotRetryForeverForNonNetworkExceptions() throws Exception {
-    YarnConfiguration conf = new YarnConfiguration();
+    YarnConfiguration conf = getConf();
     conf.setInt(YarnConfiguration.RESOURCEMANAGER_CONNECT_MAX_WAIT_MS, -1);
 
     ResourceManager rm = null;
@@ -1715,10 +1321,9 @@ public class TestYarnClient {
 
   @Test
   public void testSignalContainer() throws Exception {
-    Configuration conf = new Configuration();
     @SuppressWarnings("resource")
     final YarnClient client = new MockYarnClient();
-    client.init(conf);
+    client.init(getConf());
     client.start();
     ApplicationId applicationId = ApplicationId.newInstance(1234, 5);
     ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(
@@ -1741,7 +1346,7 @@ public class TestYarnClient {
       boolean timelineClientBestEffort,
       Throwable mockErr,
       CreateTimelineClientErrorVerifier errVerifier) throws Exception {
-    Configuration conf = new Configuration();
+    Configuration conf = getConf();
     conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED,
         timelineServiceEnabled);
     conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_CLIENT_BEST_EFFORT,

+ 521 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClientWithReservation.java

@@ -0,0 +1,521 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.client.api.impl;
+
+import com.google.common.base.Supplier;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationListRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationListResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateResponse;
+import org.apache.hadoop.yarn.api.records.ReservationDefinition;
+import org.apache.hadoop.yarn.api.records.ReservationId;
+import org.apache.hadoop.yarn.api.records.ReservationRequest;
+import org.apache.hadoop.yarn.api.records.ReservationRequestInterpreter;
+import org.apache.hadoop.yarn.api.records.ReservationRequests;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.client.api.YarnClient;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.MiniYARNCluster;
+import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSystemTestUtil;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairSchedulerConfiguration;
+import org.apache.hadoop.yarn.util.Clock;
+import org.apache.hadoop.yarn.util.UTCClock;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.stream.Collectors;
+
+
+/**
+ * This class is to test class {@link YarnClient) and {@link YarnClientImpl}
+ * with Reservation.
+ */
+@RunWith(Parameterized.class)
+public class TestYarnClientWithReservation {
+  protected final static String TEST_DIR =
+      new File(System.getProperty("test.build.data", "/tmp")).getAbsolutePath();
+  protected final static String FS_ALLOC_FILE =
+      new File(TEST_DIR, "test-fs-queues.xml").getAbsolutePath();
+
+  public enum SchedulerType {
+    CAPACITY, FAIR
+  }
+
+  private SchedulerType schedulerType;
+
+  @Parameterized.Parameters(name = "{0}")
+  public static Collection<Object[]> getParameters() {
+    return Arrays.stream(SchedulerType.values()).map(
+        type -> new Object[]{type}).collect(Collectors.toList());
+  }
+
+  public TestYarnClientWithReservation(SchedulerType scheduler) {
+    this.schedulerType = scheduler;
+  }
+
+
+  private MiniYARNCluster setupMiniYARNCluster() throws Exception {
+    MiniYARNCluster cluster =
+        new MiniYARNCluster("testReservationAPIs", 2, 1, 1);
+
+    cluster.init(getConfigurationForReservation());
+    cluster.start();
+
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        return cluster.getResourceManager().getRMContext()
+            .getReservationSystem()
+            .getPlan(ReservationSystemTestUtil.reservationQ)
+            .getTotalCapacity().getMemorySize() > 6000;
+      }
+    }, 10, 10000);
+
+    return cluster;
+  }
+
+  private Configuration getConfigurationForReservation() {
+    Configuration conf = new Configuration();
+    if (schedulerType == SchedulerType.FAIR) {
+      conf = configureReservationForFairScheduler();
+      conf.set(YarnConfiguration.RM_SCHEDULER, FairScheduler.class.getName());
+    } else if (schedulerType == SchedulerType.CAPACITY) {
+      conf = configureReservationForCapacityScheduler();
+      conf.set(YarnConfiguration.RM_SCHEDULER,
+          CapacityScheduler.class.getName());
+    }
+
+    conf.setBoolean(YarnConfiguration.RM_RESERVATION_SYSTEM_ENABLE, true);
+    return conf;
+  }
+
+  private Configuration configureReservationForCapacityScheduler() {
+    CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
+    ReservationSystemTestUtil.setupQueueConfiguration(conf);
+    return conf;
+  }
+
+  private Configuration configureReservationForFairScheduler() {
+    Configuration conf = new Configuration();
+    try {
+      PrintWriter out = new PrintWriter(new FileWriter(FS_ALLOC_FILE));
+      out.println("<?xml version=\"1.0\"?>");
+      out.println("<allocations>");
+      out.println("<queue name=\"root\">");
+      out.println("  <queue name=\"default\"></queue>");
+      out.println("  <queue name=\"dedicated\">");
+      out.println("    <reservation></reservation>");
+      // set weight to 10 to make sure this queue get enough steady fair share
+      out.println("    <weight>10</weight>");
+      out.println("  </queue>");
+      out.println("</queue>");
+      out.println("<defaultQueueSchedulingPolicy>drf" +
+          "</defaultQueueSchedulingPolicy>");
+      out.println("</allocations>");
+      out.close();
+    } catch (IOException e) {
+      Assert.fail(e.getMessage());
+    }
+
+    conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, FS_ALLOC_FILE);
+    return conf;
+  }
+
+  private YarnClient setupYarnClient(MiniYARNCluster cluster) {
+    final Configuration yarnConf = cluster.getConfig();
+    YarnClient client = YarnClient.createYarnClient();
+    client.init(yarnConf);
+    client.start();
+    return client;
+  }
+
+  private ReservationSubmissionRequest submitReservationTestHelper(
+      YarnClient client, long arrival, long deadline, long duration)
+      throws IOException, YarnException {
+    ReservationId reservationID = client.createReservation().getReservationId();
+    ReservationSubmissionRequest sRequest = createSimpleReservationRequest(
+        reservationID, 4, arrival, deadline, duration);
+    ReservationSubmissionResponse sResponse =
+        client.submitReservation(sRequest);
+    Assert.assertNotNull(sResponse);
+    Assert.assertNotNull(reservationID);
+    System.out.println("Submit reservation response: " + reservationID);
+
+    return sRequest;
+  }
+
+  @Before
+  public void setup() {
+    QueueMetrics.clearQueueMetrics();
+    DefaultMetricsSystem.setMiniClusterMode(true);
+  }
+
+  @Test
+  public void testCreateReservation() throws Exception {
+    MiniYARNCluster cluster = setupMiniYARNCluster();
+    YarnClient client = setupYarnClient(cluster);
+    try {
+      Clock clock = new UTCClock();
+      long arrival = clock.getTime();
+      long duration = 60000;
+      long deadline = (long) (arrival + 1.05 * duration);
+      ReservationSubmissionRequest sRequest =
+          submitReservationTestHelper(client, arrival, deadline, duration);
+
+      // Submit the reservation again with the same request and make sure it
+      // passes.
+      client.submitReservation(sRequest);
+
+      // Submit the reservation with the same reservation id but different
+      // reservation definition, and ensure YarnException is thrown.
+      arrival = clock.getTime();
+      ReservationDefinition rDef = sRequest.getReservationDefinition();
+      rDef.setArrival(arrival + duration);
+      sRequest.setReservationDefinition(rDef);
+      try {
+        client.submitReservation(sRequest);
+        Assert.fail("Reservation submission should fail if a duplicate "
+            + "reservation id is used, but the reservation definition has been "
+            + "updated.");
+      } catch (Exception e) {
+        Assert.assertTrue(e instanceof YarnException);
+      }
+    } finally {
+      // clean-up
+      if (client != null) {
+        client.stop();
+      }
+      cluster.stop();
+    }
+  }
+
+  @Test
+  public void testUpdateReservation() throws Exception {
+    MiniYARNCluster cluster = setupMiniYARNCluster();
+    YarnClient client = setupYarnClient(cluster);
+    try {
+      Clock clock = new UTCClock();
+      long arrival = clock.getTime();
+      long duration = 60000;
+      long deadline = (long) (arrival + 1.05 * duration);
+      ReservationSubmissionRequest sRequest =
+          submitReservationTestHelper(client, arrival, deadline, duration);
+
+      ReservationDefinition rDef = sRequest.getReservationDefinition();
+      ReservationRequest rr =
+          rDef.getReservationRequests().getReservationResources().get(0);
+      ReservationId reservationID = sRequest.getReservationId();
+      rr.setNumContainers(5);
+      arrival = clock.getTime();
+      duration = 30000;
+      deadline = (long) (arrival + 1.05 * duration);
+      rr.setDuration(duration);
+      rDef.setArrival(arrival);
+      rDef.setDeadline(deadline);
+      ReservationUpdateRequest uRequest =
+          ReservationUpdateRequest.newInstance(rDef, reservationID);
+      ReservationUpdateResponse uResponse = client.updateReservation(uRequest);
+      Assert.assertNotNull(uResponse);
+      System.out.println("Update reservation response: " + uResponse);
+    } finally {
+      // clean-up
+      if (client != null) {
+        client.stop();
+      }
+      cluster.stop();
+    }
+  }
+
+  private ReservationSubmissionRequest createSimpleReservationRequest(
+      ReservationId reservationId, int numContainers, long arrival,
+      long deadline, long duration) {
+    // create a request with a single atomic ask
+    ReservationRequest r =
+        ReservationRequest.newInstance(Resource.newInstance(1024, 1),
+            numContainers, 1, duration);
+    ReservationRequests reqs =
+        ReservationRequests.newInstance(Collections.singletonList(r),
+            ReservationRequestInterpreter.R_ALL);
+    ReservationDefinition rDef =
+        ReservationDefinition.newInstance(arrival, deadline, reqs,
+            "testYarnClient#reservation");
+    ReservationSubmissionRequest request =
+        ReservationSubmissionRequest.newInstance(rDef,
+            ReservationSystemTestUtil.reservationQ, reservationId);
+    return request;
+  }
+
+
+  @Test
+  public void testListReservationsByReservationId() throws Exception{
+    MiniYARNCluster cluster = setupMiniYARNCluster();
+    YarnClient client = setupYarnClient(cluster);
+    try {
+      Clock clock = new UTCClock();
+      long arrival = clock.getTime();
+      long duration = 60000;
+      long deadline = (long) (arrival + 1.05 * duration);
+      ReservationSubmissionRequest sRequest =
+          submitReservationTestHelper(client, arrival, deadline, duration);
+
+      ReservationId reservationID = sRequest.getReservationId();
+      ReservationListRequest request = ReservationListRequest.newInstance(
+          ReservationSystemTestUtil.reservationQ, reservationID.toString(), -1,
+          -1, false);
+      ReservationListResponse response = client.listReservations(request);
+      Assert.assertNotNull(response);
+      Assert.assertEquals(1, response.getReservationAllocationState().size());
+      Assert.assertEquals(response.getReservationAllocationState().get(0)
+          .getReservationId().getId(), reservationID.getId());
+      Assert.assertEquals(response.getReservationAllocationState().get(0)
+          .getResourceAllocationRequests().size(), 0);
+    } finally {
+      // clean-up
+      if (client != null) {
+        client.stop();
+      }
+      cluster.stop();
+    }
+  }
+
+  @Test
+  public void testListReservationsByTimeInterval() throws Exception {
+    MiniYARNCluster cluster = setupMiniYARNCluster();
+    YarnClient client = setupYarnClient(cluster);
+    try {
+      Clock clock = new UTCClock();
+      long arrival = clock.getTime();
+      long duration = 60000;
+      long deadline = (long) (arrival + 1.05 * duration);
+      ReservationSubmissionRequest sRequest =
+          submitReservationTestHelper(client, arrival, deadline, duration);
+
+      // List reservations, search by a point in time within the reservation
+      // range.
+      arrival = clock.getTime();
+      ReservationId reservationID = sRequest.getReservationId();
+      ReservationListRequest request = ReservationListRequest.newInstance(
+          ReservationSystemTestUtil.reservationQ, "", arrival + duration / 2,
+          arrival + duration / 2, true);
+
+      ReservationListResponse response = client.listReservations(request);
+      Assert.assertNotNull(response);
+      Assert.assertEquals(1, response.getReservationAllocationState().size());
+      Assert.assertEquals(response.getReservationAllocationState().get(0)
+          .getReservationId().getId(), reservationID.getId());
+      // List reservations, search by time within reservation interval.
+      request = ReservationListRequest.newInstance(
+          ReservationSystemTestUtil.reservationQ, "", 1, Long.MAX_VALUE, true);
+
+      response = client.listReservations(request);
+      Assert.assertNotNull(response);
+      Assert.assertEquals(1, response.getReservationAllocationState().size());
+      Assert.assertEquals(response.getReservationAllocationState().get(0)
+          .getReservationId().getId(), reservationID.getId());
+      // Verify that the full resource allocations exist.
+      Assert.assertTrue(response.getReservationAllocationState().get(0)
+          .getResourceAllocationRequests().size() > 0);
+
+      // Verify that the full RDL is returned.
+      ReservationRequests reservationRequests =
+          response.getReservationAllocationState().get(0)
+              .getReservationDefinition().getReservationRequests();
+      Assert.assertEquals("R_ALL",
+          reservationRequests.getInterpreter().toString());
+      Assert.assertTrue(reservationRequests.getReservationResources().get(0)
+          .getDuration() == duration);
+    } finally {
+      // clean-up
+      if (client != null) {
+        client.stop();
+      }
+      cluster.stop();
+    }
+  }
+
+  @Test
+  public void testListReservationsByInvalidTimeInterval() throws Exception {
+    MiniYARNCluster cluster = setupMiniYARNCluster();
+    YarnClient client = setupYarnClient(cluster);
+    try {
+      Clock clock = new UTCClock();
+      long arrival = clock.getTime();
+      long duration = 60000;
+      long deadline = (long) (arrival + 1.05 * duration);
+      ReservationSubmissionRequest sRequest =
+          submitReservationTestHelper(client, arrival, deadline, duration);
+
+      // List reservations, search by invalid end time == -1.
+      ReservationListRequest request = ReservationListRequest
+          .newInstance(ReservationSystemTestUtil.reservationQ, "", 1, -1, true);
+
+      ReservationListResponse response = client.listReservations(request);
+      Assert.assertNotNull(response);
+      Assert.assertEquals(1, response.getReservationAllocationState().size());
+      Assert.assertEquals(response.getReservationAllocationState().get(0)
+          .getReservationId().getId(), sRequest.getReservationId().getId());
+
+      // List reservations, search by invalid end time < -1.
+      request = ReservationListRequest.newInstance(
+          ReservationSystemTestUtil.reservationQ, "", 1, -10, true);
+
+      response = client.listReservations(request);
+      Assert.assertNotNull(response);
+      Assert.assertEquals(1, response.getReservationAllocationState().size());
+      Assert.assertEquals(response.getReservationAllocationState().get(0)
+          .getReservationId().getId(), sRequest.getReservationId().getId());
+    } finally {
+      // clean-up
+      if (client != null) {
+        client.stop();
+      }
+      cluster.stop();
+    }
+  }
+
+  @Test
+  public void testListReservationsByTimeIntervalContainingNoReservations()
+      throws Exception {
+    MiniYARNCluster cluster = setupMiniYARNCluster();
+    YarnClient client = setupYarnClient(cluster);
+    try {
+      Clock clock = new UTCClock();
+      long arrival = clock.getTime();
+      long duration = 60000;
+      long deadline = (long) (arrival + 1.05 * duration);
+      ReservationSubmissionRequest sRequest =
+          submitReservationTestHelper(client, arrival, deadline, duration);
+
+      // List reservations, search by very large start time.
+      ReservationListRequest request = ReservationListRequest.newInstance(
+          ReservationSystemTestUtil.reservationQ, "", Long.MAX_VALUE, -1,
+          false);
+
+      ReservationListResponse response = client.listReservations(request);
+
+      // Ensure all reservations are filtered out.
+      Assert.assertNotNull(response);
+      Assert.assertEquals(response.getReservationAllocationState().size(), 0);
+
+      duration = 30000;
+      deadline = sRequest.getReservationDefinition().getDeadline();
+
+      // List reservations, search by start time after the reservation
+      // end time.
+      request = ReservationListRequest.newInstance(
+          ReservationSystemTestUtil.reservationQ, "", deadline + duration,
+          deadline + 2 * duration, false);
+
+      response = client.listReservations(request);
+
+      // Ensure all reservations are filtered out.
+      Assert.assertNotNull(response);
+      Assert.assertEquals(response.getReservationAllocationState().size(), 0);
+
+      arrival = clock.getTime();
+      // List reservations, search by end time before the reservation start
+      // time.
+      request = ReservationListRequest.newInstance(
+          ReservationSystemTestUtil.reservationQ, "", 0, arrival - duration,
+          false);
+
+      response = client.listReservations(request);
+
+      // Ensure all reservations are filtered out.
+      Assert.assertNotNull(response);
+      Assert.assertEquals(response.getReservationAllocationState().size(), 0);
+
+      // List reservations, search by very small end time.
+      request = ReservationListRequest
+          .newInstance(ReservationSystemTestUtil.reservationQ, "", 0, 1, false);
+
+      response = client.listReservations(request);
+
+      // Ensure all reservations are filtered out.
+      Assert.assertNotNull(response);
+      Assert.assertEquals(response.getReservationAllocationState().size(), 0);
+
+    } finally {
+      // clean-up
+      if (client != null) {
+        client.stop();
+      }
+      cluster.stop();
+    }
+  }
+
+  @Test
+  public void testReservationDelete() throws Exception {
+    MiniYARNCluster cluster = setupMiniYARNCluster();
+    YarnClient client = setupYarnClient(cluster);
+    try {
+      Clock clock = new UTCClock();
+      long arrival = clock.getTime();
+      long duration = 60000;
+      long deadline = (long) (arrival + 1.05 * duration);
+      ReservationSubmissionRequest sRequest =
+          submitReservationTestHelper(client, arrival, deadline, duration);
+
+      ReservationId reservationID = sRequest.getReservationId();
+      // Delete the reservation
+      ReservationDeleteRequest dRequest =
+          ReservationDeleteRequest.newInstance(reservationID);
+      ReservationDeleteResponse dResponse = client.deleteReservation(dRequest);
+      Assert.assertNotNull(dResponse);
+      System.out.println("Delete reservation response: " + dResponse);
+
+      // List reservations, search by non-existent reservationID
+      ReservationListRequest request = ReservationListRequest.newInstance(
+          ReservationSystemTestUtil.reservationQ, reservationID.toString(), -1,
+          -1, false);
+
+      ReservationListResponse response =  client.listReservations(request);
+      Assert.assertNotNull(response);
+      Assert.assertEquals(0, response.getReservationAllocationState().size());
+    } finally {
+      // clean-up
+      if (client != null) {
+        client.stop();
+      }
+      cluster.stop();
+    }
+  }
+
+}

+ 8 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebApps.java

@@ -394,16 +394,22 @@ public class WebApps {
     }
 
     public WebApp start(WebApp webapp) {
-      return start(webapp, null);
+      return start(webapp, null, null);
     }
 
-    public WebApp start(WebApp webapp, WebAppContext ui2Context) {
+    public WebApp start(WebApp webapp, WebAppContext ui2Context,
+        Map<String, String> services) {
       WebApp webApp = build(webapp);
       HttpServer2 httpServer = webApp.httpServer();
       if (ui2Context != null) {
         addFiltersForNewContext(ui2Context);
         httpServer.addHandlerAtFront(ui2Context);
       }
+      if (services!=null) {
+        String packageName = services.get("PackageName");
+        String pathSpec = services.get("PathSpec");
+        httpServer.addJerseyResourcePackage(packageName, pathSpec);
+      }
       try {
         httpServer.start();
         LOG.info("Web app " + name + " started at "

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

@@ -2826,7 +2826,20 @@
    <value>300</value>
   </property>
 
+  <property>
+    <description>The registry base directory for federation.</description>
+    <name>yarn.federation.registry.base-dir</name>
+    <value>yarnfederation/</value>
+  </property>
+
   <!-- Other Configuration -->
+
+  <property>
+    <description>The registry implementation to use.</description>
+    <name>yarn.registry.class</name>
+    <value>org.apache.hadoop.registry.client.impl.FSRegistryOperationsService</value>
+  </property>
+
   <property>
     <description>The interval that the yarn client library uses to poll the
     completion status of the asynchronous API of application client protocol.
@@ -2987,6 +3000,14 @@
     <value>org.apache.hadoop.yarn.server.nodemanager.amrmproxy.DefaultRequestInterceptor</value>
   </property>
 
+  <property>
+    <description>
+    Whether AMRMProxy HA is enabled.
+    </description>
+    <name>yarn.nodemanager.amrmproxy.ha.enable</name>
+    <value>false</value>
+  </property>
+
   <property>
     <description>
     Setting that controls whether distributed scheduling is enabled.

+ 47 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestResourceRequest.java

@@ -0,0 +1,47 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.api;
+
+import org.apache.hadoop.yarn.api.records.ExecutionType;
+import org.apache.hadoop.yarn.api.records.ExecutionTypeRequest;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * The class to test {@link ResourceRequest}.
+ */
+public class TestResourceRequest {
+
+  @Test
+  public void testEqualsOnExecutionTypeRequest() {
+    ResourceRequest resourceRequestA =
+        ResourceRequest.newInstance(Priority.newInstance(0), "localhost",
+            Resource.newInstance(1024, 1), 1, false, "",
+            ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED, true));
+
+    ResourceRequest resourceRequestB =
+        ResourceRequest.newInstance(Priority.newInstance(0), "localhost",
+            Resource.newInstance(1024, 1), 1, false, "",
+            ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED, false));
+
+    Assert.assertFalse(resourceRequestA.equals(resourceRequestB));
+  }
+}

+ 5 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml

@@ -66,6 +66,11 @@
       <scope>test</scope>
     </dependency>
 
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-registry</artifactId>
+    </dependency>
+
     <dependency>
       <groupId>com.google.guava</groupId>
       <artifactId>guava</artifactId>

+ 338 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/utils/FederationRegistryClient.java

@@ -0,0 +1,338 @@
+/**
+ * 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.federation.utils;
+
+import java.io.IOException;
+import java.security.PrivilegedAction;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.registry.client.api.BindFlags;
+import org.apache.hadoop.registry.client.api.RegistryOperations;
+import org.apache.hadoop.registry.client.types.ServiceRecord;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Helper class that handles reads and writes to Yarn Registry to support UAM HA
+ * and second attempt.
+ */
+public class FederationRegistryClient {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(FederationRegistryClient.class);
+
+  private RegistryOperations registry;
+
+  private UserGroupInformation user;
+
+  // AppId -> SubClusterId -> UAM token
+  private Map<ApplicationId, Map<String, Token<AMRMTokenIdentifier>>>
+      appSubClusterTokenMap;
+
+  // Structure in registry: <registryBaseDir>/<AppId>/<SubClusterId> -> UAMToken
+  private String registryBaseDir;
+
+  public FederationRegistryClient(Configuration conf,
+      RegistryOperations registry, UserGroupInformation user) {
+    this.registry = registry;
+    this.user = user;
+    this.appSubClusterTokenMap = new ConcurrentHashMap<>();
+    this.registryBaseDir =
+        conf.get(YarnConfiguration.FEDERATION_REGISTRY_BASE_KEY,
+            YarnConfiguration.DEFAULT_FEDERATION_REGISTRY_BASE_KEY);
+    LOG.info("Using registry {} with base directory: {}",
+        this.registry.getClass().getName(), this.registryBaseDir);
+  }
+
+  /**
+   * Get the list of known applications in the registry.
+   *
+   * @return the list of known applications
+   */
+  public List<String> getAllApplications() {
+    // Suppress the exception here because it is valid that the entry does not
+    // exist
+    List<String> applications = null;
+    try {
+      applications = listDirRegistry(this.registry, this.user,
+          getRegistryKey(null, null), false);
+    } catch (YarnException e) {
+      LOG.warn("Unexpected exception from listDirRegistry", e);
+    }
+    if (applications == null) {
+      // It is valid for listDirRegistry to return null
+      return new ArrayList<>();
+    }
+    return applications;
+  }
+
+  /**
+   * For testing, delete all application records in registry.
+   */
+  @VisibleForTesting
+  public void cleanAllApplications() {
+    try {
+      removeKeyRegistry(this.registry, this.user, getRegistryKey(null, null),
+          true, false);
+    } catch (YarnException e) {
+      LOG.warn("Unexpected exception from removeKeyRegistry", e);
+    }
+  }
+
+  /**
+   * Write/update the UAM token for an application and a sub-cluster.
+   *
+   * @param subClusterId sub-cluster id of the token
+   * @param token the UAM of the application
+   * @return whether the amrmToken is added or updated to a new value
+   */
+  public boolean writeAMRMTokenForUAM(ApplicationId appId,
+      String subClusterId, Token<AMRMTokenIdentifier> token) {
+    Map<String, Token<AMRMTokenIdentifier>> subClusterTokenMap =
+        this.appSubClusterTokenMap.get(appId);
+    if (subClusterTokenMap == null) {
+      subClusterTokenMap = new ConcurrentHashMap<>();
+      this.appSubClusterTokenMap.put(appId, subClusterTokenMap);
+    }
+
+    boolean update = !token.equals(subClusterTokenMap.get(subClusterId));
+    if (!update) {
+      LOG.debug("Same amrmToken received from {}, skip writing registry for {}",
+          subClusterId, appId);
+      return update;
+    }
+
+    LOG.info("Writing/Updating amrmToken for {} to registry for {}",
+        subClusterId, appId);
+    try {
+      // First, write the token entry
+      writeRegistry(this.registry, this.user,
+          getRegistryKey(appId, subClusterId), token.encodeToUrlString(), true);
+
+      // Then update the subClusterTokenMap
+      subClusterTokenMap.put(subClusterId, token);
+    } catch (YarnException | IOException e) {
+      LOG.error(
+          "Failed writing AMRMToken to registry for subcluster " + subClusterId,
+          e);
+    }
+    return update;
+  }
+
+  /**
+   * Load the information of one application from registry.
+   *
+   * @param appId application id
+   * @return the sub-cluster to UAM token mapping
+   */
+  public Map<String, Token<AMRMTokenIdentifier>>
+      loadStateFromRegistry(ApplicationId appId) {
+    Map<String, Token<AMRMTokenIdentifier>> retMap = new HashMap<>();
+    // Suppress the exception here because it is valid that the entry does not
+    // exist
+    List<String> subclusters = null;
+    try {
+      subclusters = listDirRegistry(this.registry, this.user,
+          getRegistryKey(appId, null), false);
+    } catch (YarnException e) {
+      LOG.warn("Unexpected exception from listDirRegistry", e);
+    }
+
+    if (subclusters == null) {
+      LOG.info("Application {} does not exist in registry", appId);
+      return retMap;
+    }
+
+    // Read the amrmToken for each sub-cluster with an existing UAM
+    for (String scId : subclusters) {
+      LOG.info("Reading amrmToken for subcluster {} for {}", scId, appId);
+      String key = getRegistryKey(appId, scId);
+      try {
+        String tokenString = readRegistry(this.registry, this.user, key, true);
+        if (tokenString == null) {
+          throw new YarnException("Null string from readRegistry key " + key);
+        }
+        Token<AMRMTokenIdentifier> amrmToken = new Token<>();
+        amrmToken.decodeFromUrlString(tokenString);
+        // Clear the service field, as if RM just issued the token
+        amrmToken.setService(new Text());
+
+        retMap.put(scId, amrmToken);
+      } catch (Exception e) {
+        LOG.error("Failed reading registry key " + key
+            + ", skipping subcluster " + scId, e);
+      }
+    }
+
+    // Override existing map if there
+    this.appSubClusterTokenMap.put(appId, new ConcurrentHashMap<>(retMap));
+    return retMap;
+  }
+
+  /**
+   * Remove an application from registry.
+   *
+   * @param appId application id
+   */
+  public void removeAppFromRegistry(ApplicationId appId) {
+    Map<String, Token<AMRMTokenIdentifier>> subClusterTokenMap =
+        this.appSubClusterTokenMap.get(appId);
+    LOG.info("Removing all registry entries for {}", appId);
+
+    if (subClusterTokenMap == null || subClusterTokenMap.size() == 0) {
+      return;
+    }
+
+    // Lastly remove the application directory
+    String key = getRegistryKey(appId, null);
+    try {
+      removeKeyRegistry(this.registry, this.user, key, true, true);
+      subClusterTokenMap.clear();
+    } catch (YarnException e) {
+      LOG.error("Failed removing registry directory key " + key, e);
+    }
+  }
+
+  private String getRegistryKey(ApplicationId appId, String fileName) {
+    if (appId == null) {
+      return this.registryBaseDir;
+    }
+    if (fileName == null) {
+      return this.registryBaseDir + appId.toString();
+    }
+    return this.registryBaseDir + appId.toString() + "/" + fileName;
+  }
+
+  private String readRegistry(final RegistryOperations registryImpl,
+      UserGroupInformation ugi, final String key, final boolean throwIfFails)
+      throws YarnException {
+    // Use the ugi loaded with app credentials to access registry
+    String result = ugi.doAs(new PrivilegedAction<String>() {
+      @Override
+      public String run() {
+        try {
+          ServiceRecord value = registryImpl.resolve(key);
+          if (value != null) {
+            return value.description;
+          }
+        } catch (Throwable e) {
+          if (throwIfFails) {
+            LOG.error("Registry resolve key " + key + " failed", e);
+          }
+        }
+        return null;
+      }
+    });
+    if (result == null && throwIfFails) {
+      throw new YarnException("Registry resolve key " + key + " failed");
+    }
+    return result;
+  }
+
+  private void removeKeyRegistry(final RegistryOperations registryImpl,
+      UserGroupInformation ugi, final String key, final boolean recursive,
+      final boolean throwIfFails) throws YarnException {
+    // Use the ugi loaded with app credentials to access registry
+    boolean success = ugi.doAs(new PrivilegedAction<Boolean>() {
+      @Override
+      public Boolean run() {
+        try {
+          registryImpl.delete(key, recursive);
+          return true;
+        } catch (Throwable e) {
+          if (throwIfFails) {
+            LOG.error("Registry remove key " + key + " failed", e);
+          }
+        }
+        return false;
+      }
+    });
+    if (!success && throwIfFails) {
+      throw new YarnException("Registry remove key " + key + " failed");
+    }
+  }
+
+  /**
+   * Write registry entry, override if exists.
+   */
+  private void writeRegistry(final RegistryOperations registryImpl,
+      UserGroupInformation ugi, final String key, final String value,
+      final boolean throwIfFails) throws YarnException {
+
+    final ServiceRecord recordValue = new ServiceRecord();
+    recordValue.description = value;
+    // Use the ugi loaded with app credentials to access registry
+    boolean success = ugi.doAs(new PrivilegedAction<Boolean>() {
+      @Override
+      public Boolean run() {
+        try {
+          registryImpl.bind(key, recordValue, BindFlags.OVERWRITE);
+          return true;
+        } catch (Throwable e) {
+          if (throwIfFails) {
+            LOG.error("Registry write key " + key + " failed", e);
+          }
+        }
+        return false;
+      }
+    });
+    if (!success && throwIfFails) {
+      throw new YarnException("Registry write key " + key + " failed");
+    }
+  }
+
+  /**
+   * List the sub directories in the given directory.
+   */
+  private List<String> listDirRegistry(final RegistryOperations registryImpl,
+      UserGroupInformation ugi, final String key, final boolean throwIfFails)
+      throws YarnException {
+    List<String> result = ugi.doAs(new PrivilegedAction<List<String>>() {
+      @Override
+      public List<String> run() {
+        try {
+          return registryImpl.list(key);
+        } catch (Throwable e) {
+          if (throwIfFails) {
+            LOG.error("Registry list key " + key + " failed", e);
+          }
+        }
+        return null;
+      }
+    });
+    if (result == null && throwIfFails) {
+      throw new YarnException("Registry list key " + key + " failed");
+    }
+    return result;
+  }
+
+}

+ 109 - 32
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/uam/UnmanagedAMPoolManager.java

@@ -33,6 +33,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
@@ -44,9 +45,10 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
-import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
 import org.apache.hadoop.yarn.server.utils.AMRMClientUtils;
 import org.apache.hadoop.yarn.util.AsyncCallback;
 import org.slf4j.Logger;
@@ -67,7 +69,7 @@ public class UnmanagedAMPoolManager extends AbstractService {
   // Map from uamId to UAM instances
   private Map<String, UnmanagedApplicationManager> unmanagedAppMasterMap;
 
-  private Map<String, ApplicationAttemptId> attemptIdMap;
+  private Map<String, ApplicationId> appIdMap;
 
   private ExecutorService threadpool;
 
@@ -82,7 +84,7 @@ public class UnmanagedAMPoolManager extends AbstractService {
       this.threadpool = Executors.newCachedThreadPool();
     }
     this.unmanagedAppMasterMap = new ConcurrentHashMap<>();
-    this.attemptIdMap = new ConcurrentHashMap<>();
+    this.appIdMap = new ConcurrentHashMap<>();
     super.serviceStart();
   }
 
@@ -114,7 +116,7 @@ public class UnmanagedAMPoolManager extends AbstractService {
         public KillApplicationResponse call() throws Exception {
           try {
             LOG.info("Force-killing UAM id " + uamId + " for application "
-                + attemptIdMap.get(uamId));
+                + appIdMap.get(uamId));
             return unmanagedAppMasterMap.remove(uamId).forceKillApplication();
           } catch (Exception e) {
             LOG.error("Failed to kill unmanaged application master", e);
@@ -132,7 +134,7 @@ public class UnmanagedAMPoolManager extends AbstractService {
         LOG.error("Failed to kill unmanaged application master", e);
       }
     }
-    this.attemptIdMap.clear();
+    this.appIdMap.clear();
     super.serviceStop();
   }
 
@@ -145,13 +147,18 @@ public class UnmanagedAMPoolManager extends AbstractService {
    * @param queueName queue of the application
    * @param submitter submitter name of the UAM
    * @param appNameSuffix application name suffix for the UAM
+   * @param keepContainersAcrossApplicationAttempts keep container flag for UAM
+   *          recovery.
+   * @see ApplicationSubmissionContext
+   *          #setKeepContainersAcrossApplicationAttempts(boolean)
    * @return uamId for the UAM
    * @throws YarnException if registerApplicationMaster fails
    * @throws IOException if registerApplicationMaster fails
    */
   public String createAndRegisterNewUAM(
       RegisterApplicationMasterRequest registerRequest, Configuration conf,
-      String queueName, String submitter, String appNameSuffix)
+      String queueName, String submitter, String appNameSuffix,
+      boolean keepContainersAcrossApplicationAttempts)
       throws YarnException, IOException {
     ApplicationId appId = null;
     ApplicationClientProtocol rmClient;
@@ -173,45 +180,93 @@ public class UnmanagedAMPoolManager extends AbstractService {
       rmClient = null;
     }
 
-    createAndRegisterNewUAM(appId.toString(), registerRequest, conf, appId,
-        queueName, submitter, appNameSuffix);
+    // Launch the UAM in RM
+    launchUAM(appId.toString(), conf, appId, queueName, submitter,
+        appNameSuffix, keepContainersAcrossApplicationAttempts);
+
+    // Register the UAM application
+    registerApplicationMaster(appId.toString(), registerRequest);
+
+    // Returns the appId as uamId
     return appId.toString();
   }
 
   /**
-   * Create a new UAM and register the application, using the provided uamId and
-   * appId.
+   * Launch a new UAM, using the provided uamId and appId.
    *
-   * @param uamId identifier for the UAM
-   * @param registerRequest RegisterApplicationMasterRequest
+   * @param uamId uam Id
    * @param conf configuration for this UAM
    * @param appId application id for the UAM
    * @param queueName queue of the application
    * @param submitter submitter name of the UAM
    * @param appNameSuffix application name suffix for the UAM
-   * @return RegisterApplicationMasterResponse
-   * @throws YarnException if registerApplicationMaster fails
-   * @throws IOException if registerApplicationMaster fails
+   * @param keepContainersAcrossApplicationAttempts keep container flag for UAM
+   *          recovery.
+   * @see ApplicationSubmissionContext
+   *          #setKeepContainersAcrossApplicationAttempts(boolean)
+   * @return UAM token
+   * @throws YarnException if fails
+   * @throws IOException if fails
    */
-  public RegisterApplicationMasterResponse createAndRegisterNewUAM(String uamId,
-      RegisterApplicationMasterRequest registerRequest, Configuration conf,
+  public Token<AMRMTokenIdentifier> launchUAM(String uamId, Configuration conf,
+      ApplicationId appId, String queueName, String submitter,
+      String appNameSuffix, boolean keepContainersAcrossApplicationAttempts)
+      throws YarnException, IOException {
+
+    if (this.unmanagedAppMasterMap.containsKey(uamId)) {
+      throw new YarnException("UAM " + uamId + " already exists");
+    }
+    UnmanagedApplicationManager uam = createUAM(conf, appId, queueName,
+        submitter, appNameSuffix, keepContainersAcrossApplicationAttempts);
+    // Put the UAM into map first before initializing it to avoid additional UAM
+    // for the same uamId being created concurrently
+    this.unmanagedAppMasterMap.put(uamId, uam);
+
+    Token<AMRMTokenIdentifier> amrmToken = null;
+    try {
+      LOG.info("Launching UAM id {} for application {}", uamId, appId);
+      amrmToken = uam.launchUAM();
+    } catch (Exception e) {
+      // Add the map earlier and remove here if register failed because we want
+      // to make sure there is only one uam instance per uamId at any given time
+      this.unmanagedAppMasterMap.remove(uamId);
+      throw e;
+    }
+
+    this.appIdMap.put(uamId, uam.getAppId());
+    return amrmToken;
+  }
+
+  /**
+   * Re-attach to an existing UAM, using the provided uamIdentifier.
+   *
+   * @param uamId uam Id
+   * @param conf configuration for this UAM
+   * @param appId application id for the UAM
+   * @param queueName queue of the application
+   * @param submitter submitter name of the UAM
+   * @param appNameSuffix application name suffix for the UAM
+   * @param uamToken UAM token
+   * @throws YarnException if fails
+   * @throws IOException if fails
+   */
+  public void reAttachUAM(String uamId, Configuration conf,
       ApplicationId appId, String queueName, String submitter,
-      String appNameSuffix) throws YarnException, IOException {
+      String appNameSuffix, Token<AMRMTokenIdentifier> uamToken)
+      throws YarnException, IOException {
 
     if (this.unmanagedAppMasterMap.containsKey(uamId)) {
       throw new YarnException("UAM " + uamId + " already exists");
     }
     UnmanagedApplicationManager uam =
-        createUAM(conf, appId, queueName, submitter, appNameSuffix);
+        createUAM(conf, appId, queueName, submitter, appNameSuffix, true);
     // Put the UAM into map first before initializing it to avoid additional UAM
     // for the same uamId being created concurrently
     this.unmanagedAppMasterMap.put(uamId, uam);
 
-    RegisterApplicationMasterResponse response = null;
     try {
-      LOG.info("Creating and registering UAM id {} for application {}", uamId,
-          appId);
-      response = uam.createAndRegisterApplicationMaster(registerRequest);
+      LOG.info("Reattaching UAM id {} for application {}", uamId, appId);
+      uam.reAttachUAM(uamToken);
     } catch (Exception e) {
       // Add the map earlier and remove here if register failed because we want
       // to make sure there is only one uam instance per uamId at any given time
@@ -219,8 +274,7 @@ public class UnmanagedAMPoolManager extends AbstractService {
       throw e;
     }
 
-    this.attemptIdMap.put(uamId, uam.getAttemptId());
-    return response;
+    this.appIdMap.put(uamId, uam.getAppId());
   }
 
   /**
@@ -231,20 +285,42 @@ public class UnmanagedAMPoolManager extends AbstractService {
    * @param queueName queue of the application
    * @param submitter submitter name of the application
    * @param appNameSuffix application name suffix
+   * @param keepContainersAcrossApplicationAttempts keep container flag for UAM
    * @return the UAM instance
    */
   @VisibleForTesting
   protected UnmanagedApplicationManager createUAM(Configuration conf,
       ApplicationId appId, String queueName, String submitter,
-      String appNameSuffix) {
+      String appNameSuffix, boolean keepContainersAcrossApplicationAttempts) {
     return new UnmanagedApplicationManager(conf, appId, queueName, submitter,
-        appNameSuffix);
+        appNameSuffix, keepContainersAcrossApplicationAttempts);
+  }
+
+  /**
+   * Register application master for the UAM.
+   *
+   * @param uamId uam Id
+   * @param registerRequest RegisterApplicationMasterRequest
+   * @return register response
+   * @throws YarnException if register fails
+   * @throws IOException if register fails
+   */
+  public RegisterApplicationMasterResponse registerApplicationMaster(
+      String uamId, RegisterApplicationMasterRequest registerRequest)
+      throws YarnException, IOException {
+    if (!this.unmanagedAppMasterMap.containsKey(uamId)) {
+      throw new YarnException("UAM " + uamId + " does not exist");
+    }
+    LOG.info("Registering UAM id {} for application {}", uamId,
+        this.appIdMap.get(uamId));
+    return this.unmanagedAppMasterMap.get(uamId)
+        .registerApplicationMaster(registerRequest);
   }
 
   /**
    * AllocateAsync to an UAM.
    *
-   * @param uamId identifier for the UAM
+   * @param uamId uam Id
    * @param request AllocateRequest
    * @param callback callback for response
    * @throws YarnException if allocate fails
@@ -262,7 +338,7 @@ public class UnmanagedAMPoolManager extends AbstractService {
   /**
    * Finish an UAM/application.
    *
-   * @param uamId identifier for the UAM
+   * @param uamId uam Id
    * @param request FinishApplicationMasterRequest
    * @return FinishApplicationMasterResponse
    * @throws YarnException if finishApplicationMaster call fails
@@ -274,14 +350,15 @@ public class UnmanagedAMPoolManager extends AbstractService {
     if (!this.unmanagedAppMasterMap.containsKey(uamId)) {
       throw new YarnException("UAM " + uamId + " does not exist");
     }
-    LOG.info("Finishing application for UAM id {} ", uamId);
+    LOG.info("Finishing UAM id {} for application {}", uamId,
+        this.appIdMap.get(uamId));
     FinishApplicationMasterResponse response =
         this.unmanagedAppMasterMap.get(uamId).finishApplicationMaster(request);
 
     if (response.getIsUnregistered()) {
       // Only remove the UAM when the unregister finished
       this.unmanagedAppMasterMap.remove(uamId);
-      this.attemptIdMap.remove(uamId);
+      this.appIdMap.remove(uamId);
       LOG.info("UAM id {} is unregistered", uamId);
     }
     return response;
@@ -301,7 +378,7 @@ public class UnmanagedAMPoolManager extends AbstractService {
   /**
    * Return whether an UAM exists.
    *
-   * @param uamId identifier for the UAM
+   * @param uamId uam Id
    * @return UAM exists or not
    */
   public boolean hasUAMId(String uamId) {

+ 128 - 84
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/uam/UnmanagedApplicationManager.java

@@ -50,7 +50,9 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.hadoop.yarn.api.records.NMToken;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
@@ -90,7 +92,6 @@ public class UnmanagedApplicationManager {
   private AMRequestHandlerThread handlerThread;
   private ApplicationMasterProtocol rmProxy;
   private ApplicationId applicationId;
-  private ApplicationAttemptId attemptId;
   private String submitter;
   private String appNameSuffix;
   private Configuration conf;
@@ -101,9 +102,31 @@ public class UnmanagedApplicationManager {
   private ApplicationClientProtocol rmClient;
   private long asyncApiPollIntervalMillis;
   private RecordFactory recordFactory;
+  private boolean keepContainersAcrossApplicationAttempts;
 
+  /*
+   * This flag is used as an indication that this method launchUAM/reAttachUAM
+   * is called (and perhaps blocked in initializeUnmanagedAM below due to RM
+   * connection/failover issue and not finished yet). Set the flag before
+   * calling the blocking call to RM.
+   */
+  private boolean connectionInitiated;
+
+  /**
+   * Constructor.
+   *
+   * @param conf configuration
+   * @param appId application Id to use for this UAM
+   * @param queueName the queue of the UAM
+   * @param submitter user name of the app
+   * @param appNameSuffix the app name suffix to use
+   * @param keepContainersAcrossApplicationAttempts keep container flag for UAM
+   *          recovery. See {@link ApplicationSubmissionContext
+   *          #setKeepContainersAcrossApplicationAttempts(boolean)}
+   */
   public UnmanagedApplicationManager(Configuration conf, ApplicationId appId,
-      String queueName, String submitter, String appNameSuffix) {
+      String queueName, String submitter, String appNameSuffix,
+      boolean keepContainersAcrossApplicationAttempts) {
     Preconditions.checkNotNull(conf, "Configuration cannot be null");
     Preconditions.checkNotNull(appId, "ApplicationId cannot be null");
     Preconditions.checkNotNull(submitter, "App submitter cannot be null");
@@ -116,6 +139,7 @@ public class UnmanagedApplicationManager {
     this.handlerThread = new AMRequestHandlerThread();
     this.requestQueue = new LinkedBlockingQueue<>();
     this.rmProxy = null;
+    this.connectionInitiated = false;
     this.registerRequest = null;
     this.recordFactory = RecordFactoryProvider.getRecordFactory(conf);
     this.asyncApiPollIntervalMillis = conf.getLong(
@@ -123,45 +147,84 @@ public class UnmanagedApplicationManager {
             YARN_CLIENT_APPLICATION_CLIENT_PROTOCOL_POLL_INTERVAL_MS,
         YarnConfiguration.
             DEFAULT_YARN_CLIENT_APPLICATION_CLIENT_PROTOCOL_POLL_INTERVAL_MS);
+    this.keepContainersAcrossApplicationAttempts =
+        keepContainersAcrossApplicationAttempts;
+  }
+
+  /**
+   * Launch a new UAM in the resource manager.
+   *
+   * @return identifier uam identifier
+   * @throws YarnException if fails
+   * @throws IOException if fails
+   */
+  public Token<AMRMTokenIdentifier> launchUAM()
+      throws YarnException, IOException {
+    this.connectionInitiated = true;
+
+    // Blocking call to RM
+    Token<AMRMTokenIdentifier> amrmToken =
+        initializeUnmanagedAM(this.applicationId);
+
+    // Creates the UAM connection
+    createUAMProxy(amrmToken);
+    return amrmToken;
+  }
+
+  /**
+   * Re-attach to an existing UAM in the resource manager.
+   *
+   * @param amrmToken the UAM token
+   * @throws IOException if re-attach fails
+   * @throws YarnException if re-attach fails
+   */
+  public void reAttachUAM(Token<AMRMTokenIdentifier> amrmToken)
+      throws IOException, YarnException {
+    this.connectionInitiated = true;
+
+    // Creates the UAM connection
+    createUAMProxy(amrmToken);
+  }
+
+  protected void createUAMProxy(Token<AMRMTokenIdentifier> amrmToken)
+      throws IOException {
+    this.userUgi = UserGroupInformation.createProxyUser(
+        this.applicationId.toString(), UserGroupInformation.getCurrentUser());
+    this.rmProxy = createRMProxy(ApplicationMasterProtocol.class, this.conf,
+        this.userUgi, amrmToken);
   }
 
   /**
    * Registers this {@link UnmanagedApplicationManager} with the resource
    * manager.
    *
-   * @param request the register request
-   * @return the register response
+   * @param request RegisterApplicationMasterRequest
+   * @return register response
    * @throws YarnException if register fails
    * @throws IOException if register fails
    */
-  public RegisterApplicationMasterResponse createAndRegisterApplicationMaster(
+  public RegisterApplicationMasterResponse registerApplicationMaster(
       RegisterApplicationMasterRequest request)
       throws YarnException, IOException {
-    // This need to be done first in this method, because it is used as an
-    // indication that this method is called (and perhaps blocked due to RM
-    // connection and not finished yet)
+    // Save the register request for re-register later
     this.registerRequest = request;
 
-    // attemptId will be available after this call
-    UnmanagedAMIdentifier identifier =
-        initializeUnmanagedAM(this.applicationId);
-
-    try {
-      this.userUgi = UserGroupInformation.createProxyUser(
-          identifier.getAttemptId().toString(),
-          UserGroupInformation.getCurrentUser());
-    } catch (IOException e) {
-      LOG.error("Exception while trying to get current user", e);
-      throw new YarnRuntimeException(e);
-    }
-
-    this.rmProxy = createRMProxy(ApplicationMasterProtocol.class, this.conf,
-        this.userUgi, identifier.getToken());
-
-    LOG.info("Registering the Unmanaged application master {}", this.attemptId);
+    // Since we have setKeepContainersAcrossApplicationAttempts = true for UAM.
+    // We do not expect application already registered exception here
+    LOG.info("Registering the Unmanaged application master {}",
+        this.applicationId);
     RegisterApplicationMasterResponse response =
         this.rmProxy.registerApplicationMaster(this.registerRequest);
 
+    for (Container container : response.getContainersFromPreviousAttempts()) {
+      LOG.info("RegisterUAM returned existing running container "
+          + container.getId());
+    }
+    for (NMToken nmToken : response.getNMTokensFromPreviousAttempts()) {
+      LOG.info("RegisterUAM returned existing NM token for node "
+          + nmToken.getNodeId());
+    }
+
     // Only when register succeed that we start the heartbeat thread
     this.handlerThread.setUncaughtExceptionHandler(
         new HeartBeatThreadUncaughtExceptionHandler());
@@ -187,11 +250,11 @@ public class UnmanagedApplicationManager {
     this.handlerThread.shutdown();
 
     if (this.rmProxy == null) {
-      if (this.registerRequest != null) {
-        // This is possible if the async registerApplicationMaster is still
+      if (this.connectionInitiated) {
+        // This is possible if the async launchUAM is still
         // blocked and retrying. Return a dummy response in this case.
         LOG.warn("Unmanaged AM still not successfully launched/registered yet."
-            + " Stopping the UAM client thread anyways.");
+            + " Stopping the UAM heartbeat thread anyways.");
         return FinishApplicationMasterResponse.newInstance(false);
       } else {
         throw new YarnException("finishApplicationMaster should not "
@@ -199,7 +262,7 @@ public class UnmanagedApplicationManager {
       }
     }
     return AMRMClientUtils.finishAMWithReRegister(request, this.rmProxy,
-        this.registerRequest, this.attemptId);
+        this.registerRequest, this.applicationId);
   }
 
   /**
@@ -212,7 +275,7 @@ public class UnmanagedApplicationManager {
   public KillApplicationResponse forceKillApplication()
       throws IOException, YarnException {
     KillApplicationRequest request =
-        KillApplicationRequest.newInstance(this.attemptId.getApplicationId());
+        KillApplicationRequest.newInstance(this.applicationId);
 
     this.handlerThread.shutdown();
 
@@ -240,29 +303,29 @@ public class UnmanagedApplicationManager {
       LOG.debug("Interrupted while waiting to put on response queue", ex);
     }
     // Two possible cases why the UAM is not successfully registered yet:
-    // 1. registerApplicationMaster is not called at all. Should throw here.
-    // 2. registerApplicationMaster is called but hasn't successfully returned.
+    // 1. launchUAM is not called at all. Should throw here.
+    // 2. launchUAM is called but hasn't successfully returned.
     //
     // In case 2, we have already save the allocate request above, so if the
     // registration succeed later, no request is lost.
     if (this.rmProxy == null) {
-      if (this.registerRequest != null) {
+      if (this.connectionInitiated) {
         LOG.info("Unmanaged AM still not successfully launched/registered yet."
             + " Saving the allocate request and send later.");
       } else {
         throw new YarnException(
-            "AllocateAsync should not be called before createAndRegister");
+            "AllocateAsync should not be called before launchUAM");
       }
     }
   }
 
   /**
-   * Returns the application attempt id of the UAM.
+   * Returns the application id of the UAM.
    *
-   * @return attempt id of the UAM
+   * @return application id of the UAM
    */
-  public ApplicationAttemptId getAttemptId() {
-    return this.attemptId;
+  public ApplicationId getAppId() {
+    return this.applicationId;
   }
 
   /**
@@ -287,15 +350,15 @@ public class UnmanagedApplicationManager {
    * Launch and initialize an unmanaged AM. First, it creates a new application
    * on the RM and negotiates a new attempt id. Then it waits for the RM
    * application attempt state to reach YarnApplicationAttemptState.LAUNCHED
-   * after which it returns the AM-RM token and the attemptId.
+   * after which it returns the AM-RM token.
    *
    * @param appId application id
-   * @return the UAM identifier
+   * @return the UAM token
    * @throws IOException if initialize fails
    * @throws YarnException if initialize fails
    */
-  protected UnmanagedAMIdentifier initializeUnmanagedAM(ApplicationId appId)
-      throws IOException, YarnException {
+  protected Token<AMRMTokenIdentifier> initializeUnmanagedAM(
+      ApplicationId appId) throws IOException, YarnException {
     try {
       UserGroupInformation appSubmitter =
           UserGroupInformation.createRemoteUser(this.submitter);
@@ -306,13 +369,12 @@ public class UnmanagedApplicationManager {
       submitUnmanagedApp(appId);
 
       // Monitor the application attempt to wait for launch state
-      ApplicationAttemptReport attemptReport = monitorCurrentAppAttempt(appId,
+      monitorCurrentAppAttempt(appId,
           EnumSet.of(YarnApplicationState.ACCEPTED,
               YarnApplicationState.RUNNING, YarnApplicationState.KILLED,
               YarnApplicationState.FAILED, YarnApplicationState.FINISHED),
           YarnApplicationAttemptState.LAUNCHED);
-      this.attemptId = attemptReport.getApplicationAttemptId();
-      return getUAMIdentifier();
+      return getUAMToken();
     } finally {
       this.rmClient = null;
     }
@@ -343,6 +405,8 @@ public class UnmanagedApplicationManager {
     submitRequest.setApplicationSubmissionContext(context);
 
     context.setUnmanagedAM(true);
+    context.setKeepContainersAcrossApplicationAttempts(
+        this.keepContainersAcrossApplicationAttempts);
 
     LOG.info("Submitting unmanaged application {}", appId);
     this.rmClient.submitApplication(submitRequest);
@@ -374,8 +438,10 @@ public class UnmanagedApplicationManager {
         if (appStates.contains(state)) {
           if (state != YarnApplicationState.ACCEPTED) {
             throw new YarnRuntimeException(
-                "Received non-accepted application state: " + state
-                    + ". Application " + appId + " not the first attempt?");
+                "Received non-accepted application state: " + state + " for "
+                    + appId + ". This is likely because this is not the first "
+                    + "app attempt in home sub-cluster, and AMRMProxy HA "
+                    + "(yarn.nodemanager.amrmproxy.ha.enable) is not enabled.");
           }
           appAttemptId =
               getApplicationReport(appId).getCurrentApplicationAttemptId();
@@ -415,25 +481,25 @@ public class UnmanagedApplicationManager {
   }
 
   /**
-   * Gets the identifier of the unmanaged AM.
+   * Gets the amrmToken of the unmanaged AM.
    *
-   * @return the identifier of the unmanaged AM.
+   * @return the amrmToken of the unmanaged AM.
    * @throws IOException if getApplicationReport fails
    * @throws YarnException if getApplicationReport fails
    */
-  protected UnmanagedAMIdentifier getUAMIdentifier()
+  protected Token<AMRMTokenIdentifier> getUAMToken()
       throws IOException, YarnException {
     Token<AMRMTokenIdentifier> token = null;
     org.apache.hadoop.yarn.api.records.Token amrmToken =
-        getApplicationReport(this.attemptId.getApplicationId()).getAMRMToken();
+        getApplicationReport(this.applicationId).getAMRMToken();
     if (amrmToken != null) {
       token = ConverterUtils.convertFromYarn(amrmToken, (Text) null);
     } else {
       LOG.warn(
           "AMRMToken not found in the application report for application: {}",
-          this.attemptId.getApplicationId());
+          this.applicationId);
     }
-    return new UnmanagedAMIdentifier(this.attemptId, token);
+    return token;
   }
 
   private ApplicationReport getApplicationReport(ApplicationId appId)
@@ -444,29 +510,6 @@ public class UnmanagedApplicationManager {
     return this.rmClient.getApplicationReport(request).getApplicationReport();
   }
 
-  /**
-   * Data structure that encapsulates the application attempt identifier and the
-   * AMRMTokenIdentifier. Make it public because clients with HA need it.
-   */
-  public static class UnmanagedAMIdentifier {
-    private ApplicationAttemptId attemptId;
-    private Token<AMRMTokenIdentifier> token;
-
-    public UnmanagedAMIdentifier(ApplicationAttemptId attemptId,
-        Token<AMRMTokenIdentifier> token) {
-      this.attemptId = attemptId;
-      this.token = token;
-    }
-
-    public ApplicationAttemptId getAttemptId() {
-      return this.attemptId;
-    }
-
-    public Token<AMRMTokenIdentifier> getToken() {
-      return this.token;
-    }
-  }
-
   /**
    * Data structure that encapsulates AllocateRequest and AsyncCallback
    * instance.
@@ -549,8 +592,10 @@ public class UnmanagedApplicationManager {
           }
 
           request.setResponseId(lastResponseId);
+
           AllocateResponse response = AMRMClientUtils.allocateWithReRegister(
-              request, rmProxy, registerRequest, attemptId);
+              request, rmProxy, registerRequest, applicationId);
+
           if (response == null) {
             throw new YarnException("Null allocateResponse from allocate");
           }
@@ -578,18 +623,17 @@ public class UnmanagedApplicationManager {
             LOG.debug("Interrupted while waiting for queue", ex);
           }
         } catch (IOException ex) {
-          LOG.warn(
-              "IO Error occurred while processing heart beat for " + attemptId,
-              ex);
+          LOG.warn("IO Error occurred while processing heart beat for "
+              + applicationId, ex);
         } catch (Throwable ex) {
           LOG.warn(
-              "Error occurred while processing heart beat for " + attemptId,
+              "Error occurred while processing heart beat for " + applicationId,
               ex);
         }
       }
 
       LOG.info("UnmanagedApplicationManager has been stopped for {}. "
-          + "AMRequestHandlerThread thread is exiting", attemptId);
+          + "AMRequestHandlerThread thread is exiting", applicationId);
     }
   }
 
@@ -600,8 +644,8 @@ public class UnmanagedApplicationManager {
       implements UncaughtExceptionHandler {
     @Override
     public void uncaughtException(Thread t, Throwable e) {
-      LOG.error("Heartbeat thread {} for application attempt {} crashed!",
-          t.getName(), attemptId, e);
+      LOG.error("Heartbeat thread {} for application {} crashed!",
+          t.getName(), applicationId, e);
     }
   }
 }

+ 14 - 16
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/AMRMClientUtils.java

@@ -36,7 +36,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
-import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.client.ClientRMProxy;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.ApplicationMasterNotRegisteredException;
@@ -63,16 +63,16 @@ public final class AMRMClientUtils {
   /**
    * Handle ApplicationNotRegistered exception and re-register.
    *
-   * @param attemptId app attemptId
+   * @param appId application Id
    * @param rmProxy RM proxy instance
    * @param registerRequest the AM re-register request
    * @throws YarnException if re-register fails
    */
   public static void handleNotRegisteredExceptionAndReRegister(
-      ApplicationAttemptId attemptId, ApplicationMasterProtocol rmProxy,
+      ApplicationId appId, ApplicationMasterProtocol rmProxy,
       RegisterApplicationMasterRequest registerRequest) throws YarnException {
     LOG.info("App attempt {} not registered, most likely due to RM failover. "
-        + " Trying to re-register.", attemptId);
+        + " Trying to re-register.", appId);
     try {
       rmProxy.registerApplicationMaster(registerRequest);
     } catch (Exception e) {
@@ -93,25 +93,24 @@ public final class AMRMClientUtils {
    * @param request allocate request
    * @param rmProxy RM proxy
    * @param registerRequest the register request for re-register
-   * @param attemptId application attempt id
+   * @param appId application id
    * @return allocate response
    * @throws YarnException if RM call fails
    * @throws IOException if RM call fails
    */
   public static AllocateResponse allocateWithReRegister(AllocateRequest request,
       ApplicationMasterProtocol rmProxy,
-      RegisterApplicationMasterRequest registerRequest,
-      ApplicationAttemptId attemptId) throws YarnException, IOException {
+      RegisterApplicationMasterRequest registerRequest, ApplicationId appId)
+      throws YarnException, IOException {
     try {
       return rmProxy.allocate(request);
     } catch (ApplicationMasterNotRegisteredException e) {
-      handleNotRegisteredExceptionAndReRegister(attemptId, rmProxy,
+      handleNotRegisteredExceptionAndReRegister(appId, rmProxy,
           registerRequest);
       // reset responseId after re-register
       request.setResponseId(0);
       // retry allocate
-      return allocateWithReRegister(request, rmProxy, registerRequest,
-          attemptId);
+      return allocateWithReRegister(request, rmProxy, registerRequest, appId);
     }
   }
 
@@ -123,23 +122,22 @@ public final class AMRMClientUtils {
    * @param request finishApplicationMaster request
    * @param rmProxy RM proxy
    * @param registerRequest the register request for re-register
-   * @param attemptId application attempt id
+   * @param appId application id
    * @return finishApplicationMaster response
    * @throws YarnException if RM call fails
    * @throws IOException if RM call fails
    */
   public static FinishApplicationMasterResponse finishAMWithReRegister(
       FinishApplicationMasterRequest request, ApplicationMasterProtocol rmProxy,
-      RegisterApplicationMasterRequest registerRequest,
-      ApplicationAttemptId attemptId) throws YarnException, IOException {
+      RegisterApplicationMasterRequest registerRequest, ApplicationId appId)
+      throws YarnException, IOException {
     try {
       return rmProxy.finishApplicationMaster(request);
     } catch (ApplicationMasterNotRegisteredException ex) {
-      handleNotRegisteredExceptionAndReRegister(attemptId, rmProxy,
+      handleNotRegisteredExceptionAndReRegister(appId, rmProxy,
           registerRequest);
       // retry finishAM after re-register
-      return finishAMWithReRegister(request, rmProxy, registerRequest,
-          attemptId);
+      return finishAMWithReRegister(request, rmProxy, registerRequest, appId);
     }
   }
 

+ 56 - 47
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java

@@ -111,6 +111,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.NMToken;
@@ -177,10 +178,9 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
       LoggerFactory.getLogger(MockResourceManagerFacade.class);
 
   private HashSet<ApplicationId> applicationMap = new HashSet<>();
-  private HashMap<String, List<ContainerId>> applicationContainerIdMap =
-      new HashMap<String, List<ContainerId>>();
-  private HashMap<ContainerId, Container> allocatedContainerMap =
-      new HashMap<ContainerId, Container>();
+  private HashSet<ApplicationId> keepContainerOnUams = new HashSet<>();
+  private HashMap<ApplicationAttemptId, List<ContainerId>>
+      applicationContainerIdMap = new HashMap<>();
   private AtomicInteger containerIndex = new AtomicInteger(0);
   private Configuration conf;
   private int subClusterId;
@@ -221,7 +221,7 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
     this.isRunning = mode;
   }
 
-  private static String getAppIdentifier() throws IOException {
+  private static ApplicationAttemptId getAppIdentifier() throws IOException {
     AMRMTokenIdentifier result = null;
     UserGroupInformation remoteUgi = UserGroupInformation.getCurrentUser();
     Set<TokenIdentifier> tokenIds = remoteUgi.getTokenIdentifiers();
@@ -231,7 +231,8 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
         break;
       }
     }
-    return result != null ? result.getApplicationAttemptId().toString() : "";
+    return result != null ? result.getApplicationAttemptId()
+        : ApplicationAttemptId.newInstance(ApplicationId.newInstance(0, 0), 0);
   }
 
   private void validateRunning() throws ConnectException {
@@ -246,19 +247,32 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
       throws YarnException, IOException {
 
     validateRunning();
-
-    String amrmToken = getAppIdentifier();
-    LOG.info("Registering application attempt: " + amrmToken);
+    ApplicationAttemptId attemptId = getAppIdentifier();
+    LOG.info("Registering application attempt: " + attemptId);
 
     shouldReRegisterNext = false;
 
+    List<Container> containersFromPreviousAttempt = null;
+
     synchronized (applicationContainerIdMap) {
-      if (applicationContainerIdMap.containsKey(amrmToken)) {
-        throw new InvalidApplicationMasterRequestException(
-            AMRMClientUtils.APP_ALREADY_REGISTERED_MESSAGE);
+      if (applicationContainerIdMap.containsKey(attemptId)) {
+        if (keepContainerOnUams.contains(attemptId.getApplicationId())) {
+          // For UAM with the keepContainersFromPreviousAttempt flag, return all
+          // running containers
+          containersFromPreviousAttempt = new ArrayList<>();
+          for (ContainerId containerId : applicationContainerIdMap
+              .get(attemptId)) {
+            containersFromPreviousAttempt.add(Container.newInstance(containerId,
+                null, null, null, null, null));
+          }
+        } else {
+          throw new InvalidApplicationMasterRequestException(
+              AMRMClientUtils.APP_ALREADY_REGISTERED_MESSAGE);
+        }
+      } else {
+        // Keep track of the containers that are returned to this application
+        applicationContainerIdMap.put(attemptId, new ArrayList<ContainerId>());
       }
-      // Keep track of the containers that are returned to this application
-      applicationContainerIdMap.put(amrmToken, new ArrayList<ContainerId>());
     }
 
     // Make sure we wait for certain test cases last in the method
@@ -278,7 +292,7 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
     }
 
     return RegisterApplicationMasterResponse.newInstance(null, null, null, null,
-        null, request.getHost(), null);
+        containersFromPreviousAttempt, request.getHost(), null);
   }
 
   @Override
@@ -288,8 +302,8 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
 
     validateRunning();
 
-    String amrmToken = getAppIdentifier();
-    LOG.info("Finishing application attempt: " + amrmToken);
+    ApplicationAttemptId attemptId = getAppIdentifier();
+    LOG.info("Finishing application attempt: " + attemptId);
 
     if (shouldReRegisterNext) {
       String message = "AM is not registered, should re-register.";
@@ -299,12 +313,9 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
 
     synchronized (applicationContainerIdMap) {
       // Remove the containers that were being tracked for this application
-      Assert.assertTrue("The application id is NOT registered: " + amrmToken,
-          applicationContainerIdMap.containsKey(amrmToken));
-      List<ContainerId> ids = applicationContainerIdMap.remove(amrmToken);
-      for (ContainerId c : ids) {
-        allocatedContainerMap.remove(c);
-      }
+      Assert.assertTrue("The application id is NOT registered: " + attemptId,
+          applicationContainerIdMap.containsKey(attemptId));
+      applicationContainerIdMap.remove(attemptId);
     }
 
     return FinishApplicationMasterResponse.newInstance(
@@ -334,8 +345,8 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
           + "askList and releaseList in the same heartbeat");
     }
 
-    String amrmToken = getAppIdentifier();
-    LOG.info("Allocate from application attempt: " + amrmToken);
+    ApplicationAttemptId attemptId = getAppIdentifier();
+    LOG.info("Allocate from application attempt: " + attemptId);
 
     if (shouldReRegisterNext) {
       String message = "AM is not registered, should re-register.";
@@ -367,16 +378,16 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
             // will need it in future
             Assert.assertTrue(
                 "The application id is Not registered before allocate(): "
-                    + amrmToken,
-                applicationContainerIdMap.containsKey(amrmToken));
-            List<ContainerId> ids = applicationContainerIdMap.get(amrmToken);
+                    + attemptId,
+                applicationContainerIdMap.containsKey(attemptId));
+            List<ContainerId> ids = applicationContainerIdMap.get(attemptId);
             ids.add(containerId);
-            this.allocatedContainerMap.put(containerId, container);
           }
         }
       }
     }
 
+    List<ContainerStatus> completedList = new ArrayList<>();
     if (request.getReleaseList() != null
         && request.getReleaseList().size() > 0) {
       LOG.info("Releasing containers: " + request.getReleaseList().size());
@@ -384,9 +395,9 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
         Assert
             .assertTrue(
                 "The application id is not registered before allocate(): "
-                    + amrmToken,
-                applicationContainerIdMap.containsKey(amrmToken));
-        List<ContainerId> ids = applicationContainerIdMap.get(amrmToken);
+                    + attemptId,
+                applicationContainerIdMap.containsKey(attemptId));
+        List<ContainerId> ids = applicationContainerIdMap.get(attemptId);
 
         for (ContainerId id : request.getReleaseList()) {
           boolean found = false;
@@ -402,18 +413,8 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
               + conf.get("AMRMTOKEN"), found);
 
           ids.remove(id);
-
-          // Return the released container back to the AM with new fake Ids. The
-          // test case does not care about the IDs. The IDs are faked because
-          // otherwise the LRM will throw duplication identifier exception. This
-          // returning of fake containers is ONLY done for testing purpose - for
-          // the test code to get confirmation that the sub-cluster resource
-          // managers received the release request
-          ContainerId fakeContainerId = ContainerId.newInstance(
-              getApplicationAttemptId(1), containerIndex.incrementAndGet());
-          Container fakeContainer = allocatedContainerMap.get(id);
-          fakeContainer.setId(fakeContainerId);
-          containerList.add(fakeContainer);
+          completedList.add(
+              ContainerStatus.newInstance(id, ContainerState.COMPLETE, "", 0));
         }
       }
     }
@@ -424,9 +425,9 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
     // Always issue a new AMRMToken as if RM rolled master key
     Token newAMRMToken = Token.newInstance(new byte[0], "", new byte[0], "");
 
-    return AllocateResponse.newInstance(0, new ArrayList<ContainerStatus>(),
-        containerList, new ArrayList<NodeReport>(), null, AMCommand.AM_RESYNC,
-        1, null, new ArrayList<NMToken>(), newAMRMToken,
+    return AllocateResponse.newInstance(0, completedList, containerList,
+        new ArrayList<NodeReport>(), null, AMCommand.AM_RESYNC, 1, null,
+        new ArrayList<NMToken>(), newAMRMToken,
         new ArrayList<UpdatedContainer>());
   }
 
@@ -443,6 +444,7 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
     report.setApplicationId(request.getApplicationId());
     report.setCurrentApplicationAttemptId(
         ApplicationAttemptId.newInstance(request.getApplicationId(), 1));
+    report.setAMRMToken(Token.newInstance(new byte[0], "", new byte[0], ""));
     response.setApplicationReport(report);
     return response;
   }
@@ -486,6 +488,12 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
     }
     LOG.info("Application submitted: " + appId);
     applicationMap.add(appId);
+
+    if (request.getApplicationSubmissionContext().getUnmanagedAM()
+        || request.getApplicationSubmissionContext()
+            .getKeepContainersAcrossApplicationAttempts()) {
+      keepContainerOnUams.add(appId);
+    }
     return SubmitApplicationResponse.newInstance();
   }
 
@@ -502,6 +510,7 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
         throw new ApplicationNotFoundException(
             "Trying to kill an absent application: " + appId);
       }
+      keepContainerOnUams.remove(appId);
     }
     LOG.info("Force killing application: " + appId);
     return KillApplicationResponse.newInstance(true);

+ 90 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/utils/TestFederationRegistryClient.java

@@ -0,0 +1,90 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     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.federation.utils;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.registry.client.api.RegistryOperations;
+import org.apache.hadoop.registry.client.impl.FSRegistryOperationsService;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Unit test for FederationRegistryClient.
+ */
+public class TestFederationRegistryClient {
+  private Configuration conf;
+  private UserGroupInformation user;
+  private RegistryOperations registry;
+  private FederationRegistryClient registryClient;
+
+  @Before
+  public void setup() throws Exception {
+    this.conf = new YarnConfiguration();
+
+    this.registry = new FSRegistryOperationsService();
+    this.registry.init(this.conf);
+    this.registry.start();
+
+    this.user = UserGroupInformation.getCurrentUser();
+    this.registryClient =
+        new FederationRegistryClient(this.conf, this.registry, this.user);
+    this.registryClient.cleanAllApplications();
+    Assert.assertEquals(0, this.registryClient.getAllApplications().size());
+  }
+
+  @After
+  public void breakDown() {
+    registryClient.cleanAllApplications();
+    Assert.assertEquals(0, registryClient.getAllApplications().size());
+    registry.stop();
+  }
+
+  @Test
+  public void testBasicCase() {
+    ApplicationId appId = ApplicationId.newInstance(0, 0);
+    String scId1 = "subcluster1";
+    String scId2 = "subcluster2";
+
+    this.registryClient.writeAMRMTokenForUAM(appId, scId1,
+        new Token<AMRMTokenIdentifier>());
+    this.registryClient.writeAMRMTokenForUAM(appId, scId2,
+        new Token<AMRMTokenIdentifier>());
+    // Duplicate entry, should overwrite
+    this.registryClient.writeAMRMTokenForUAM(appId, scId1,
+        new Token<AMRMTokenIdentifier>());
+
+    Assert.assertEquals(1, this.registryClient.getAllApplications().size());
+    Assert.assertEquals(2,
+        this.registryClient.loadStateFromRegistry(appId).size());
+
+    this.registryClient.removeAppFromRegistry(appId);
+
+    Assert.assertEquals(0, this.registryClient.getAllApplications().size());
+    Assert.assertEquals(0,
+        this.registryClient.loadStateFromRegistry(appId).size());
+  }
+
+}

+ 85 - 15
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/uam/TestUnmanagedApplicationManager.java

@@ -65,7 +65,7 @@ public class TestUnmanagedApplicationManager {
         ApplicationAttemptId.newInstance(ApplicationId.newInstance(0, 1), 1);
 
     uam = new TestableUnmanagedApplicationManager(conf,
-        attemptId.getApplicationId(), null, "submitter", "appNameSuffix");
+        attemptId.getApplicationId(), null, "submitter", "appNameSuffix", true);
   }
 
   protected void waitForCallBackCountAndCheckZeroPending(
@@ -88,7 +88,8 @@ public class TestUnmanagedApplicationManager {
   public void testBasicUsage()
       throws YarnException, IOException, InterruptedException {
 
-    createAndRegisterApplicationMaster(
+    launchUAM(attemptId);
+    registerApplicationMaster(
         RegisterApplicationMasterRequest.newInstance(null, 0, null), attemptId);
 
     allocateAsync(AllocateRequest.newInstance(0, 0, null, null, null), callback,
@@ -102,11 +103,48 @@ public class TestUnmanagedApplicationManager {
         attemptId);
   }
 
+  /*
+   * Test re-attaching of an existing UAM. This is for HA of UAM client.
+   */
+  @Test(timeout = 5000)
+  public void testUAMReAttach()
+      throws YarnException, IOException, InterruptedException {
+
+    launchUAM(attemptId);
+    registerApplicationMaster(
+        RegisterApplicationMasterRequest.newInstance(null, 0, null), attemptId);
+
+    allocateAsync(AllocateRequest.newInstance(0, 0, null, null, null), callback,
+        attemptId);
+    // Wait for outstanding async allocate callback
+    waitForCallBackCountAndCheckZeroPending(callback, 1);
+
+    MockResourceManagerFacade rmProxy = uam.getRMProxy();
+    uam = new TestableUnmanagedApplicationManager(conf,
+        attemptId.getApplicationId(), null, "submitter", "appNameSuffix", true);
+    uam.setRMProxy(rmProxy);
+
+    reAttachUAM(null, attemptId);
+    registerApplicationMaster(
+        RegisterApplicationMasterRequest.newInstance(null, 0, null), attemptId);
+
+    allocateAsync(AllocateRequest.newInstance(0, 0, null, null, null), callback,
+        attemptId);
+
+    // Wait for outstanding async allocate callback
+    waitForCallBackCountAndCheckZeroPending(callback, 2);
+
+    finishApplicationMaster(
+        FinishApplicationMasterRequest.newInstance(null, null, null),
+        attemptId);
+  }
+
   @Test(timeout = 5000)
   public void testReRegister()
       throws YarnException, IOException, InterruptedException {
 
-    createAndRegisterApplicationMaster(
+    launchUAM(attemptId);
+    registerApplicationMaster(
         RegisterApplicationMasterRequest.newInstance(null, 0, null), attemptId);
 
     uam.setShouldReRegisterNext();
@@ -137,7 +175,8 @@ public class TestUnmanagedApplicationManager {
       @Override
       public void run() {
         try {
-          createAndRegisterApplicationMaster(
+          launchUAM(attemptId);
+          registerApplicationMaster(
               RegisterApplicationMasterRequest.newInstance(null, 1001, null),
               attemptId);
         } catch (Exception e) {
@@ -221,7 +260,8 @@ public class TestUnmanagedApplicationManager {
   @Test
   public void testForceKill()
       throws YarnException, IOException, InterruptedException {
-    createAndRegisterApplicationMaster(
+    launchUAM(attemptId);
+    registerApplicationMaster(
         RegisterApplicationMasterRequest.newInstance(null, 0, null), attemptId);
     uam.forceKillApplication();
 
@@ -241,19 +281,40 @@ public class TestUnmanagedApplicationManager {
     return ugi;
   }
 
-  protected RegisterApplicationMasterResponse
-      createAndRegisterApplicationMaster(
-          final RegisterApplicationMasterRequest request,
-          ApplicationAttemptId appAttemptId)
-          throws YarnException, IOException, InterruptedException {
+  protected Token<AMRMTokenIdentifier> launchUAM(
+      ApplicationAttemptId appAttemptId)
+      throws IOException, InterruptedException {
+    return getUGIWithToken(appAttemptId)
+        .doAs(new PrivilegedExceptionAction<Token<AMRMTokenIdentifier>>() {
+          @Override
+          public Token<AMRMTokenIdentifier> run() throws Exception {
+            return uam.launchUAM();
+          }
+        });
+  }
+
+  protected void reAttachUAM(final Token<AMRMTokenIdentifier> uamToken,
+      ApplicationAttemptId appAttemptId)
+      throws IOException, InterruptedException {
+    getUGIWithToken(appAttemptId).doAs(new PrivilegedExceptionAction<Object>() {
+      @Override
+      public Token<AMRMTokenIdentifier> run() throws Exception {
+        uam.reAttachUAM(uamToken);
+        return null;
+      }
+    });
+  }
+
+  protected RegisterApplicationMasterResponse registerApplicationMaster(
+      final RegisterApplicationMasterRequest request,
+      ApplicationAttemptId appAttemptId)
+      throws YarnException, IOException, InterruptedException {
     return getUGIWithToken(appAttemptId).doAs(
         new PrivilegedExceptionAction<RegisterApplicationMasterResponse>() {
           @Override
           public RegisterApplicationMasterResponse run()
               throws YarnException, IOException {
-            RegisterApplicationMasterResponse response =
-                uam.createAndRegisterApplicationMaster(request);
-            return response;
+            return uam.registerApplicationMaster(request);
           }
         });
   }
@@ -311,8 +372,9 @@ public class TestUnmanagedApplicationManager {
 
     public TestableUnmanagedApplicationManager(Configuration conf,
         ApplicationId appId, String queueName, String submitter,
-        String appNameSuffix) {
-      super(conf, appId, queueName, submitter, appNameSuffix);
+        String appNameSuffix, boolean keepContainersAcrossApplicationAttempts) {
+      super(conf, appId, queueName, submitter, appNameSuffix,
+          keepContainersAcrossApplicationAttempts);
     }
 
     @SuppressWarnings("unchecked")
@@ -330,6 +392,14 @@ public class TestUnmanagedApplicationManager {
         rmProxy.setShouldReRegisterNext();
       }
     }
+
+    public MockResourceManagerFacade getRMProxy() {
+      return rmProxy;
+    }
+
+    public void setRMProxy(MockResourceManagerFacade proxy) {
+      this.rmProxy = proxy;
+    }
   }
 
 }

+ 16 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/AMRMProxyApplicationContext.java

@@ -19,6 +19,8 @@
 package org.apache.hadoop.yarn.server.nodemanager.amrmproxy;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.registry.client.api.RegistryOperations;
+import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
@@ -67,4 +69,18 @@ public interface AMRMProxyApplicationContext {
    */
   Context getNMCotext();
 
+  /**
+   * Gets the credentials of this application.
+   *
+   * @return the credentials.
+   */
+  Credentials getCredentials();
+
+  /**
+   * Gets the registry client.
+   *
+   * @return the registry.
+   */
+  RegistryOperations getRegistryClient();
+
 }

+ 29 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/AMRMProxyApplicationContextImpl.java

@@ -22,6 +22,8 @@ import java.io.IOException;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.registry.client.api.RegistryOperations;
+import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
@@ -42,6 +44,8 @@ public class AMRMProxyApplicationContextImpl implements
   private Integer localTokenKeyId;
   private Token<AMRMTokenIdentifier> amrmToken;
   private Token<AMRMTokenIdentifier> localToken;
+  private Credentials credentials;
+  private RegistryOperations registry;
 
   /**
    * Create an instance of the AMRMProxyApplicationContext.
@@ -52,17 +56,23 @@ public class AMRMProxyApplicationContextImpl implements
    * @param user user name of the application
    * @param amrmToken amrmToken issued by RM
    * @param localToken amrmToken issued by AMRMProxy
+   * @param credentials application credentials
+   * @param registry Yarn Registry client
    */
-  public AMRMProxyApplicationContextImpl(Context nmContext,
-      Configuration conf, ApplicationAttemptId applicationAttemptId,
-      String user, Token<AMRMTokenIdentifier> amrmToken,
-      Token<AMRMTokenIdentifier> localToken) {
+  @SuppressWarnings("checkstyle:parameternumber")
+  public AMRMProxyApplicationContextImpl(Context nmContext, Configuration conf,
+      ApplicationAttemptId applicationAttemptId, String user,
+      Token<AMRMTokenIdentifier> amrmToken,
+      Token<AMRMTokenIdentifier> localToken, Credentials credentials,
+      RegistryOperations registry) {
     this.nmContext = nmContext;
     this.conf = conf;
     this.applicationAttemptId = applicationAttemptId;
     this.user = user;
     this.amrmToken = amrmToken;
     this.localToken = localToken;
+    this.credentials = credentials;
+    this.registry = registry;
   }
 
   @Override
@@ -88,11 +98,14 @@ public class AMRMProxyApplicationContextImpl implements
   /**
    * Sets the application's AMRMToken.
    *
-   * @param amrmToken amrmToken issued by RM
+   * @param amrmToken the new amrmToken from RM
+   * @return whether the saved token is updated to a different value
    */
-  public synchronized void setAMRMToken(
+  public synchronized boolean setAMRMToken(
       Token<AMRMTokenIdentifier> amrmToken) {
+    Token<AMRMTokenIdentifier> oldValue = this.amrmToken;
     this.amrmToken = amrmToken;
+    return !this.amrmToken.equals(oldValue);
   }
 
   @Override
@@ -134,4 +147,14 @@ public class AMRMProxyApplicationContextImpl implements
   public Context getNMCotext() {
     return nmContext;
   }
+
+  @Override
+  public Credentials getCredentials() {
+    return this.credentials;
+  }
+
+  @Override
+  public RegistryOperations getRegistryClient() {
+    return this.registry;
+  }
 }

+ 65 - 18
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/AMRMProxyService.java

@@ -34,12 +34,13 @@ import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.ipc.Server;
+import org.apache.hadoop.registry.client.api.RegistryOperations;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.SaslRpcServer;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
-import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.service.CompositeService;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
@@ -60,15 +61,19 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
 import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
+import org.apache.hadoop.yarn.server.api.ContainerType;
+import org.apache.hadoop.yarn.server.federation.utils.FederationStateStoreFacade;
 import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEventType;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService.RecoveredAMRMProxyState;
 import org.apache.hadoop.yarn.server.nodemanager.scheduler.DistributedScheduler;
 import org.apache.hadoop.yarn.server.security.MasterKeyData;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.server.utils.YarnServerSecurityUtils;
+import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -82,7 +87,7 @@ import com.google.common.base.Preconditions;
  * pipeline is a chain of interceptor instances that can inspect and modify the
  * request/response as needed.
  */
-public class AMRMProxyService extends AbstractService implements
+public class AMRMProxyService extends CompositeService implements
     ApplicationMasterProtocol {
   private static final Logger LOG = LoggerFactory
       .getLogger(AMRMProxyService.class);
@@ -96,6 +101,7 @@ public class AMRMProxyService extends AbstractService implements
   private InetSocketAddress listenerEndpoint;
   private AMRMProxyTokenSecretManager secretManager;
   private Map<ApplicationId, RequestInterceptorChainWrapper> applPipelineMap;
+  private RegistryOperations registry;
 
   /**
    * Creates an instance of the service.
@@ -118,10 +124,23 @@ public class AMRMProxyService extends AbstractService implements
 
   @Override
   protected void serviceInit(Configuration conf) throws Exception {
-    super.serviceInit(conf);
     this.secretManager =
         new AMRMProxyTokenSecretManager(this.nmContext.getNMStateStore());
     this.secretManager.init(conf);
+
+    // Both second app attempt and NM restart within Federation need registry
+    if (conf.getBoolean(YarnConfiguration.AMRM_PROXY_HA_ENABLED,
+        YarnConfiguration.DEFAULT_AMRM_PROXY_HA_ENABLED)
+        || conf.getBoolean(YarnConfiguration.NM_RECOVERY_ENABLED,
+            YarnConfiguration.DEFAULT_NM_RECOVERY_ENABLED)) {
+      this.registry = FederationStateStoreFacade.createInstance(conf,
+          YarnConfiguration.YARN_REGISTRY_CLASS,
+          YarnConfiguration.DEFAULT_YARN_REGISTRY_CLASS,
+          RegistryOperations.class);
+      addService(this.registry);
+    }
+
+    super.serviceInit(conf);
   }
 
   @Override
@@ -203,6 +222,8 @@ public class AMRMProxyService extends AbstractService implements
             amrmToken = new Token<>();
             amrmToken.decodeFromUrlString(
                 new String(contextEntry.getValue(), "UTF-8"));
+            // Clear the service field, as if RM just issued the token
+            amrmToken.setService(new Text());
           }
         }
 
@@ -214,12 +235,36 @@ public class AMRMProxyService extends AbstractService implements
           throw new IOException("No user found for app attempt " + attemptId);
         }
 
+        // Regenerate the local AMRMToken for the AM
         Token<AMRMTokenIdentifier> localToken =
             this.secretManager.createAndGetAMRMToken(attemptId);
 
+        // Retrieve the AM container credentials from NM context
+        Credentials amCred = null;
+        for (Container container : this.nmContext.getContainers().values()) {
+          LOG.debug("From NM Context container " + container.getContainerId());
+          if (container.getContainerId().getApplicationAttemptId().equals(
+              attemptId) && container.getContainerTokenIdentifier() != null) {
+            LOG.debug("Container type "
+                + container.getContainerTokenIdentifier().getContainerType());
+            if (container.getContainerTokenIdentifier()
+                .getContainerType() == ContainerType.APPLICATION_MASTER) {
+              LOG.info("AM container {} found in context, has credentials: {}",
+                  container.getContainerId(),
+                  (container.getCredentials() != null));
+              amCred = container.getCredentials();
+            }
+          }
+        }
+        if (amCred == null) {
+          LOG.error("No credentials found for AM container of {}. "
+              + "Yarn registry access might not work", attemptId);
+        }
+
+        // Create the intercepter pipeline for the AM
         initializePipeline(attemptId, user, amrmToken, localToken,
-            entry.getValue(), true);
-      } catch (Exception e) {
+            entry.getValue(), true, amCred);
+      } catch (IOException e) {
         LOG.error("Exception when recovering " + attemptId
             + ", removing it from NMStateStore and move on", e);
         this.nmContext.getNMStateStore().removeAMRMProxyAppContext(attemptId);
@@ -326,7 +371,7 @@ public class AMRMProxyService extends AbstractService implements
 
     initializePipeline(appAttemptId,
         containerTokenIdentifierForKey.getApplicationSubmitter(), amrmToken,
-        localToken, null, false);
+        localToken, null, false, credentials);
   }
 
   /**
@@ -342,7 +387,8 @@ public class AMRMProxyService extends AbstractService implements
   protected void initializePipeline(ApplicationAttemptId applicationAttemptId,
       String user, Token<AMRMTokenIdentifier> amrmToken,
       Token<AMRMTokenIdentifier> localToken,
-      Map<String, byte[]> recoveredDataMap, boolean isRecovery) {
+      Map<String, byte[]> recoveredDataMap, boolean isRecovery,
+      Credentials credentials) {
     RequestInterceptorChainWrapper chainWrapper = null;
     synchronized (applPipelineMap) {
       if (applPipelineMap
@@ -404,8 +450,9 @@ public class AMRMProxyService extends AbstractService implements
     try {
       RequestInterceptor interceptorChain =
           this.createRequestInterceptorChain();
-      interceptorChain.init(createApplicationMasterContext(this.nmContext,
-          applicationAttemptId, user, amrmToken, localToken));
+      interceptorChain.init(
+          createApplicationMasterContext(this.nmContext, applicationAttemptId,
+              user, amrmToken, localToken, credentials, this.registry));
       if (isRecovery) {
         if (recoveredDataMap == null) {
           throw new YarnRuntimeException(
@@ -497,14 +544,12 @@ public class AMRMProxyService extends AbstractService implements
       allocateResponse.setAMRMToken(null);
 
       org.apache.hadoop.security.token.Token<AMRMTokenIdentifier> newToken =
-          new org.apache.hadoop.security.token.Token<AMRMTokenIdentifier>(
-              token.getIdentifier().array(), token.getPassword().array(),
-              new Text(token.getKind()), new Text(token.getService()));
-
-      context.setAMRMToken(newToken);
+          ConverterUtils.convertFromYarn(token, (Text) null);
 
-      // Update the AMRMToken in context map in NM state store
-      if (this.nmContext.getNMStateStore() != null) {
+      // Update the AMRMToken in context map, and in NM state store if it is
+      // different
+      if (context.setAMRMToken(newToken)
+          && this.nmContext.getNMStateStore() != null) {
         try {
           this.nmContext.getNMStateStore().storeAMRMProxyAppContextEntry(
               context.getApplicationAttemptId(), NMSS_AMRMTOKEN_KEY,
@@ -547,10 +592,12 @@ public class AMRMProxyService extends AbstractService implements
   private AMRMProxyApplicationContext createApplicationMasterContext(
       Context context, ApplicationAttemptId applicationAttemptId, String user,
       Token<AMRMTokenIdentifier> amrmToken,
-      Token<AMRMTokenIdentifier> localToken) {
+      Token<AMRMTokenIdentifier> localToken, Credentials credentials,
+      RegistryOperations registryImpl) {
     AMRMProxyApplicationContextImpl appContext =
         new AMRMProxyApplicationContextImpl(context, getConfig(),
-            applicationAttemptId, user, amrmToken, localToken);
+            applicationAttemptId, user, amrmToken, localToken, credentials,
+            registryImpl);
     return appContext;
   }
 

+ 204 - 17
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/FederationInterceptor.java

@@ -34,6 +34,8 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
@@ -42,6 +44,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest
 import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
@@ -56,17 +59,20 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.InvalidApplicationMasterRequestException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
 import org.apache.hadoop.yarn.server.federation.failover.FederationProxyProviderUtil;
 import org.apache.hadoop.yarn.server.federation.policies.FederationPolicyUtils;
 import org.apache.hadoop.yarn.server.federation.policies.amrmproxy.FederationAMRMProxyPolicy;
 import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
 import org.apache.hadoop.yarn.server.federation.resolver.SubClusterResolver;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
+import org.apache.hadoop.yarn.server.federation.utils.FederationRegistryClient;
 import org.apache.hadoop.yarn.server.federation.utils.FederationStateStoreFacade;
 import org.apache.hadoop.yarn.server.uam.UnmanagedAMPoolManager;
 import org.apache.hadoop.yarn.server.utils.AMRMClientUtils;
 import org.apache.hadoop.yarn.server.utils.YarnServerSecurityUtils;
 import org.apache.hadoop.yarn.util.AsyncCallback;
+import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -145,6 +151,8 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
    */
   private UserGroupInformation appOwner;
 
+  private FederationRegistryClient registryClient;
+
   /**
    * Creates an instance of the FederationInterceptor class.
    */
@@ -179,6 +187,10 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
     } catch (Exception ex) {
       throw new YarnRuntimeException(ex);
     }
+    // Add all app tokens for Yarn Registry access
+    if (this.registryClient != null && appContext.getCredentials() != null) {
+      this.appOwner.addCredentials(appContext.getCredentials());
+    }
 
     this.homeSubClusterId =
         SubClusterId.newInstance(YarnConfiguration.getClusterId(conf));
@@ -192,6 +204,11 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
 
     this.uamPool.init(conf);
     this.uamPool.start();
+
+    if (appContext.getRegistryClient() != null) {
+      this.registryClient = new FederationRegistryClient(conf,
+          appContext.getRegistryClient(), this.appOwner);
+    }
   }
 
   /**
@@ -250,20 +267,27 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
      */
     this.amRegistrationResponse =
         this.homeRM.registerApplicationMaster(request);
+    if (this.amRegistrationResponse
+        .getContainersFromPreviousAttempts() != null) {
+      cacheAllocatedContainers(
+          this.amRegistrationResponse.getContainersFromPreviousAttempts(),
+          this.homeSubClusterId);
+    }
+
+    ApplicationId appId =
+        getApplicationContext().getApplicationAttemptId().getApplicationId();
+    reAttachUAMAndMergeRegisterResponse(this.amRegistrationResponse, appId);
 
     // the queue this application belongs will be used for getting
     // AMRMProxy policy from state store.
     String queue = this.amRegistrationResponse.getQueue();
     if (queue == null) {
-      LOG.warn("Received null queue for application "
-          + getApplicationContext().getApplicationAttemptId().getApplicationId()
-          + " from home sub-cluster. Will use default queue name "
+      LOG.warn("Received null queue for application " + appId
+          + " from home subcluster. Will use default queue name "
           + YarnConfiguration.DEFAULT_QUEUE_NAME
           + " for getting AMRMProxyPolicy");
     } else {
-      LOG.info("Application "
-          + getApplicationContext().getApplicationAttemptId().getApplicationId()
-          + " belongs to queue " + queue);
+      LOG.info("Application " + appId + " belongs to queue " + queue);
     }
 
     // Initialize the AMRMProxyPolicy
@@ -304,7 +328,7 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
       AllocateResponse homeResponse = AMRMClientUtils.allocateWithReRegister(
           requests.get(this.homeSubClusterId), this.homeRM,
           this.amRegistrationRequest,
-          getApplicationContext().getApplicationAttemptId());
+          getApplicationContext().getApplicationAttemptId().getApplicationId());
 
       // Notify policy of home response
       try {
@@ -393,8 +417,8 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
     // request to the home resource manager on this thread.
     FinishApplicationMasterResponse homeResponse =
         AMRMClientUtils.finishAMWithReRegister(request, this.homeRM,
-            this.amRegistrationRequest,
-            getApplicationContext().getApplicationAttemptId());
+            this.amRegistrationRequest, getApplicationContext()
+                .getApplicationAttemptId().getApplicationId());
 
     if (subClusterIds.size() > 0) {
       // Wait for other sub-cluster resource managers to return the
@@ -425,6 +449,14 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
 
     if (failedToUnRegister) {
       homeResponse.setIsUnregistered(false);
+    } else {
+      // Clean up UAMs only when the app finishes successfully, so that no more
+      // attempt will be launched.
+      this.uamPool.stop();
+      if (this.registryClient != null) {
+        this.registryClient.removeAppFromRegistry(getApplicationContext()
+            .getApplicationAttemptId().getApplicationId());
+      }
     }
     return homeResponse;
   }
@@ -442,9 +474,8 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
    */
   @Override
   public void shutdown() {
-    if (this.uamPool != null) {
-      this.uamPool.stop();
-    }
+    // Do not stop uamPool service and kill UAMs here because of possible second
+    // app attempt
     if (threadpool != null) {
       try {
         threadpool.shutdown();
@@ -455,6 +486,16 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
     super.shutdown();
   }
 
+  /**
+   * Only for unit test cleanup.
+   */
+  @VisibleForTesting
+  protected void cleanupRegistry() {
+    if (this.registryClient != null) {
+      this.registryClient.cleanAllApplications();
+    }
+  }
+
   /**
    * Create the UAM pool manager for secondary sub-clsuters. For unit test to
    * override.
@@ -486,6 +527,120 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
     }
   }
 
+  private void mergeRegisterResponse(
+      RegisterApplicationMasterResponse homeResponse,
+      RegisterApplicationMasterResponse otherResponse) {
+
+    if (!isNullOrEmpty(otherResponse.getContainersFromPreviousAttempts())) {
+      if (!isNullOrEmpty(homeResponse.getContainersFromPreviousAttempts())) {
+        homeResponse.getContainersFromPreviousAttempts()
+            .addAll(otherResponse.getContainersFromPreviousAttempts());
+      } else {
+        homeResponse.setContainersFromPreviousAttempts(
+            otherResponse.getContainersFromPreviousAttempts());
+      }
+    }
+
+    if (!isNullOrEmpty(otherResponse.getNMTokensFromPreviousAttempts())) {
+      if (!isNullOrEmpty(homeResponse.getNMTokensFromPreviousAttempts())) {
+        homeResponse.getNMTokensFromPreviousAttempts()
+            .addAll(otherResponse.getNMTokensFromPreviousAttempts());
+      } else {
+        homeResponse.setNMTokensFromPreviousAttempts(
+            otherResponse.getNMTokensFromPreviousAttempts());
+      }
+    }
+  }
+
+  /**
+   * Try re-attach to all existing and running UAMs in secondary sub-clusters
+   * launched by previous application attempts if any. All running containers in
+   * the UAMs will be combined into the registerResponse. For the first attempt,
+   * the registry will be empty for this application and thus no-op here.
+   */
+  protected void reAttachUAMAndMergeRegisterResponse(
+      RegisterApplicationMasterResponse homeResponse,
+      final ApplicationId appId) {
+
+    if (this.registryClient == null) {
+      // Both AMRMProxy HA and NM work preserving restart is not enabled
+      LOG.warn("registryClient is null, skip attaching existing UAM if any");
+      return;
+    }
+
+    // Load existing running UAMs from the previous attempts from
+    // registry, if any
+    Map<String, Token<AMRMTokenIdentifier>> uamMap =
+        this.registryClient.loadStateFromRegistry(appId);
+    if (uamMap.size() == 0) {
+      LOG.info("No existing UAM for application {} found in Yarn Registry",
+          appId);
+      return;
+    }
+    LOG.info("Found {} existing UAMs for application {} in Yarn Registry. "
+        + "Reattaching in parallel", uamMap.size(), appId);
+
+    ExecutorCompletionService<RegisterApplicationMasterResponse>
+        completionService = new ExecutorCompletionService<>(threadpool);
+
+    for (Entry<String, Token<AMRMTokenIdentifier>> entry : uamMap.entrySet()) {
+      final SubClusterId subClusterId =
+          SubClusterId.newInstance(entry.getKey());
+      final Token<AMRMTokenIdentifier> amrmToken = entry.getValue();
+
+      completionService
+          .submit(new Callable<RegisterApplicationMasterResponse>() {
+            @Override
+            public RegisterApplicationMasterResponse call() throws Exception {
+              RegisterApplicationMasterResponse response = null;
+              try {
+                // Create a config loaded with federation on and subclusterId
+                // for each UAM
+                YarnConfiguration config = new YarnConfiguration(getConf());
+                FederationProxyProviderUtil.updateConfForFederation(config,
+                    subClusterId.getId());
+
+                uamPool.reAttachUAM(subClusterId.getId(), config, appId,
+                    amRegistrationResponse.getQueue(),
+                    getApplicationContext().getUser(), homeSubClusterId.getId(),
+                    amrmToken);
+
+                response = uamPool.registerApplicationMaster(
+                    subClusterId.getId(), amRegistrationRequest);
+
+                if (response != null
+                    && response.getContainersFromPreviousAttempts() != null) {
+                  cacheAllocatedContainers(
+                      response.getContainersFromPreviousAttempts(),
+                      subClusterId);
+                }
+                LOG.info("UAM {} reattached for {}", subClusterId, appId);
+              } catch (Throwable e) {
+                LOG.error(
+                    "Reattaching UAM " + subClusterId + " failed for " + appId,
+                    e);
+              }
+              return response;
+            }
+          });
+    }
+
+    // Wait for the re-attach responses
+    for (int i = 0; i < uamMap.size(); i++) {
+      try {
+        Future<RegisterApplicationMasterResponse> future =
+            completionService.take();
+        RegisterApplicationMasterResponse registerResponse = future.get();
+        if (registerResponse != null) {
+          LOG.info("Merging register response for {}", appId);
+          mergeRegisterResponse(homeResponse, registerResponse);
+        }
+      } catch (Exception e) {
+        LOG.warn("Reattaching UAM failed for ApplicationId: " + appId, e);
+      }
+    }
+  }
+
   private SubClusterId getSubClusterForNode(String nodeName) {
     SubClusterId subClusterId = null;
     try {
@@ -655,6 +810,20 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
                 responses.add(response);
               }
 
+              // Save the new AMRMToken for the UAM in registry if present
+              if (response.getAMRMToken() != null) {
+                Token<AMRMTokenIdentifier> newToken = ConverterUtils
+                    .convertFromYarn(response.getAMRMToken(), (Text) null);
+                // Update the token in registry
+                if (registryClient != null) {
+                  registryClient
+                      .writeAMRMTokenForUAM(
+                          getApplicationContext().getApplicationAttemptId()
+                              .getApplicationId(),
+                          subClusterId.getId(), newToken);
+                }
+              }
+
               // Notify policy of secondary sub-cluster responses
               try {
                 policyInterpreter.notifyOfResponse(subClusterId, response);
@@ -714,20 +883,23 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
                     subClusterId);
 
                 RegisterApplicationMasterResponse uamResponse = null;
+                Token<AMRMTokenIdentifier> token = null;
                 try {
                   // For appNameSuffix, use subClusterId of the home sub-cluster
-                  uamResponse = uamPool.createAndRegisterNewUAM(subClusterId,
-                      registerRequest, config,
+                  token = uamPool.launchUAM(subClusterId, config,
                       appContext.getApplicationAttemptId().getApplicationId(),
                       amRegistrationResponse.getQueue(), appContext.getUser(),
-                      homeSubClusterId.toString());
+                      homeSubClusterId.toString(), registryClient != null);
+
+                  uamResponse = uamPool.registerApplicationMaster(subClusterId,
+                      registerRequest);
                 } catch (Throwable e) {
                   LOG.error("Failed to register application master: "
                       + subClusterId + " Application: "
                       + appContext.getApplicationAttemptId(), e);
                 }
                 return new RegisterApplicationMasterResponseInfo(uamResponse,
-                    SubClusterId.newInstance(subClusterId));
+                    SubClusterId.newInstance(subClusterId), token);
               }
             });
       }
@@ -752,6 +924,14 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
                 + getApplicationContext().getApplicationAttemptId());
             successfulRegistrations.put(uamResponse.getSubClusterId(),
                 uamResponse.getResponse());
+
+            if (registryClient != null) {
+              registryClient.writeAMRMTokenForUAM(
+                  getApplicationContext().getApplicationAttemptId()
+                      .getApplicationId(),
+                  uamResponse.getSubClusterId().getId(),
+                  uamResponse.getUamToken());
+            }
           }
         } catch (Exception e) {
           LOG.warn("Failed to register unmanaged application master: "
@@ -1087,11 +1267,14 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
   private static class RegisterApplicationMasterResponseInfo {
     private RegisterApplicationMasterResponse response;
     private SubClusterId subClusterId;
+    private Token<AMRMTokenIdentifier> uamToken;
 
     RegisterApplicationMasterResponseInfo(
-        RegisterApplicationMasterResponse response, SubClusterId subClusterId) {
+        RegisterApplicationMasterResponse response, SubClusterId subClusterId,
+        Token<AMRMTokenIdentifier> uamToken) {
       this.response = response;
       this.subClusterId = subClusterId;
+      this.uamToken = uamToken;
     }
 
     public RegisterApplicationMasterResponse getResponse() {
@@ -1101,6 +1284,10 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
     public SubClusterId getSubClusterId() {
       return subClusterId;
     }
+
+    public Token<AMRMTokenIdentifier> getUamToken() {
+      return uamToken;
+    }
   }
 
   /**

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

@@ -353,10 +353,6 @@ public class ContainerManagerImpl extends CompositeService implements
       rsrcLocalizationSrvc.recoverLocalizedResources(
           stateStore.loadLocalizationState());
 
-      if (this.amrmProxyEnabled) {
-        this.getAMRMProxyService().recover();
-      }
-
       RecoveredApplicationsState appsState = stateStore.loadApplicationsState();
       for (ContainerManagerApplicationProto proto :
            appsState.getApplications()) {
@@ -373,6 +369,11 @@ public class ContainerManagerImpl extends CompositeService implements
         recoverContainer(rcs);
       }
 
+      // Recovery AMRMProxy state after apps and containers are recovered
+      if (this.amrmProxyEnabled) {
+        this.getAMRMProxyService().recover();
+      }
+
       //Dispatching the RECOVERY_COMPLETED event through the dispatcher
       //so that all the paused, scheduled and queued containers will
       //be scheduled for execution on availability of resources.

+ 12 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/BaseAMRMProxyTest.java

@@ -31,6 +31,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRespo
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Priority;
@@ -56,10 +57,10 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Cont
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.resourceplugin.ResourcePluginManager;
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMMemoryStateStoreService;
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService;
+import org.apache.hadoop.yarn.server.scheduler.OpportunisticContainerAllocator;
 import org.apache.hadoop.yarn.server.nodemanager.security.NMContainerTokenSecretManager;
 import org.apache.hadoop.yarn.server.nodemanager.security.NMTokenSecretManagerInNM;
 import org.apache.hadoop.yarn.server.nodemanager.timelineservice.NMTimelinePublisher;
-import org.apache.hadoop.yarn.server.scheduler.OpportunisticContainerAllocator;
 import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
 import org.apache.hadoop.yarn.util.Records;
 import org.junit.After;
@@ -179,6 +180,15 @@ public abstract class BaseAMRMProxyTest {
     return new NMContext(null, null, null, null, stateStore, false, this.conf);
   }
 
+  protected List<ContainerId> getCompletedContainerIds(
+      List<ContainerStatus> containerStatus) {
+    List<ContainerId> ret = new ArrayList<>();
+    for (ContainerStatus status : containerStatus) {
+      ret.add(status.getContainerId());
+    }
+    return ret;
+  }
+
   /**
    * This helper method will invoke the specified function in parallel for each
    * end point in the specified list using a thread pool and return the
@@ -623,7 +633,7 @@ public abstract class BaseAMRMProxyTest {
      */
     public void initApp(ApplicationAttemptId applicationId, String user) {
       super.initializePipeline(applicationId, user,
-          new Token<AMRMTokenIdentifier>(), null, null, false);
+          new Token<AMRMTokenIdentifier>(), null, null, false, null);
     }
 
     public void stopApp(ApplicationId applicationId) {

+ 10 - 11
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestAMRMProxyService.java

@@ -444,7 +444,7 @@ public class TestAMRMProxyService extends BaseAMRMProxyTest {
 
     applicationAttemptId = ApplicationAttemptId.newInstance(appId, 2);
     getAMRMProxyService().initializePipeline(applicationAttemptId, user,
-        new Token<AMRMTokenIdentifier>(), null, null, false);
+        new Token<AMRMTokenIdentifier>(), null, null, false, null);
 
     RequestInterceptorChainWrapper chain2 =
         getAMRMProxyService().getPipelines().get(appId);
@@ -531,16 +531,14 @@ public class TestAMRMProxyService extends BaseAMRMProxyTest {
         "new AMRMToken from RM should have been nulled by AMRMProxyService",
         allocateResponse.getAMRMToken());
 
-    // The way the mock resource manager is setup, it will return the containers
-    // that were released in the response. This is done because the UAMs run
-    // asynchronously and we need to if all the resource managers received the
-    // release it. The containers sent by the mock resource managers will be
+    // We need to make sure all the resource managers received the
+    // release list. The containers sent by the mock resource managers will be
     // aggregated and returned back to us and we can assert if all the release
     // lists reached the sub-clusters
-    List<Container> containersForReleasedContainerIds =
-        new ArrayList<Container>();
-    containersForReleasedContainerIds.addAll(allocateResponse
-        .getAllocatedContainers());
+    List<ContainerId> containersForReleasedContainerIds = new ArrayList<>();
+    List<ContainerId> newlyFinished = getCompletedContainerIds(
+        allocateResponse.getCompletedContainersStatuses());
+    containersForReleasedContainerIds.addAll(newlyFinished);
 
     // Send max 10 heart beats to receive all the containers. If not, we will
     // fail the test
@@ -554,8 +552,9 @@ public class TestAMRMProxyService extends BaseAMRMProxyTest {
           "new AMRMToken from RM should have been nulled by AMRMProxyService",
           allocateResponse.getAMRMToken());
 
-      containersForReleasedContainerIds.addAll(allocateResponse
-          .getAllocatedContainers());
+      newlyFinished = getCompletedContainerIds(
+          allocateResponse.getCompletedContainersStatuses());
+      containersForReleasedContainerIds.addAll(newlyFinished);
 
       LOG.info("Number of containers received in this request: "
           + Integer.toString(allocateResponse.getAllocatedContainers()

+ 110 - 16
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestFederationInterceptor.java

@@ -19,16 +19,20 @@
 package org.apache.hadoop.yarn.server.nodemanager.amrmproxy;
 
 import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorCompletionService;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 
+import org.apache.hadoop.registry.client.api.RegistryOperations;
+import org.apache.hadoop.registry.client.impl.FSRegistryOperationsService;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
@@ -59,6 +63,10 @@ import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterRegisterRequest;
 import org.apache.hadoop.yarn.server.federation.store.records.SubClusterState;
 import org.apache.hadoop.yarn.server.federation.utils.FederationStateStoreFacade;
+import org.apache.hadoop.yarn.server.nodemanager.Context;
+import org.apache.hadoop.yarn.server.nodemanager.NodeManager.NMContext;
+import org.apache.hadoop.yarn.server.nodemanager.recovery.NMMemoryStateStoreService;
+import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService;
 import org.apache.hadoop.yarn.util.Records;
 import org.junit.Assert;
 import org.junit.Test;
@@ -79,7 +87,10 @@ public class TestFederationInterceptor extends BaseAMRMProxyTest {
 
   private TestableFederationInterceptor interceptor;
   private MemoryFederationStateStore stateStore;
+  private NMStateStoreService nmStateStore;
+  private RegistryOperations registry;
 
+  private Context nmContext;
   private int testAppId;
   private ApplicationAttemptId attemptId;
 
@@ -93,15 +104,28 @@ public class TestFederationInterceptor extends BaseAMRMProxyTest {
     FederationStateStoreFacade.getInstance().reinitialize(stateStore,
         getConf());
 
+    nmStateStore = new NMMemoryStateStoreService();
+    nmStateStore.init(getConf());
+    nmStateStore.start();
+
+    registry = new FSRegistryOperationsService();
+    registry.init(getConf());
+    registry.start();
+
     testAppId = 1;
     attemptId = getApplicationAttemptId(testAppId);
-    interceptor.init(new AMRMProxyApplicationContextImpl(null, getConf(),
-        attemptId, "test-user", null, null));
+    nmContext =
+        new NMContext(null, null, null, null, nmStateStore, false, getConf());
+    interceptor.init(new AMRMProxyApplicationContextImpl(nmContext, getConf(),
+        attemptId, "test-user", null, null, null, registry));
+    interceptor.cleanupRegistry();
   }
 
   @Override
   public void tearDown() {
+    interceptor.cleanupRegistry();
     interceptor.shutdown();
+    registry.stop();
     super.tearDown();
   }
 
@@ -207,18 +231,17 @@ public class TestFederationInterceptor extends BaseAMRMProxyTest {
     AllocateResponse allocateResponse = interceptor.allocate(allocateRequest);
     Assert.assertNotNull(allocateResponse);
 
-    // The way the mock resource manager is setup, it will return the containers
-    // that were released in the allocated containers. The release request will
-    // be split and handled by the corresponding UAM. The release containers
-    // returned by the mock resource managers will be aggregated and returned
-    // back to us and we can check if total request size and returned size are
-    // the same
-    List<Container> containersForReleasedContainerIds =
-        new ArrayList<Container>();
-    containersForReleasedContainerIds
-        .addAll(allocateResponse.getAllocatedContainers());
+    // The release request will be split and handled by the corresponding UAM.
+    // The release containers returned by the mock resource managers will be
+    // aggregated and returned back to us and we can check if total request size
+    // and returned size are the same
+    List<ContainerId> containersForReleasedContainerIds =
+        new ArrayList<ContainerId>();
+    List<ContainerId> newlyFinished = getCompletedContainerIds(
+        allocateResponse.getCompletedContainersStatuses());
+    containersForReleasedContainerIds.addAll(newlyFinished);
     LOG.info("Number of containers received in the original request: "
-        + Integer.toString(allocateResponse.getAllocatedContainers().size()));
+        + Integer.toString(newlyFinished.size()));
 
     // Send max 10 heart beats to receive all the containers. If not, we will
     // fail the test
@@ -228,11 +251,12 @@ public class TestFederationInterceptor extends BaseAMRMProxyTest {
       allocateResponse =
           interceptor.allocate(Records.newRecord(AllocateRequest.class));
       Assert.assertNotNull(allocateResponse);
-      containersForReleasedContainerIds
-          .addAll(allocateResponse.getAllocatedContainers());
+      newlyFinished = getCompletedContainerIds(
+          allocateResponse.getCompletedContainersStatuses());
+      containersForReleasedContainerIds.addAll(newlyFinished);
 
       LOG.info("Number of containers received in this request: "
-          + Integer.toString(allocateResponse.getAllocatedContainers().size()));
+          + Integer.toString(newlyFinished.size()));
       LOG.info("Total number of containers received: "
           + Integer.toString(containersForReleasedContainerIds.size()));
       Thread.sleep(10);
@@ -547,4 +571,74 @@ public class TestFederationInterceptor extends BaseAMRMProxyTest {
     Assert.assertEquals(1, response.getUpdatedContainers().size());
     Assert.assertEquals(1, response.getUpdateErrors().size());
   }
+
+  @Test
+  public void testSecondAttempt() throws Exception {
+    ApplicationUserInfo userInfo = getApplicationUserInfo(testAppId);
+    userInfo.getUser().doAs(new PrivilegedExceptionAction<Object>() {
+      @Override
+      public Object run() throws Exception {
+        // Register the application
+        RegisterApplicationMasterRequest registerReq =
+            Records.newRecord(RegisterApplicationMasterRequest.class);
+        registerReq.setHost(Integer.toString(testAppId));
+        registerReq.setRpcPort(testAppId);
+        registerReq.setTrackingUrl("");
+
+        RegisterApplicationMasterResponse registerResponse =
+            interceptor.registerApplicationMaster(registerReq);
+        Assert.assertNotNull(registerResponse);
+
+        Assert.assertEquals(0, interceptor.getUnmanagedAMPoolSize());
+
+        // Allocate one batch of containers
+        registerSubCluster(SubClusterId.newInstance("SC-1"));
+        registerSubCluster(SubClusterId.newInstance(HOME_SC_ID));
+
+        int numberOfContainers = 3;
+        List<Container> containers =
+            getContainersAndAssert(numberOfContainers, numberOfContainers * 2);
+        for (Container c : containers) {
+          System.out.println(c.getId() + " ha");
+        }
+        Assert.assertEquals(1, interceptor.getUnmanagedAMPoolSize());
+
+        // Preserve the mock RM instances for secondaries
+        ConcurrentHashMap<String, MockResourceManagerFacade> secondaries =
+            interceptor.getSecondaryRMs();
+
+        // Increase the attemptId and create a new intercepter instance for it
+        attemptId = ApplicationAttemptId.newInstance(
+            attemptId.getApplicationId(), attemptId.getAttemptId() + 1);
+
+        interceptor = new TestableFederationInterceptor(null, secondaries);
+        interceptor.init(new AMRMProxyApplicationContextImpl(nmContext,
+            getConf(), attemptId, "test-user", null, null, null, registry));
+        registerResponse = interceptor.registerApplicationMaster(registerReq);
+
+        // Should re-attach secondaries and get the three running containers
+        Assert.assertEquals(1, interceptor.getUnmanagedAMPoolSize());
+        Assert.assertEquals(numberOfContainers,
+            registerResponse.getContainersFromPreviousAttempts().size());
+
+        // Release all containers
+        releaseContainersAndAssert(
+            registerResponse.getContainersFromPreviousAttempts());
+
+        // Finish the application
+        FinishApplicationMasterRequest finishReq =
+            Records.newRecord(FinishApplicationMasterRequest.class);
+        finishReq.setDiagnostics("");
+        finishReq.setTrackingUrl("");
+        finishReq.setFinalApplicationStatus(FinalApplicationStatus.SUCCEEDED);
+
+        FinishApplicationMasterResponse finshResponse =
+            interceptor.finishApplicationMaster(finishReq);
+        Assert.assertNotNull(finshResponse);
+        Assert.assertEquals(true, finshResponse.getIsUnregistered());
+        return null;
+      }
+    });
+  }
+
 }

+ 24 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/TestableFederationInterceptor.java

@@ -44,6 +44,15 @@ public class TestableFederationInterceptor extends FederationInterceptor {
   private AtomicInteger runningIndex = new AtomicInteger(0);
   private MockResourceManagerFacade mockRm;
 
+  public TestableFederationInterceptor() {
+  }
+
+  public TestableFederationInterceptor(MockResourceManagerFacade homeRM,
+      ConcurrentHashMap<String, MockResourceManagerFacade> secondaries) {
+    mockRm = homeRM;
+    secondaryResourceManagers = secondaries;
+  }
+
   @Override
   protected UnmanagedAMPoolManager createUnmanagedAMPoolManager(
       ExecutorService threadPool) {
@@ -68,7 +77,7 @@ public class TestableFederationInterceptor extends FederationInterceptor {
     // We create one instance of the mock resource manager per sub cluster. Keep
     // track of the instances of the RMs in the map keyed by the sub cluster id
     synchronized (this.secondaryResourceManagers) {
-      if (this.secondaryResourceManagers.contains(subClusterId)) {
+      if (this.secondaryResourceManagers.containsKey(subClusterId)) {
         return (T) this.secondaryResourceManagers.get(subClusterId);
       } else {
         // The running index here is used to simulate different RM_EPOCH to
@@ -91,6 +100,15 @@ public class TestableFederationInterceptor extends FederationInterceptor {
     }
   }
 
+  protected MockResourceManagerFacade getHomeRM() {
+    return mockRm;
+  }
+
+  protected ConcurrentHashMap<String, MockResourceManagerFacade>
+      getSecondaryRMs() {
+    return secondaryResourceManagers;
+  }
+
   /**
    * Extends the UnmanagedAMPoolManager and overrides methods to provide a
    * testable implementation of UnmanagedAMPoolManager.
@@ -104,9 +122,9 @@ public class TestableFederationInterceptor extends FederationInterceptor {
     @Override
     public UnmanagedApplicationManager createUAM(Configuration conf,
         ApplicationId appId, String queueName, String submitter,
-        String appNameSuffix) {
+        String appNameSuffix, boolean keepContainersAcrossApplicationAttempts) {
       return new TestableUnmanagedApplicationManager(conf, appId, queueName,
-          submitter, appNameSuffix);
+          submitter, appNameSuffix, keepContainersAcrossApplicationAttempts);
     }
   }
 
@@ -119,8 +137,9 @@ public class TestableFederationInterceptor extends FederationInterceptor {
 
     public TestableUnmanagedApplicationManager(Configuration conf,
         ApplicationId appId, String queueName, String submitter,
-        String appNameSuffix) {
-      super(conf, appId, queueName, submitter, appNameSuffix);
+        String appNameSuffix, boolean keepContainersAcrossApplicationAttempts) {
+      super(conf, appId, queueName, submitter, appNameSuffix,
+          keepContainersAcrossApplicationAttempts);
     }
 
     /**

+ 1 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java

@@ -360,13 +360,8 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
   private RMAppImpl createAndPopulateNewRMApp(
       ApplicationSubmissionContext submissionContext, long submitTime,
       String user, boolean isRecovery, long startTime) throws YarnException {
+
     if (!isRecovery) {
-      // Do queue mapping
-      if (rmContext.getQueuePlacementManager() != null) {
-        // We only do queue mapping when it's a new application
-        rmContext.getQueuePlacementManager().placeApplication(
-            submissionContext, user);
-      }
       // fail the submission if configured application timeout value is invalid
       RMServerUtils.validateApplicationTimeouts(
           submissionContext.getApplicationTimeouts());

+ 12 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java

@@ -126,7 +126,9 @@ import java.nio.charset.Charset;
 import java.security.PrivilegedExceptionAction;
 import java.security.SecureRandom;
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 /**
@@ -1062,7 +1064,7 @@ public class ResourceManager extends CompositeService implements Recoverable {
   }
 
   protected void startWepApp() {
-
+    Map<String, String> serviceConfig = null;
     Configuration conf = getConfig();
 
     RMWebAppUtil.setupSecurityAndFilters(conf,
@@ -1128,7 +1130,15 @@ public class ResourceManager extends CompositeService implements Recoverable {
       }
     }
 
-    webApp = builder.start(new RMWebApp(this), uiWebAppContext);
+    if (getConfig().getBoolean(YarnConfiguration.YARN_API_SERVICES_ENABLE,
+        false)) {
+      serviceConfig = new HashMap<String, String>();
+      String apiPackages = "org.apache.hadoop.yarn.service.webapp;" +
+          "org.apache.hadoop.yarn.webapp";
+      serviceConfig.put("PackageName", apiPackages);
+      serviceConfig.put("PathSpec", "/app/*");
+    }
+    webApp = builder.start(new RMWebApp(this), uiWebAppContext, serviceConfig);
   }
 
   private String getWebAppsPath(String appName) {

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

@@ -203,6 +203,12 @@ public class FifoIntraQueuePreemptionPlugin
       Resources.subtractFromNonNegative(preemtableFromApp, tmpApp.selected);
       Resources.subtractFromNonNegative(preemtableFromApp, tmpApp.getAMUsed());
 
+      if (context.getIntraQueuePreemptionOrderPolicy()
+            .equals(IntraQueuePreemptionOrderPolicy.USERLIMIT_FIRST)) {
+        Resources.subtractFromNonNegative(preemtableFromApp,
+          tmpApp.getFiCaSchedulerApp().getCSLeafQueue().getMinimumAllocation());
+      }
+
       // Calculate toBePreempted from apps as follows:
       // app.preemptable = min(max(app.used - app.selected - app.ideal, 0),
       // intra_q_preemptable)

+ 52 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/placement/ApplicationPlacementContext.java

@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.placement;
+
+/**
+ * Each placement rule when it successfully places an application onto a queue
+ * returns a PlacementRuleContext which encapsulates the queue the
+ * application was mapped to and any parent queue for the queue (if configured)
+ */
+public class ApplicationPlacementContext {
+
+  private String queue;
+
+  private String parentQueue;
+
+  public ApplicationPlacementContext(String queue) {
+    this(queue,null);
+  }
+
+  public ApplicationPlacementContext(String queue, String parentQueue) {
+    this.queue = queue;
+    this.parentQueue = parentQueue;
+  }
+
+  public String getQueue() {
+    return queue;
+  }
+
+  public String getParentQueue() {
+    return parentQueue;
+  }
+
+  public boolean hasParentQueue() {
+    return parentQueue != null;
+  }
+}

+ 15 - 19
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/placement/PlacementManager.java

@@ -23,7 +23,6 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
 
-import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
@@ -53,36 +52,33 @@ public class PlacementManager {
     }
   }
 
-  public void placeApplication(ApplicationSubmissionContext asc, String user)
-      throws YarnException {
+  public ApplicationPlacementContext placeApplication(
+      ApplicationSubmissionContext asc, String user) throws YarnException {
+
     try {
       readLock.lock();
+
       if (null == rules || rules.isEmpty()) {
-        return;
+        return null;
       }
-      
-      String newQueueName = null;
+
+      ApplicationPlacementContext placement = null;
       for (PlacementRule rule : rules) {
-        newQueueName = rule.getQueueForApp(asc, user);
-        if (newQueueName != null) {
+        placement = rule.getPlacementForApp(asc, user);
+        if (placement != null) {
           break;
         }
       }
-      
+
       // Failed to get where to place application
-      if (null == newQueueName && null == asc.getQueue()) {
-        String msg = "Failed to get where to place application="
-            + asc.getApplicationId();
+      if (null == placement && null == asc.getQueue()) {
+        String msg = "Failed to get where to place application=" + asc
+            .getApplicationId();
         LOG.error(msg);
         throw new YarnException(msg);
       }
-      
-      // Set it to ApplicationSubmissionContext
-      if (!StringUtils.equals(asc.getQueue(), newQueueName)) {
-        LOG.info("Placed application=" + asc.getApplicationId() + " to queue="
-            + newQueueName + ", original queue=" + asc.getQueue());
-        asc.setQueue(newQueueName);
-      }
+
+      return placement;
     } finally {
       readLock.unlock();
     }

+ 4 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/placement/PlacementRule.java

@@ -25,6 +25,7 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 
 public abstract class PlacementRule {
+
   public String getName() {
     return this.getClass().getName();
   }
@@ -50,6 +51,6 @@ public abstract class PlacementRule {
    *         in the {@link PlacementManager} will take care
    *         </p>
    */
-  public abstract String getQueueForApp(ApplicationSubmissionContext asc,
-      String user) throws YarnException;
-}
+  public abstract ApplicationPlacementContext getPlacementForApp(
+      ApplicationSubmissionContext asc, String user) throws YarnException;
+}

+ 271 - 13
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/placement/UserGroupMappingPlacementRule.java

@@ -19,8 +19,10 @@
 package org.apache.hadoop.yarn.server.resourcemanager.placement;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
@@ -32,6 +34,15 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.resourcemanager.placement.UserGroupMappingPlacementRule.QueueMapping.MappingType;
 
 import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.AutoCreatedLeafQueue;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerContext;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueManager;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ManagedParentQueue;
+
+import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.DOT;
 
 public class UserGroupMappingPlacementRule extends PlacementRule {
   private static final Log LOG = LogFactory
@@ -66,17 +77,41 @@ public class UserGroupMappingPlacementRule extends PlacementRule {
     MappingType type;
     String source;
     String queue;
+    String parentQueue;
+
+    public final static String DELIMITER = ":";
 
     public QueueMapping(MappingType type, String source, String queue) {
       this.type = type;
       this.source = source;
       this.queue = queue;
+      this.parentQueue = null;
     }
-    
+
+    public QueueMapping(MappingType type, String source,
+        String queue, String parentQueue) {
+      this.type = type;
+      this.source = source;
+      this.queue = queue;
+      this.parentQueue = parentQueue;
+    }
+
     public String getQueue() {
       return queue;
     }
-    
+
+    public String getParentQueue() {
+      return parentQueue;
+    }
+
+    public MappingType getType() {
+      return type;
+    }
+
+    public String getSource() {
+      return source;
+    }
+
     @Override
     public int hashCode() {
       return super.hashCode();
@@ -93,6 +128,13 @@ public class UserGroupMappingPlacementRule extends PlacementRule {
         return false;
       }
     }
+
+    public String toString() {
+      return type.toString() + DELIMITER + source + DELIMITER +
+        (parentQueue != null ?
+        parentQueue + "." + queue :
+        queue);
+    }
   }
 
   public UserGroupMappingPlacementRule(boolean overrideWithQueueMappings,
@@ -102,26 +144,27 @@ public class UserGroupMappingPlacementRule extends PlacementRule {
     this.groups = groups;
   }
 
-  private String getMappedQueue(String user) throws IOException {
+  private ApplicationPlacementContext getPlacementForUser(String user)
+      throws IOException {
     for (QueueMapping mapping : mappings) {
       if (mapping.type == MappingType.USER) {
         if (mapping.source.equals(CURRENT_USER_MAPPING)) {
           if (mapping.queue.equals(CURRENT_USER_MAPPING)) {
-            return user;
+            return getPlacementContext(mapping, user);
           } else if (mapping.queue.equals(PRIMARY_GROUP_MAPPING)) {
-            return groups.getGroups(user).get(0);
+            return getPlacementContext(mapping, groups.getGroups(user).get(0));
           } else {
-            return mapping.queue;
+            return getPlacementContext(mapping);
           }
         }
         if (user.equals(mapping.source)) {
-          return mapping.queue;
+          return getPlacementContext(mapping);
         }
       }
       if (mapping.type == MappingType.GROUP) {
         for (String userGroups : groups.getGroups(user)) {
           if (userGroups.equals(mapping.source)) {
-            return mapping.queue;
+            return getPlacementContext(mapping);
           }
         }
       }
@@ -130,13 +173,14 @@ public class UserGroupMappingPlacementRule extends PlacementRule {
   }
 
   @Override
-  public String getQueueForApp(ApplicationSubmissionContext asc, String user)
+  public ApplicationPlacementContext getPlacementForApp(
+      ApplicationSubmissionContext asc, String user)
       throws YarnException {
     String queueName = asc.getQueue();
     ApplicationId applicationId = asc.getApplicationId();
     if (mappings != null && mappings.size() > 0) {
       try {
-        String mappedQueue = getMappedQueue(user);
+        ApplicationPlacementContext mappedQueue = getPlacementForUser(user);
         if (mappedQueue != null) {
           // We have a mapping, should we use it?
           if (queueName.equals(YarnConfiguration.DEFAULT_QUEUE_NAME)
@@ -153,10 +197,224 @@ public class UserGroupMappingPlacementRule extends PlacementRule {
         throw new YarnException(message);
       }
     }
-    
-    return queueName;
+    return null;
   }
-  
+
+  private ApplicationPlacementContext getPlacementContext(
+      QueueMapping mapping) {
+    return getPlacementContext(mapping, mapping.getQueue());
+  }
+
+  private ApplicationPlacementContext getPlacementContext(QueueMapping mapping,
+      String leafQueueName) {
+    if (!StringUtils.isEmpty(mapping.parentQueue)) {
+      return new ApplicationPlacementContext(leafQueueName,
+          mapping.getParentQueue());
+    } else{
+      return new ApplicationPlacementContext(leafQueueName);
+    }
+  }
+
+  @VisibleForTesting
+  public static UserGroupMappingPlacementRule get(
+      CapacitySchedulerContext schedulerContext) throws IOException {
+    CapacitySchedulerConfiguration conf = schedulerContext.getConfiguration();
+    boolean overrideWithQueueMappings = conf.getOverrideWithQueueMappings();
+    LOG.info(
+        "Initialized queue mappings, override: " + overrideWithQueueMappings);
+
+    List<QueueMapping> queueMappings = conf.getQueueMappings();
+
+    // Get new user/group mappings
+    List<QueueMapping> newMappings = new ArrayList<>();
+
+    CapacitySchedulerQueueManager queueManager =
+        schedulerContext.getCapacitySchedulerQueueManager();
+
+    // check if mappings refer to valid queues
+    for (QueueMapping mapping : queueMappings) {
+
+      QueuePath queuePath = extractQueuePath(mapping.getQueue());
+      if (isStaticQueueMapping(mapping)) {
+        //Try getting queue by its leaf queue name
+        // without splitting into parent/leaf queues
+        CSQueue queue = queueManager.getQueue(mapping.getQueue());
+        if (ifQueueDoesNotExist(queue)) {
+          //Try getting the queue by extracting leaf and parent queue names
+          //Assuming its a potential auto created leaf queue
+          queue = queueManager.getQueue(queuePath.getLeafQueue());
+
+          if (ifQueueDoesNotExist(queue)) {
+            //if leaf queue does not exist,
+            // this could be a potential auto created leaf queue
+            //validate if parent queue is specified,
+            // then it should exist and
+            // be an instance of AutoCreateEnabledParentQueue
+            QueueMapping newMapping = validateAndGetAutoCreatedQueueMapping(
+                queueManager, mapping, queuePath);
+            if (newMapping == null) {
+              throw new IOException(
+                  "mapping contains invalid or non-leaf queue " + mapping
+                      .getQueue());
+            }
+            newMappings.add(newMapping);
+          } else{
+            QueueMapping newMapping = validateAndGetQueueMapping(queueManager,
+                queue, mapping, queuePath);
+            newMappings.add(newMapping);
+          }
+        } else{
+          // if queue exists, validate
+          //   if its an instance of leaf queue
+          //   if its an instance of auto created leaf queue,
+          // then extract parent queue name and update queue mapping
+          QueueMapping newMapping = validateAndGetQueueMapping(queueManager,
+              queue, mapping, queuePath);
+          newMappings.add(newMapping);
+        }
+      } else{
+        //If it is a dynamic queue mapping,
+        // we can safely assume leaf queue name does not have '.' in it
+        // validate
+        // if parent queue is specified, then
+        //  parent queue exists and an instance of AutoCreateEnabledParentQueue
+        //
+        QueueMapping newMapping = validateAndGetAutoCreatedQueueMapping(
+            queueManager, mapping, queuePath);
+        if (newMapping != null) {
+          newMappings.add(newMapping);
+        } else{
+          newMappings.add(mapping);
+        }
+      }
+    }
+
+    // initialize groups if mappings are present
+    if (newMappings.size() > 0) {
+      Groups groups = new Groups(conf);
+      return new UserGroupMappingPlacementRule(overrideWithQueueMappings,
+          newMappings, groups);
+    }
+
+    return null;
+  }
+
+  private static QueueMapping validateAndGetQueueMapping(
+      CapacitySchedulerQueueManager queueManager, CSQueue queue,
+      QueueMapping mapping, QueuePath queuePath) throws IOException {
+    if (!(queue instanceof LeafQueue)) {
+      throw new IOException(
+          "mapping contains invalid or non-leaf queue : " + mapping.getQueue());
+    }
+
+    if (queue instanceof AutoCreatedLeafQueue && queue
+        .getParent() instanceof ManagedParentQueue) {
+
+      QueueMapping newMapping = validateAndGetAutoCreatedQueueMapping(
+          queueManager, mapping, queuePath);
+      if (newMapping == null) {
+        throw new IOException(
+            "mapping contains invalid or non-leaf queue " + mapping.getQueue());
+      }
+      return newMapping;
+    }
+    return mapping;
+  }
+
+  private static boolean ifQueueDoesNotExist(CSQueue queue) {
+    return queue == null;
+  }
+
+  private static QueueMapping validateAndGetAutoCreatedQueueMapping(
+      CapacitySchedulerQueueManager queueManager, QueueMapping mapping,
+      QueuePath queuePath) throws IOException {
+    if (queuePath.hasParentQueue()) {
+      //if parent queue is specified,
+      // then it should exist and be an instance of ManagedParentQueue
+      validateParentQueue(queueManager.getQueue(queuePath.getParentQueue()),
+          queuePath.getParentQueue(), queuePath.getLeafQueue());
+      return new QueueMapping(mapping.getType(), mapping.getSource(),
+          queuePath.getLeafQueue(), queuePath.getParentQueue());
+    }
+
+    return null;
+  }
+
+  private static boolean isStaticQueueMapping(QueueMapping mapping) {
+    return !mapping.getQueue().contains(
+        UserGroupMappingPlacementRule.CURRENT_USER_MAPPING) && !mapping
+        .getQueue().contains(
+            UserGroupMappingPlacementRule.PRIMARY_GROUP_MAPPING);
+  }
+
+  private static class QueuePath {
+
+    public String parentQueue;
+    public String leafQueue;
+
+    public QueuePath(final String leafQueue) {
+      this.leafQueue = leafQueue;
+    }
+
+    public QueuePath(final String parentQueue, final String leafQueue) {
+      this.parentQueue = parentQueue;
+      this.leafQueue = leafQueue;
+    }
+
+    public String getParentQueue() {
+      return parentQueue;
+    }
+
+    public String getLeafQueue() {
+      return leafQueue;
+    }
+
+    public boolean hasParentQueue() {
+      return parentQueue != null;
+    }
+
+    @Override
+    public String toString() {
+      return parentQueue + DOT + leafQueue;
+    }
+  }
+
+  private static QueuePath extractQueuePath(String queueName)
+      throws IOException {
+    int parentQueueNameEndIndex = queueName.lastIndexOf(DOT);
+
+    if (parentQueueNameEndIndex > -1) {
+      final String parentQueue = queueName.substring(0, parentQueueNameEndIndex)
+          .trim();
+      final String leafQueue = queueName.substring(parentQueueNameEndIndex + 1)
+          .trim();
+      return new QueuePath(parentQueue, leafQueue);
+    }
+
+    return new QueuePath(queueName);
+  }
+
+  private static void validateParentQueue(CSQueue parentQueue,
+      String parentQueueName, String leafQueueName) throws IOException {
+    if (parentQueue == null) {
+      throw new IOException(
+          "mapping contains invalid or non-leaf queue [" + leafQueueName
+              + "] and invalid parent queue [" + parentQueueName + "]");
+    } else if (!(parentQueue instanceof ManagedParentQueue)) {
+      throw new IOException("mapping contains leaf queue [" + leafQueueName
+          + "] and invalid parent queue which "
+          + "does not have auto creation of leaf queues enabled ["
+          + parentQueueName + "]");
+    } else if (!parentQueue.getQueueName().equals(parentQueueName)) {
+      throw new IOException(
+          "mapping contains invalid or non-leaf queue [" + leafQueueName
+              + "] and invalid parent queue "
+              + "which does not match existing leaf queue's parent : ["
+              + parentQueueName + "] does not match [ " + parentQueue
+              .getQueueName() + "]");
+    }
+  }
+
   @VisibleForTesting
   public List<QueueMapping> getQueueMappings() {
     return mappings;

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

@@ -68,6 +68,7 @@ import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
 import org.apache.hadoop.yarn.security.client.ClientToAMTokenIdentifier;
@@ -83,6 +84,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMServerUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.blacklist.BlacklistManager;
 import org.apache.hadoop.yarn.server.resourcemanager.blacklist.DisabledBlacklistManager;
 import org.apache.hadoop.yarn.server.resourcemanager.blacklist.SimpleBlacklistManager;
+import org.apache.hadoop.yarn.server.resourcemanager.placement.ApplicationPlacementContext;
+import org.apache.hadoop.yarn.server.resourcemanager.placement.PlacementManager;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.Recoverable;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.ApplicationStateData;
@@ -158,6 +161,8 @@ public class RMAppImpl implements RMApp, Recoverable {
 
   private boolean isNumAttemptsBeyondThreshold = false;
 
+
+
   // Mutable fields
   private long startTime;
   private long finishTime = 0;
@@ -1073,38 +1078,51 @@ public class RMAppImpl implements RMApp, Recoverable {
                   app.getUser(),
                   BuilderUtils.parseTokensConf(app.submissionContext));
         } catch (Exception e) {
-          String msg = "Failed to fetch user credentials from application:"
-              + e.getMessage();
+          String msg = "Failed to fetch user credentials from application:" + e
+              .getMessage();
           app.diagnostics.append(msg);
           LOG.error(msg, e);
         }
       }
 
-      for (Map.Entry<ApplicationTimeoutType, Long> timeout :
-        app.applicationTimeouts.entrySet()) {
+      for (Map.Entry<ApplicationTimeoutType, Long> timeout : app.applicationTimeouts
+          .entrySet()) {
         app.rmContext.getRMAppLifetimeMonitor().registerApp(app.applicationId,
             timeout.getKey(), timeout.getValue());
         if (LOG.isDebugEnabled()) {
           long remainingTime = timeout.getValue() - app.systemClock.getTime();
           LOG.debug("Application " + app.applicationId
               + " is registered for timeout monitor, type=" + timeout.getKey()
-              + " remaining timeout="
-              + (remainingTime > 0 ? remainingTime / 1000 : 0) + " seconds");
+              + " remaining timeout=" + (remainingTime > 0 ?
+              remainingTime / 1000 :
+              0) + " seconds");
         }
       }
 
+      ApplicationPlacementContext placementContext = null;
+      try {
+        placementContext = placeApplication(app.rmContext,
+            app.submissionContext, app.user);
+      } catch (Exception e) {
+        String msg = "Failed to place application to queue :" + e.getMessage();
+        app.diagnostics.append(msg);
+        LOG.error(msg, e);
+      }
+
       // No existent attempts means the attempt associated with this app was not
       // started or started but not yet saved.
       if (app.attempts.isEmpty()) {
-        app.scheduler.handle(new AppAddedSchedulerEvent(app.user,
-            app.submissionContext, false, app.applicationPriority));
+        app.scheduler.handle(
+            new AppAddedSchedulerEvent(app.user, app.submissionContext, false,
+                app.applicationPriority, placementContext));
         return RMAppState.SUBMITTED;
       }
 
       // Add application to scheduler synchronously to guarantee scheduler
       // knows applications before AM or NM re-registers.
-      app.scheduler.handle(new AppAddedSchedulerEvent(app.user,
-          app.submissionContext, true, app.applicationPriority));
+      app.scheduler.handle(
+          new AppAddedSchedulerEvent(app.user, app.submissionContext, true,
+              app.applicationPriority, placementContext));
 
       // recover attempts
       app.recoverAppAttempts();
@@ -1120,8 +1138,20 @@ public class RMAppImpl implements RMApp, Recoverable {
       RMAppTransition {
     @Override
     public void transition(RMAppImpl app, RMAppEvent event) {
-      app.handler.handle(new AppAddedSchedulerEvent(app.user,
-          app.submissionContext, false, app.applicationPriority));
+      ApplicationPlacementContext placementContext = null;
+      try {
+        placementContext = placeApplication(app.rmContext,
+            app.submissionContext, app.user);
+        replaceQueueFromPlacementContext(placementContext,
+            app.submissionContext);
+      } catch (YarnException e) {
+        String msg = "Failed to place application to queue :" + e.getMessage();
+        app.diagnostics.append(msg);
+        LOG.error(msg, e);
+      }
+      app.handler.handle(
+          new AppAddedSchedulerEvent(app.user, app.submissionContext, false,
+              app.applicationPriority, placementContext));
       // send the ATS create Event
       app.sendATSCreateEvent();
     }
@@ -1611,35 +1641,39 @@ public class RMAppImpl implements RMApp, Recoverable {
     int numNonAMContainerPreempted = 0;
     Map<String, Long> resourceSecondsMap = new HashMap<>();
     Map<String, Long> preemptedSecondsMap = new HashMap<>();
-
-    for (RMAppAttempt attempt : attempts.values()) {
-      if (null != attempt) {
-        RMAppAttemptMetrics attemptMetrics =
-            attempt.getRMAppAttemptMetrics();
-        Resources.addTo(resourcePreempted,
-            attemptMetrics.getResourcePreempted());
-        numAMContainerPreempted += attemptMetrics.getIsPreempted() ? 1 : 0;
-        numNonAMContainerPreempted +=
-            attemptMetrics.getNumNonAMContainersPreempted();
-        // getAggregateAppResourceUsage() will calculate resource usage stats
-        // for both running and finished containers.
-        AggregateAppResourceUsage resUsage =
-            attempt.getRMAppAttemptMetrics().getAggregateAppResourceUsage();
-        for (Map.Entry<String, Long> entry : resUsage
-            .getResourceUsageSecondsMap().entrySet()) {
-          long value = RMServerUtils
-              .getOrDefault(resourceSecondsMap, entry.getKey(), 0L);
-          value += entry.getValue();
-          resourceSecondsMap.put(entry.getKey(), value);
-        }
-        for (Map.Entry<String, Long> entry : attemptMetrics
-            .getPreemptedResourceSecondsMap().entrySet()) {
-          long value = RMServerUtils
-              .getOrDefault(preemptedSecondsMap, entry.getKey(), 0L);
-          value += entry.getValue();
-          preemptedSecondsMap.put(entry.getKey(), value);
+    this.readLock.lock();
+    try {
+      for (RMAppAttempt attempt : attempts.values()) {
+        if (null != attempt) {
+          RMAppAttemptMetrics attemptMetrics =
+              attempt.getRMAppAttemptMetrics();
+          Resources.addTo(resourcePreempted,
+              attemptMetrics.getResourcePreempted());
+          numAMContainerPreempted += attemptMetrics.getIsPreempted() ? 1 : 0;
+          numNonAMContainerPreempted +=
+              attemptMetrics.getNumNonAMContainersPreempted();
+          // getAggregateAppResourceUsage() will calculate resource usage stats
+          // for both running and finished containers.
+          AggregateAppResourceUsage resUsage =
+              attempt.getRMAppAttemptMetrics().getAggregateAppResourceUsage();
+          for (Map.Entry<String, Long> entry : resUsage
+              .getResourceUsageSecondsMap().entrySet()) {
+            long value = RMServerUtils
+                .getOrDefault(resourceSecondsMap, entry.getKey(), 0L);
+            value += entry.getValue();
+            resourceSecondsMap.put(entry.getKey(), value);
+          }
+          for (Map.Entry<String, Long> entry : attemptMetrics
+              .getPreemptedResourceSecondsMap().entrySet()) {
+            long value = RMServerUtils
+                .getOrDefault(preemptedSecondsMap, entry.getKey(), 0L);
+            value += entry.getValue();
+            preemptedSecondsMap.put(entry.getKey(), value);
+          }
         }
       }
+    } finally {
+      this.readLock.unlock();
     }
 
     return new RMAppMetrics(resourcePreempted, numNonAMContainerPreempted,
@@ -2013,4 +2047,37 @@ public class RMAppImpl implements RMApp, Recoverable {
     this.submissionContext.setAMContainerSpec(null);
     this.submissionContext.setLogAggregationContext(null);
   }
+
+  @VisibleForTesting
+  static ApplicationPlacementContext placeApplication(RMContext rmContext,
+      ApplicationSubmissionContext context, String user) throws YarnException {
+
+    ApplicationPlacementContext placementContext = null;
+    PlacementManager placementManager = rmContext.getQueuePlacementManager();
+
+    if (placementManager != null) {
+      placementContext = placementManager.placeApplication(context, user);
+    } else{
+      LOG.error(
+          "Queue Placement Manager is null. Cannot place application :" + " "
+              + context.getApplicationId() + " to queue ");
+    }
+
+    return placementContext;
+  }
+
+  static void replaceQueueFromPlacementContext(
+      ApplicationPlacementContext placementContext,
+      ApplicationSubmissionContext context) {
+    // Set it to ApplicationSubmissionContext
+    //apply queue mapping only to new application submissions
+    if (placementContext != null && !StringUtils.equals(context.getQueue(),
+        placementContext.getQueue())) {
+      LOG.info("Placed application=" + context.getApplicationId() + " to queue="
+          + placementContext.getQueue() + ", original queue=" + context
+          .getQueue());
+      context.setQueue(placementContext.getQueue());
+    }
+  }
+
 }

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

@@ -78,7 +78,7 @@ public abstract class AbstractCSQueue implements CSQueue {
   final String queueName;
   private final String queuePath;
   volatile int numContainers;
-  
+
   final Resource minimumAllocation;
   volatile Resource maximumAllocation;
   private volatile QueueState state = null;

+ 120 - 76
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractManagedParentQueue.java

@@ -35,31 +35,13 @@ public abstract class AbstractManagedParentQueue extends ParentQueue {
   private static final Logger LOG = LoggerFactory.getLogger(
       AbstractManagedParentQueue.class);
 
-  private int maxAppsForAutoCreatedQueues;
-  private int maxAppsPerUserForAutoCreatedQueues;
-  private int userLimit;
-  private float userLimitFactor;
+  protected AutoCreatedLeafQueueTemplate leafQueueTemplate;
 
   public AbstractManagedParentQueue(CapacitySchedulerContext cs,
       String queueName, CSQueue parent, CSQueue old) throws IOException {
     super(cs, queueName, parent, old);
 
     super.setupQueueConfigs(csContext.getClusterResource());
-    initializeLeafQueueConfigs();
-
-    StringBuffer queueInfo = new StringBuffer();
-    queueInfo.append("Created Managed Parent Queue: ").append(queueName)
-        .append("\nof type : [" + getClass())
-        .append("]\nwith capacity: [")
-        .append(super.getCapacity()).append("]\nwith max capacity: [")
-        .append(super.getMaximumCapacity()).append("\nwith max apps: [")
-        .append(getMaxApplicationsForAutoCreatedQueues())
-        .append("]\nwith max apps per user: [")
-        .append(getMaxApplicationsPerUserForAutoCreatedQueues())
-        .append("]\nwith user limit: [").append(getUserLimit())
-        .append("]\nwith user limit factor: [")
-        .append(getUserLimitFactor()).append("].");
-    LOG.info(queueInfo.toString());
   }
 
   @Override
@@ -71,8 +53,6 @@ public abstract class AbstractManagedParentQueue extends ParentQueue {
       // Set new configs
       setupQueueConfigs(clusterResource);
 
-      initializeLeafQueueConfigs();
-
       // run reinitialize on each existing queue, to trigger absolute cap
       // recomputations
       for (CSQueue res : this.getChildQueues()) {
@@ -87,72 +67,29 @@ public abstract class AbstractManagedParentQueue extends ParentQueue {
    * Initialize leaf queue configs from template configurations specified on
    * parent queue.
    */
-  protected void initializeLeafQueueConfigs() {
+  protected AutoCreatedLeafQueueTemplate.Builder initializeLeafQueueConfigs
+    (String queuePath) {
 
     CapacitySchedulerConfiguration conf = csContext.getConfiguration();
 
-    final String queuePath = super.getQueuePath();
+    AutoCreatedLeafQueueTemplate.Builder leafQueueTemplateBuilder = new
+        AutoCreatedLeafQueueTemplate.Builder();
     int maxApps = conf.getMaximumApplicationsPerQueue(queuePath);
     if (maxApps < 0) {
       maxApps = (int) (
           CapacitySchedulerConfiguration.DEFAULT_MAXIMUM_SYSTEM_APPLICATIIONS
               * getAbsoluteCapacity());
     }
-    userLimit = conf.getUserLimit(queuePath);
-    userLimitFactor = conf.getUserLimitFactor(queuePath);
-    maxAppsForAutoCreatedQueues = maxApps;
-    maxAppsPerUserForAutoCreatedQueues =
-        (int) (maxApps * (userLimit / 100.0f) * userLimitFactor);
-
-  }
-
-  /**
-   * Number of maximum applications for each of the auto created leaf queues.
-   *
-   * @return maxAppsForAutoCreatedQueues
-   */
-  public int getMaxApplicationsForAutoCreatedQueues() {
-    return maxAppsForAutoCreatedQueues;
-  }
-
-  /**
-   * Number of maximum applications per user for each of the auto created
-   * leaf queues.
-   *
-   * @return maxAppsPerUserForAutoCreatedQueues
-   */
-  public int getMaxApplicationsPerUserForAutoCreatedQueues() {
-    return maxAppsPerUserForAutoCreatedQueues;
-  }
-
-  /**
-   * User limit value for each of the  auto created leaf queues.
-   *
-   * @return userLimit
-   */
-  public int getUserLimitForAutoCreatedQueues() {
-    return userLimit;
-  }
-
-  /**
-   * User limit factor value for each of the  auto created leaf queues.
-   *
-   * @return userLimitFactor
-   */
-  public float getUserLimitFactor() {
-    return userLimitFactor;
-  }
 
-  public int getMaxAppsForAutoCreatedQueues() {
-    return maxAppsForAutoCreatedQueues;
-  }
+    int userLimit = conf.getUserLimit(queuePath);
+    float userLimitFactor = conf.getUserLimitFactor(queuePath);
+    leafQueueTemplateBuilder.userLimit(userLimit)
+          .userLimitFactor(userLimitFactor)
+          .maxApps(maxApps)
+          .maxAppsPerUser(
+              (int) (maxApps * (userLimit / 100.0f) * userLimitFactor));
 
-  public int getMaxAppsPerUserForAutoCreatedQueues() {
-    return maxAppsPerUserForAutoCreatedQueues;
-  }
-
-  public int getUserLimit() {
-    return userLimit;
+    return leafQueueTemplateBuilder;
   }
 
   /**
@@ -229,4 +166,111 @@ public abstract class AbstractManagedParentQueue extends ParentQueue {
     }
     return childQueue;
   }
+
+  protected float sumOfChildCapacities() {
+    try {
+      writeLock.lock();
+      float ret = 0;
+      for (CSQueue l : childQueues) {
+        ret += l.getCapacity();
+      }
+      return ret;
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  protected float sumOfChildAbsCapacities() {
+    try {
+      writeLock.lock();
+      float ret = 0;
+      for (CSQueue l : childQueues) {
+        ret += l.getAbsoluteCapacity();
+      }
+      return ret;
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  public static class AutoCreatedLeafQueueTemplate {
+
+    private QueueCapacities queueCapacities;
+
+    private int maxApps;
+    private int maxAppsPerUser;
+    private int userLimit;
+    private float userLimitFactor;
+
+    AutoCreatedLeafQueueTemplate(Builder builder) {
+      this.maxApps = builder.maxApps;
+      this.maxAppsPerUser = builder.maxAppsPerUser;
+      this.userLimit = builder.userLimit;
+      this.userLimitFactor = builder.userLimitFactor;
+      this.queueCapacities = builder.queueCapacities;
+    }
+
+    public static class Builder {
+      private int maxApps;
+      private int maxAppsPerUser;
+
+      private int userLimit;
+      private float userLimitFactor;
+
+      private QueueCapacities queueCapacities;
+
+      Builder maxApps(int maxApplications) {
+        this.maxApps =  maxApplications;
+        return this;
+      }
+
+      Builder maxAppsPerUser(int maxApplicationsPerUser) {
+        this.maxAppsPerUser = maxApplicationsPerUser;
+        return this;
+      }
+
+      Builder userLimit(int usrLimit) {
+        this.userLimit = usrLimit;
+        return this;
+      }
+
+      Builder userLimitFactor(float ulf) {
+        this.userLimitFactor = ulf;
+        return this;
+      }
+
+      Builder capacities(QueueCapacities capacities) {
+        this.queueCapacities = capacities;
+        return this;
+      }
+
+      AutoCreatedLeafQueueTemplate build() {
+        return new AutoCreatedLeafQueueTemplate(this);
+      }
+    }
+
+    public int getUserLimit() {
+      return userLimit;
+    }
+
+    public float getUserLimitFactor() {
+      return userLimitFactor;
+    }
+
+    public QueueCapacities getQueueCapacities() {
+      return queueCapacities;
+    }
+
+    public int getMaxApps() {
+      return maxApps;
+    }
+
+    public int getMaxAppsPerUser() {
+      return maxAppsPerUser;
+    }
+  }
+
+  public AutoCreatedLeafQueueTemplate getLeafQueueTemplate() {
+    return leafQueueTemplate;
+  }
 }

+ 16 - 11
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AutoCreatedLeafQueue.java

@@ -21,6 +21,8 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerDynamicEditException;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.QueueEntitlement;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity
+    .AbstractManagedParentQueue.AutoCreatedLeafQueueTemplate;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -42,17 +44,18 @@ public class AutoCreatedLeafQueue extends LeafQueue {
       AbstractManagedParentQueue parent) throws IOException {
     super(cs, queueName, parent, null);
 
-    updateApplicationAndUserLimits(parent.getUserLimitForAutoCreatedQueues(),
-        parent.getUserLimitFactor(),
-        parent.getMaxApplicationsForAutoCreatedQueues(),
-        parent.getMaxApplicationsPerUserForAutoCreatedQueues());
-
+    AutoCreatedLeafQueueTemplate leafQueueTemplate =
+        parent.getLeafQueueTemplate();
+    updateApplicationAndUserLimits(leafQueueTemplate.getUserLimit(),
+        leafQueueTemplate.getUserLimitFactor(),
+        leafQueueTemplate.getMaxApps(),
+        leafQueueTemplate.getMaxAppsPerUser());
     this.parent = parent;
   }
 
   @Override
-  public void reinitialize(CSQueue newlyParsedQueue,
-      Resource clusterResource) throws IOException {
+  public void reinitialize(CSQueue newlyParsedQueue, Resource clusterResource)
+      throws IOException {
     try {
       writeLock.lock();
 
@@ -62,10 +65,12 @@ public class AutoCreatedLeafQueue extends LeafQueue {
       CSQueueUtils.updateQueueStatistics(resourceCalculator, clusterResource,
           this, labelManager, null);
 
-      updateApplicationAndUserLimits(parent.getUserLimitForAutoCreatedQueues(),
-          parent.getUserLimitFactor(),
-          parent.getMaxApplicationsForAutoCreatedQueues(),
-          parent.getMaxApplicationsPerUserForAutoCreatedQueues());
+      AutoCreatedLeafQueueTemplate leafQueueTemplate =
+          parent.getLeafQueueTemplate();
+      updateApplicationAndUserLimits(leafQueueTemplate.getUserLimit(),
+          leafQueueTemplate.getUserLimitFactor(),
+          leafQueueTemplate.getMaxApps(),
+          leafQueueTemplate.getMaxAppsPerUser());
 
     } finally {
       writeLock.unlock();

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

@@ -41,7 +41,6 @@ import org.apache.hadoop.classification.InterfaceStability.Evolving;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.AccessControlException;
-import org.apache.hadoop.security.Groups;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
@@ -64,10 +63,10 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.SchedulerResourceTypes;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.placement.ApplicationPlacementContext;
 import org.apache.hadoop.yarn.server.resourcemanager.placement.PlacementFactory;
 import org.apache.hadoop.yarn.server.resourcemanager.placement.PlacementRule;
 import org.apache.hadoop.yarn.server.resourcemanager.placement.UserGroupMappingPlacementRule;
-import org.apache.hadoop.yarn.server.resourcemanager.placement.UserGroupMappingPlacementRule.QueueMapping;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.ApplicationStateData;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationConstants;
@@ -146,6 +145,8 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.util.concurrent.SettableFuture;
 
+import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.QUEUE_MAPPING;
+
 @LimitedPrivate("yarn")
 @Evolving
 @SuppressWarnings("unchecked")
@@ -560,44 +561,17 @@ public class CapacityScheduler extends
   }
 
   @VisibleForTesting
-  public UserGroupMappingPlacementRule
-      getUserGroupMappingPlacementRule() throws IOException {
+  public PlacementRule getUserGroupMappingPlacementRule() throws IOException {
     try {
       readLock.lock();
-      boolean overrideWithQueueMappings = conf.getOverrideWithQueueMappings();
-      LOG.info(
-          "Initialized queue mappings, override: " + overrideWithQueueMappings);
-
-      // Get new user/group mappings
-      List<QueueMapping> newMappings = conf.getQueueMappings();
-      // check if mappings refer to valid queues
-      for (QueueMapping mapping : newMappings) {
-        String mappingQueue = mapping.getQueue();
-        if (!mappingQueue.equals(
-            UserGroupMappingPlacementRule.CURRENT_USER_MAPPING) && !mappingQueue
-            .equals(UserGroupMappingPlacementRule.PRIMARY_GROUP_MAPPING)) {
-          CSQueue queue = getQueue(mappingQueue);
-          if (queue == null || !(queue instanceof LeafQueue)) {
-            throw new IOException(
-                "mapping contains invalid or non-leaf queue " + mappingQueue);
-          }
-        }
-      }
-
-      // initialize groups if mappings are present
-      if (newMappings.size() > 0) {
-        Groups groups = new Groups(conf);
-        return new UserGroupMappingPlacementRule(overrideWithQueueMappings,
-            newMappings, groups);
-      }
-
-      return null;
+      return UserGroupMappingPlacementRule.get(this);
     } finally {
       readLock.unlock();
     }
   }
 
-  private void updatePlacementRules() throws IOException {
+  @VisibleForTesting
+  void updatePlacementRules() throws IOException {
     // Initialize placement rules
     Collection<String> placementRuleStrs = conf.getStringCollection(
         YarnConfiguration.QUEUE_PLACEMENT_RULES);
@@ -731,37 +705,92 @@ public class CapacityScheduler extends
     }
   }
 
-  private void addApplication(ApplicationId applicationId,
-      String queueName, String user, Priority priority) {
+  private void addApplication(ApplicationId applicationId, String queueName,
+      String user, Priority priority,
+      ApplicationPlacementContext placementContext) {
     try {
       writeLock.lock();
       if (isSystemAppsLimitReached()) {
         String message = "Maximum system application limit reached,"
             + "cannot accept submission of application: " + applicationId;
-        this.rmContext.getDispatcher().getEventHandler().handle(new RMAppEvent(
-            applicationId, RMAppEventType.APP_REJECTED, message));
+        this.rmContext.getDispatcher().getEventHandler().handle(
+            new RMAppEvent(applicationId, RMAppEventType.APP_REJECTED,
+                message));
         return;
       }
       // Sanity checks.
       CSQueue queue = getQueue(queueName);
+
+      if (queue == null && placementContext != null) {
+        //Could be a potential auto-created leaf queue
+        try {
+          queue = autoCreateLeafQueue(placementContext);
+        } catch (YarnException | IOException e) {
+          LOG.error("Could not auto-create leaf queue due to : ", e);
+          final String message =
+              "Application " + applicationId + " submission by user : " + user
+                  + " to  queue : " + queueName + " failed : " + e.getMessage();
+          this.rmContext.getDispatcher().getEventHandler().handle(
+              new RMAppEvent(applicationId, RMAppEventType.APP_REJECTED,
+                  message));
+        }
+      }
+
       if (queue == null) {
-        String message =
+        final String message =
             "Application " + applicationId + " submitted by user " + user
                 + " to unknown queue: " + queueName;
+
         this.rmContext.getDispatcher().getEventHandler().handle(
             new RMAppEvent(applicationId, RMAppEventType.APP_REJECTED,
                 message));
         return;
       }
+
       if (!(queue instanceof LeafQueue)) {
         String message =
-            "Application " + applicationId + " submitted by user " + user
-                + " to non-leaf queue: " + queueName;
+            "Application " + applicationId + " submitted by user : " + user
+                + " to non-leaf queue : " + queueName;
         this.rmContext.getDispatcher().getEventHandler().handle(
             new RMAppEvent(applicationId, RMAppEventType.APP_REJECTED,
                 message));
         return;
+      } else if (queue instanceof AutoCreatedLeafQueue && queue
+          .getParent() instanceof ManagedParentQueue) {
+
+        //If queue already exists and auto-queue creation was not required,
+        //placement context should not be null
+        if (placementContext == null) {
+          String message =
+              "Application " + applicationId + " submission by user : " + user
+                  + " to specified queue : " + queueName + "  is prohibited. "
+                  + "Verify automatic queue mapping for user exists in " +
+                  QUEUE_MAPPING;
+          this.rmContext.getDispatcher().getEventHandler().handle(
+              new RMAppEvent(applicationId, RMAppEventType.APP_REJECTED,
+                  message));
+          return;
+          // For a queue which exists already and
+          // not auto-created above, then its parent queue should match
+          // the parent queue specified in queue mapping
+        } else if (!queue.getParent().getQueueName().equals(
+            placementContext.getParentQueue())) {
+          String message =
+              "Auto created Leaf queue " + placementContext.getQueue() + " "
+                  + "already exists under queue : " + queue
+                  .getParent().getQueuePath()
+                  + ".But Queue mapping configuration " +
+                   CapacitySchedulerConfiguration.QUEUE_MAPPING + " has been "
+                  + "updated to a different parent queue : "
+                  + placementContext.getParentQueue()
+                  + " for the specified user : " + user;
+          this.rmContext.getDispatcher().getEventHandler().handle(
+              new RMAppEvent(applicationId, RMAppEventType.APP_REJECTED,
+                  message));
+          return;
+        }
       }
+
       // Submit to the queue
       try {
         queue.submitApplication(applicationId, user, queueName);
@@ -1483,7 +1512,8 @@ public class CapacityScheduler extends
       if (queueName != null) {
         if (!appAddedEvent.getIsAppRecovering()) {
           addApplication(appAddedEvent.getApplicationId(), queueName,
-              appAddedEvent.getUser(), appAddedEvent.getApplicatonPriority());
+              appAddedEvent.getUser(), appAddedEvent.getApplicatonPriority(),
+              appAddedEvent.getPlacementContext());
         } else {
           addApplicationOnRecovery(appAddedEvent.getApplicationId(), queueName,
               appAddedEvent.getUser(), appAddedEvent.getApplicatonPriority());
@@ -2001,7 +2031,8 @@ public class CapacityScheduler extends
     try {
       writeLock.lock();
       LeafQueue queue = this.queueManager.getAndCheckLeafQueue(inQueue);
-      ParentQueue parent = (ParentQueue) queue.getParent();
+      AbstractManagedParentQueue parent = (AbstractManagedParentQueue) queue
+          .getParent();
 
       if (!(queue instanceof AutoCreatedLeafQueue)) {
         throw new SchedulerDynamicEditException(
@@ -2010,7 +2041,8 @@ public class CapacityScheduler extends
       }
 
       if (parent == null
-          || !(AbstractManagedParentQueue.class.isAssignableFrom(parent.getClass()))) {
+          || !(AbstractManagedParentQueue.class.isAssignableFrom(
+              parent.getClass()))) {
         throw new SchedulerDynamicEditException(
             "The parent of AutoCreatedLeafQueue " + inQueue
                 + " must be a PlanQueue/ManagedParentQueue");
@@ -2655,4 +2687,43 @@ public class CapacityScheduler extends
     }
     return null;
   }
+
+  private LeafQueue autoCreateLeafQueue(
+      ApplicationPlacementContext placementContext)
+      throws IOException, YarnException {
+
+    AutoCreatedLeafQueue autoCreatedLeafQueue = null;
+
+    String leafQueueName = placementContext.getQueue();
+    String parentQueueName = placementContext.getParentQueue();
+
+    if (!StringUtils.isEmpty(parentQueueName)) {
+      CSQueue parentQueue = getQueue(parentQueueName);
+
+      if (parentQueue != null && conf.isAutoCreateChildQueueEnabled(
+          parentQueue.getQueuePath())) {
+
+        ManagedParentQueue autoCreateEnabledParentQueue =
+            (ManagedParentQueue) parentQueue;
+        autoCreatedLeafQueue = new AutoCreatedLeafQueue(this, leafQueueName,
+            autoCreateEnabledParentQueue);
+
+        addQueue(autoCreatedLeafQueue);
+
+        //TODO - Set entitlement through capacity management policy
+      } else{
+        throw new SchedulerDynamicEditException(
+            "Could not auto-create leaf queue for " + leafQueueName
+                + ". Queue mapping specifies an invalid parent queue "
+                + "which does not exist "
+                + parentQueueName);
+      }
+    } else{
+      throw new SchedulerDynamicEditException(
+          "Could not auto-create leaf queue for " + leafQueueName
+              + ". Queue mapping does not specify"
+              + " which parent queue it needs to be created under.");
+    }
+    return autoCreatedLeafQueue;
+  }
 }

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

@@ -907,6 +907,12 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
         DEFAULT_ENABLE_QUEUE_MAPPING_OVERRIDE);
   }
 
+  @Private
+  @VisibleForTesting
+  public void setOverrideWithQueueMappings(boolean overrideWithQueueMappings) {
+    setBoolean(ENABLE_QUEUE_MAPPING_OVERRIDE, overrideWithQueueMappings);
+  }
+
   /**
    * Returns a collection of strings, trimming leading and trailing whitespeace
    * on each value
@@ -981,6 +987,31 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
     return mappings;
   }
 
+  @Private
+  @VisibleForTesting
+  public void setQueuePlacementRules(Collection<String> queuePlacementRules) {
+    if (queuePlacementRules == null) {
+      return;
+    }
+    String str = StringUtils.join(",", queuePlacementRules);
+    setStrings(YarnConfiguration.QUEUE_PLACEMENT_RULES, str);
+  }
+
+  @Private
+  @VisibleForTesting
+  public void setQueueMappings(List<QueueMapping> queueMappings) {
+    if (queueMappings == null) {
+      return;
+    }
+
+    List<String> queueMappingStrs = new ArrayList<>();
+    for (QueueMapping mapping : queueMappings) {
+      queueMappingStrs.add(mapping.toString());
+    }
+
+    setStrings(QUEUE_MAPPING, StringUtils.join(",", queueMappingStrs));
+  }
+
   public boolean isReservable(String queue) {
     boolean isReservable =
         getBoolean(getQueuePrefix(queue) + IS_RESERVABLE, false);
@@ -1523,4 +1554,126 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
   public void setDefaultLifetimePerQueue(String queue, long defaultLifetime) {
     setLong(getQueuePrefix(queue) + DEFAULT_LIFETIME_SUFFIX, defaultLifetime);
   }
+
+  @Private
+  public static final boolean DEFAULT_AUTO_CREATE_CHILD_QUEUE_ENABLED = false;
+
+  @Private
+  public static final String AUTO_CREATE_CHILD_QUEUE_ENABLED =
+      "auto-create-child-queue.enabled";
+
+  @Private
+  public static final String AUTO_CREATED_LEAF_QUEUE_TEMPLATE_PREFIX =
+      "leaf-queue-template";
+
+  @Private
+  public static final String AUTO_CREATE_QUEUE_MAX_QUEUES =
+      "auto-create-child-queue.max-queues";
+
+  @Private
+  public static final int DEFAULT_AUTO_CREATE_QUEUE_MAX_QUEUES = 1000;
+
+  /**
+   * If true, this queue will be created as a Parent Queue which Auto Created
+   * leaf child queues
+   *
+   * @param queuePath The queues path
+   * @return true if auto create is enabled for child queues else false. Default
+   * is false
+   */
+  @Private
+  public boolean isAutoCreateChildQueueEnabled(String queuePath) {
+    boolean isAutoCreateEnabled = getBoolean(
+        getQueuePrefix(queuePath) + AUTO_CREATE_CHILD_QUEUE_ENABLED,
+        DEFAULT_AUTO_CREATE_CHILD_QUEUE_ENABLED);
+    return isAutoCreateEnabled;
+  }
+
+  @Private
+  @VisibleForTesting
+  public void setAutoCreateChildQueueEnabled(String queuePath,
+      boolean autoCreationEnabled) {
+    setBoolean(getQueuePrefix(queuePath) +
+            AUTO_CREATE_CHILD_QUEUE_ENABLED,
+        autoCreationEnabled);
+  }
+
+  /**
+   * Get the auto created leaf queue's template configuration prefix
+   * Leaf queue's template capacities are configured at the parent queue
+   *
+   * @param queuePath parent queue's path
+   * @return Config prefix for leaf queue template configurations
+   */
+  @Private
+  public String getAutoCreatedQueueTemplateConfPrefix(String queuePath) {
+    return queuePath + DOT + AUTO_CREATED_LEAF_QUEUE_TEMPLATE_PREFIX;
+  }
+
+  @Private
+  public static final String FAIL_AUTO_CREATION_ON_EXCEEDING_CAPACITY =
+      "auto-create-child-queue.fail-on-exceeding-parent-capacity";
+
+  @Private
+  public static final boolean DEFAULT_FAIL_AUTO_CREATION_ON_EXCEEDING_CAPACITY =
+      false;
+
+  /**
+   * Fail further auto leaf queue creation when parent's guaranteed capacity is
+   * exceeded.
+   *
+   * @param queuePath the parent queue's path
+   * @return true if configured to fail else false
+   */
+  @Private
+  public boolean getShouldFailAutoQueueCreationWhenGuaranteedCapacityExceeded(
+      String queuePath) {
+    boolean shouldFailAutoQueueCreationOnExceedingGuaranteedCapacity =
+        getBoolean(getQueuePrefix(queuePath)
+                + FAIL_AUTO_CREATION_ON_EXCEEDING_CAPACITY,
+            DEFAULT_FAIL_AUTO_CREATION_ON_EXCEEDING_CAPACITY);
+    return shouldFailAutoQueueCreationOnExceedingGuaranteedCapacity;
+  }
+
+  @VisibleForTesting
+  @Private
+  public void setShouldFailAutoQueueCreationWhenGuaranteedCapacityExceeded(
+      String queuePath, boolean autoCreationEnabled) {
+    setBoolean(
+        getQueuePrefix(queuePath) +
+            FAIL_AUTO_CREATION_ON_EXCEEDING_CAPACITY,
+        autoCreationEnabled);
+  }
+
+  /**
+   * Get the max number of leaf queues that are allowed to be created under
+   * a parent queue
+   *
+   * @param queuePath the paret queue's path
+   * @return the max number of leaf queues allowed to be auto created
+   */
+  @Private
+  public int getAutoCreatedQueuesMaxChildQueuesLimit(String queuePath) {
+    return getInt(getQueuePrefix(queuePath) +
+            AUTO_CREATE_QUEUE_MAX_QUEUES,
+        DEFAULT_AUTO_CREATE_QUEUE_MAX_QUEUES);
+  }
+
+  @Private
+  @VisibleForTesting
+  public void setAutoCreatedLeafQueueTemplateCapacity(String queuePath,
+      float val) {
+    String leafQueueConfPrefix = getAutoCreatedQueueTemplateConfPrefix(
+        queuePath);
+    setCapacity(leafQueueConfPrefix, val);
+  }
+
+  @Private
+  @VisibleForTesting
+  public void setAutoCreatedLeafQueueTemplateMaxCapacity(String queuePath,
+      float val) {
+    String leafQueueConfPrefix = getAutoCreatedQueueTemplateConfPrefix(
+        queuePath);
+    setMaximumCapacity(leafQueueConfPrefix, val);
+  }
 }

+ 69 - 34
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerQueueManager.java

@@ -154,7 +154,7 @@ public class CapacitySchedulerQueueManager implements SchedulerQueueManager<
    * @throws IOException if fails to initialize queues
    */
   public void initializeQueues(CapacitySchedulerConfiguration conf)
-      throws IOException {
+    throws IOException {
     root = parseQueue(this.csContext, conf, null,
         CapacitySchedulerConfiguration.ROOT, queues, queues, NOOP);
     setQueueAcls(authorizer, appPriorityACLManager, queues);
@@ -176,7 +176,7 @@ public class CapacitySchedulerQueueManager implements SchedulerQueueManager<
     if (!csContext.isConfigurationMutable() ||
         csContext.getRMContext().getHAServiceState()
             != HAServiceProtocol.HAServiceState.STANDBY) {
-      // Ensure queue hiearchy in the new XML file is proper.
+      // Ensure queue hierarchy in the new XML file is proper.
       validateQueueHierarchy(queues, newQueues);
     }
 
@@ -216,11 +216,13 @@ public class CapacitySchedulerQueueManager implements SchedulerQueueManager<
       Map<String, CSQueue> oldQueues,
       QueueHook hook) throws IOException {
     CSQueue queue;
-    String fullQueueName =
-        (parent == null) ? queueName
-            : (parent.getQueuePath() + "." + queueName);
+    String fullQueueName = (parent == null) ?
+        queueName :
+        (parent.getQueuePath() + "." + queueName);
     String[] childQueueNames = conf.getQueues(fullQueueName);
     boolean isReservableQueue = conf.isReservable(fullQueueName);
+    boolean isAutoCreateEnabled = conf.isAutoCreateChildQueueEnabled(
+        fullQueueName);
     if (childQueueNames == null || childQueueNames.length == 0) {
       if (null == parent) {
         throw new IllegalStateException(
@@ -229,9 +231,8 @@ public class CapacitySchedulerQueueManager implements SchedulerQueueManager<
       // Check if the queue will be dynamically managed by the Reservation
       // system
       if (isReservableQueue) {
-        queue =
-            new PlanQueue(csContext, queueName, parent,
-                oldQueues.get(queueName));
+        queue = new PlanQueue(csContext, queueName, parent,
+            oldQueues.get(queueName));
 
         //initializing the "internal" default queue, for SLS compatibility
         String defReservationId =
@@ -249,38 +250,46 @@ public class CapacitySchedulerQueueManager implements SchedulerQueueManager<
         ((PlanQueue) queue).setChildQueues(childQueues);
         queues.put(defReservationId, resQueue);
 
-      } else {
-        queue =
-            new LeafQueue(csContext, queueName, parent,
-                oldQueues.get(queueName));
+      } else if (isAutoCreateEnabled) {
+        queue = new ManagedParentQueue(csContext, queueName, parent,
+            oldQueues.get(queueName));
 
+      } else{
+        queue = new LeafQueue(csContext, queueName, parent,
+            oldQueues.get(queueName));
         // Used only for unit tests
         queue = hook.hook(queue);
       }
-    } else {
+    } else{
       if (isReservableQueue) {
         throw new IllegalStateException(
             "Only Leaf Queues can be reservable for " + queueName);
       }
-      ParentQueue parentQueue =
-          new ParentQueue(csContext, queueName, parent,
-              oldQueues.get(queueName));
+
+      ParentQueue parentQueue;
+      if (isAutoCreateEnabled) {
+        parentQueue = new ManagedParentQueue(csContext, queueName, parent,
+            oldQueues.get(queueName));
+      } else{
+        parentQueue = new ParentQueue(csContext, queueName, parent,
+            oldQueues.get(queueName));
+      }
 
       // Used only for unit tests
       queue = hook.hook(parentQueue);
 
       List<CSQueue> childQueues = new ArrayList<>();
       for (String childQueueName : childQueueNames) {
-        CSQueue childQueue =
-            parseQueue(csContext, conf, queue, childQueueName,
-              queues, oldQueues, hook);
+        CSQueue childQueue = parseQueue(csContext, conf, queue, childQueueName,
+            queues, oldQueues, hook);
         childQueues.add(childQueue);
       }
       parentQueue.setChildQueues(childQueues);
+
     }
 
-    if (queue instanceof LeafQueue && queues.containsKey(queueName)
-        && queues.get(queueName) instanceof LeafQueue) {
+    if (queue instanceof LeafQueue && queues.containsKey(queueName) && queues
+        .get(queueName) instanceof LeafQueue) {
       throw new IOException("Two leaf queues were named " + queueName
           + ". Leaf queue names must be distinct");
     }
@@ -312,27 +321,46 @@ public class CapacitySchedulerQueueManager implements SchedulerQueueManager<
           if (oldQueue.getState() == QueueState.STOPPED) {
             LOG.info("Deleting Queue " + queueName + ", as it is not"
                 + " present in the modified capacity configuration xml");
-          } else {
+          } else{
             throw new IOException(oldQueue.getQueuePath() + " is deleted from"
                 + " the new capacity scheduler configuration, but the"
-                + " queue is not yet in stopped state. "
-                + "Current State : " + oldQueue.getState());
+                + " queue is not yet in stopped state. " + "Current State : "
+                + oldQueue.getState());
           }
         } else if (!oldQueue.getQueuePath().equals(newQueue.getQueuePath())) {
           //Queue's cannot be moved from one hierarchy to other
-          throw new IOException(queueName + " is moved from:"
-              + oldQueue.getQueuePath() + " to:" + newQueue.getQueuePath()
-              + " after refresh, which is not allowed.");
-        } else  if (oldQueue instanceof LeafQueue
+          throw new IOException(
+              queueName + " is moved from:" + oldQueue.getQueuePath() + " to:"
+                  + newQueue.getQueuePath()
+                  + " after refresh, which is not allowed.");
+        } else if (oldQueue instanceof ParentQueue
+            && !(oldQueue instanceof ManagedParentQueue)
+            && newQueue instanceof ManagedParentQueue) {
+          throw new IOException(
+              "Can not convert parent queue: " + oldQueue.getQueuePath()
+                  + " to auto create enabled parent queue since "
+                  + "it could have other pre-configured queues which is not "
+                  + "supported");
+        } else if (oldQueue instanceof ManagedParentQueue
+            && !(newQueue instanceof ManagedParentQueue)) {
+          throw new IOException(
+              "Cannot convert auto create enabled parent queue: " + oldQueue
+                  .getQueuePath() + " to leaf queue. Please check "
+                  + " parent queue's configuration "
+                  + CapacitySchedulerConfiguration
+                  .AUTO_CREATE_CHILD_QUEUE_ENABLED
+                  + " is set to true");
+        } else if (oldQueue instanceof LeafQueue
             && newQueue instanceof ParentQueue) {
           if (oldQueue.getState() == QueueState.STOPPED) {
             LOG.info("Converting the leaf queue: " + oldQueue.getQueuePath()
                 + " to parent queue.");
-          } else {
-            throw new IOException("Can not convert the leaf queue: "
-                + oldQueue.getQueuePath() + " to parent queue since "
-                + "it is not yet in stopped state. Current State : "
-                + oldQueue.getState());
+          } else{
+            throw new IOException(
+                "Can not convert the leaf queue: " + oldQueue.getQueuePath()
+                    + " to parent queue since "
+                    + "it is not yet in stopped state. Current State : "
+                    + oldQueue.getState());
           }
         } else if (oldQueue instanceof ParentQueue
             && newQueue instanceof LeafQueue) {
@@ -352,6 +380,7 @@ public class CapacitySchedulerQueueManager implements SchedulerQueueManager<
    */
   private void updateQueues(Map<String, CSQueue> existingQueues,
       Map<String, CSQueue> newQueues) {
+    CapacitySchedulerConfiguration conf = csContext.getConfiguration();
     for (Map.Entry<String, CSQueue> e : newQueues.entrySet()) {
       String queueName = e.getKey();
       CSQueue queue = e.getValue();
@@ -363,7 +392,13 @@ public class CapacitySchedulerQueueManager implements SchedulerQueueManager<
         .iterator(); itr.hasNext();) {
       Map.Entry<String, CSQueue> e = itr.next();
       String queueName = e.getKey();
-      if (!newQueues.containsKey(queueName)) {
+      CSQueue existingQueue = e.getValue();
+
+      //TODO - Handle case when auto create is disabled on parent queues
+      if (!newQueues.containsKey(queueName) && !(
+          existingQueue instanceof AutoCreatedLeafQueue && conf
+              .isAutoCreateChildQueueEnabled(
+                  existingQueue.getParent().getQueuePath()))) {
         itr.remove();
       }
     }

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

@@ -0,0 +1,158 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
+
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler
+    .SchedulerDynamicEditException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+/**
+ * Auto Creation enabled Parent queue. This queue initially does not have any
+ * children to start with and all child
+ * leaf queues will be auto created. Currently this does not allow other
+ * pre-configured leaf or parent queues to
+ * co-exist along with auto-created leaf queues. The auto creation is limited
+ * to leaf queues currently.
+ */
+public class ManagedParentQueue extends AbstractManagedParentQueue {
+
+  private boolean shouldFailAutoCreationWhenGuaranteedCapacityExceeded = false;
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+      ManagedParentQueue.class);
+
+  public ManagedParentQueue(final CapacitySchedulerContext cs,
+      final String queueName, final CSQueue parent, final CSQueue old)
+      throws IOException {
+    super(cs, queueName, parent, old);
+    String leafQueueTemplateConfPrefix = getLeafQueueConfigPrefix(
+        csContext.getConfiguration());
+    this.leafQueueTemplate = initializeLeafQueueConfigs(
+        leafQueueTemplateConfPrefix).build();
+
+    StringBuffer queueInfo = new StringBuffer();
+    queueInfo.append("Created Managed Parent Queue: ").append(queueName).append(
+        "]\nwith capacity: [").append(super.getCapacity()).append(
+        "]\nwith max capacity: [").append(super.getMaximumCapacity()).append(
+        "\nwith max apps: [").append(leafQueueTemplate.getMaxApps()).append(
+        "]\nwith max apps per user: [").append(
+        leafQueueTemplate.getMaxAppsPerUser()).append("]\nwith user limit: [")
+        .append(leafQueueTemplate.getUserLimit()).append(
+        "]\nwith user limit factor: [").append(
+        leafQueueTemplate.getUserLimitFactor()).append("].");
+    LOG.info(queueInfo.toString());
+  }
+
+  @Override
+  public void reinitialize(CSQueue newlyParsedQueue, Resource clusterResource)
+      throws IOException {
+    validate(newlyParsedQueue);
+    super.reinitialize(newlyParsedQueue, clusterResource);
+    String leafQueueTemplateConfPrefix = getLeafQueueConfigPrefix(
+        csContext.getConfiguration());
+    this.leafQueueTemplate = initializeLeafQueueConfigs(
+        leafQueueTemplateConfPrefix).build();
+  }
+
+  @Override
+  protected AutoCreatedLeafQueueTemplate.Builder initializeLeafQueueConfigs(
+      String queuePath) {
+
+    AutoCreatedLeafQueueTemplate.Builder leafQueueTemplate =
+        super.initializeLeafQueueConfigs(queuePath);
+
+    CapacitySchedulerConfiguration conf = csContext.getConfiguration();
+    String leafQueueTemplateConfPrefix = getLeafQueueConfigPrefix(conf);
+    QueueCapacities queueCapacities = new QueueCapacities(false);
+    CSQueueUtils.loadUpdateAndCheckCapacities(leafQueueTemplateConfPrefix,
+        csContext.getConfiguration(), queueCapacities, getQueueCapacities());
+    leafQueueTemplate.capacities(queueCapacities);
+
+    shouldFailAutoCreationWhenGuaranteedCapacityExceeded =
+        conf.getShouldFailAutoQueueCreationWhenGuaranteedCapacityExceeded(
+            getQueuePath());
+
+    return leafQueueTemplate;
+  }
+
+  protected void validate(final CSQueue newlyParsedQueue) throws IOException {
+    // Sanity check
+    if (!(newlyParsedQueue instanceof ManagedParentQueue) || !newlyParsedQueue
+        .getQueuePath().equals(getQueuePath())) {
+      throw new IOException(
+          "Trying to reinitialize " + getQueuePath() + " from "
+              + newlyParsedQueue.getQueuePath());
+    }
+  }
+
+  @Override
+  public void addChildQueue(CSQueue childQueue)
+      throws SchedulerDynamicEditException {
+    try {
+      writeLock.lock();
+
+      if (childQueue == null || !(childQueue instanceof AutoCreatedLeafQueue)) {
+        throw new SchedulerDynamicEditException(
+            "Expected child queue to be an instance of AutoCreatedLeafQueue");
+      }
+
+      CapacitySchedulerConfiguration conf = csContext.getConfiguration();
+      ManagedParentQueue parentQueue =
+          (ManagedParentQueue) childQueue.getParent();
+
+      String leafQueueName = childQueue.getQueueName();
+      int maxQueues = conf.getAutoCreatedQueuesMaxChildQueuesLimit(
+          parentQueue.getQueuePath());
+
+      if (parentQueue.getChildQueues().size() >= maxQueues) {
+        throw new SchedulerDynamicEditException(
+            "Cannot auto create leaf queue " + leafQueueName + ".Max Child "
+                + "Queue limit exceeded which is configured as : " + maxQueues
+                + " and number of child queues is : " + parentQueue
+                .getChildQueues().size());
+      }
+
+      if (shouldFailAutoCreationWhenGuaranteedCapacityExceeded) {
+        if (getLeafQueueTemplate().getQueueCapacities().getAbsoluteCapacity()
+            + parentQueue.sumOfChildAbsCapacities() > parentQueue
+            .getAbsoluteCapacity()) {
+          throw new SchedulerDynamicEditException(
+              "Cannot auto create leaf queue " + leafQueueName + ". Child "
+                  + "queues capacities have reached parent queue : "
+                  + parentQueue.getQueuePath() + " guaranteed capacity");
+        }
+      }
+
+      AutoCreatedLeafQueue leafQueue = (AutoCreatedLeafQueue) childQueue;
+      super.addChildQueue(leafQueue);
+      //TODO - refresh policy queue after capacity management is added
+
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  private String getLeafQueueConfigPrefix(CapacitySchedulerConfiguration conf) {
+    return conf.getAutoCreatedQueueTemplateConfPrefix(getQueuePath());
+  }
+
+}

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

@@ -1081,17 +1081,4 @@ public class ParentQueue extends AbstractCSQueue {
   public QueueOrderingPolicy getQueueOrderingPolicy() {
     return queueOrderingPolicy;
   }
-
-  protected float sumOfChildCapacities() {
-    try {
-      writeLock.lock();
-      float ret = 0;
-      for (CSQueue l : childQueues) {
-        ret += l.getCapacity();
-      }
-      return ret;
-    } finally {
-      writeLock.unlock();
-    }
-  }
 }

+ 21 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/PlanQueue.java

@@ -40,6 +40,19 @@ public class PlanQueue extends AbstractManagedParentQueue {
   public PlanQueue(CapacitySchedulerContext cs, String queueName,
       CSQueue parent, CSQueue old) throws IOException {
     super(cs, queueName, parent, old);
+    this.leafQueueTemplate = initializeLeafQueueConfigs(getQueuePath()).build();
+
+    StringBuffer queueInfo = new StringBuffer();
+    queueInfo.append("Created Plan Queue: ").append(queueName).append(
+        "]\nwith capacity: [").append(super.getCapacity()).append(
+        "]\nwith max capacity: [").append(super.getMaximumCapacity()).append(
+        "\nwith max apps: [").append(leafQueueTemplate.getMaxApps()).append(
+        "]\nwith max apps per user: [").append(
+        leafQueueTemplate.getMaxAppsPerUser()).append("]\nwith user limit: [")
+        .append(leafQueueTemplate.getUserLimit()).append(
+        "]\nwith user limit factor: [").append(
+        leafQueueTemplate.getUserLimitFactor()).append("].");
+    LOG.info(queueInfo.toString());
   }
 
   @Override
@@ -47,17 +60,21 @@ public class PlanQueue extends AbstractManagedParentQueue {
       throws IOException {
     validate(newlyParsedQueue);
     super.reinitialize(newlyParsedQueue, clusterResource);
+    this.leafQueueTemplate = initializeLeafQueueConfigs(getQueuePath()).build();
   }
 
   @Override
-  protected void initializeLeafQueueConfigs() {
-    String queuePath = super.getQueuePath();
+  protected AutoCreatedLeafQueueTemplate.Builder initializeLeafQueueConfigs
+      (String queuePath) {
+    AutoCreatedLeafQueueTemplate.Builder leafQueueTemplate = super
+        .initializeLeafQueueConfigs
+        (queuePath);
     showReservationsAsQueues = csContext.getConfiguration()
         .getShowReservationAsQueues(queuePath);
-    super.initializeLeafQueueConfigs();
+    return leafQueueTemplate;
   }
 
-  private void validate(final CSQueue newlyParsedQueue) throws IOException {
+  protected void validate(final CSQueue newlyParsedQueue) throws IOException {
     // Sanity check
     if (!(newlyParsedQueue instanceof PlanQueue) || !newlyParsedQueue
         .getQueuePath().equals(getQueuePath())) {

Algunos archivos no se mostraron porque demasiados archivos cambiaron en este cambio