Browse Source

Merge branch 'trunk' into HADOOP-13345

Steve Loughran 8 years ago
parent
commit
d0bca3557f
100 changed files with 3122 additions and 756 deletions
  1. 9 6
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
  2. 3 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/LoadBalancingKMSClientProvider.java
  3. 21 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
  4. 2 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java
  5. 2 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ftp/FTPFileSystem.java
  6. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/SequenceFile.java
  7. 3 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/CompositeGroupsMapping.java
  8. 294 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/curator/ZKCuratorManager.java
  9. 27 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/curator/package-info.java
  10. 94 1
      hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
  11. 6 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java
  12. 44 4
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java
  13. 3 3
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalFileSystem.java
  14. 95 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/curator/TestZKCuratorManager.java
  15. 7 4
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
  16. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  17. 0 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
  18. 12 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
  19. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
  20. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsPermissionsGuide.md
  21. 2 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/TestAclCLI.java
  22. 63 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java
  23. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSAclBaseTest.java
  24. 8 6
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageWithAcl.java
  25. 34 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java
  26. 113 26
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestJobImpl.java
  27. 5 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
  28. 8 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
  29. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/io/FileBench.java
  30. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/MiniMRClientClusterFactory.java
  31. 3 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestCombineFileInputFormat.java
  32. 3 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestCombineSequenceFileInputFormat.java
  33. 3 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestCombineTextInputFormat.java
  34. 3 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestConcatenatedCompressedInput.java
  35. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMapRed.java
  36. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMiniMRChildTask.java
  37. 4 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTextInputFormat.java
  38. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/join/TestWrappedRecordReaderClassloader.java
  39. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/join/TestWrappedRRClassloader.java
  40. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/util/MRAsyncDiskService.java
  41. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobs.java
  42. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobsWithHistoryService.java
  43. 4 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/FadvisedFileRegion.java
  44. 1 1
      hadoop-tools/hadoop-archives/src/main/java/org/apache/hadoop/tools/HadoopArchives.java
  45. 1 1
      hadoop-tools/hadoop-azure-datalake/pom.xml
  46. 8 0
      hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlConfKeys.java
  47. 21 0
      hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlFileSystem.java
  48. 2 0
      hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/TokenProviderType.java
  49. 90 8
      hadoop-tools/hadoop-azure-datalake/src/site/markdown/index.md
  50. 40 0
      hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestAzureADTokenProvider.java
  51. 0 17
      hadoop-tools/hadoop-azure/src/main/resources/META-INF/services/org.apache.hadoop.fs.FileSystem
  52. 1 1
      hadoop-tools/hadoop-gridmix/src/main/java/org/apache/hadoop/mapred/gridmix/Gridmix.java
  53. 7 1
      hadoop-tools/hadoop-gridmix/src/main/java/org/apache/hadoop/mapred/gridmix/PseudoLocalFs.java
  54. 2 2
      hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestFilePool.java
  55. 4 4
      hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestFileQueue.java
  56. 1 1
      hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestPseudoLocalFs.java
  57. 2 2
      hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestUserResolve.java
  58. 1 1
      hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/util/SwiftTestUtils.java
  59. 1 1
      hadoop-tools/hadoop-openstack/src/test/java/org/apache/hadoop/fs/swift/SwiftFileSystemBaseTest.java
  60. 2 2
      hadoop-tools/hadoop-openstack/src/test/java/org/apache/hadoop/fs/swift/TestSwiftFileSystemPartitionedUploads.java
  61. 3 3
      hadoop-tools/hadoop-rumen/src/test/java/org/apache/hadoop/tools/rumen/TestHistograms.java
  62. 13 14
      hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/StreamJob.java
  63. 22 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
  64. 11 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
  65. 0 53
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
  66. 38 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml
  67. 22 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/CMakeLists.txt
  68. 476 196
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/configuration.c
  69. 138 44
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/configuration.h
  70. 23 16
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c
  71. 1 51
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.h
  72. 1 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/get_executable.c
  73. 9 8
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/main.c
  74. 134 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/util.c
  75. 115 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/util.h
  76. 31 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/resources/test/test-configurations/configuration-1.cfg
  77. 28 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/resources/test/test-configurations/configuration-2.cfg
  78. 25 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/resources/test/test-configurations/old-config.cfg
  79. 9 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test-container-executor.c
  80. 432 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test_configuration.cc
  81. 29 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test_main.cc
  82. 138 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test_util.cc
  83. 5 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java
  84. 3 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ActiveStandbyElectorBasedElectorService.java
  85. 0 81
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMZKUtils.java
  86. 43 49
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
  87. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/SchedulingEditPolicy.java
  88. 1 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/SchedulingMonitor.java
  89. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.java
  90. 5 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/invariants/InvariantsChecker.java
  91. 3 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/invariants/MetricsInvariantChecker.java
  92. 17 21
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKRMStateStore.java
  93. 13 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationInputValidator.java
  94. 8 8
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
  95. 4 0
      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
  96. 20 19
      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
  97. 6 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationFileLoaderService.java
  98. 3 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/RMHATestBase.java
  99. 93 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestReservationInputValidator.java
  100. 113 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java

+ 9 - 6
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java

@@ -3146,7 +3146,8 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
       JsonGenerator dumpGenerator = dumpFactory.createGenerator(out);
       JsonGenerator dumpGenerator = dumpFactory.createGenerator(out);
       dumpGenerator.writeStartObject();
       dumpGenerator.writeStartObject();
       dumpGenerator.writeFieldName("property");
       dumpGenerator.writeFieldName("property");
-      appendJSONProperty(dumpGenerator, config, propertyName);
+      appendJSONProperty(dumpGenerator, config, propertyName,
+          new ConfigRedactor(config));
       dumpGenerator.writeEndObject();
       dumpGenerator.writeEndObject();
       dumpGenerator.flush();
       dumpGenerator.flush();
     }
     }
@@ -3186,11 +3187,11 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
     dumpGenerator.writeFieldName("properties");
     dumpGenerator.writeFieldName("properties");
     dumpGenerator.writeStartArray();
     dumpGenerator.writeStartArray();
     dumpGenerator.flush();
     dumpGenerator.flush();
+    ConfigRedactor redactor = new ConfigRedactor(config);
     synchronized (config) {
     synchronized (config) {
       for (Map.Entry<Object,Object> item: config.getProps().entrySet()) {
       for (Map.Entry<Object,Object> item: config.getProps().entrySet()) {
-        appendJSONProperty(dumpGenerator,
-            config,
-            item.getKey().toString());
+        appendJSONProperty(dumpGenerator, config, item.getKey().toString(),
+            redactor);
       }
       }
     }
     }
     dumpGenerator.writeEndArray();
     dumpGenerator.writeEndArray();
@@ -3208,12 +3209,14 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
    * @throws IOException
    * @throws IOException
    */
    */
   private static void appendJSONProperty(JsonGenerator jsonGen,
   private static void appendJSONProperty(JsonGenerator jsonGen,
-      Configuration config, String name) throws IOException {
+      Configuration config, String name, ConfigRedactor redactor)
+      throws IOException {
     // skip writing if given property name is empty or null
     // skip writing if given property name is empty or null
     if(!Strings.isNullOrEmpty(name) && jsonGen != null) {
     if(!Strings.isNullOrEmpty(name) && jsonGen != null) {
       jsonGen.writeStartObject();
       jsonGen.writeStartObject();
       jsonGen.writeStringField("key", name);
       jsonGen.writeStringField("key", name);
-      jsonGen.writeStringField("value", config.get(name));
+      jsonGen.writeStringField("value",
+          redactor.redact(name, config.get(name)));
       jsonGen.writeBooleanField("isFinal",
       jsonGen.writeBooleanField("isFinal",
           config.finalParameters.contains(name));
           config.finalParameters.contains(name));
       String[] resources = config.updatingResource.get(name);
       String[] resources = config.updatingResource.get(name);

+ 3 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/LoadBalancingKMSClientProvider.java

@@ -292,7 +292,9 @@ public class LoadBalancingKMSClientProvider extends KeyProvider implements
     }
     }
   }
   }
 
 
-  public EncryptedKeyVersion reencryptEncryptedKey(EncryptedKeyVersion ekv)
+  @Override
+  public EncryptedKeyVersion reencryptEncryptedKey(
+      final EncryptedKeyVersion ekv)
       throws IOException, GeneralSecurityException {
       throws IOException, GeneralSecurityException {
     try {
     try {
       return doOp(new ProviderCallable<EncryptedKeyVersion>() {
       return doOp(new ProviderCallable<EncryptedKeyVersion>() {

+ 21 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java

@@ -377,4 +377,25 @@ public class CommonConfigurationKeys extends CommonConfigurationKeysPublic {
 
 
   // HDFS client HTrace configuration.
   // HDFS client HTrace configuration.
   public static final String  FS_CLIENT_HTRACE_PREFIX = "fs.client.htrace.";
   public static final String  FS_CLIENT_HTRACE_PREFIX = "fs.client.htrace.";
+
+  // Global ZooKeeper configuration keys
+  public static final String ZK_PREFIX = "hadoop.zk.";
+  /** ACL for the ZooKeeper ensemble. */
+  public static final String ZK_ACL = ZK_PREFIX + "acl";
+  public static final String ZK_ACL_DEFAULT = "world:anyone:rwcda";
+  /** Authentication for the ZooKeeper ensemble. */
+  public static final String ZK_AUTH = ZK_PREFIX + "auth";
+
+  /** Address of the ZooKeeper ensemble. */
+  public static final String ZK_ADDRESS = ZK_PREFIX + "address";
+  /** Maximum number of retries for a ZooKeeper operation. */
+  public static final String ZK_NUM_RETRIES = ZK_PREFIX + "num-retries";
+  public static final int    ZK_NUM_RETRIES_DEFAULT = 1000;
+  /** Timeout for a ZooKeeper operation in ZooKeeper in milliseconds. */
+  public static final String ZK_TIMEOUT_MS = ZK_PREFIX + "timeout-ms";
+  public static final int    ZK_TIMEOUT_MS_DEFAULT = 10000;
+  /** How often to retry a ZooKeeper operation  in milliseconds. */
+  public static final String ZK_RETRY_INTERVAL_MS =
+      ZK_PREFIX + "retry-interval-ms";
+  public static final int    ZK_RETRY_INTERVAL_MS_DEFAULT = 1000;
 }
 }

+ 2 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java

@@ -295,8 +295,8 @@ public class FileUtil {
                                         Path dst)
                                         Path dst)
                                         throws IOException {
                                         throws IOException {
     if (srcFS == dstFS) {
     if (srcFS == dstFS) {
-      String srcq = src.makeQualified(srcFS).toString() + Path.SEPARATOR;
-      String dstq = dst.makeQualified(dstFS).toString() + Path.SEPARATOR;
+      String srcq = srcFS.makeQualified(src).toString() + Path.SEPARATOR;
+      String dstq = dstFS.makeQualified(dst).toString() + Path.SEPARATOR;
       if (dstq.startsWith(srcq)) {
       if (dstq.startsWith(srcq)) {
         if (srcq.length() == dstq.length()) {
         if (srcq.length() == dstq.length()) {
           throw new IOException("Cannot copy " + src + " to itself.");
           throw new IOException("Cannot copy " + src + " to itself.");

+ 2 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ftp/FTPFileSystem.java

@@ -505,7 +505,7 @@ public class FTPFileSystem extends FileSystem {
       long modTime = -1; // Modification time of root dir not known.
       long modTime = -1; // Modification time of root dir not known.
       Path root = new Path("/");
       Path root = new Path("/");
       return new FileStatus(length, isDir, blockReplication, blockSize,
       return new FileStatus(length, isDir, blockReplication, blockSize,
-          modTime, root.makeQualified(this));
+          modTime, this.makeQualified(root));
     }
     }
     String pathName = parentPath.toUri().getPath();
     String pathName = parentPath.toUri().getPath();
     FTPFile[] ftpFiles = client.listFiles(pathName);
     FTPFile[] ftpFiles = client.listFiles(pathName);
@@ -546,7 +546,7 @@ public class FTPFileSystem extends FileSystem {
     String group = ftpFile.getGroup();
     String group = ftpFile.getGroup();
     Path filePath = new Path(parentPath, ftpFile.getName());
     Path filePath = new Path(parentPath, ftpFile.getName());
     return new FileStatus(length, isDir, blockReplication, blockSize, modTime,
     return new FileStatus(length, isDir, blockReplication, blockSize, modTime,
-        accessTime, permission, user, group, filePath.makeQualified(this));
+        accessTime, permission, user, group, this.makeQualified(filePath));
   }
   }
 
 
   @Override
   @Override

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/SequenceFile.java

@@ -1883,7 +1883,7 @@ public class SequenceFile {
     @Deprecated
     @Deprecated
     public Reader(FileSystem fs, Path file, 
     public Reader(FileSystem fs, Path file, 
                   Configuration conf) throws IOException {
                   Configuration conf) throws IOException {
-      this(conf, file(file.makeQualified(fs)));
+      this(conf, file(fs.makeQualified(file)));
     }
     }
 
 
     /**
     /**

+ 3 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/CompositeGroupsMapping.java

@@ -74,7 +74,9 @@ public class CompositeGroupsMapping
       try {
       try {
         groups = provider.getGroups(user);
         groups = provider.getGroups(user);
       } catch (Exception e) {
       } catch (Exception e) {
-        //LOG.warn("Exception trying to get groups for user " + user, e);      
+        LOG.warn("Unable to get groups for user {} via {} because: {}",
+            user, provider.getClass().getSimpleName(), e.toString());
+        LOG.debug("Stacktrace: ", e);
       }        
       }        
       if (groups != null && ! groups.isEmpty()) {
       if (groups != null && ! groups.isEmpty()) {
         groupSet.addAll(groups);
         groupSet.addAll(groups);

+ 294 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/curator/ZKCuratorManager.java

@@ -0,0 +1,294 @@
+/**
+ * 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.util.curator;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.curator.framework.AuthInfo;
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.retry.RetryNTimes;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.util.ZKUtil;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.data.ACL;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Helper class that provides utility methods specific to ZK operations.
+ */
+@InterfaceAudience.Private
+public final class ZKCuratorManager {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ZKCuratorManager.class);
+
+  /** Configuration for the ZooKeeper connection. */
+  private final Configuration conf;
+
+  /** Curator for ZooKeeper. */
+  private CuratorFramework curator;
+
+
+  public ZKCuratorManager(Configuration config) throws IOException {
+    this.conf = config;
+  }
+
+  /**
+   * Get the curator framework managing the ZooKeeper connection.
+   * @return Curator framework.
+   */
+  public CuratorFramework getCurator() {
+    return curator;
+  }
+
+  /**
+   * Close the connection with ZooKeeper.
+   */
+  public void close() {
+    if (curator != null) {
+      curator.close();
+    }
+  }
+
+  /**
+   * Utility method to fetch the ZK ACLs from the configuration.
+   * @throws java.io.IOException if the Zookeeper ACLs configuration file
+   * cannot be read
+   */
+  public static List<ACL> getZKAcls(Configuration conf) throws IOException {
+    // Parse authentication from configuration.
+    String zkAclConf = conf.get(CommonConfigurationKeys.ZK_ACL,
+        CommonConfigurationKeys.ZK_ACL_DEFAULT);
+    try {
+      zkAclConf = ZKUtil.resolveConfIndirection(zkAclConf);
+      return ZKUtil.parseACLs(zkAclConf);
+    } catch (IOException | ZKUtil.BadAclFormatException e) {
+      LOG.error("Couldn't read ACLs based on {}",
+          CommonConfigurationKeys.ZK_ACL);
+      throw e;
+    }
+  }
+
+  /**
+   * Utility method to fetch ZK auth info from the configuration.
+   * @throws java.io.IOException if the Zookeeper ACLs configuration file
+   * cannot be read
+   */
+  public static List<ZKUtil.ZKAuthInfo> getZKAuths(Configuration conf)
+      throws IOException {
+    String zkAuthConf = conf.get(CommonConfigurationKeys.ZK_AUTH);
+    try {
+      zkAuthConf = ZKUtil.resolveConfIndirection(zkAuthConf);
+      if (zkAuthConf != null) {
+        return ZKUtil.parseAuth(zkAuthConf);
+      } else {
+        return Collections.emptyList();
+      }
+    } catch (IOException | ZKUtil.BadAuthFormatException e) {
+      LOG.error("Couldn't read Auth based on {}",
+          CommonConfigurationKeys.ZK_AUTH);
+      throw e;
+    }
+  }
+
+  /**
+   * Start the connection to the ZooKeeper ensemble.
+   * @param conf Configuration for the connection.
+   * @throws IOException If the connection cannot be started.
+   */
+  public void start() throws IOException {
+    this.start(new ArrayList<>());
+  }
+
+  /**
+   * Start the connection to the ZooKeeper ensemble.
+   * @param conf Configuration for the connection.
+   * @param authInfos List of authentication keys.
+   * @throws IOException If the connection cannot be started.
+   */
+  public void start(List<AuthInfo> authInfos) throws IOException {
+
+    // Connect to the ZooKeeper ensemble
+    String zkHostPort = conf.get(CommonConfigurationKeys.ZK_ADDRESS);
+    if (zkHostPort == null) {
+      throw new IOException(
+          CommonConfigurationKeys.ZK_ADDRESS + " is not configured.");
+    }
+    int numRetries = conf.getInt(CommonConfigurationKeys.ZK_NUM_RETRIES,
+        CommonConfigurationKeys.ZK_NUM_RETRIES_DEFAULT);
+    int zkSessionTimeout = conf.getInt(CommonConfigurationKeys.ZK_TIMEOUT_MS,
+        CommonConfigurationKeys.ZK_TIMEOUT_MS_DEFAULT);
+    int zkRetryInterval = conf.getInt(
+        CommonConfigurationKeys.ZK_RETRY_INTERVAL_MS,
+        CommonConfigurationKeys.ZK_RETRY_INTERVAL_MS_DEFAULT);
+    RetryNTimes retryPolicy = new RetryNTimes(numRetries, zkRetryInterval);
+
+    // Set up ZK auths
+    List<ZKUtil.ZKAuthInfo> zkAuths = getZKAuths(conf);
+    if (authInfos == null) {
+      authInfos = new ArrayList<>();
+    }
+    for (ZKUtil.ZKAuthInfo zkAuth : zkAuths) {
+      authInfos.add(new AuthInfo(zkAuth.getScheme(), zkAuth.getAuth()));
+    }
+
+    CuratorFramework client = CuratorFrameworkFactory.builder()
+        .connectString(zkHostPort)
+        .sessionTimeoutMs(zkSessionTimeout)
+        .retryPolicy(retryPolicy)
+        .authorization(authInfos)
+        .build();
+    client.start();
+
+    this.curator = client;
+  }
+
+  /**
+   * Get ACLs for a ZNode.
+   * @param path Path of the ZNode.
+   * @return The list of ACLs.
+   * @throws Exception
+   */
+  public List<ACL> getACL(final String path) throws Exception {
+    return curator.getACL().forPath(path);
+  }
+
+  /**
+   * Get the data in a ZNode.
+   * @param path Path of the ZNode.
+   * @param stat Output statistics of the ZNode.
+   * @return The data in the ZNode.
+   * @throws Exception If it cannot contact Zookeeper.
+   */
+  public byte[] getData(final String path) throws Exception {
+    return curator.getData().forPath(path);
+  }
+
+  /**
+   * Get the data in a ZNode.
+   * @param path Path of the ZNode.
+   * @param stat Output statistics of the ZNode.
+   * @return The data in the ZNode.
+   * @throws Exception If it cannot contact Zookeeper.
+   */
+  public String getSringData(final String path) throws Exception {
+    byte[] bytes = getData(path);
+    return new String(bytes, Charset.forName("UTF-8"));
+  }
+
+  /**
+   * Set data into a ZNode.
+   * @param path Path of the ZNode.
+   * @param data Data to set.
+   * @param version Version of the data to store.
+   * @throws Exception If it cannot contact Zookeeper.
+   */
+  public void setData(String path, byte[] data, int version) throws Exception {
+    curator.setData().withVersion(version).forPath(path, data);
+  }
+
+  /**
+   * Set data into a ZNode.
+   * @param path Path of the ZNode.
+   * @param data Data to set as String.
+   * @param version Version of the data to store.
+   * @throws Exception If it cannot contact Zookeeper.
+   */
+  public void setData(String path, String data, int version) throws Exception {
+    byte[] bytes = data.getBytes(Charset.forName("UTF-8"));
+    setData(path, bytes, version);
+  }
+
+  /**
+   * Get children of a ZNode.
+   * @param path Path of the ZNode.
+   * @return The list of children.
+   * @throws Exception If it cannot contact Zookeeper.
+   */
+  public List<String> getChildren(final String path) throws Exception {
+    return curator.getChildren().forPath(path);
+  }
+
+  /**
+   * Check if a ZNode exists.
+   * @param path Path of the ZNode.
+   * @return If the ZNode exists.
+   * @throws Exception If it cannot contact Zookeeper.
+   */
+  public boolean exists(final String path) throws Exception {
+    return curator.checkExists().forPath(path) != null;
+  }
+
+  /**
+   * Create a ZNode.
+   * @param path Path of the ZNode.
+   * @return If the ZNode was created.
+   * @throws Exception If it cannot contact Zookeeper.
+   */
+  public boolean create(final String path) throws Exception {
+    return create(path, null);
+  }
+
+  /**
+   * Create a ZNode.
+   * @param path Path of the ZNode.
+   * @param zkAcl ACL for the node.
+   * @return If the ZNode was created.
+   * @throws Exception If it cannot contact Zookeeper.
+   */
+  public boolean create(final String path, List<ACL> zkAcl) throws Exception {
+    boolean created = false;
+    if (!exists(path)) {
+      curator.create()
+          .withMode(CreateMode.PERSISTENT)
+          .withACL(zkAcl)
+          .forPath(path, null);
+      created = true;
+    }
+    return created;
+  }
+
+  /**
+   * Delete a ZNode.
+   * @param path Path of the ZNode.
+   * @throws Exception If it cannot contact ZooKeeper.
+   */
+  public void delete(final String path) throws Exception {
+    if (exists(path)) {
+      curator.delete().deletingChildrenIfNeeded().forPath(path);
+    }
+  }
+
+  /**
+   * Get the path for a ZNode.
+   * @param root Root of the ZNode.
+   * @param nodeName Name of the ZNode.
+   * @return Path for the ZNode.
+   */
+  public static String getNodePath(String root, String nodeName) {
+    return root + "/" + nodeName;
+  }
+}

+ 27 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/curator/package-info.java

@@ -0,0 +1,27 @@
+/**
+ * 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.
+ */
+
+/**
+ * This package provides utilities to interact with Curator ZooKeeper.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+package org.apache.hadoop.util.curator;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;

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

@@ -1429,6 +1429,18 @@
 </property>
 </property>
 
 
 <!-- Azure file system properties -->
 <!-- Azure file system properties -->
+<property>
+  <name>fs.wasb.impl</name>
+  <value>org.apache.hadoop.fs.azure.NativeAzureFileSystem</value>
+  <description>The implementation class of the Native Azure Filesystem</description>
+</property>
+
+<property>
+  <name>fs.wasbs.impl</name>
+  <value>org.apache.hadoop.fs.azure.NativeAzureFileSystem$Secure</value>
+  <description>The implementation class of the Secure Native Azure Filesystem</description>
+</property>
+
 <property>
 <property>
   <name>fs.azure.secure.mode</name>
   <name>fs.azure.secure.mode</name>
   <value>false</value>
   <value>false</value>
@@ -2682,11 +2694,16 @@
     <value>ClientCredential</value>
     <value>ClientCredential</value>
     <description>
     <description>
       Defines Azure Active Directory OAuth2 access token provider type.
       Defines Azure Active Directory OAuth2 access token provider type.
-      Supported types are ClientCredential, RefreshToken, and Custom.
+      Supported types are ClientCredential, RefreshToken, MSI, DeviceCode,
+      and Custom.
       The ClientCredential type requires property fs.adl.oauth2.client.id,
       The ClientCredential type requires property fs.adl.oauth2.client.id,
       fs.adl.oauth2.credential, and fs.adl.oauth2.refresh.url.
       fs.adl.oauth2.credential, and fs.adl.oauth2.refresh.url.
       The RefreshToken type requires property fs.adl.oauth2.client.id and
       The RefreshToken type requires property fs.adl.oauth2.client.id and
       fs.adl.oauth2.refresh.token.
       fs.adl.oauth2.refresh.token.
+      The MSI type requires properties fs.adl.oauth2.msi.port and
+      fs.adl.oauth2.msi.tenantguid.
+      The DeviceCode type requires property
+      fs.adl.oauth2.devicecode.clientapp.id.
       The Custom type requires property fs.adl.oauth2.access.token.provider.
       The Custom type requires property fs.adl.oauth2.access.token.provider.
     </description>
     </description>
   </property>
   </property>
@@ -2723,6 +2740,36 @@
     </description>
     </description>
   </property>
   </property>
 
 
+  <property>
+    <name>fs.adl.oauth2.msi.port</name>
+    <value></value>
+    <description>
+      The localhost port for the MSI token service. This is the port specified
+      when creating the Azure VM.
+      Used by MSI token provider.
+    </description>
+  </property>
+
+  <property>
+    <name>fs.adl.oauth2.msi.tenantguid</name>
+    <value></value>
+    <description>
+      The tenant guid for the Azure AAD tenant under which the azure data lake
+      store account is created.
+      Used by MSI token provider.
+    </description>
+  </property>
+
+  <property>
+    <name>fs.adl.oauth2.devicecode.clientapp.id</name>
+    <value></value>
+    <description>
+      The app id of the AAD native app in whose context the auth request
+      should be made.
+      Used by DeviceCode token provider.
+    </description>
+  </property>
+
   <!-- Azure Data Lake File System Configurations Ends Here-->
   <!-- Azure Data Lake File System Configurations Ends Here-->
 
 
   <property>
   <property>
@@ -2771,4 +2818,50 @@
       This determines the number of open file handles.
       This determines the number of open file handles.
     </description>
     </description>
   </property>
   </property>
+
+  <property>
+    <description>Host:Port of the ZooKeeper server to be used.
+    </description>
+    <name>hadoop.zk.address</name>
+    <!--value>127.0.0.1:2181</value-->
+  </property>
+
+  <property>
+    <description>Number of tries to connect to ZooKeeper.</description>
+    <name>hadoop.zk.num-retries</name>
+    <value>1000</value>
+  </property>
+
+  <property>
+    <description>Retry interval in milliseconds when connecting to ZooKeeper.
+    </description>
+    <name>hadoop.zk.retry-interval-ms</name>
+    <value>1000</value>
+  </property>
+
+  <property>
+    <description>ZooKeeper session timeout in milliseconds. Session expiration
+    is managed by the ZooKeeper cluster itself, not by the client. This value is
+    used by the cluster to determine when the client's session expires.
+    Expirations happens when the cluster does not hear from the client within
+    the specified session timeout period (i.e. no heartbeat).</description>
+    <name>hadoop.zk.timeout-ms</name>
+    <value>10000</value>
+  </property>
+
+  <property>
+    <description>ACL's to be used for ZooKeeper znodes.</description>
+    <name>hadoop.zk.acl</name>
+    <value>world:anyone:rwcda</value>
+  </property>
+
+  <property>
+    <description>
+        Specify the auths to be used for the ACL's specified in hadoop.zk.acl.
+        This takes a comma-separated list of authentication mechanisms, each of the
+        form 'scheme:auth' (the same syntax used for the 'addAuth' command in
+        the ZK CLI).
+    </description>
+    <name>hadoop.zk.auth</name>
+  </property>
 </configuration>
 </configuration>

+ 6 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java

@@ -103,6 +103,12 @@ public class TestCommonConfigurationFields extends TestConfigurationFieldsBase {
     xmlPrefixToSkipCompare.add("fs.s3n.");
     xmlPrefixToSkipCompare.add("fs.s3n.");
     xmlPrefixToSkipCompare.add("s3native.");
     xmlPrefixToSkipCompare.add("s3native.");
 
 
+    // WASB properties are in a different subtree.
+    // - org.apache.hadoop.fs.azure.NativeAzureFileSystem
+    xmlPrefixToSkipCompare.add("fs.wasb.impl");
+    xmlPrefixToSkipCompare.add("fs.wasbs.impl");
+    xmlPrefixToSkipCompare.add("fs.azure.");
+
     // ADL properties are in a different subtree
     // ADL properties are in a different subtree
     // - org.apache.hadoop.hdfs.web.ADLConfKeys
     // - org.apache.hadoop.hdfs.web.ADLConfKeys
     xmlPrefixToSkipCompare.add("adl.");
     xmlPrefixToSkipCompare.add("adl.");

+ 44 - 4
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java

@@ -49,6 +49,7 @@ import static org.junit.Assert.assertArrayEquals;
 
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration.IntegerRanges;
 import org.apache.hadoop.conf.Configuration.IntegerRanges;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetUtils;
@@ -82,6 +83,11 @@ public class TestConfiguration extends TestCase {
   /** Four apostrophes. */
   /** Four apostrophes. */
   public static final String ESCAPED = "&apos;&#39;&#0039;&#x27;";
   public static final String ESCAPED = "&apos;&#39;&#0039;&#x27;";
 
 
+  private static final String SENSITIVE_CONFIG_KEYS =
+      CommonConfigurationKeysPublic.HADOOP_SECURITY_SENSITIVE_CONFIG_KEYS;
+
+  private BufferedWriter out;
+
   @Override
   @Override
   protected void setUp() throws Exception {
   protected void setUp() throws Exception {
     super.setUp();
     super.setUp();
@@ -90,6 +96,9 @@ public class TestConfiguration extends TestCase {
   
   
   @Override
   @Override
   protected void tearDown() throws Exception {
   protected void tearDown() throws Exception {
+    if(out != null) {
+      out.close();
+    }
     super.tearDown();
     super.tearDown();
     new File(CONFIG).delete();
     new File(CONFIG).delete();
     new File(CONFIG2).delete();
     new File(CONFIG2).delete();
@@ -878,8 +887,6 @@ public class TestConfiguration extends TestCase {
     new File(new File(relConfig).getParent()).delete();
     new File(new File(relConfig).getParent()).delete();
   }
   }
 
 
-  BufferedWriter out;
-	
   public void testIntegerRanges() {
   public void testIntegerRanges() {
     Configuration conf = new Configuration();
     Configuration conf = new Configuration();
     conf.set("first", "-100");
     conf.set("first", "-100");
@@ -1787,8 +1794,41 @@ public class TestConfiguration extends TestCase {
       assertEquals(fileResource.toString(),prop.getResource());
       assertEquals(fileResource.toString(),prop.getResource());
     }
     }
   }
   }
-  
-    
+
+  public void testDumpSensitiveProperty() throws IOException {
+    final String myPassword = "ThisIsMyPassword";
+    Configuration testConf = new Configuration(false);
+    out = new BufferedWriter(new FileWriter(CONFIG));
+    startConfig();
+    appendProperty("test.password", myPassword);
+    endConfig();
+    Path fileResource = new Path(CONFIG);
+    testConf.addResource(fileResource);
+
+    try (StringWriter outWriter = new StringWriter()) {
+      testConf.set(SENSITIVE_CONFIG_KEYS, "password$");
+      Configuration.dumpConfiguration(testConf, "test.password", outWriter);
+      assertFalse(outWriter.toString().contains(myPassword));
+    }
+  }
+
+  public void testDumpSensitiveConfiguration() throws IOException {
+    final String myPassword = "ThisIsMyPassword";
+    Configuration testConf = new Configuration(false);
+    out = new BufferedWriter(new FileWriter(CONFIG));
+    startConfig();
+    appendProperty("test.password", myPassword);
+    endConfig();
+    Path fileResource = new Path(CONFIG);
+    testConf.addResource(fileResource);
+
+    try (StringWriter outWriter = new StringWriter()) {
+      testConf.set(SENSITIVE_CONFIG_KEYS, "password$");
+      Configuration.dumpConfiguration(testConf, outWriter);
+      assertFalse(outWriter.toString().contains(myPassword));
+    }
+  }
+
   public void testGetValByRegex() {
   public void testGetValByRegex() {
     Configuration conf = new Configuration();
     Configuration conf = new Configuration();
     String key1 = "t.abc.key1";
     String key1 = "t.abc.key1";

+ 3 - 3
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalFileSystem.java

@@ -218,8 +218,8 @@ public class TestLocalFileSystem {
 
 
   @Test
   @Test
   public void testHomeDirectory() throws IOException {
   public void testHomeDirectory() throws IOException {
-    Path home = new Path(System.getProperty("user.home"))
-      .makeQualified(fileSys);
+    Path home = fileSys.makeQualified(
+        new Path(System.getProperty("user.home")));
     Path fsHome = fileSys.getHomeDirectory();
     Path fsHome = fileSys.getHomeDirectory();
     assertEquals(home, fsHome);
     assertEquals(home, fsHome);
   }
   }
@@ -229,7 +229,7 @@ public class TestLocalFileSystem {
     Path path = new Path(TEST_ROOT_DIR, "foo%bar");
     Path path = new Path(TEST_ROOT_DIR, "foo%bar");
     writeFile(fileSys, path, 1);
     writeFile(fileSys, path, 1);
     FileStatus status = fileSys.getFileStatus(path);
     FileStatus status = fileSys.getFileStatus(path);
-    assertEquals(path.makeQualified(fileSys), status.getPath());
+    assertEquals(fileSys.makeQualified(path), status.getPath());
     cleanupFile(fileSys, path);
     cleanupFile(fileSys, path);
   }
   }
   
   

+ 95 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/curator/TestZKCuratorManager.java

@@ -0,0 +1,95 @@
+/**
+ * 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.util.curator;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.List;
+
+import org.apache.curator.test.TestingServer;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test the manager for ZooKeeper Curator.
+ */
+public class TestZKCuratorManager {
+
+  private TestingServer server;
+  private ZKCuratorManager curator;
+
+  @Before
+  public void setup() throws Exception {
+    this.server = new TestingServer();
+
+    Configuration conf = new Configuration();
+    conf.set(
+        CommonConfigurationKeys.ZK_ADDRESS, this.server.getConnectString());
+
+    this.curator = new ZKCuratorManager(conf);
+    this.curator.start();
+  }
+
+  @After
+  public void teardown() throws Exception {
+    this.curator.close();
+    if (this.server != null) {
+      this.server.close();
+      this.server = null;
+    }
+  }
+
+  @Test
+  public void testReadWriteData() throws Exception {
+    String testZNode = "/test";
+    String expectedString = "testString";
+    assertFalse(curator.exists(testZNode));
+    curator.create(testZNode);
+    assertTrue(curator.exists(testZNode));
+    curator.setData(testZNode, expectedString, -1);
+    String testString = curator.getSringData("/test");
+    assertEquals(expectedString, testString);
+  }
+
+  @Test
+  public void testChildren() throws Exception {
+    List<String> children = curator.getChildren("/");
+    assertEquals(1, children.size());
+
+    assertFalse(curator.exists("/node1"));
+    curator.create("/node1");
+    assertTrue(curator.exists("/node1"));
+
+    assertFalse(curator.exists("/node2"));
+    curator.create("/node2");
+    assertTrue(curator.exists("/node2"));
+
+    children = curator.getChildren("/");
+    assertEquals(3, children.size());
+
+    curator.delete("/node2");
+    assertFalse(curator.exists("/node2"));
+    children = curator.getChildren("/");
+    assertEquals(2, children.size());
+  }
+}

+ 7 - 4
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java

@@ -1131,8 +1131,9 @@ public class DFSInputStream extends FSInputStream
         Future<ByteBuffer> firstRequest = hedgedService
         Future<ByteBuffer> firstRequest = hedgedService
             .submit(getFromDataNodeCallable);
             .submit(getFromDataNodeCallable);
         futures.add(firstRequest);
         futures.add(firstRequest);
+        Future<ByteBuffer> future = null;
         try {
         try {
-          Future<ByteBuffer> future = hedgedService.poll(
+          future = hedgedService.poll(
               conf.getHedgedReadThresholdMillis(), TimeUnit.MILLISECONDS);
               conf.getHedgedReadThresholdMillis(), TimeUnit.MILLISECONDS);
           if (future != null) {
           if (future != null) {
             ByteBuffer result = future.get();
             ByteBuffer result = future.get();
@@ -1142,16 +1143,18 @@ public class DFSInputStream extends FSInputStream
           }
           }
           DFSClient.LOG.debug("Waited {}ms to read from {}; spawning hedged "
           DFSClient.LOG.debug("Waited {}ms to read from {}; spawning hedged "
               + "read", conf.getHedgedReadThresholdMillis(), chosenNode.info);
               + "read", conf.getHedgedReadThresholdMillis(), chosenNode.info);
-          // Ignore this node on next go around.
-          ignored.add(chosenNode.info);
           dfsClient.getHedgedReadMetrics().incHedgedReadOps();
           dfsClient.getHedgedReadMetrics().incHedgedReadOps();
           // continue; no need to refresh block locations
           // continue; no need to refresh block locations
         } catch (ExecutionException e) {
         } catch (ExecutionException e) {
-          // Ignore
+          futures.remove(future);
         } catch (InterruptedException e) {
         } catch (InterruptedException e) {
           throw new InterruptedIOException(
           throw new InterruptedIOException(
               "Interrupted while waiting for reading task");
               "Interrupted while waiting for reading task");
         }
         }
+        // Ignore this node on next go around.
+        // If poll timeout and the request still ongoing, don't consider it
+        // again. If read data failed, don't consider it either.
+        ignored.add(chosenNode.info);
       } else {
       } else {
         // We are starting up a 'hedged' read. We have a read already
         // We are starting up a 'hedged' read. We have a read already
         // ongoing. Call getBestNodeDNAddrPair instead of chooseDataNode.
         // ongoing. Call getBestNodeDNAddrPair instead of chooseDataNode.

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

@@ -269,7 +269,7 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String DFS_NAMENODE_POSIX_ACL_INHERITANCE_ENABLED_KEY =
   public static final String DFS_NAMENODE_POSIX_ACL_INHERITANCE_ENABLED_KEY =
       "dfs.namenode.posix.acl.inheritance.enabled";
       "dfs.namenode.posix.acl.inheritance.enabled";
   public static final boolean
   public static final boolean
-      DFS_NAMENODE_POSIX_ACL_INHERITANCE_ENABLED_DEFAULT = false;
+      DFS_NAMENODE_POSIX_ACL_INHERITANCE_ENABLED_DEFAULT = true;
   public static final String  DFS_NAMENODE_XATTRS_ENABLED_KEY = "dfs.namenode.xattrs.enabled";
   public static final String  DFS_NAMENODE_XATTRS_ENABLED_KEY = "dfs.namenode.xattrs.enabled";
   public static final boolean DFS_NAMENODE_XATTRS_ENABLED_DEFAULT = true;
   public static final boolean DFS_NAMENODE_XATTRS_ENABLED_DEFAULT = true;
   public static final String  DFS_ADMIN = "dfs.cluster.administrators";
   public static final String  DFS_ADMIN = "dfs.cluster.administrators";

+ 0 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java

@@ -212,8 +212,6 @@ public class DatanodeManager {
     this.namesystem = namesystem;
     this.namesystem = namesystem;
     this.blockManager = blockManager;
     this.blockManager = blockManager;
 
 
-    // TODO: Enables DFSNetworkTopology by default after more stress
-    // testings/validations.
     this.useDfsNetworkTopology = conf.getBoolean(
     this.useDfsNetworkTopology = conf.getBoolean(
         DFSConfigKeys.DFS_USE_DFS_NETWORK_TOPOLOGY_KEY,
         DFSConfigKeys.DFS_USE_DFS_NETWORK_TOPOLOGY_KEY,
         DFSConfigKeys.DFS_USE_DFS_NETWORK_TOPOLOGY_DEFAULT);
         DFSConfigKeys.DFS_USE_DFS_NETWORK_TOPOLOGY_DEFAULT);

+ 12 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java

@@ -26,10 +26,11 @@ import java.util.Collections;
 import java.util.Comparator;
 import java.util.Comparator;
 import java.util.HashSet;
 import java.util.HashSet;
 import java.util.List;
 import java.util.List;
-import java.util.PriorityQueue;
+import java.util.NavigableSet;
 import java.util.Set;
 import java.util.Set;
 import java.util.SortedMap;
 import java.util.SortedMap;
 import java.util.TreeMap;
 import java.util.TreeMap;
+import java.util.TreeSet;
 import java.util.concurrent.Callable;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Executors;
@@ -87,11 +88,15 @@ public class LeaseManager {
   // Mapping: leaseHolder -> Lease
   // Mapping: leaseHolder -> Lease
   private final SortedMap<String, Lease> leases = new TreeMap<>();
   private final SortedMap<String, Lease> leases = new TreeMap<>();
   // Set of: Lease
   // Set of: Lease
-  private final PriorityQueue<Lease> sortedLeases = new PriorityQueue<>(512,
+  private final NavigableSet<Lease> sortedLeases = new TreeSet<>(
       new Comparator<Lease>() {
       new Comparator<Lease>() {
         @Override
         @Override
         public int compare(Lease o1, Lease o2) {
         public int compare(Lease o1, Lease o2) {
-          return Long.signum(o1.getLastUpdate() - o2.getLastUpdate());
+          if (o1.getLastUpdate() != o2.getLastUpdate()) {
+            return Long.signum(o1.getLastUpdate() - o2.getLastUpdate());
+          } else {
+            return o1.holder.compareTo(o2.holder);
+          }
         }
         }
   });
   });
   // INodeID -> Lease
   // INodeID -> Lease
@@ -528,9 +533,10 @@ public class LeaseManager {
 
 
     long start = monotonicNow();
     long start = monotonicNow();
 
 
-    while(!sortedLeases.isEmpty() && sortedLeases.peek().expiredHardLimit()
-      && !isMaxLockHoldToReleaseLease(start)) {
-      Lease leaseToCheck = sortedLeases.peek();
+    while(!sortedLeases.isEmpty() &&
+        sortedLeases.first().expiredHardLimit()
+        && !isMaxLockHoldToReleaseLease(start)) {
+      Lease leaseToCheck = sortedLeases.first();
       LOG.info(leaseToCheck + " has expired hard limit");
       LOG.info(leaseToCheck + " has expired hard limit");
 
 
       final List<Long> removing = new ArrayList<>();
       final List<Long> removing = new ArrayList<>();

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

@@ -459,7 +459,7 @@
 
 
   <property>
   <property>
     <name>dfs.namenode.posix.acl.inheritance.enabled</name>
     <name>dfs.namenode.posix.acl.inheritance.enabled</name>
-    <value>false</value>
+    <value>true</value>
     <description>
     <description>
       Set to true to enable POSIX style ACL inheritance. When it is enabled
       Set to true to enable POSIX style ACL inheritance. When it is enabled
       and the create request comes from a compatible client, the NameNode
       and the create request comes from a compatible client, the NameNode

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsPermissionsGuide.md

@@ -322,7 +322,7 @@ Configuration Parameters
 
 
 *   `dfs.namenode.posix.acl.inheritance.enabled`
 *   `dfs.namenode.posix.acl.inheritance.enabled`
 
 
-    Set to true to enable POSIX style ACL inheritance. Disabled by default.
+    Set to true to enable POSIX style ACL inheritance. Enabled by default.
     When it is enabled and the create request comes from a compatible client,
     When it is enabled and the create request comes from a compatible client,
     the NameNode will apply default ACLs from the parent directory to
     the NameNode will apply default ACLs from the parent directory to
     the create mode and ignore the client umask. If no default ACL is found,
     the create mode and ignore the client umask. If no default ACL is found,

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/TestAclCLI.java

@@ -34,6 +34,8 @@ public class TestAclCLI extends CLITestHelperDFS {
 
 
   protected void initConf() {
   protected void initConf() {
     conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY, true);
     conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY, true);
+    conf.setBoolean(
+        DFSConfigKeys.DFS_NAMENODE_POSIX_ACL_INHERITANCE_ENABLED_KEY, false);
   }
   }
 
 
   @Before
   @Before

+ 63 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java

@@ -59,6 +59,8 @@ import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 import org.mockito.stubbing.Answer;
 
 
 import com.google.common.base.Supplier;
 import com.google.common.base.Supplier;
+import org.slf4j.LoggerFactory;
+import org.slf4j.Logger;
 
 
 /**
 /**
  * This class tests the DFS positional read functionality in a single node
  * This class tests the DFS positional read functionality in a single node
@@ -72,6 +74,9 @@ public class TestPread {
   boolean simulatedStorage;
   boolean simulatedStorage;
   boolean isHedgedRead;
   boolean isHedgedRead;
 
 
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestPread.class.getName());
+
   @Before
   @Before
   public void setup() {
   public void setup() {
     simulatedStorage = false;
     simulatedStorage = false;
@@ -551,6 +556,64 @@ public class TestPread {
     }
     }
   }
   }
 
 
+  @Test(timeout=30000)
+  public void testHedgedReadFromAllDNFailed() throws IOException {
+    Configuration conf = new Configuration();
+    int numHedgedReadPoolThreads = 5;
+    final int hedgedReadTimeoutMillis = 50;
+
+    conf.setInt(HdfsClientConfigKeys.HedgedRead.THREADPOOL_SIZE_KEY,
+        numHedgedReadPoolThreads);
+    conf.setLong(HdfsClientConfigKeys.HedgedRead.THRESHOLD_MILLIS_KEY,
+        hedgedReadTimeoutMillis);
+    conf.setInt(HdfsClientConfigKeys.Retry.WINDOW_BASE_KEY, 0);
+    // Set up the InjectionHandler
+    DFSClientFaultInjector.set(Mockito.mock(DFSClientFaultInjector.class));
+    DFSClientFaultInjector injector = DFSClientFaultInjector.get();
+    Mockito.doAnswer(new Answer<Void>() {
+      @Override
+      public Void answer(InvocationOnMock invocation) throws Throwable {
+        if (true) {
+          LOG.info("-------------- throw Checksum Exception");
+          throw new ChecksumException("ChecksumException test", 100);
+        }
+        return null;
+      }
+    }).when(injector).fetchFromDatanodeException();
+
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3)
+        .format(true).build();
+    DistributedFileSystem fileSys = cluster.getFileSystem();
+    DFSClient dfsClient = fileSys.getClient();
+    FSDataOutputStream output = null;
+    DFSInputStream input = null;
+    String filename = "/hedgedReadMaxOut.dat";
+    DFSHedgedReadMetrics metrics = dfsClient.getHedgedReadMetrics();
+    // Metrics instance is static, so we need to reset counts from prior tests.
+    metrics.hedgedReadOps.set(0);
+    try {
+      Path file = new Path(filename);
+      output = fileSys.create(file, (short) 2);
+      byte[] data = new byte[64 * 1024];
+      output.write(data);
+      output.flush();
+      output.close();
+      byte[] buffer = new byte[64 * 1024];
+      input = dfsClient.open(filename);
+      input.read(0, buffer, 0, 1024);
+      Assert.fail("Reading the block should have thrown BlockMissingException");
+    } catch (BlockMissingException e) {
+      assertEquals(3, input.getHedgedReadOpsLoopNumForTesting());
+      assertTrue(metrics.getHedgedReadOps() == 0);
+    } finally {
+      Mockito.reset(injector);
+      IOUtils.cleanupWithLogger(LOG, input);
+      IOUtils.cleanupWithLogger(LOG, output);
+      fileSys.close();
+      cluster.shutdown();
+    }
+  }
+
   /**
   /**
    * Scenario: 1. Write a file with RF=2, DN1 and DN2<br>
    * Scenario: 1. Write a file with RF=2, DN1 and DN2<br>
    * 2. Open the stream, Consider Locations are [DN1, DN2] in LocatedBlock.<br>
    * 2. Open the stream, Consider Locations are [DN1, DN2] in LocatedBlock.<br>

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

@@ -903,7 +903,7 @@ public abstract class FSAclBaseTest {
     assertArrayEquals(new AclEntry[] {
     assertArrayEquals(new AclEntry[] {
       aclEntry(ACCESS, USER, "foo", ALL),
       aclEntry(ACCESS, USER, "foo", ALL),
       aclEntry(ACCESS, GROUP, READ_EXECUTE) }, returned);
       aclEntry(ACCESS, GROUP, READ_EXECUTE) }, returned);
-    assertPermission(filePath, (short)010640);
+    assertPermission(filePath, (short)010660);
     assertAclFeature(filePath, true);
     assertAclFeature(filePath, true);
   }
   }
 
 
@@ -1003,7 +1003,7 @@ public abstract class FSAclBaseTest {
       aclEntry(DEFAULT, GROUP, READ_EXECUTE),
       aclEntry(DEFAULT, GROUP, READ_EXECUTE),
       aclEntry(DEFAULT, MASK, ALL),
       aclEntry(DEFAULT, MASK, ALL),
       aclEntry(DEFAULT, OTHER, NONE) }, returned);
       aclEntry(DEFAULT, OTHER, NONE) }, returned);
-    assertPermission(dirPath, (short)010750);
+    assertPermission(dirPath, (short)010770);
     assertAclFeature(dirPath, true);
     assertAclFeature(dirPath, true);
   }
   }
 
 
@@ -1120,7 +1120,7 @@ public abstract class FSAclBaseTest {
     s = fs.getAclStatus(filePath);
     s = fs.getAclStatus(filePath);
     returned = s.getEntries().toArray(new AclEntry[0]);
     returned = s.getEntries().toArray(new AclEntry[0]);
     assertArrayEquals(expected, returned);
     assertArrayEquals(expected, returned);
-    assertPermission(filePath, (short)010640);
+    assertPermission(filePath, (short)010660);
     assertAclFeature(filePath, true);
     assertAclFeature(filePath, true);
   }
   }
 
 
@@ -1149,7 +1149,7 @@ public abstract class FSAclBaseTest {
     s = fs.getAclStatus(subdirPath);
     s = fs.getAclStatus(subdirPath);
     returned = s.getEntries().toArray(new AclEntry[0]);
     returned = s.getEntries().toArray(new AclEntry[0]);
     assertArrayEquals(expected, returned);
     assertArrayEquals(expected, returned);
-    assertPermission(subdirPath, (short)010750);
+    assertPermission(subdirPath, (short)010770);
     assertAclFeature(subdirPath, true);
     assertAclFeature(subdirPath, true);
   }
   }
 
 

+ 8 - 6
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageWithAcl.java

@@ -138,13 +138,15 @@ public class TestFSImageWithAcl {
       aclEntry(DEFAULT, MASK, ALL),
       aclEntry(DEFAULT, MASK, ALL),
       aclEntry(DEFAULT, OTHER, READ_EXECUTE) };
       aclEntry(DEFAULT, OTHER, READ_EXECUTE) };
 
 
+    short permExpected = (short)010775;
+
     AclEntry[] fileReturned = fs.getAclStatus(filePath).getEntries()
     AclEntry[] fileReturned = fs.getAclStatus(filePath).getEntries()
       .toArray(new AclEntry[0]);
       .toArray(new AclEntry[0]);
     Assert.assertArrayEquals(fileExpected, fileReturned);
     Assert.assertArrayEquals(fileExpected, fileReturned);
     AclEntry[] subdirReturned = fs.getAclStatus(subdirPath).getEntries()
     AclEntry[] subdirReturned = fs.getAclStatus(subdirPath).getEntries()
       .toArray(new AclEntry[0]);
       .toArray(new AclEntry[0]);
     Assert.assertArrayEquals(subdirExpected, subdirReturned);
     Assert.assertArrayEquals(subdirExpected, subdirReturned);
-    assertPermission(fs, subdirPath, (short)010755);
+    assertPermission(fs, subdirPath, permExpected);
 
 
     restart(fs, persistNamespace);
     restart(fs, persistNamespace);
 
 
@@ -154,7 +156,7 @@ public class TestFSImageWithAcl {
     subdirReturned = fs.getAclStatus(subdirPath).getEntries()
     subdirReturned = fs.getAclStatus(subdirPath).getEntries()
       .toArray(new AclEntry[0]);
       .toArray(new AclEntry[0]);
     Assert.assertArrayEquals(subdirExpected, subdirReturned);
     Assert.assertArrayEquals(subdirExpected, subdirReturned);
-    assertPermission(fs, subdirPath, (short)010755);
+    assertPermission(fs, subdirPath, permExpected);
 
 
     aclSpec = Lists.newArrayList(aclEntry(DEFAULT, USER, "foo", READ_WRITE));
     aclSpec = Lists.newArrayList(aclEntry(DEFAULT, USER, "foo", READ_WRITE));
     fs.modifyAclEntries(dirPath, aclSpec);
     fs.modifyAclEntries(dirPath, aclSpec);
@@ -165,7 +167,7 @@ public class TestFSImageWithAcl {
     subdirReturned = fs.getAclStatus(subdirPath).getEntries()
     subdirReturned = fs.getAclStatus(subdirPath).getEntries()
       .toArray(new AclEntry[0]);
       .toArray(new AclEntry[0]);
     Assert.assertArrayEquals(subdirExpected, subdirReturned);
     Assert.assertArrayEquals(subdirExpected, subdirReturned);
-    assertPermission(fs, subdirPath, (short)010755);
+    assertPermission(fs, subdirPath, permExpected);
 
 
     restart(fs, persistNamespace);
     restart(fs, persistNamespace);
 
 
@@ -175,7 +177,7 @@ public class TestFSImageWithAcl {
     subdirReturned = fs.getAclStatus(subdirPath).getEntries()
     subdirReturned = fs.getAclStatus(subdirPath).getEntries()
       .toArray(new AclEntry[0]);
       .toArray(new AclEntry[0]);
     Assert.assertArrayEquals(subdirExpected, subdirReturned);
     Assert.assertArrayEquals(subdirExpected, subdirReturned);
-    assertPermission(fs, subdirPath, (short)010755);
+    assertPermission(fs, subdirPath, permExpected);
 
 
     fs.removeAcl(dirPath);
     fs.removeAcl(dirPath);
 
 
@@ -185,7 +187,7 @@ public class TestFSImageWithAcl {
     subdirReturned = fs.getAclStatus(subdirPath).getEntries()
     subdirReturned = fs.getAclStatus(subdirPath).getEntries()
       .toArray(new AclEntry[0]);
       .toArray(new AclEntry[0]);
     Assert.assertArrayEquals(subdirExpected, subdirReturned);
     Assert.assertArrayEquals(subdirExpected, subdirReturned);
-    assertPermission(fs, subdirPath, (short)010755);
+    assertPermission(fs, subdirPath, permExpected);
 
 
     restart(fs, persistNamespace);
     restart(fs, persistNamespace);
 
 
@@ -195,7 +197,7 @@ public class TestFSImageWithAcl {
     subdirReturned = fs.getAclStatus(subdirPath).getEntries()
     subdirReturned = fs.getAclStatus(subdirPath).getEntries()
       .toArray(new AclEntry[0]);
       .toArray(new AclEntry[0]);
     Assert.assertArrayEquals(subdirExpected, subdirReturned);
     Assert.assertArrayEquals(subdirExpected, subdirReturned);
-    assertPermission(fs, subdirPath, (short)010755);
+    assertPermission(fs, subdirPath, permExpected);
   }
   }
 
 
   @Test
   @Test

+ 34 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java

@@ -644,6 +644,8 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
   private float reduceProgress;
   private float reduceProgress;
   private float cleanupProgress;
   private float cleanupProgress;
   private boolean isUber = false;
   private boolean isUber = false;
+  private boolean finishJobWhenReducersDone;
+  private boolean completingJob = false;
 
 
   private Credentials jobCredentials;
   private Credentials jobCredentials;
   private Token<JobTokenIdentifier> jobToken;
   private Token<JobTokenIdentifier> jobToken;
@@ -717,6 +719,9 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
     this.maxFetchFailuresNotifications = conf.getInt(
     this.maxFetchFailuresNotifications = conf.getInt(
         MRJobConfig.MAX_FETCH_FAILURES_NOTIFICATIONS,
         MRJobConfig.MAX_FETCH_FAILURES_NOTIFICATIONS,
         MRJobConfig.DEFAULT_MAX_FETCH_FAILURES_NOTIFICATIONS);
         MRJobConfig.DEFAULT_MAX_FETCH_FAILURES_NOTIFICATIONS);
+    this.finishJobWhenReducersDone = conf.getBoolean(
+        MRJobConfig.FINISH_JOB_WHEN_REDUCERS_DONE,
+        MRJobConfig.DEFAULT_FINISH_JOB_WHEN_REDUCERS_DONE);
   }
   }
 
 
   protected StateMachine<JobStateInternal, JobEventType, JobEvent> getStateMachine() {
   protected StateMachine<JobStateInternal, JobEventType, JobEvent> getStateMachine() {
@@ -2021,7 +2026,9 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
                 TimeUnit.MILLISECONDS);
                 TimeUnit.MILLISECONDS);
         return JobStateInternal.FAIL_WAIT;
         return JobStateInternal.FAIL_WAIT;
       }
       }
-      
+
+      checkReadyForCompletionWhenAllReducersDone(job);
+
       return job.checkReadyForCommit();
       return job.checkReadyForCommit();
     }
     }
 
 
@@ -2052,6 +2059,32 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
       }
       }
       job.metrics.killedTask(task);
       job.metrics.killedTask(task);
     }
     }
+
+   /** Improvement: if all reducers have finished, we check if we have
+       restarted mappers that are still running. This can happen in a
+       situation when a node becomes UNHEALTHY and mappers are rescheduled.
+       See MAPREDUCE-6870 for details */
+    private void checkReadyForCompletionWhenAllReducersDone(JobImpl job) {
+      if (job.finishJobWhenReducersDone) {
+        int totalReduces = job.getTotalReduces();
+        int completedReduces = job.getCompletedReduces();
+
+        if (totalReduces > 0 && totalReduces == completedReduces
+            && !job.completingJob) {
+
+          for (TaskId mapTaskId : job.mapTasks) {
+            MapTaskImpl task = (MapTaskImpl) job.tasks.get(mapTaskId);
+            if (!task.isFinished()) {
+              LOG.info("Killing map task " + task.getID());
+              job.eventHandler.handle(
+                  new TaskEvent(task.getID(), TaskEventType.T_KILL));
+            }
+          }
+
+          job.completingJob = true;
+        }
+      }
+    }
   }
   }
 
 
   // Transition class for handling jobs with no tasks
   // Transition class for handling jobs with no tasks

+ 113 - 26
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestJobImpl.java

@@ -564,33 +564,13 @@ public class TestJobImpl {
     dispatcher.register(TaskAttemptEventType.class, taskAttemptEventHandler);
     dispatcher.register(TaskAttemptEventType.class, taskAttemptEventHandler);
 
 
     // replace the tasks with spied versions to return the right attempts
     // replace the tasks with spied versions to return the right attempts
-    Map<TaskId,Task> spiedTasks = new HashMap<TaskId,Task>();
-    List<NodeReport> nodeReports = new ArrayList<NodeReport>();
-    Map<NodeReport,TaskId> nodeReportsToTaskIds =
-        new HashMap<NodeReport,TaskId>();
-    for (Map.Entry<TaskId,Task> e: job.tasks.entrySet()) {
-      TaskId taskId = e.getKey();
-      Task task = e.getValue();
-      if (taskId.getTaskType() == TaskType.MAP) {
-        // add an attempt to the task to simulate nodes
-        NodeId nodeId = mock(NodeId.class);
-        TaskAttempt attempt = mock(TaskAttempt.class);
-        when(attempt.getNodeId()).thenReturn(nodeId);
-        TaskAttemptId attemptId = MRBuilderUtils.newTaskAttemptId(taskId, 0);
-        when(attempt.getID()).thenReturn(attemptId);
-        // create a spied task
-        Task spied = spy(task);
-        doReturn(attempt).when(spied).getAttempt(any(TaskAttemptId.class));
-        spiedTasks.put(taskId, spied);
+    Map<TaskId, Task> spiedTasks = new HashMap<>();
+    List<NodeReport> nodeReports = new ArrayList<>();
+    Map<NodeReport, TaskId> nodeReportsToTaskIds = new HashMap<>();
+
+    createSpiedMapTasks(nodeReportsToTaskIds, spiedTasks, job,
+        NodeState.UNHEALTHY, nodeReports);
 
 
-        // create a NodeReport based on the node id
-        NodeReport report = mock(NodeReport.class);
-        when(report.getNodeState()).thenReturn(NodeState.UNHEALTHY);
-        when(report.getNodeId()).thenReturn(nodeId);
-        nodeReports.add(report);
-        nodeReportsToTaskIds.put(report, taskId);
-      }
-    }
     // replace the tasks with the spied tasks
     // replace the tasks with the spied tasks
     job.tasks.putAll(spiedTasks);
     job.tasks.putAll(spiedTasks);
 
 
@@ -641,6 +621,82 @@ public class TestJobImpl {
     commitHandler.stop();
     commitHandler.stop();
   }
   }
 
 
+  @Test
+  public void testJobNCompletedWhenAllReducersAreFinished()
+      throws Exception {
+    testJobCompletionWhenReducersAreFinished(true);
+  }
+
+  @Test
+  public void testJobNotCompletedWhenAllReducersAreFinished()
+      throws Exception {
+    testJobCompletionWhenReducersAreFinished(false);
+  }
+
+  private void testJobCompletionWhenReducersAreFinished(boolean killMappers)
+      throws InterruptedException, BrokenBarrierException {
+    Configuration conf = new Configuration();
+    conf.setBoolean(MRJobConfig.FINISH_JOB_WHEN_REDUCERS_DONE, killMappers);
+    conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir);
+    conf.setInt(MRJobConfig.NUM_REDUCES, 1);
+    DrainDispatcher dispatcher = new DrainDispatcher();
+    dispatcher.init(conf);
+    final List<TaskEvent> killedEvents =
+        Collections.synchronizedList(new ArrayList<TaskEvent>());
+    dispatcher.register(TaskEventType.class, new EventHandler<TaskEvent>() {
+      @Override
+      public void handle(TaskEvent event) {
+        if (event.getType() == TaskEventType.T_KILL) {
+          killedEvents.add(event);
+        }
+      }
+    });
+    dispatcher.start();
+    CyclicBarrier syncBarrier = new CyclicBarrier(2);
+    OutputCommitter committer = new TestingOutputCommitter(syncBarrier, true);
+    CommitterEventHandler commitHandler =
+        createCommitterEventHandler(dispatcher, committer);
+    commitHandler.init(conf);
+    commitHandler.start();
+
+    final JobImpl job = createRunningStubbedJob(conf, dispatcher, 2, null);
+
+    // replace the tasks with spied versions to return the right attempts
+    Map<TaskId, Task> spiedTasks = new HashMap<>();
+    List<NodeReport> nodeReports = new ArrayList<>();
+    Map<NodeReport, TaskId> nodeReportsToTaskIds = new HashMap<>();
+
+    createSpiedMapTasks(nodeReportsToTaskIds, spiedTasks, job,
+        NodeState.RUNNING, nodeReports);
+
+    // replace the tasks with the spied tasks
+    job.tasks.putAll(spiedTasks);
+
+    // finish reducer
+    for (TaskId taskId: job.tasks.keySet()) {
+      if (taskId.getTaskType() == TaskType.REDUCE) {
+        job.handle(new JobTaskEvent(taskId, TaskState.SUCCEEDED));
+      }
+    }
+
+    dispatcher.await();
+
+    /*
+     * StubbedJob cannot finish in this test - we'd have to generate the
+     * necessary events in this test manually, but that wouldn't add too
+     * much value. Instead, we validate the T_KILL events.
+     */
+    if (killMappers) {
+      Assert.assertEquals("Number of killed events", 2, killedEvents.size());
+      Assert.assertEquals("AttemptID", "task_1234567890000_0001_m_000000",
+          killedEvents.get(0).getTaskID().toString());
+      Assert.assertEquals("AttemptID", "task_1234567890000_0001_m_000001",
+          killedEvents.get(1).getTaskID().toString());
+    } else {
+      Assert.assertEquals("Number of killed events", 0, killedEvents.size());
+    }
+  }
+
   public static void main(String[] args) throws Exception {
   public static void main(String[] args) throws Exception {
     TestJobImpl t = new TestJobImpl();
     TestJobImpl t = new TestJobImpl();
     t.testJobNoTasks();
     t.testJobNoTasks();
@@ -1021,6 +1077,37 @@ public class TestJobImpl {
     Assert.assertEquals(state, job.getInternalState());
     Assert.assertEquals(state, job.getInternalState());
   }
   }
 
 
+  private void createSpiedMapTasks(Map<NodeReport, TaskId>
+      nodeReportsToTaskIds, Map<TaskId, Task> spiedTasks, JobImpl job,
+      NodeState nodeState, List<NodeReport> nodeReports) {
+    for (Map.Entry<TaskId, Task> e: job.tasks.entrySet()) {
+      TaskId taskId = e.getKey();
+      Task task = e.getValue();
+      if (taskId.getTaskType() == TaskType.MAP) {
+        // add an attempt to the task to simulate nodes
+        NodeId nodeId = mock(NodeId.class);
+        TaskAttempt attempt = mock(TaskAttempt.class);
+        when(attempt.getNodeId()).thenReturn(nodeId);
+        TaskAttemptId attemptId = MRBuilderUtils.newTaskAttemptId(taskId, 0);
+        when(attempt.getID()).thenReturn(attemptId);
+        // create a spied task
+        Task spied = spy(task);
+        Map<TaskAttemptId, TaskAttempt> attemptMap = new HashMap<>();
+        attemptMap.put(attemptId, attempt);
+        when(spied.getAttempts()).thenReturn(attemptMap);
+        doReturn(attempt).when(spied).getAttempt(any(TaskAttemptId.class));
+        spiedTasks.put(taskId, spied);
+
+        // create a NodeReport based on the node id
+        NodeReport report = mock(NodeReport.class);
+        when(report.getNodeState()).thenReturn(nodeState);
+        when(report.getNodeId()).thenReturn(nodeId);
+        nodeReports.add(report);
+        nodeReportsToTaskIds.put(report, taskId);
+      }
+    }
+  }
+
   private static class JobSubmittedEventHandler implements
   private static class JobSubmittedEventHandler implements
       EventHandler<JobHistoryEvent> {
       EventHandler<JobHistoryEvent> {
 
 

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

@@ -431,7 +431,7 @@ public interface MRJobConfig {
   public static final String JOB_ACL_MODIFY_JOB = "mapreduce.job.acl-modify-job";
   public static final String JOB_ACL_MODIFY_JOB = "mapreduce.job.acl-modify-job";
 
 
   public static final String DEFAULT_JOB_ACL_MODIFY_JOB = " ";
   public static final String DEFAULT_JOB_ACL_MODIFY_JOB = " ";
-  
+
   public static final String JOB_RUNNING_MAP_LIMIT =
   public static final String JOB_RUNNING_MAP_LIMIT =
       "mapreduce.job.running.map.limit";
       "mapreduce.job.running.map.limit";
   public static final int DEFAULT_JOB_RUNNING_MAP_LIMIT = 0;
   public static final int DEFAULT_JOB_RUNNING_MAP_LIMIT = 0;
@@ -1033,4 +1033,8 @@ public interface MRJobConfig {
   String MR_JOB_REDACTED_PROPERTIES = "mapreduce.job.redacted-properties";
   String MR_JOB_REDACTED_PROPERTIES = "mapreduce.job.redacted-properties";
 
 
   String MR_JOB_SEND_TOKEN_CONF = "mapreduce.job.send-token-conf";
   String MR_JOB_SEND_TOKEN_CONF = "mapreduce.job.send-token-conf";
+
+  String FINISH_JOB_WHEN_REDUCERS_DONE =
+      "mapreduce.job.finish-when-all-reducers-done";
+  boolean DEFAULT_FINISH_JOB_WHEN_REDUCERS_DONE = true;
 }
 }

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

@@ -1125,6 +1125,14 @@
   </description>
   </description>
 </property>
 </property>
 
 
+<property>
+  <name>mapreduce.job.finish-when-all-reducers-done</name>
+  <value>true</value>
+  <description>Specifies whether the job should complete once all reducers
+     have finished, regardless of whether there are still running mappers.
+  </description>
+</property>
+
 <property>
 <property>
   <name>mapreduce.job.token.tracking.ids.enabled</name>
   <name>mapreduce.job.token.tracking.ids.enabled</name>
   <value>false</value>
   <value>false</value>

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/io/FileBench.java

@@ -170,7 +170,7 @@ public class FileBench extends Configured implements Tool {
     for(int i = 0; i < argv.length; ++i) {
     for(int i = 0; i < argv.length; ++i) {
       try {
       try {
         if ("-dir".equals(argv[i])) {
         if ("-dir".equals(argv[i])) {
-          root = new Path(argv[++i]).makeQualified(fs);
+          root = fs.makeQualified(new Path(argv[++i]));
           System.out.println("DIR: " + root.toString());
           System.out.println("DIR: " + root.toString());
         } else if ("-seed".equals(argv[i])) {
         } else if ("-seed".equals(argv[i])) {
           job.setLong("filebench.seed", Long.valueOf(argv[++i]));
           job.setLong("filebench.seed", Long.valueOf(argv[++i]));

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/MiniMRClientClusterFactory.java

@@ -50,8 +50,8 @@ public class MiniMRClientClusterFactory {
 
 
     FileSystem fs = FileSystem.get(conf);
     FileSystem fs = FileSystem.get(conf);
 
 
-    Path testRootDir = new Path("target", identifier + "-tmpDir")
-        .makeQualified(fs);
+    Path testRootDir = fs.makeQualified(
+        new Path("target", identifier + "-tmpDir"));
     Path appJar = new Path(testRootDir, "MRAppJar.jar");
     Path appJar = new Path(testRootDir, "MRAppJar.jar");
 
 
     // Copy MRAppJar and make it private.
     // Copy MRAppJar and make it private.

+ 3 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestCombineFileInputFormat.java

@@ -47,9 +47,9 @@ public class TestCombineFileInputFormat {
       throw new RuntimeException("init failure", e);
       throw new RuntimeException("init failure", e);
     }
     }
   }
   }
-  private static Path workDir =
-    new Path(new Path(System.getProperty("test.build.data", "/tmp")),
-             "TestCombineFileInputFormat").makeQualified(localFs);
+  private static Path workDir = localFs.makeQualified(new Path(
+      System.getProperty("test.build.data", "/tmp"),
+      "TestCombineFileInputFormat"));
 
 
   private static void writeFile(FileSystem fs, Path name, 
   private static void writeFile(FileSystem fs, Path name, 
                                 String contents) throws IOException {
                                 String contents) throws IOException {

+ 3 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestCombineSequenceFileInputFormat.java

@@ -53,10 +53,9 @@ public class TestCombineSequenceFileInputFormat {
     }
     }
   }
   }
 
 
-  @SuppressWarnings("deprecation")
-  private static Path workDir =
-    new Path(new Path(System.getProperty("test.build.data", "/tmp")),
-             "TestCombineSequenceFileInputFormat").makeQualified(localFs);
+  private static Path workDir = localFs.makeQualified(new Path(
+      System.getProperty("test.build.data", "/tmp"),
+      "TestCombineSequenceFileInputFormat"));
 
 
   @Test(timeout=10000)
   @Test(timeout=10000)
   public void testFormat() throws Exception {
   public void testFormat() throws Exception {

+ 3 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestCombineTextInputFormat.java

@@ -60,10 +60,9 @@ public class TestCombineTextInputFormat {
     }
     }
   }
   }
 
 
-  @SuppressWarnings("deprecation")
-  private static Path workDir =
-    new Path(new Path(System.getProperty("test.build.data", "/tmp")),
-             "TestCombineTextInputFormat").makeQualified(localFs);
+  private static Path workDir = localFs.makeQualified(new Path(
+      System.getProperty("test.build.data", "/tmp"),
+      "TestCombineTextInputFormat"));
 
 
   // A reporter that does nothing
   // A reporter that does nothing
   private static final Reporter voidReporter = Reporter.NULL;
   private static final Reporter voidReporter = Reporter.NULL;

+ 3 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestConcatenatedCompressedInput.java

@@ -84,9 +84,9 @@ public class TestConcatenatedCompressedInput {
   public void after() {
   public void after() {
     ZlibFactory.loadNativeZLib();
     ZlibFactory.loadNativeZLib();
   }
   }
-  private static Path workDir =
-    new Path(new Path(System.getProperty("test.build.data", "/tmp")),
-             "TestConcatenatedCompressedInput").makeQualified(localFs);
+  private static Path workDir = localFs.makeQualified(new Path(
+      System.getProperty("test.build.data", "/tmp"),
+      "TestConcatenatedCompressedInput"));
 
 
   private static LineReader makeStream(String str) throws IOException {
   private static LineReader makeStream(String str) throws IOException {
     return new LineReader(new ByteArrayInputStream(str.getBytes("UTF-8")),
     return new LineReader(new ByteArrayInputStream(str.getBytes("UTF-8")),

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMapRed.java

@@ -342,8 +342,8 @@ public class TestMapRed extends Configured implements Tool {
       values.add(m);
       values.add(m);
       m = m.replace((char)('A' + i - 1), (char)('A' + i));
       m = m.replace((char)('A' + i - 1), (char)('A' + i));
     }
     }
-    Path testdir = new Path(
-        System.getProperty("test.build.data","/tmp")).makeQualified(fs);
+    Path testdir = fs.makeQualified(new Path(
+        System.getProperty("test.build.data","/tmp")));
     fs.delete(testdir, true);
     fs.delete(testdir, true);
     Path inFile = new Path(testdir, "nullin/blah");
     Path inFile = new Path(testdir, "nullin/blah");
     SequenceFile.Writer w = SequenceFile.createWriter(fs, conf, inFile,
     SequenceFile.Writer w = SequenceFile.createWriter(fs, conf, inFile,

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMiniMRChildTask.java

@@ -75,8 +75,8 @@ public class TestMiniMRChildTask {
     }
     }
   }
   }
 
 
-  private static Path TEST_ROOT_DIR = new Path("target",
-      TestMiniMRChildTask.class.getName() + "-tmpDir").makeQualified(localFs);
+  private static Path TEST_ROOT_DIR = localFs.makeQualified(
+      new Path("target", TestMiniMRChildTask.class.getName() + "-tmpDir"));
   static Path APP_JAR = new Path(TEST_ROOT_DIR, "MRAppJar.jar");
   static Path APP_JAR = new Path(TEST_ROOT_DIR, "MRAppJar.jar");
 
 
   /**
   /**

+ 4 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestTextInputFormat.java

@@ -61,10 +61,10 @@ public class TestTextInputFormat {
       throw new RuntimeException("init failure", e);
       throw new RuntimeException("init failure", e);
     }
     }
   }
   }
-  @SuppressWarnings("deprecation")
-  private static Path workDir =
-    new Path(new Path(System.getProperty("test.build.data", "/tmp")),
-             "TestTextInputFormat").makeQualified(localFs);
+
+  private static Path workDir = localFs.makeQualified(new Path(
+      System.getProperty("test.build.data", "/tmp"),
+      "TestTextInputFormat"));
 
 
   @Test (timeout=500000)
   @Test (timeout=500000)
   public void testFormat() throws Exception {
   public void testFormat() throws Exception {

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/join/TestWrappedRecordReaderClassloader.java

@@ -50,8 +50,8 @@ public class TestWrappedRecordReaderClassloader {
     assertTrue(job.getClassLoader() instanceof Fake_ClassLoader);
     assertTrue(job.getClassLoader() instanceof Fake_ClassLoader);
 
 
     FileSystem fs = FileSystem.get(job);
     FileSystem fs = FileSystem.get(job);
-    Path testdir = new Path(System.getProperty("test.build.data", "/tmp"))
-        .makeQualified(fs);
+    Path testdir = fs.makeQualified(new Path(
+        System.getProperty("test.build.data", "/tmp")));
 
 
     Path base = new Path(testdir, "/empty");
     Path base = new Path(testdir, "/empty");
     Path[] src = { new Path(base, "i0"), new Path("i1"), new Path("i2") };
     Path[] src = { new Path(base, "i0"), new Path("i1"), new Path("i2") };

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/join/TestWrappedRRClassloader.java

@@ -50,8 +50,8 @@ public class TestWrappedRRClassloader {
     assertTrue(conf.getClassLoader() instanceof Fake_ClassLoader);
     assertTrue(conf.getClassLoader() instanceof Fake_ClassLoader);
 
 
     FileSystem fs = FileSystem.get(conf);
     FileSystem fs = FileSystem.get(conf);
-    Path testdir = new Path(System.getProperty("test.build.data", "/tmp"))
-        .makeQualified(fs);
+    Path testdir = fs.makeQualified(new Path(
+        System.getProperty("test.build.data", "/tmp")));
 
 
     Path base = new Path(testdir, "/empty");
     Path base = new Path(testdir, "/empty");
     Path[] src = { new Path(base, "i0"), new Path("i1"), new Path("i2") };
     Path[] src = { new Path(base, "i0"), new Path("i1"), new Path("i2") };

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/util/MRAsyncDiskService.java

@@ -330,7 +330,7 @@ public class MRAsyncDiskService {
    * Returns the normalized path of a path.
    * Returns the normalized path of a path.
    */
    */
   private String normalizePath(String path) {
   private String normalizePath(String path) {
-    return (new Path(path)).makeQualified(this.localFileSystem)
+    return this.localFileSystem.makeQualified(new Path(path))
         .toUri().getPath();
         .toUri().getPath();
   }
   }
   
   

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

@@ -128,8 +128,8 @@ public class TestMRJobs {
     }
     }
   }
   }
 
 
-  private static Path TEST_ROOT_DIR = new Path("target",
-      TestMRJobs.class.getName() + "-tmpDir").makeQualified(localFs);
+  private static Path TEST_ROOT_DIR = localFs.makeQualified(
+      new Path("target", TestMRJobs.class.getName() + "-tmpDir"));
   static Path APP_JAR = new Path(TEST_ROOT_DIR, "MRAppJar.jar");
   static Path APP_JAR = new Path(TEST_ROOT_DIR, "MRAppJar.jar");
   private static final String OUTPUT_ROOT_DIR = "/tmp/" +
   private static final String OUTPUT_ROOT_DIR = "/tmp/" +
     TestMRJobs.class.getSimpleName();
     TestMRJobs.class.getSimpleName();

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobsWithHistoryService.java

@@ -73,8 +73,8 @@ public class TestMRJobsWithHistoryService {
     }
     }
   }
   }
 
 
-  private static Path TEST_ROOT_DIR = new Path("target",
-      TestMRJobs.class.getName() + "-tmpDir").makeQualified(localFs);
+  private static Path TEST_ROOT_DIR = localFs.makeQualified(
+      new Path("target", TestMRJobs.class.getName() + "-tmpDir"));
   static Path APP_JAR = new Path(TEST_ROOT_DIR, "MRAppJar.jar");
   static Path APP_JAR = new Path(TEST_ROOT_DIR, "MRAppJar.jar");
 
 
   @Before
   @Before

+ 4 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/FadvisedFileRegion.java

@@ -111,7 +111,10 @@ public class FadvisedFileRegion extends DefaultFileRegion {
     
     
     long trans = actualCount;
     long trans = actualCount;
     int readSize;
     int readSize;
-    ByteBuffer byteBuffer = ByteBuffer.allocate(this.shuffleBufferSize);
+    ByteBuffer byteBuffer = ByteBuffer.allocate(
+        Math.min(
+            this.shuffleBufferSize,
+            trans > Integer.MAX_VALUE ? Integer.MAX_VALUE : (int) trans));
     
     
     while(trans > 0L &&
     while(trans > 0L &&
         (readSize = fileChannel.read(byteBuffer, this.position+position)) > 0) {
         (readSize = fileChannel.read(byteBuffer, this.position+position)) > 0) {

+ 1 - 1
hadoop-tools/hadoop-archives/src/main/java/org/apache/hadoop/tools/HadoopArchives.java

@@ -473,7 +473,7 @@ public class HadoopArchives implements Tool {
     conf.setLong(HAR_BLOCKSIZE_LABEL, blockSize);
     conf.setLong(HAR_BLOCKSIZE_LABEL, blockSize);
     conf.setLong(HAR_PARTSIZE_LABEL, partSize);
     conf.setLong(HAR_PARTSIZE_LABEL, partSize);
     conf.set(DST_HAR_LABEL, archiveName);
     conf.set(DST_HAR_LABEL, archiveName);
-    conf.set(SRC_PARENT_LABEL, parentPath.makeQualified(fs).toString());
+    conf.set(SRC_PARENT_LABEL, fs.makeQualified(parentPath).toString());
     conf.setInt(HAR_REPLICATION_LABEL, repl);
     conf.setInt(HAR_REPLICATION_LABEL, repl);
     Path outputPath = new Path(dest, archiveName);
     Path outputPath = new Path(dest, archiveName);
     FileOutputFormat.setOutputPath(conf, outputPath);
     FileOutputFormat.setOutputPath(conf, outputPath);

+ 1 - 1
hadoop-tools/hadoop-azure-datalake/pom.xml

@@ -110,7 +110,7 @@
     <dependency>
     <dependency>
       <groupId>com.microsoft.azure</groupId>
       <groupId>com.microsoft.azure</groupId>
       <artifactId>azure-data-lake-store-sdk</artifactId>
       <artifactId>azure-data-lake-store-sdk</artifactId>
-      <version>2.1.4</version>
+      <version>2.2.1</version>
     </dependency>
     </dependency>
     <!--  ENDS HERE-->
     <!--  ENDS HERE-->
     <dependency>
     <dependency>

+ 8 - 0
hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlConfKeys.java

@@ -54,6 +54,14 @@ public final class AdlConfKeys {
   public static final String TOKEN_PROVIDER_TYPE_CLIENT_CRED =
   public static final String TOKEN_PROVIDER_TYPE_CLIENT_CRED =
       "ClientCredential";
       "ClientCredential";
 
 
+  // MSI Auth Configuration
+  public static final String MSI_PORT = "fs.adl.oauth2.msi.port";
+  public static final String MSI_TENANT_GUID = "fs.adl.oauth2.msi.tenantguid";
+
+  // DeviceCode Auth configuration
+  public static final String DEVICE_CODE_CLIENT_APP_ID =
+      "fs.adl.oauth2.devicecode.clientapp.id";
+
   public static final String READ_AHEAD_BUFFER_SIZE_KEY =
   public static final String READ_AHEAD_BUFFER_SIZE_KEY =
       "adl.feature.client.cache.readahead";
       "adl.feature.client.cache.readahead";
 
 

+ 21 - 0
hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/AdlFileSystem.java

@@ -34,6 +34,8 @@ import com.microsoft.azure.datalake.store.LatencyTracker;
 import com.microsoft.azure.datalake.store.UserGroupRepresentation;
 import com.microsoft.azure.datalake.store.UserGroupRepresentation;
 import com.microsoft.azure.datalake.store.oauth2.AccessTokenProvider;
 import com.microsoft.azure.datalake.store.oauth2.AccessTokenProvider;
 import com.microsoft.azure.datalake.store.oauth2.ClientCredsTokenProvider;
 import com.microsoft.azure.datalake.store.oauth2.ClientCredsTokenProvider;
+import com.microsoft.azure.datalake.store.oauth2.DeviceCodeTokenProvider;
+import com.microsoft.azure.datalake.store.oauth2.MsiTokenProvider;
 import com.microsoft.azure.datalake.store.oauth2.RefreshTokenBasedTokenProvider;
 import com.microsoft.azure.datalake.store.oauth2.RefreshTokenBasedTokenProvider;
 
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.lang.StringUtils;
@@ -254,6 +256,12 @@ public class AdlFileSystem extends FileSystem {
     case ClientCredential:
     case ClientCredential:
       tokenProvider = getConfCredentialBasedTokenProvider(conf);
       tokenProvider = getConfCredentialBasedTokenProvider(conf);
       break;
       break;
+    case MSI:
+      tokenProvider = getMsiBasedTokenProvider(conf);
+      break;
+    case DeviceCode:
+      tokenProvider = getDeviceCodeTokenProvider(conf);
+      break;
     case Custom:
     case Custom:
     default:
     default:
       AzureADTokenProvider azureADTokenProvider = getCustomAccessTokenProvider(
       AzureADTokenProvider azureADTokenProvider = getCustomAccessTokenProvider(
@@ -280,6 +288,19 @@ public class AdlFileSystem extends FileSystem {
     return new RefreshTokenBasedTokenProvider(clientId, refreshToken);
     return new RefreshTokenBasedTokenProvider(clientId, refreshToken);
   }
   }
 
 
+  private AccessTokenProvider getMsiBasedTokenProvider(
+          Configuration conf) throws IOException {
+    int port = Integer.parseInt(getNonEmptyVal(conf, MSI_PORT));
+    String tenantGuid = getPasswordString(conf, MSI_TENANT_GUID);
+    return new MsiTokenProvider(port, tenantGuid);
+  }
+
+  private AccessTokenProvider getDeviceCodeTokenProvider(
+          Configuration conf) throws IOException {
+    String clientAppId = getNonEmptyVal(conf, DEVICE_CODE_CLIENT_APP_ID);
+    return new DeviceCodeTokenProvider(clientAppId);
+  }
+
   @VisibleForTesting
   @VisibleForTesting
   AccessTokenProvider getTokenProvider() {
   AccessTokenProvider getTokenProvider() {
     return tokenProvider;
     return tokenProvider;

+ 2 - 0
hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/fs/adl/TokenProviderType.java

@@ -21,5 +21,7 @@ package org.apache.hadoop.fs.adl;
 enum TokenProviderType {
 enum TokenProviderType {
   RefreshToken,
   RefreshToken,
   ClientCredential,
   ClientCredential,
+  MSI,
+  DeviceCode,
   Custom
   Custom
 }
 }

+ 90 - 8
hadoop-tools/hadoop-azure-datalake/src/site/markdown/index.md

@@ -111,20 +111,24 @@ service associated with the client id. See [*Active Directory Library For Java*]
 ##### Generating the Service Principal
 ##### Generating the Service Principal
 
 
 1.  Go to [the portal](https://portal.azure.com)
 1.  Go to [the portal](https://portal.azure.com)
-2.  Under "Browse", look for Active Directory and click on it.
-3.  Create "Web Application". Remember the name you create here - that is what you will add to your ADL account as authorized user.
+2.  Under services in left nav, look for Azure Active Directory and click it.
+3.  Using "App Registrations" in the menu, create "Web Application". Remember
+    the name you create here - that is what you will add to your ADL account
+    as authorized user.
 4.  Go through the wizard
 4.  Go through the wizard
-5.  Once app is created, Go to app configuration, and find the section on "keys"
+5.  Once app is created, go to "keys" under "settings" for the app
 6.  Select a key duration and hit save. Save the generated keys.
 6.  Select a key duration and hit save. Save the generated keys.
-7. Note down the properties you will need to auth:
-    -  The client ID
+7.  Go back to the App Registrations page, and click on the "Endpoints" button
+    at the top
+    a. Note down the  "Token Endpoint" URL
+8. Note down the properties you will need to auth:
+    -  The "Application ID" of the Web App you created above
     -  The key you just generated above
     -  The key you just generated above
-    -  The token endpoint (select "View endpoints" at the bottom of the page and copy/paste the OAuth2 .0 Token Endpoint value)
-    -  Resource: Always https://management.core.windows.net/ , for all customers
+    -  The token endpoint
 
 
 ##### Adding the service principal to your ADL Account
 ##### Adding the service principal to your ADL Account
 1.  Go to the portal again, and open your ADL account
 1.  Go to the portal again, and open your ADL account
-2.  Select Users under Settings
+2.  Select `Access control (IAM)`
 3.  Add your user name you created in Step 6 above (note that it does not show up in the list, but will be found if you searched for the name)
 3.  Add your user name you created in Step 6 above (note that it does not show up in the list, but will be found if you searched for the name)
 4.  Add "Owner" role
 4.  Add "Owner" role
 
 
@@ -153,6 +157,84 @@ Add the following properties to your `core-site.xml`
 </property>
 </property>
 ```
 ```
 
 
+#### Using MSI (Managed Service Identity)
+
+Azure VMs can be provisioned with "service identities" that are managed by the
+Identity extension within the VM. The advantage of doing this is that the
+credentials are managed by the extension, and do not have to be put into
+core-site.xml.
+
+To use MSI, the following two steps are needed:
+1. Modify the VM deployment template to specify the port number of the token
+ service exposed to localhost by the identity extension in the VM.
+2. Get your Azure ActiveDirectory Tenant ID:
+   1. Go to [the portal](https://portal.azure.com)
+   2. Under services in left nav, look for Azure Active Directory and click on it.
+   3. Click on Properties
+   4. Note down the GUID shown under "Directory ID" - this is your AAD tenant ID
+
+
+##### Configure core-site.xml
+Add the following properties to your `core-site.xml`
+
+```xml
+<property>
+  <name>fs.adl.oauth2.access.token.provider.type</name>
+  <value>Msi</value>
+</property>
+
+<property>
+  <name>fs.adl.oauth2.msi.port</name>
+  <value>PORT NUMBER FROM STEP 1 ABOVE</value>
+</property>
+
+<property>
+  <name>fs.adl.oauth2.msi.TenantGuid</name>
+  <value>AAD TENANT ID GUID FROM STEP 2 ABOVE</value>
+</property>
+```
+
+### Using Device Code Auth for interactive login
+
+**Note:** This auth method is suitable for running interactive tools, but will
+not work for jobs submitted to a cluster.
+
+To use user-based login, Azure ActiveDirectory provides login flow using
+device code.
+
+To use device code flow, user must first create a **Native** app registration
+in the Azure portal, and provide the client ID for the app as a config. Here
+are the steps:
+
+1.  Go to [the portal](https://portal.azure.com)
+2.  Under services in left nav, look for Azure Active Directory and click on it.
+3.  Using "App Registrations" in the menu, create "Native Application".
+4.  Go through the wizard
+5.  Once app is created, note down the "Appplication ID" of the app
+6. Grant permissions to the app:
+    1. Click on "Permissions" for the app, and then add "Azure Data Lake" and
+       "Windows Azure Service Management API" permissions
+    2. Click on "Grant Permissions" to add the permissions to the app
+
+Add the following properties to your `core-site.xml`
+
+```xml
+<property>
+  <name>fs.adl.oauth2.devicecode.clientappid</name>
+  <value>APP ID FROM STEP 5 ABOVE</value>
+</property>
+```
+
+It is usually not desirable to add DeviceCode as the default token provider
+type. But it can be used when using a local command:
+```
+ hadoop fs -Dfs.adl.oauth2.access.token.provider.type=DeviceCode -ls ...
+```
+Running this will print a URL and device code that can be used to login from
+any browser (even on a different machine, outside of the ssh session). Once
+the login is done, the command continues.
+
+
 #### Protecting the Credentials with Credential Providers
 #### Protecting the Credentials with Credential Providers
 
 
 In many Hadoop clusters, the `core-site.xml` file is world-readable. To protect
 In many Hadoop clusters, the `core-site.xml` file is world-readable. To protect

+ 40 - 0
hadoop-tools/hadoop-azure-datalake/src/test/java/org/apache/hadoop/fs/adl/TestAzureADTokenProvider.java

@@ -23,6 +23,8 @@ import java.io.IOException;
 import java.net.URI;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URISyntaxException;
 
 
+import com.microsoft.azure.datalake.store.oauth2.DeviceCodeTokenProvider;
+import com.microsoft.azure.datalake.store.oauth2.MsiTokenProvider;
 import org.apache.commons.lang.builder.EqualsBuilder;
 import org.apache.commons.lang.builder.EqualsBuilder;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.adl.common.CustomMockTokenProvider;
 import org.apache.hadoop.fs.adl.common.CustomMockTokenProvider;
@@ -40,6 +42,9 @@ import static org.apache.hadoop.fs.adl.AdlConfKeys
     .AZURE_AD_TOKEN_PROVIDER_CLASS_KEY;
     .AZURE_AD_TOKEN_PROVIDER_CLASS_KEY;
 import static org.apache.hadoop.fs.adl.AdlConfKeys
 import static org.apache.hadoop.fs.adl.AdlConfKeys
     .AZURE_AD_TOKEN_PROVIDER_TYPE_KEY;
     .AZURE_AD_TOKEN_PROVIDER_TYPE_KEY;
+import static org.apache.hadoop.fs.adl.AdlConfKeys.DEVICE_CODE_CLIENT_APP_ID;
+import static org.apache.hadoop.fs.adl.AdlConfKeys.MSI_PORT;
+import static org.apache.hadoop.fs.adl.AdlConfKeys.MSI_TENANT_GUID;
 import static org.apache.hadoop.fs.adl.TokenProviderType.*;
 import static org.apache.hadoop.fs.adl.TokenProviderType.*;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertEquals;
 
 
@@ -97,6 +102,41 @@ public class TestAzureADTokenProvider {
     Assert.assertTrue(tokenProvider instanceof ClientCredsTokenProvider);
     Assert.assertTrue(tokenProvider instanceof ClientCredsTokenProvider);
   }
   }
 
 
+  @Test
+  public void testMSITokenProvider()
+          throws IOException, URISyntaxException {
+    Configuration conf = new Configuration();
+    conf.setEnum(AZURE_AD_TOKEN_PROVIDER_TYPE_KEY, MSI);
+    conf.set(MSI_PORT, "54321");
+    conf.set(MSI_TENANT_GUID, "TENANT_GUID");
+
+    URI uri = new URI("adl://localhost:8080");
+    AdlFileSystem fileSystem = new AdlFileSystem();
+    fileSystem.initialize(uri, conf);
+    AccessTokenProvider tokenProvider = fileSystem.getTokenProvider();
+    Assert.assertTrue(tokenProvider instanceof MsiTokenProvider);
+  }
+
+  @Test
+  public void testDeviceCodeTokenProvider()
+          throws IOException, URISyntaxException {
+    boolean runTest = false;
+    if (runTest) {
+      // Device code auth method causes an interactive prompt, so run this only
+      // when running the test interactively at a local terminal. Disabling
+      // test by default, to not break any automation.
+      Configuration conf = new Configuration();
+      conf.setEnum(AZURE_AD_TOKEN_PROVIDER_TYPE_KEY, DeviceCode);
+      conf.set(DEVICE_CODE_CLIENT_APP_ID, "CLIENT_APP_ID_GUID");
+
+      URI uri = new URI("adl://localhost:8080");
+      AdlFileSystem fileSystem = new AdlFileSystem();
+      fileSystem.initialize(uri, conf);
+      AccessTokenProvider tokenProvider = fileSystem.getTokenProvider();
+      Assert.assertTrue(tokenProvider instanceof DeviceCodeTokenProvider);
+    }
+  }
+
   @Test
   @Test
   public void testCustomCredTokenProvider()
   public void testCustomCredTokenProvider()
       throws URISyntaxException, IOException {
       throws URISyntaxException, IOException {

+ 0 - 17
hadoop-tools/hadoop-azure/src/main/resources/META-INF/services/org.apache.hadoop.fs.FileSystem

@@ -1,17 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one or more
-# contributor license agreements.  See the NOTICE file distributed with
-# this work for additional information regarding copyright ownership.
-# The ASF licenses this file to You under the Apache License, Version 2.0
-# (the "License"); you may not use this file except in compliance with
-# the License.  You may obtain a copy of the License at
-#
-#     http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
-org.apache.hadoop.fs.azure.NativeAzureFileSystem
-org.apache.hadoop.fs.azure.NativeAzureFileSystem$Secure

+ 1 - 1
hadoop-tools/hadoop-gridmix/src/main/java/org/apache/hadoop/mapred/gridmix/Gridmix.java

@@ -447,7 +447,7 @@ public class Gridmix extends Configured implements Tool {
 
 
     // Create <ioPath> with 777 permissions
     // Create <ioPath> with 777 permissions
     final FileSystem inputFs = ioPath.getFileSystem(conf);
     final FileSystem inputFs = ioPath.getFileSystem(conf);
-    ioPath = ioPath.makeQualified(inputFs);
+    ioPath = inputFs.makeQualified(ioPath);
     boolean succeeded = false;
     boolean succeeded = false;
     try {
     try {
       succeeded = FileSystem.mkdirs(inputFs, ioPath,
       succeeded = FileSystem.mkdirs(inputFs, ioPath,

+ 7 - 1
hadoop-tools/hadoop-gridmix/src/main/java/org/apache/hadoop/mapred/gridmix/PseudoLocalFs.java

@@ -116,7 +116,7 @@ class PseudoLocalFs extends FileSystem {
    * @throws FileNotFoundException
    * @throws FileNotFoundException
    */
    */
   long validateFileNameFormat(Path path) throws FileNotFoundException {
   long validateFileNameFormat(Path path) throws FileNotFoundException {
-    path = path.makeQualified(this);
+    path = this.makeQualified(path);
     boolean valid = true;
     boolean valid = true;
     long fileSize = 0;
     long fileSize = 0;
     if (!path.toUri().getScheme().equals(getUri().getScheme())) {
     if (!path.toUri().getScheme().equals(getUri().getScheme())) {
@@ -329,4 +329,10 @@ class PseudoLocalFs extends FileSystem {
     throw new UnsupportedOperationException("SetWorkingDirectory "
     throw new UnsupportedOperationException("SetWorkingDirectory "
         + "is not supported in pseudo local file system.");
         + "is not supported in pseudo local file system.");
   }
   }
+
+  @Override
+  public Path makeQualified(Path path) {
+    // skip FileSystem#checkPath() to validate some other Filesystems
+    return path.makeQualified(this.getUri(), this.getWorkingDirectory());
+  }
 }
 }

+ 2 - 2
hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestFilePool.java

@@ -48,8 +48,8 @@ public class TestFilePool {
     try {
     try {
       final Configuration conf = new Configuration();
       final Configuration conf = new Configuration();
       final FileSystem fs = FileSystem.getLocal(conf).getRaw();
       final FileSystem fs = FileSystem.getLocal(conf).getRaw();
-      return new Path(System.getProperty("test.build.data", "/tmp"),
-          "testFilePool").makeQualified(fs);
+      return fs.makeQualified(new Path(
+          System.getProperty("test.build.data", "/tmp"), "testFilePool"));
     } catch (IOException e) {
     } catch (IOException e) {
       fail();
       fail();
     }
     }

+ 4 - 4
hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestFileQueue.java

@@ -48,8 +48,8 @@ public class TestFileQueue {
   public static void setup() throws IOException {
   public static void setup() throws IOException {
     final Configuration conf = new Configuration();
     final Configuration conf = new Configuration();
     final FileSystem fs = FileSystem.getLocal(conf).getRaw();
     final FileSystem fs = FileSystem.getLocal(conf).getRaw();
-    final Path p = new Path(System.getProperty("test.build.data", "/tmp"),
-        "testFileQueue").makeQualified(fs);
+    final Path p = fs.makeQualified(new Path(
+        System.getProperty("test.build.data", "/tmp"), "testFileQueue"));
     fs.delete(p, true);
     fs.delete(p, true);
     final byte[] b = new byte[BLOCK];
     final byte[] b = new byte[BLOCK];
     for (int i = 0; i < NFILES; ++i) {
     for (int i = 0; i < NFILES; ++i) {
@@ -71,8 +71,8 @@ public class TestFileQueue {
   public static void cleanup() throws IOException {
   public static void cleanup() throws IOException {
     final Configuration conf = new Configuration();
     final Configuration conf = new Configuration();
     final FileSystem fs = FileSystem.getLocal(conf).getRaw();
     final FileSystem fs = FileSystem.getLocal(conf).getRaw();
-    final Path p = new Path(System.getProperty("test.build.data", "/tmp"),
-        "testFileQueue").makeQualified(fs);
+    final Path p = fs.makeQualified(new Path(
+        System.getProperty("test.build.data", "/tmp"), "testFileQueue"));
     fs.delete(p, true);
     fs.delete(p, true);
   }
   }
 
 

+ 1 - 1
hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestPseudoLocalFs.java

@@ -224,7 +224,7 @@ public class TestPseudoLocalFs {
 
 
     // Validate operations on valid qualified path
     // Validate operations on valid qualified path
     path = new Path("myPsedoFile.1237");
     path = new Path("myPsedoFile.1237");
-    path = path.makeQualified(pfs);
+    path = pfs.makeQualified(path);
     validateGetFileStatus(pfs, path, true);
     validateGetFileStatus(pfs, path, true);
     validateCreate(pfs, path, true);
     validateCreate(pfs, path, true);
     validateOpen(pfs, path, true);
     validateOpen(pfs, path, true);

+ 2 - 2
hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/TestUserResolve.java

@@ -40,8 +40,8 @@ public class TestUserResolve {
   public static void createRootDir() throws IOException {
   public static void createRootDir() throws IOException {
     conf = new Configuration();
     conf = new Configuration();
     fs = FileSystem.getLocal(conf);
     fs = FileSystem.getLocal(conf);
-    rootDir = new Path(new Path(System.getProperty("test.build.data", "/tmp"))
-                 .makeQualified(fs), "gridmixUserResolve");
+    rootDir = new Path(fs.makeQualified(new Path(
+        System.getProperty("test.build.data", "/tmp"))), "gridmixUserResolve");
   }
   }
 
 
   /**
   /**

+ 1 - 1
hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/util/SwiftTestUtils.java

@@ -278,7 +278,7 @@ public class SwiftTestUtils extends org.junit.Assert {
     noteAction(action);
     noteAction(action);
     try {
     try {
       if (fileSystem != null) {
       if (fileSystem != null) {
-        fileSystem.delete(new Path(cleanupPath).makeQualified(fileSystem),
+        fileSystem.delete(fileSystem.makeQualified(new Path(cleanupPath)),
                           true);
                           true);
       }
       }
     } catch (Exception e) {
     } catch (Exception e) {

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

@@ -159,7 +159,7 @@ public class SwiftFileSystemBaseTest extends Assert implements
    * @return a qualified path instance
    * @return a qualified path instance
    */
    */
   protected Path path(String pathString) {
   protected Path path(String pathString) {
-    return new Path(pathString).makeQualified(fs);
+    return fs.makeQualified(new Path(pathString));
   }
   }
 
 
   /**
   /**

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

@@ -126,7 +126,7 @@ public class TestSwiftFileSystemPartitionedUploads extends
       SwiftTestUtils.compareByteArrays(src, dest, len);
       SwiftTestUtils.compareByteArrays(src, dest, len);
       FileStatus status;
       FileStatus status;
 
 
-      final Path qualifiedPath = path.makeQualified(fs);
+      final Path qualifiedPath = fs.makeQualified(path);
       status = fs.getFileStatus(qualifiedPath);
       status = fs.getFileStatus(qualifiedPath);
       //now see what block location info comes back.
       //now see what block location info comes back.
       //This will vary depending on the Swift version, so the results
       //This will vary depending on the Swift version, so the results
@@ -216,7 +216,7 @@ public class TestSwiftFileSystemPartitionedUploads extends
 
 
   private FileStatus validatePathLen(Path path, int len) throws IOException {
   private FileStatus validatePathLen(Path path, int len) throws IOException {
     //verify that the length is what was written in a direct status check
     //verify that the length is what was written in a direct status check
-    final Path qualifiedPath = path.makeQualified(fs);
+    final Path qualifiedPath = fs.makeQualified(path);
     FileStatus[] parentDirListing = fs.listStatus(qualifiedPath.getParent());
     FileStatus[] parentDirListing = fs.listStatus(qualifiedPath.getParent());
     StringBuilder listing = lsToString(parentDirListing);
     StringBuilder listing = lsToString(parentDirListing);
     String parentDirLS = listing.toString();
     String parentDirLS = listing.toString();

+ 3 - 3
hadoop-tools/hadoop-rumen/src/test/java/org/apache/hadoop/tools/rumen/TestHistograms.java

@@ -57,8 +57,8 @@ public class TestHistograms {
   public void testHistograms() throws IOException {
   public void testHistograms() throws IOException {
     final Configuration conf = new Configuration();
     final Configuration conf = new Configuration();
     final FileSystem lfs = FileSystem.getLocal(conf);
     final FileSystem lfs = FileSystem.getLocal(conf);
-    final Path rootInputDir = new Path(
-        System.getProperty("test.tools.input.dir", "")).makeQualified(lfs);
+    final Path rootInputDir = lfs.makeQualified(new Path(
+        System.getProperty("test.tools.input.dir", "target/input")));
     final Path rootInputFile = new Path(rootInputDir, "rumen/histogram-tests");
     final Path rootInputFile = new Path(rootInputDir, "rumen/histogram-tests");
 
 
 
 
@@ -132,7 +132,7 @@ public class TestHistograms {
     final FileSystem lfs = FileSystem.getLocal(conf);
     final FileSystem lfs = FileSystem.getLocal(conf);
 
 
     for (String arg : args) {
     for (String arg : args) {
-      Path filePath = new Path(arg).makeQualified(lfs);
+      Path filePath = lfs.makeQualified(new Path(arg));
       String fileName = filePath.getName();
       String fileName = filePath.getName();
       if (fileName.startsWith("input")) {
       if (fileName.startsWith("input")) {
         LoggedDiscreteCDF newResult = histogramFileToCDF(filePath, lfs);
         LoggedDiscreteCDF newResult = histogramFileToCDF(filePath, lfs);

+ 13 - 14
hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/StreamJob.java

@@ -22,13 +22,11 @@ import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.IOException;
 import java.net.URI;
 import java.net.URI;
-import java.net.URISyntaxException;
 import java.net.URLEncoder;
 import java.net.URLEncoder;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Arrays;
 import java.util.List;
 import java.util.List;
 import java.util.Map;
 import java.util.Map;
-import java.util.regex.Pattern;
 import java.util.TreeMap;
 import java.util.TreeMap;
 import java.util.TreeSet;
 import java.util.TreeSet;
 
 
@@ -41,12 +39,12 @@ import org.apache.commons.cli.Options;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.mapreduce.MRConfig;
 import org.apache.hadoop.mapreduce.MRConfig;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.filecache.DistributedCache;
 import org.apache.hadoop.mapreduce.filecache.DistributedCache;
 import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
 import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.mapred.FileInputFormat;
 import org.apache.hadoop.mapred.FileInputFormat;
@@ -56,7 +54,6 @@ import org.apache.hadoop.mapred.JobClient;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.JobID;
 import org.apache.hadoop.mapred.JobID;
 import org.apache.hadoop.mapred.KeyValueTextInputFormat;
 import org.apache.hadoop.mapred.KeyValueTextInputFormat;
-import org.apache.hadoop.mapred.OutputFormat;
 import org.apache.hadoop.mapred.RunningJob;
 import org.apache.hadoop.mapred.RunningJob;
 import org.apache.hadoop.mapred.SequenceFileAsTextInputFormat;
 import org.apache.hadoop.mapred.SequenceFileAsTextInputFormat;
 import org.apache.hadoop.mapred.SequenceFileInputFormat;
 import org.apache.hadoop.mapred.SequenceFileInputFormat;
@@ -65,6 +62,7 @@ import org.apache.hadoop.mapred.TextOutputFormat;
 import org.apache.hadoop.mapred.lib.LazyOutputFormat;
 import org.apache.hadoop.mapred.lib.LazyOutputFormat;
 import org.apache.hadoop.mapred.lib.aggregate.ValueAggregatorCombiner;
 import org.apache.hadoop.mapred.lib.aggregate.ValueAggregatorCombiner;
 import org.apache.hadoop.mapred.lib.aggregate.ValueAggregatorReducer;
 import org.apache.hadoop.mapred.lib.aggregate.ValueAggregatorReducer;
+import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.streaming.io.IdentifierResolver;
 import org.apache.hadoop.streaming.io.IdentifierResolver;
 import org.apache.hadoop.streaming.io.InputWriter;
 import org.apache.hadoop.streaming.io.InputWriter;
 import org.apache.hadoop.streaming.io.OutputReader;
 import org.apache.hadoop.streaming.io.OutputReader;
@@ -297,7 +295,10 @@ public class StreamJob implements Tool {
           try {
           try {
             Path path = new Path(file);
             Path path = new Path(file);
             FileSystem localFs = FileSystem.getLocal(config_);
             FileSystem localFs = FileSystem.getLocal(config_);
-            String finalPath = path.makeQualified(localFs).toString();
+            Path qualifiedPath = path.makeQualified(
+                localFs.getUri(), localFs.getWorkingDirectory());
+            validate(qualifiedPath);
+            String finalPath = qualifiedPath.toString();
             if(fileList.length() > 0) {
             if(fileList.length() > 0) {
               fileList.append(',');
               fileList.append(',');
             }
             }
@@ -313,7 +314,6 @@ public class StreamJob implements Tool {
           tmpFiles = tmpFiles + "," + fileList;
           tmpFiles = tmpFiles + "," + fileList;
         }
         }
         config_.set("tmpfiles", tmpFiles);
         config_.set("tmpfiles", tmpFiles);
-        validate(packageFiles_);
       }
       }
 
 
       String fsName = cmdLine.getOptionValue("dfs");
       String fsName = cmdLine.getOptionValue("dfs");
@@ -391,14 +391,13 @@ public class StreamJob implements Tool {
     return OptionBuilder.withDescription(desc).create(name);
     return OptionBuilder.withDescription(desc).create(name);
   }
   }
 
 
-  private void validate(final List<String> values)
-  throws IllegalArgumentException {
-    for (String file : values) {
-      File f = new File(file);
-      if (!FileUtil.canRead(f)) {
-        fail("File: " + f.getAbsolutePath()
-          + " does not exist, or is not readable.");
-      }
+  private void validate(final Path path) throws IOException {
+    try {
+      path.getFileSystem(config_).access(path, FsAction.READ);
+    } catch (FileNotFoundException e) {
+      fail("File: " + path + " does not exist.");
+    } catch (AccessControlException e) {
+      fail("File: " + path + " is not readable.");
     }
     }
   }
   }
 
 

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

@@ -29,6 +29,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.ha.ActiveStandbyElector;
 import org.apache.hadoop.ha.ActiveStandbyElector;
 import org.apache.hadoop.http.HttpConfig;
 import org.apache.hadoop.http.HttpConfig;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetUtils;
@@ -87,7 +88,17 @@ public class YarnConfiguration extends Configuration {
     });
     });
     Configuration.addDeprecations(new DeprecationDelta[] {
     Configuration.addDeprecations(new DeprecationDelta[] {
         new DeprecationDelta(RM_SYSTEM_METRICS_PUBLISHER_ENABLED,
         new DeprecationDelta(RM_SYSTEM_METRICS_PUBLISHER_ENABLED,
-            SYSTEM_METRICS_PUBLISHER_ENABLED)
+            SYSTEM_METRICS_PUBLISHER_ENABLED),
+        new DeprecationDelta(RM_ZK_ACL, CommonConfigurationKeys.ZK_ACL),
+        new DeprecationDelta(RM_ZK_AUTH, CommonConfigurationKeys.ZK_AUTH),
+        new DeprecationDelta(RM_ZK_ADDRESS,
+            CommonConfigurationKeys.ZK_ADDRESS),
+        new DeprecationDelta(RM_ZK_NUM_RETRIES,
+            CommonConfigurationKeys.ZK_NUM_RETRIES),
+        new DeprecationDelta(RM_ZK_TIMEOUT_MS,
+            CommonConfigurationKeys.ZK_TIMEOUT_MS),
+        new DeprecationDelta(RM_ZK_RETRY_INTERVAL_MS,
+            CommonConfigurationKeys.ZK_RETRY_INTERVAL_MS),
     });
     });
   }
   }
 
 
@@ -2710,6 +2721,16 @@ public class YarnConfiguration extends Configuration {
       "org.apache.hadoop.yarn.server.router.webapp."
       "org.apache.hadoop.yarn.server.router.webapp."
           + "DefaultRequestInterceptorREST";
           + "DefaultRequestInterceptorREST";
 
 
+  /**
+   * The interceptor class used in FederationInterceptorREST to communicate with
+   * each SubCluster.
+   */
+  public static final String ROUTER_WEBAPP_DEFAULT_INTERCEPTOR_CLASS =
+      ROUTER_WEBAPP_PREFIX + "default-interceptor-class";
+  public static final String DEFAULT_ROUTER_WEBAPP_DEFAULT_INTERCEPTOR_CLASS =
+      "org.apache.hadoop.yarn.server.router.webapp."
+          + "DefaultRequestInterceptorREST";
+
   ////////////////////////////////
   ////////////////////////////////
   // Other Configs
   // Other Configs
   ////////////////////////////////
   ////////////////////////////////

+ 11 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java

@@ -81,6 +81,8 @@ public class TestYarnConfigurationFields extends TestConfigurationFieldsBase {
         .add(YarnConfiguration.ROUTER_CLIENTRM_ADDRESS);
         .add(YarnConfiguration.ROUTER_CLIENTRM_ADDRESS);
     configurationPropsToSkipCompare
     configurationPropsToSkipCompare
         .add(YarnConfiguration.ROUTER_RMADMIN_ADDRESS);
         .add(YarnConfiguration.ROUTER_RMADMIN_ADDRESS);
+    configurationPropsToSkipCompare
+        .add(YarnConfiguration.ROUTER_WEBAPP_DEFAULT_INTERCEPTOR_CLASS);
 
 
     // Federation policies configs to be ignored
     // Federation policies configs to be ignored
     configurationPropsToSkipCompare
     configurationPropsToSkipCompare
@@ -121,6 +123,15 @@ public class TestYarnConfigurationFields extends TestConfigurationFieldsBase {
     configurationPropsToSkipCompare
     configurationPropsToSkipCompare
         .add(YarnConfiguration.RM_SYSTEM_METRICS_PUBLISHER_ENABLED);
         .add(YarnConfiguration.RM_SYSTEM_METRICS_PUBLISHER_ENABLED);
 
 
+    // skip deprecated ZooKeeper settings
+    configurationPropsToSkipCompare.add(YarnConfiguration.RM_ZK_ADDRESS);
+    configurationPropsToSkipCompare.add(YarnConfiguration.RM_ZK_NUM_RETRIES);
+    configurationPropsToSkipCompare.add(YarnConfiguration.RM_ZK_TIMEOUT_MS);
+    configurationPropsToSkipCompare.add(
+        YarnConfiguration.RM_ZK_RETRY_INTERVAL_MS);
+    configurationPropsToSkipCompare.add(YarnConfiguration.RM_ZK_AUTH);
+    configurationPropsToSkipCompare.add(YarnConfiguration.RM_ZK_ACL);
+
     // Used as Java command line properties, not XML
     // Used as Java command line properties, not XML
     configurationPrefixToSkipCompare.add("yarn.app.container");
     configurationPrefixToSkipCompare.add("yarn.app.container");
 
 

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

@@ -451,31 +451,6 @@
     <value>${yarn.resourcemanager.max-completed-applications}</value>
     <value>${yarn.resourcemanager.max-completed-applications}</value>
   </property>
   </property>
 
 
-  <property>
-    <description>Host:Port of the ZooKeeper server to be used by the RM. This
-      must be supplied when using the ZooKeeper based implementation of the
-      RM state store and/or embedded automatic failover in an HA setting.
-    </description>
-    <name>yarn.resourcemanager.zk-address</name>
-    <!--value>127.0.0.1:2181</value-->
-  </property>
-
-  <property>
-    <description>Number of times RM tries to connect to ZooKeeper.</description>
-    <name>yarn.resourcemanager.zk-num-retries</name>
-    <value>1000</value>
-  </property>
-
-  <property>
-    <description>Retry interval in milliseconds when connecting to ZooKeeper.
-      When HA is enabled, the value here is NOT used. It is generated
-      automatically from yarn.resourcemanager.zk-timeout-ms and
-      yarn.resourcemanager.zk-num-retries.
-    </description>
-    <name>yarn.resourcemanager.zk-retry-interval-ms</name>
-    <value>1000</value>
-  </property>
-
   <property>
   <property>
     <description>Full path of the ZooKeeper znode where RM state will be
     <description>Full path of the ZooKeeper znode where RM state will be
     stored. This must be supplied when using
     stored. This must be supplied when using
@@ -485,22 +460,6 @@
     <value>/rmstore</value>
     <value>/rmstore</value>
   </property>
   </property>
 
 
-  <property>
-    <description>ZooKeeper session timeout in milliseconds. Session expiration
-    is managed by the ZooKeeper cluster itself, not by the client. This value is
-    used by the cluster to determine when the client's session expires.
-    Expirations happens when the cluster does not hear from the client within
-    the specified session timeout period (i.e. no heartbeat).</description>
-    <name>yarn.resourcemanager.zk-timeout-ms</name>
-    <value>10000</value>
-  </property>
-
-  <property>
-    <description>ACL's to be used for ZooKeeper znodes.</description>
-    <name>yarn.resourcemanager.zk-acl</name>
-    <value>world:anyone:rwcda</value>
-  </property>
-
   <property>
   <property>
     <description>
     <description>
       ACLs to be used for the root znode when using ZKRMStateStore in an HA
       ACLs to be used for the root znode when using ZKRMStateStore in an HA
@@ -526,18 +485,6 @@
     <name>yarn.resourcemanager.zk-state-store.root-node.acl</name>
     <name>yarn.resourcemanager.zk-state-store.root-node.acl</name>
   </property>
   </property>
 
 
-  <property>
-    <description>
-        Specify the auths to be used for the ACL's specified in both the
-        yarn.resourcemanager.zk-acl and
-        yarn.resourcemanager.zk-state-store.root-node.acl properties.  This
-        takes a comma-separated list of authentication mechanisms, each of the
-        form 'scheme:auth' (the same syntax used for the 'addAuth' command in
-        the ZK CLI).
-    </description>
-    <name>yarn.resourcemanager.zk-auth</name>
-  </property>
-
   <property>
   <property>
     <description>URI pointing to the location of the FileSystem path where
     <description>URI pointing to the location of the FileSystem path where
     RM state will be stored. This must be supplied when using
     RM state will be stored. This must be supplied when using

+ 38 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml

@@ -215,6 +215,44 @@
                   <results>${project.build.directory}/native-results</results>
                   <results>${project.build.directory}/native-results</results>
                 </configuration>
                 </configuration>
               </execution>
               </execution>
+              <execution>
+                <id>cetest</id>
+                <goals><goal>cmake-test</goal></goals>
+                <phase>test</phase>
+                <configuration>
+                  <!-- this should match the xml name without the TEST-part down below -->
+                  <testName>cetest</testName>
+                  <workingDirectory>${project.build.directory}/native/test</workingDirectory>
+                  <source>${basedir}/src</source>
+                  <binary>${project.build.directory}/native/test/cetest</binary>
+                  <args>
+                    <arg>--gtest_filter=-Perf.</arg>
+                    <arg>--gtest_output=xml:${project.build.directory}/surefire-reports/TEST-cetest.xml</arg>
+                  </args>
+                  <results>${project.build.directory}/surefire-reports</results>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-antrun-plugin</artifactId>
+            <executions>
+              <execution>
+                <id>make</id>
+                <phase>compile</phase>
+                <goals>
+                  <goal>run</goal>
+                </goals>
+                <configuration>
+                  <target>
+                    <copy todir="${project.build.directory}/native/test/"
+                      overwrite="true">
+                      <fileset dir="${basedir}/src/main/native/container-executor/resources/test" />
+                    </copy>
+                  </target>
+                </configuration>
+              </execution>
             </executions>
             </executions>
           </plugin>
           </plugin>
         </plugins>
         </plugins>

+ 22 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/CMakeLists.txt

@@ -19,6 +19,9 @@ cmake_minimum_required(VERSION 2.6 FATAL_ERROR)
 list(APPEND CMAKE_MODULE_PATH ${CMAKE_SOURCE_DIR}/../../../../../hadoop-common-project/hadoop-common)
 list(APPEND CMAKE_MODULE_PATH ${CMAKE_SOURCE_DIR}/../../../../../hadoop-common-project/hadoop-common)
 include(HadoopCommon)
 include(HadoopCommon)
 
 
+# Set gtest path
+set(GTEST_SRC_DIR ${CMAKE_SOURCE_DIR}/../../../../../hadoop-common-project/hadoop-common/src/main/native/gtest)
+
 # determine if container-executor.conf.dir is an absolute
 # determine if container-executor.conf.dir is an absolute
 # path in case the OS we're compiling on doesn't have
 # path in case the OS we're compiling on doesn't have
 # a hook in get_executable. We'll use this define
 # a hook in get_executable. We'll use this define
@@ -80,12 +83,20 @@ endfunction()
 include_directories(
 include_directories(
     ${CMAKE_CURRENT_SOURCE_DIR}
     ${CMAKE_CURRENT_SOURCE_DIR}
     ${CMAKE_BINARY_DIR}
     ${CMAKE_BINARY_DIR}
+    ${GTEST_SRC_DIR}/include
     main/native/container-executor
     main/native/container-executor
     main/native/container-executor/impl
     main/native/container-executor/impl
 )
 )
+# add gtest as system library to suppress gcc warnings
+include_directories(SYSTEM ${GTEST_SRC_DIR}/include)
+
 configure_file(${CMAKE_SOURCE_DIR}/config.h.cmake ${CMAKE_BINARY_DIR}/config.h)
 configure_file(${CMAKE_SOURCE_DIR}/config.h.cmake ${CMAKE_BINARY_DIR}/config.h)
 
 
+add_library(gtest ${GTEST_SRC_DIR}/gtest-all.cc)
+set_target_properties(gtest PROPERTIES COMPILE_FLAGS "-w")
+
 add_library(container
 add_library(container
+    main/native/container-executor/impl/util.c
     main/native/container-executor/impl/configuration.c
     main/native/container-executor/impl/configuration.c
     main/native/container-executor/impl/container-executor.c
     main/native/container-executor/impl/container-executor.c
     main/native/container-executor/impl/get_executable.c
     main/native/container-executor/impl/get_executable.c
@@ -95,9 +106,11 @@ add_library(container
 add_executable(container-executor
 add_executable(container-executor
     main/native/container-executor/impl/main.c
     main/native/container-executor/impl/main.c
 )
 )
+
 target_link_libraries(container-executor
 target_link_libraries(container-executor
     container
     container
 )
 )
+
 output_directory(container-executor target/usr/local/bin)
 output_directory(container-executor target/usr/local/bin)
 
 
 add_executable(test-container-executor
 add_executable(test-container-executor
@@ -107,3 +120,12 @@ target_link_libraries(test-container-executor
     container ${EXTRA_LIBS}
     container ${EXTRA_LIBS}
 )
 )
 output_directory(test-container-executor target/usr/local/bin)
 output_directory(test-container-executor target/usr/local/bin)
+
+# unit tests for container executor
+add_executable(cetest
+        main/native/container-executor/impl/util.c
+        main/native/container-executor/test/test_configuration.cc
+        main/native/container-executor/test/test_main.cc
+        main/native/container-executor/test/test_util.cc)
+target_link_libraries(cetest gtest)
+output_directory(cetest test)

+ 476 - 196
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/configuration.c

@@ -20,35 +20,55 @@
 #include <libgen.h>
 #include <libgen.h>
 
 
 #include "configuration.h"
 #include "configuration.h"
-#include "container-executor.h"
+#include "util.h"
 
 
+#define __STDC_FORMAT_MACROS
 #include <inttypes.h>
 #include <inttypes.h>
 #include <errno.h>
 #include <errno.h>
 #include <unistd.h>
 #include <unistd.h>
-#include <stdio.h>
 #include <stdlib.h>
 #include <stdlib.h>
 #include <string.h>
 #include <string.h>
 #include <sys/stat.h>
 #include <sys/stat.h>
-#include <sys/types.h>
-#include <limits.h>
-#include <ctype.h>
 
 
 #define MAX_SIZE 10
 #define MAX_SIZE 10
 
 
+static const char COMMENT_BEGIN_CHAR = '#';
+static const char SECTION_LINE_BEGIN_CHAR = '[';
+static const char SECTION_LINE_END_CHAR = ']';
+
+//clean up method for freeing section
+void free_section(struct section *section) {
+  int i = 0;
+  for (i = 0; i < section->size; i++) {
+    if (section->kv_pairs[i]->key != NULL) {
+      free((void *) section->kv_pairs[i]->key);
+    }
+    if (section->kv_pairs[i]->value != NULL) {
+      free((void *) section->kv_pairs[i]->value);
+    }
+    free(section->kv_pairs[i]);
+  }
+  if (section->kv_pairs) {
+    free(section->kv_pairs);
+    section->kv_pairs = NULL;
+  }
+  if (section->name) {
+    free(section->name);
+    section->name = NULL;
+  }
+  section->size = 0;
+}
+
 //clean up method for freeing configuration
 //clean up method for freeing configuration
-void free_configurations(struct configuration *cfg) {
+void free_configuration(struct configuration *cfg) {
   int i = 0;
   int i = 0;
   for (i = 0; i < cfg->size; i++) {
   for (i = 0; i < cfg->size; i++) {
-    if (cfg->confdetails[i]->key != NULL) {
-      free((void *)cfg->confdetails[i]->key);
+    if (cfg->sections[i] != NULL) {
+      free_section(cfg->sections[i]);
     }
     }
-    if (cfg->confdetails[i]->value != NULL) {
-      free((void *)cfg->confdetails[i]->value);
-    }
-    free(cfg->confdetails[i]);
   }
   }
-  if (cfg->size > 0) {
-    free(cfg->confdetails);
+  if (cfg->sections) {
+    free(cfg->sections);
   }
   }
   cfg->size = 0;
   cfg->size = 0;
 }
 }
@@ -65,13 +85,13 @@ static int is_only_root_writable(const char *file) {
   }
   }
   if (file_stat.st_uid != 0) {
   if (file_stat.st_uid != 0) {
     fprintf(ERRORFILE, "File %s must be owned by root, but is owned by %" PRId64 "\n",
     fprintf(ERRORFILE, "File %s must be owned by root, but is owned by %" PRId64 "\n",
-            file, (int64_t)file_stat.st_uid);
+            file, (int64_t) file_stat.st_uid);
     return 0;
     return 0;
   }
   }
   if ((file_stat.st_mode & (S_IWGRP | S_IWOTH)) != 0) {
   if ((file_stat.st_mode & (S_IWGRP | S_IWOTH)) != 0) {
     fprintf(ERRORFILE,
     fprintf(ERRORFILE,
-	    "File %s must not be world or group writable, but is %03lo\n",
-	    file, (unsigned long)file_stat.st_mode & (~S_IFMT));
+            "File %s must not be world or group writable, but is %03lo\n",
+            file, (unsigned long) file_stat.st_mode & (~S_IFMT));
     return 0;
     return 0;
   }
   }
   return 1;
   return 1;
@@ -82,9 +102,9 @@ static int is_only_root_writable(const char *file) {
  *
  *
  * NOTE: relative path names are resolved relative to the second argument not getwd(3)
  * NOTE: relative path names are resolved relative to the second argument not getwd(3)
  */
  */
-char *resolve_config_path(const char* file_name, const char *root) {
+char *resolve_config_path(const char *file_name, const char *root) {
   const char *real_fname = NULL;
   const char *real_fname = NULL;
-  char buffer[EXECUTOR_PATH_MAX*2 + 1];
+  char buffer[EXECUTOR_PATH_MAX * 2 + 1];
 
 
   if (file_name[0] == '/') {
   if (file_name[0] == '/') {
     real_fname = file_name;
     real_fname = file_name;
@@ -96,7 +116,7 @@ char *resolve_config_path(const char* file_name, const char *root) {
 #ifdef HAVE_CANONICALIZE_FILE_NAME
 #ifdef HAVE_CANONICALIZE_FILE_NAME
   char * ret = (real_fname == NULL) ? NULL : canonicalize_file_name(real_fname);
   char * ret = (real_fname == NULL) ? NULL : canonicalize_file_name(real_fname);
 #else
 #else
-  char * ret = (real_fname == NULL) ? NULL : realpath(real_fname, NULL);
+  char *ret = (real_fname == NULL) ? NULL : realpath(real_fname, NULL);
 #endif
 #endif
 #ifdef DEBUG
 #ifdef DEBUG
   fprintf(stderr,"ret = %s\n", ret);
   fprintf(stderr,"ret = %s\n", ret);
@@ -112,10 +132,19 @@ char *resolve_config_path(const char* file_name, const char *root) {
  * configuration and potentially cause damage.
  * configuration and potentially cause damage.
  * returns 0 if permissions are ok
  * returns 0 if permissions are ok
  */
  */
-int check_configuration_permissions(const char* file_name) {
+int check_configuration_permissions(const char *file_name) {
+  if (!file_name) {
+    return -1;
+  }
+
   // copy the input so that we can modify it with dirname
   // copy the input so that we can modify it with dirname
-  char* dir = strdup(file_name);
-  char* buffer = dir;
+  char *dir = strdup(file_name);
+  if (!dir) {
+    fprintf(stderr, "Failed to make a copy of filename in %s.\n", __func__);
+    return -1;
+  }
+
+  char *buffer = dir;
   do {
   do {
     if (!is_only_root_writable(dir)) {
     if (!is_only_root_writable(dir)) {
       free(buffer);
       free(buffer);
@@ -128,167 +157,396 @@ int check_configuration_permissions(const char* file_name) {
 }
 }
 
 
 /**
 /**
- * Trim whitespace from beginning and end.
-*/
-char* trim(char* input)
-{
-    char *val_begin;
-    char *val_end;
-    char *ret;
-
-    if (input == NULL) {
-      return NULL;
+ * Read a line from the the config file and return it without the newline.
+ * The caller must free the memory allocated.
+ */
+static char *read_config_line(FILE *conf_file) {
+  char *line = NULL;
+  size_t linesize = 100000;
+  ssize_t size_read = 0;
+  size_t eol = 0;
+
+  line = (char *) malloc(linesize);
+  if (line == NULL) {
+    fprintf(ERRORFILE, "malloc failed while reading configuration file.\n");
+    exit(OUT_OF_MEMORY);
+  }
+  size_read = getline(&line, &linesize, conf_file);
+
+  //feof returns true only after we read past EOF.
+  //so a file with no new line, at last can reach this place
+  //if size_read returns negative check for eof condition
+  if (size_read == -1) {
+    free(line);
+    line = NULL;
+    if (!feof(conf_file)) {
+      fprintf(ERRORFILE, "Line read returned -1 without eof\n");
+      exit(INVALID_CONFIG_FILE);
+    }
+  } else {
+    eol = strlen(line) - 1;
+    if (line[eol] == '\n') {
+      //trim the ending new line
+      line[eol] = '\0';
     }
     }
+  }
+  return line;
+}
 
 
-    val_begin = input;
-    val_end = input + strlen(input);
+/**
+ * Return if the given line is a comment line.
+ *
+ * @param line the line to check
+ *
+ * @return 1 if the line is a comment line, 0 otherwise
+ */
+static int is_comment_line(const char *line) {
+  if (line != NULL) {
+    return (line[0] == COMMENT_BEGIN_CHAR);
+  }
+  return 0;
+}
 
 
-    while (val_begin < val_end && isspace(*val_begin))
-      val_begin++;
-    while (val_end > val_begin && isspace(*(val_end - 1)))
-      val_end--;
+/**
+ * Return if the given line is a section start line.
+ *
+ * @param line the line to check
+ *
+ * @return 1 if the line is a section start line, 0 otherwise
+ */
+static int is_section_start_line(const char *line) {
+  size_t len = 0;
+  if (line != NULL) {
+    len = strlen(line) - 1;
+    return (line[0] == SECTION_LINE_BEGIN_CHAR
+            && line[len] == SECTION_LINE_END_CHAR);
+  }
+  return 0;
+}
 
 
-    ret = (char *) malloc(
-            sizeof(char) * (val_end - val_begin + 1));
-    if (ret == NULL) {
-      fprintf(ERRORFILE, "Allocation error\n");
+/**
+ * Return the name of the section from the given section start line. The
+ * caller must free the memory used.
+ *
+ * @param line the line to extract the section name from
+ *
+ * @return string with the name of the section, NULL otherwise
+ */
+static char *get_section_name(const char *line) {
+  char *name = NULL;
+  size_t len;
+
+  if (is_section_start_line(line)) {
+    // length of the name is the line - 2(to account for '[' and ']')
+    len = strlen(line) - 2;
+    name = (char *) malloc(len + 1);
+    if (name == NULL) {
+      fprintf(ERRORFILE, "malloc failed while reading section name.\n");
       exit(OUT_OF_MEMORY);
       exit(OUT_OF_MEMORY);
     }
     }
-
-    strncpy(ret, val_begin, val_end - val_begin);
-    ret[val_end - val_begin] = '\0';
-    return ret;
+    strncpy(name, line + sizeof(char), len);
+    name[len] = '\0';
+  }
+  return name;
 }
 }
 
 
-void read_config(const char* file_name, struct configuration *cfg) {
-  FILE *conf_file;
-  char *line;
+/**
+ * Read an entry for the section from the line. Function returns 0 if an entry
+ * was found, non-zero otherwise. Return values less than 0 indicate an error
+ * with the config file.
+ *
+ * @param line the line to read the entry from
+ * @param section the struct to read the entry into
+ *
+ * @return 0 if an entry was found
+ *         <0 for config file errors
+ *         >0 for issues such as empty line
+ *
+ */
+static int read_section_entry(const char *line, struct section *section) {
   char *equaltok;
   char *equaltok;
   char *temp_equaltok;
   char *temp_equaltok;
-  size_t linesize = 1000;
-  int size_read = 0;
-
-  if (file_name == NULL) {
-    fprintf(ERRORFILE, "Null configuration filename passed in\n");
-    exit(INVALID_CONFIG_FILE);
+  const char *splitter = "=";
+  char *buffer;
+  size_t len = 0;
+  if (line == NULL || section == NULL) {
+    fprintf(ERRORFILE, "NULL params passed to read_section_entry");
+    return -1;
+  }
+  len = strlen(line);
+  if (len == 0) {
+    return 1;
+  }
+  if ((section->size) % MAX_SIZE == 0) {
+    section->kv_pairs = (struct kv_pair **) realloc(
+        section->kv_pairs,
+        sizeof(struct kv_pair *) * (MAX_SIZE + section->size));
+    if (section->kv_pairs == NULL) {
+      fprintf(ERRORFILE,
+              "Failed re-allocating memory for configuration items\n");
+      exit(OUT_OF_MEMORY);
+    }
   }
   }
 
 
-  #ifdef DEBUG
-    fprintf(LOGFILE, "read_config :Conf file name is : %s \n", file_name);
-  #endif
+  buffer = strdup(line);
+  if (!buffer) {
+    fprintf(ERRORFILE, "Failed to allocating memory for line, %s\n", __func__);
+    exit(OUT_OF_MEMORY);
+  }
 
 
-  //allocate space for ten configuration items.
-  cfg->confdetails = (struct confentry **) malloc(sizeof(struct confentry *)
-      * MAX_SIZE);
-  cfg->size = 0;
-  conf_file = fopen(file_name, "r");
-  if (conf_file == NULL) {
-    fprintf(ERRORFILE, "Invalid conf file provided : %s \n", file_name);
+  //tokenize first to get key and list of values.
+  //if no equals is found ignore this line, can be an empty line also
+  equaltok = strtok_r(buffer, splitter, &temp_equaltok);
+  if (equaltok == NULL) {
+    fprintf(ERRORFILE, "Error with line '%s', no '=' found\n", buffer);
     exit(INVALID_CONFIG_FILE);
     exit(INVALID_CONFIG_FILE);
   }
   }
-  while(!feof(conf_file)) {
-    line = (char *) malloc(linesize);
-    if(line == NULL) {
-      fprintf(ERRORFILE, "malloc failed while reading configuration file.\n");
-      exit(OUT_OF_MEMORY);
+  section->kv_pairs[section->size] = (struct kv_pair *) malloc(
+      sizeof(struct kv_pair));
+  if (section->kv_pairs[section->size] == NULL) {
+    fprintf(ERRORFILE, "Failed allocating memory for single section item\n");
+    exit(OUT_OF_MEMORY);
+  }
+  memset(section->kv_pairs[section->size], 0,
+         sizeof(struct kv_pair));
+  section->kv_pairs[section->size]->key = trim(equaltok);
+
+  equaltok = strtok_r(NULL, splitter, &temp_equaltok);
+  if (equaltok == NULL) {
+    // this can happen because no value was set
+    // e.g. banned.users=#this is a comment
+    int has_values = 1;
+    if (strstr(line, splitter) == NULL) {
+      fprintf(ERRORFILE, "configuration tokenization failed, error with line %s\n", line);
+      has_values = 0;
     }
     }
-    size_read = getline(&line,&linesize,conf_file);
 
 
-    //feof returns true only after we read past EOF.
-    //so a file with no new line, at last can reach this place
-    //if size_read returns negative check for eof condition
-    if (size_read == -1) {
-      free(line);
-      if(!feof(conf_file)){
-        exit(INVALID_CONFIG_FILE);
-      } else {
-        break;
-      }
-    }
-    int eol = strlen(line) - 1;
-    if(line[eol] == '\n') {
-        //trim the ending new line
-        line[eol] = '\0';
+    // It is not a valid line, free memory.
+    free((void *) section->kv_pairs[section->size]->key);
+    free((void *) section->kv_pairs[section->size]);
+    section->kv_pairs[section->size] = NULL;
+    free(buffer);
+
+    // Return -1 when no values
+    if (!has_values) {
+      return -1;
     }
     }
-    //comment line
-    if(line[0] == '#') {
-      free(line);
-      continue;
+
+    // Return 2 for comments
+    return 2;
+  }
+
+#ifdef DEBUG
+  fprintf(LOGFILE, "read_config : Adding conf value : %s \n", equaltok);
+#endif
+
+  section->kv_pairs[section->size]->value = trim(equaltok);
+  section->size++;
+  free(buffer);
+  return 0;
+}
+
+/**
+ * Remove any trailing comment from the supplied line. Function modifies the
+ * argument provided.
+ *
+ * @param line the line from which to remove the comment
+ */
+static void trim_comment(char *line) {
+  char *begin_comment = NULL;
+  if (line != NULL) {
+    begin_comment = strchr(line, COMMENT_BEGIN_CHAR);
+    if (begin_comment != NULL) {
+      *begin_comment = '\0';
     }
     }
-    //tokenize first to get key and list of values.
-    //if no equals is found ignore this line, can be an empty line also
-    equaltok = strtok_r(line, "=", &temp_equaltok);
-    if(equaltok == NULL) {
+  }
+}
+
+/**
+ * Allocate a section struct and initialize it. The memory must be freed by
+ * the caller. Function calls exit if any error occurs.
+ *
+ * @return pointer to the allocated section struct
+ *
+ */
+static struct section *allocate_section() {
+  struct section *section = (struct section *) malloc(sizeof(struct section));
+  if (section == NULL) {
+    fprintf(ERRORFILE, "malloc failed while allocating section.\n");
+    exit(OUT_OF_MEMORY);
+  }
+  section->name = NULL;
+  section->kv_pairs = NULL;
+  section->size = 0;
+  return section;
+}
+
+/**
+ * Populate the given section struct with fields from the config file.
+ *
+ * @param conf_file the file to read from
+ * @param section pointer to the section struct to populate
+ *
+ */
+static void populate_section_fields(FILE *conf_file, struct section *section) {
+  char *line;
+  long int offset = 0;
+  while (!feof(conf_file)) {
+    offset = ftell(conf_file);
+    line = read_config_line(conf_file);
+    if (line != NULL) {
+      if (!is_comment_line(line)) {
+        trim_comment(line);
+        if (!is_section_start_line(line)) {
+          if (section->name != NULL) {
+            if (read_section_entry(line, section) < 0) {
+              fprintf(ERRORFILE, "Error parsing line %s", line);
+              exit(INVALID_CONFIG_FILE);
+            }
+          } else {
+            fprintf(ERRORFILE, "Line '%s' doesn't belong to a section\n",
+                    line);
+            exit(INVALID_CONFIG_FILE);
+          }
+        } else {
+          if (section->name == NULL) {
+            section->name = get_section_name(line);
+            if (strlen(section->name) == 0) {
+              fprintf(ERRORFILE, "Empty section name");
+              exit(INVALID_CONFIG_FILE);
+            }
+          } else {
+            // we've reached the next section
+            fseek(conf_file, offset, SEEK_SET);
+            free(line);
+            return;
+          }
+        }
+      }
       free(line);
       free(line);
-      continue;
-    }
-    cfg->confdetails[cfg->size] = (struct confentry *) malloc(
-            sizeof(struct confentry));
-    if(cfg->confdetails[cfg->size] == NULL) {
-      fprintf(LOGFILE,
-          "Failed allocating memory for single configuration item\n");
-      goto cleanup;
     }
     }
+  }
+}
 
 
-    #ifdef DEBUG
-      fprintf(LOGFILE, "read_config : Adding conf key : %s \n", equaltok);
-    #endif
+/**
+ * Read the section current section from the conf file. Section start is
+ * marked by lines of the form '[section-name]' and continue till the next
+ * section.
+ */
+static struct section *read_section(FILE *conf_file) {
+  struct section *section = allocate_section();
+  populate_section_fields(conf_file, section);
+  if (section->name == NULL) {
+    free_section(section);
+    section = NULL;
+  }
+  return section;
+}
+
+/**
+ * Merge two sections and free the second one after the merge, if desired.
+ * @param section1 the first section
+ * @param section2 the second section
+ * @param free_second_section free the second section if set
+ */
+static void merge_sections(struct section *section1, struct section *section2, const int free_second_section) {
+  int i = 0;
+  section1->kv_pairs = (struct kv_pair **) realloc(
+            section1->kv_pairs,
+            sizeof(struct kv_pair *) * (section1->size + section2->size));
+  if (section1->kv_pairs == NULL) {
+    fprintf(ERRORFILE,
+                "Failed re-allocating memory for configuration items\n");
+    exit(OUT_OF_MEMORY);
+  }
+  for (i = 0; i < section2->size; ++i) {
+    section1->kv_pairs[section1->size + i] = section2->kv_pairs[i];
+  }
+  section1->size += section2->size;
+  if (free_second_section) {
+    free(section2->name);
+    memset(section2, 0, sizeof(*section2));
+    free(section2);
+  }
+}
 
 
-    memset(cfg->confdetails[cfg->size], 0, sizeof(struct confentry));
-    cfg->confdetails[cfg->size]->key = trim(equaltok);
+int read_config(const char *file_path, struct configuration *cfg) {
+  FILE *conf_file;
 
 
-    equaltok = strtok_r(NULL, "=", &temp_equaltok);
-    if (equaltok == NULL) {
-      fprintf(LOGFILE, "configuration tokenization failed \n");
-      goto cleanup;
-    }
-    //means value is commented so don't store the key
-    if(equaltok[0] == '#') {
-      free(line);
-      free((void *)cfg->confdetails[cfg->size]->key);
-      free(cfg->confdetails[cfg->size]);
-      continue;
+  if (file_path == NULL) {
+    fprintf(ERRORFILE, "Null configuration filename passed in\n");
+    return INVALID_CONFIG_FILE;
+  }
+
+#ifdef DEBUG
+  fprintf(LOGFILE, "read_config :Conf file name is : %s \n", file_path);
+#endif
+
+  cfg->size = 0;
+  conf_file = fopen(file_path, "r");
+  if (conf_file == NULL) {
+    fprintf(ERRORFILE, "Invalid conf file provided, unable to open file"
+        " : %s \n", file_path);
+    return (INVALID_CONFIG_FILE);
+  }
+
+  cfg->sections = (struct section **) malloc(
+        sizeof(struct section *) * MAX_SIZE);
+  if (!cfg->sections) {
+    fprintf(ERRORFILE,
+            "Failed to allocate memory for configuration sections\n");
+    exit(OUT_OF_MEMORY);
+  }
+
+  // populate any entries in the older format(no sections)
+  cfg->sections[cfg->size] = allocate_section();
+  cfg->sections[cfg->size]->name = strdup("");
+  populate_section_fields(conf_file, cfg->sections[cfg->size]);
+  if (cfg->sections[cfg->size]) {
+    if (cfg->sections[cfg->size]->size) {
+      cfg->size++;
+    } else {
+      free_section(cfg->sections[cfg->size]);
     }
     }
+  }
 
 
-    #ifdef DEBUG
-      fprintf(LOGFILE, "read_config : Adding conf value : %s \n", equaltok);
-    #endif
-
-    cfg->confdetails[cfg->size]->value = trim(equaltok);
-    if((cfg->size + 1) % MAX_SIZE  == 0) {
-      cfg->confdetails = (struct confentry **) realloc(cfg->confdetails,
-          sizeof(struct confentry **) * (MAX_SIZE + cfg->size));
-      if (cfg->confdetails == NULL) {
-        fprintf(LOGFILE,
-            "Failed re-allocating memory for configuration items\n");
-        goto cleanup;
+  // populate entries in the sections format
+  while (!feof(conf_file)) {
+    cfg->sections[cfg->size] = NULL;
+    struct section *new_section = read_section(conf_file);
+    if (new_section != NULL) {
+      struct section *existing_section =
+          get_configuration_section(new_section->name, cfg);
+      if (existing_section != NULL) {
+        merge_sections((struct section *) existing_section, new_section, 1);
+      } else {
+        cfg->sections[cfg->size] = new_section;
       }
       }
     }
     }
-    if(cfg->confdetails[cfg->size]) {
-        cfg->size++;
-    }
 
 
-    free(line);
+    // Check if we need to expand memory for sections.
+    if (cfg->sections[cfg->size]) {
+      if ((cfg->size + 1) % MAX_SIZE == 0) {
+        cfg->sections = (struct section **) realloc(cfg->sections,
+                           sizeof(struct sections *) * (MAX_SIZE + cfg->size));
+        if (cfg->sections == NULL) {
+          fprintf(ERRORFILE,
+                  "Failed re-allocating memory for configuration items\n");
+          exit(OUT_OF_MEMORY);
+        }
+      }
+      cfg->size++;
+    }
   }
   }
 
 
-  //close the file
   fclose(conf_file);
   fclose(conf_file);
 
 
   if (cfg->size == 0) {
   if (cfg->size == 0) {
-    fprintf(ERRORFILE, "Invalid configuration provided in %s\n", file_name);
-    exit(INVALID_CONFIG_FILE);
-  }
-
-  //clean up allocated file name
-  return;
-  //free spaces alloced.
-  cleanup:
-  if (line != NULL) {
-    free(line);
+    free_configuration(cfg);
+    fprintf(ERRORFILE, "Invalid configuration provided in %s\n", file_path);
+    return INVALID_CONFIG_FILE;
   }
   }
-  fclose(conf_file);
-  free_configurations(cfg);
-  return;
+  return 0;
 }
 }
 
 
 /*
 /*
@@ -297,11 +555,14 @@ void read_config(const char* file_name, struct configuration *cfg) {
  * array, next time onwards used the populated array.
  * array, next time onwards used the populated array.
  *
  *
  */
  */
-char * get_value(const char* key, struct configuration *cfg) {
+char *get_section_value(const char *key, const struct section *section) {
   int count;
   int count;
-  for (count = 0; count < cfg->size; count++) {
-    if (strcmp(cfg->confdetails[count]->key, key) == 0) {
-      return strdup(cfg->confdetails[count]->value);
+  if (key == NULL || section == NULL) {
+    return NULL;
+  }
+  for (count = 0; count < section->size; count++) {
+    if (strcmp(section->kv_pairs[count]->key, key) == 0) {
+      return strdup(section->kv_pairs[count]->value);
     }
     }
   }
   }
   return NULL;
   return NULL;
@@ -311,61 +572,80 @@ char * get_value(const char* key, struct configuration *cfg) {
  * Function to return an array of values for a key.
  * Function to return an array of values for a key.
  * Value delimiter is assumed to be a ','.
  * Value delimiter is assumed to be a ','.
  */
  */
-char ** get_values(const char * key, struct configuration *cfg) {
-  char *value = get_value(key, cfg);
-  return extract_values_delim(value, ",");
+char **get_section_values(const char *key, const struct section *cfg) {
+  return get_section_values_delimiter(key, cfg, ",");
 }
 }
 
 
 /**
 /**
  * Function to return an array of values for a key, using the specified
  * Function to return an array of values for a key, using the specified
  delimiter.
  delimiter.
  */
  */
-char ** get_values_delim(const char * key, struct configuration *cfg,
-    const char *delim) {
-  char *value = get_value(key, cfg);
-  return extract_values_delim(value, delim);
+char **get_section_values_delimiter(const char *key, const struct section *cfg,
+                                    const char *delim) {
+  if (key == NULL || cfg == NULL || delim == NULL) {
+    return NULL;
+  }
+  char *value = get_section_value(key, cfg);
+  char **split_values = split_delimiter(value, delim);
+
+  if (value) {
+    free(value);
+  }
+
+  return split_values;
 }
 }
 
 
-char ** extract_values_delim(char *value, const char *delim) {
-  char ** toPass = NULL;
-  char *tempTok = NULL;
-  char *tempstr = NULL;
-  int size = 0;
-  int toPassSize = MAX_SIZE;
-  //first allocate any array of 10
-  if(value != NULL) {
-    toPass = (char **) malloc(sizeof(char *) * toPassSize);
-    tempTok = strtok_r((char *)value, delim, &tempstr);
-    while (tempTok != NULL) {
-      toPass[size++] = tempTok;
-      if(size == toPassSize) {
-        toPassSize += MAX_SIZE;
-        toPass = (char **) realloc(toPass,(sizeof(char *) * toPassSize));
-      }
-      tempTok = strtok_r(NULL, delim, &tempstr);
-    }
+char *get_configuration_value(const char *key, const char *section,
+                              const struct configuration *cfg) {
+  const struct section *section_ptr;
+  if (key == NULL || section == NULL || cfg == NULL) {
+    return NULL;
   }
   }
-  if (toPass != NULL) {
-    toPass[size] = NULL;
+  section_ptr = get_configuration_section(section, cfg);
+  if (section_ptr != NULL) {
+    return get_section_value(key, section_ptr);
   }
   }
-  return toPass;
+  return NULL;
 }
 }
 
 
-/**
- * Extracts array of values from the '%' separated list of values.
- */
-char ** extract_values(char *value) {
-  return extract_values_delim(value, "%");
+char **get_configuration_values(const char *key, const char *section,
+                                const struct configuration *cfg) {
+  const struct section *section_ptr;
+  if (key == NULL || section == NULL || cfg == NULL) {
+    return NULL;
+  }
+  section_ptr = get_configuration_section(section, cfg);
+  if (section_ptr != NULL) {
+    return get_section_values(key, section_ptr);
+  }
+  return NULL;
+}
+
+char **get_configuration_values_delimiter(const char *key, const char *section,
+                                          const struct configuration *cfg, const char *delim) {
+  const struct section *section_ptr;
+  if (key == NULL || section == NULL || cfg == NULL || delim == NULL) {
+    return NULL;
+  }
+  section_ptr = get_configuration_section(section, cfg);
+  if (section_ptr != NULL) {
+    return get_section_values_delimiter(key, section_ptr, delim);
+  }
+  return NULL;
 }
 }
 
 
-// free an entry set of values
-void free_values(char** values) {
-  if (*values != NULL) {
-    free(*values);
+struct section *get_configuration_section(const char *section,
+                                          const struct configuration *cfg) {
+  int i = 0;
+  if (cfg == NULL || section == NULL) {
+    return NULL;
   }
   }
-  if (values != NULL) {
-    free(values);
+  for (i = 0; i < cfg->size; ++i) {
+    if (strcmp(cfg->sections[i]->name, section) == 0) {
+      return cfg->sections[i];
+    }
   }
   }
+  return NULL;
 }
 }
 
 
 /**
 /**
@@ -376,12 +656,12 @@ int get_kv_key(const char *input, char *out, size_t out_len) {
   if (input == NULL)
   if (input == NULL)
     return -EINVAL;
     return -EINVAL;
 
 
-  char *split = strchr(input, '=');
+  const char *split = strchr(input, '=');
 
 
   if (split == NULL)
   if (split == NULL)
     return -EINVAL;
     return -EINVAL;
 
 
-  int key_len = split - input;
+  unsigned long key_len = split - input;
 
 
   if (out_len < (key_len + 1) || out == NULL)
   if (out_len < (key_len + 1) || out == NULL)
     return -ENAMETOOLONG;
     return -ENAMETOOLONG;
@@ -400,13 +680,13 @@ int get_kv_value(const char *input, char *out, size_t out_len) {
   if (input == NULL)
   if (input == NULL)
     return -EINVAL;
     return -EINVAL;
 
 
-  char *split = strchr(input, '=');
+  const char *split = strchr(input, '=');
 
 
   if (split == NULL)
   if (split == NULL)
     return -EINVAL;
     return -EINVAL;
 
 
   split++; // advance past '=' to the value
   split++; // advance past '=' to the value
-  int val_len = (input + strlen(input)) - split;
+  unsigned long val_len = (input + strlen(input)) - split;
 
 
   if (out_len < (val_len + 1) || out == NULL)
   if (out_len < (val_len + 1) || out == NULL)
     return -ENAMETOOLONG;
     return -ENAMETOOLONG;

+ 138 - 44
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/configuration.h

@@ -16,6 +16,9 @@
  * limitations under the License.
  * limitations under the License.
  */
  */
 
 
+#ifndef __YARN_CONTAINER_EXECUTOR_CONFIG_H__
+#define __YARN_CONTAINER_EXECUTOR_CONFIG_H__
+
 #ifdef __FreeBSD__
 #ifdef __FreeBSD__
 #define _WITH_GETLINE
 #define _WITH_GETLINE
 #endif
 #endif
@@ -23,62 +26,160 @@
 #include <stddef.h>
 #include <stddef.h>
 
 
 /** Define a platform-independent constant instead of using PATH_MAX */
 /** Define a platform-independent constant instead of using PATH_MAX */
-
 #define EXECUTOR_PATH_MAX 4096
 #define EXECUTOR_PATH_MAX 4096
 
 
-/**
- * Ensure that the configuration file and all of the containing directories
- * are only writable by root. Otherwise, an attacker can change the
- * configuration and potentially cause damage.
- * returns 0 if permissions are ok
- */
-int check_configuration_permissions(const char* file_name);
-
-/**
- * Return a string with the configuration file path name resolved via realpath(3)
- *
- * NOTE: relative path names are resolved relative to the second argument not getwd(3)
- */
-char *resolve_config_path(const char* file_name, const char *root);
-
-// Config data structures.
-struct confentry {
+// Configuration data structures.
+struct kv_pair {
   const char *key;
   const char *key;
   const char *value;
   const char *value;
 };
 };
 
 
+struct section {
+  int size;
+  char *name;
+  struct kv_pair **kv_pairs;
+};
+
 struct configuration {
 struct configuration {
   int size;
   int size;
-  struct confentry **confdetails;
+  struct section **sections;
 };
 };
 
 
-// read the given configuration file into the specified config struct.
-void read_config(const char* config_file, struct configuration *cfg);
+/**
+ * Function to ensure that the configuration file and all of the containing
+ * directories are only writable by root. Otherwise, an attacker can change
+ * the configuration and potentially cause damage.
+ *
+ * @param file_name name of the config file
+ *
+ * @returns 0 if permissions are correct, non-zero on error
+ */
+int check_configuration_permissions(const char *file_name);
+
+/**
+ * Return a string with the configuration file path name resolved via
+ * realpath(3). Relative path names are resolved relative to the second
+ * argument and not getwd(3). It's up to the caller to free the returned
+ * value.
+ *
+ * @param file_name name of the config file
+ * @param root the path against which relative path names are to be resolved
+ *
+ * @returns the resolved configuration file path
+ */
+char* resolve_config_path(const char *file_name, const char *root);
 
 
-//method exposed to get the configurations
-char *get_value(const char* key, struct configuration *cfg);
+/**
+ * Read the given configuration file into the specified configuration struct.
+ * It's the responsibility of the caller to call free_configurations to free
+ * the allocated memory. The function will check to ensure that the
+ * configuration file has the appropriate owner and permissions.
+ *
+ * @param file_path name of the configuration file to be read
+ * @param cfg the configuration structure to be filled.
+ *
+ * @return 0 on success, non-zero if there was an error
+ */
+int read_config(const char *file_path, struct configuration *cfg);
+
+/**
+ * Get the value for a key in the specified section. It's up to the caller to
+ * free the memory used for storing the return value.
+ *
+ * @param key key the name of the key
+ * @param section the section to be looked up
+ *
+ * @return pointer to the value if the key was found, null otherwise
+ */
+char* get_section_value(const char *key, const struct section *section);
 
 
-//function to return array of values pointing to the key. Values are
-//comma seperated strings.
-char ** get_values(const char* key, struct configuration *cfg);
+/**
+ * Function to get the values for a given key in the specified section.
+ * The value is split by ",". It's up to the caller to free the memory used
+ * for storing the return values.
+ *
+ * @param key the key to be looked up
+ * @param section the section to be looked up
+ *
+ * @return array of values, null if the key was not found
+ */
+char** get_section_values(const char *key, const struct section *section);
 
 
 /**
 /**
- * Function to return an array of values for a key, using the specified
- delimiter.
+ * Function to get the values for a given key in the specified section.
+ * The value is split by the specified delimiter. It's up to the caller to
+ * free the memory used for storing the return values.
+ *
+ * @param key the key to be looked up
+ * @param section the section to be looked up
+ * @param delimiter the delimiter to be used to split the value
+ *
+ * @return array of values, null if the key was not found
  */
  */
-char ** get_values_delim(const char * key, struct configuration *cfg,
+char** get_section_values_delimiter(const char *key, const struct section *section,
     const char *delim);
     const char *delim);
 
 
-// Extracts array of values from the comma separated list of values.
-char ** extract_values(char *value);
+/**
+ * Get the value for a key in the specified section in the specified
+ * configuration. It's up to the caller to free the memory used for storing
+ * the return value.
+ *
+ * @param key key the name of the key
+ * @param section the name section to be looked up
+ * @param cfg the configuration to be used
+ *
+ * @return pointer to the value if the key was found, null otherwise
+ */
+char* get_configuration_value(const char *key, const char* section,
+    const struct configuration *cfg);
+
+/**
+ * Function to get the values for a given key in the specified section in the
+ * specified configuration. The value is split by ",". It's up to the caller to
+ * free the memory used for storing the return values.
+ *
+ * @param key the key to be looked up
+ * @param section the name of the section to be looked up
+ * @param cfg the configuration to be looked up
+ *
+ * @return array of values, null if the key was not found
+ */
+char** get_configuration_values(const char *key, const char* section,
+    const struct configuration *cfg);
 
 
-char ** extract_values_delim(char *value, const char *delim);
+/**
+ * Function to get the values for a given key in the specified section in the
+ * specified configuration. The value is split by the specified delimiter.
+ * It's up to the caller to free the memory used for storing the return values.
+ *
+ * @param key the key to be looked up
+ * @param section the name of the section to be looked up
+ * @param cfg the section to be looked up
+ * @param delimiter the delimiter to be used to split the value
+ *
+ * @return array of values, null if the key was not found
+ */
+char** get_configuration_values_delimiter(const char *key, const char* section,
+    const struct configuration *cfg, const char *delimiter);
 
 
-// free the memory returned by get_values
-void free_values(char** values);
+/**
+ * Function to retrieve the specified section from the configuration.
+ *
+ * @param section the name of the section to retrieve
+ * @param cfg the configuration structure to use
+ *
+ * @return pointer to section struct containing details of the section
+ *         null on error
+ */
+struct section* get_configuration_section(const char *section,
+    const struct configuration *cfg);
 
 
-//method to free allocated configuration
-void free_configurations(struct configuration *cfg);
+/**
+ * Method to free an allocated config struct.
+ *
+ * @param cfg pointer to the structure to free
+ */
+void free_configuration(struct configuration *cfg);
 
 
 /**
 /**
  * If str is a string of the form key=val, find 'key'
  * If str is a string of the form key=val, find 'key'
@@ -106,11 +207,4 @@ int get_kv_key(const char *input, char *out, size_t out_len);
  */
  */
 int get_kv_value(const char *input, char *out, size_t out_len);
 int get_kv_value(const char *input, char *out, size_t out_len);
 
 
-/**
- * Trim whitespace from beginning and end.
- *
- * @param input    Input string that needs to be trimmed
- *
- * @return the trimmed string allocated with malloc. I has to be freed by the caller
-*/
-char* trim(char* input);
+#endif

+ 23 - 16
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c

@@ -19,6 +19,8 @@
 #include "configuration.h"
 #include "configuration.h"
 #include "container-executor.h"
 #include "container-executor.h"
 #include "utils/string-utils.h"
 #include "utils/string-utils.h"
+#include "util.h"
+#include "config.h"
 
 
 #include <inttypes.h>
 #include <inttypes.h>
 #include <libgen.h>
 #include <libgen.h>
@@ -43,8 +45,6 @@
 #include <getopt.h>
 #include <getopt.h>
 #include <regex.h>
 #include <regex.h>
 
 
-#include "config.h"
-
 #ifndef HAVE_FCHMODAT
 #ifndef HAVE_FCHMODAT
 #include "compat/fchmodat.h"
 #include "compat/fchmodat.h"
 #endif
 #endif
@@ -92,7 +92,8 @@ FILE* ERRORFILE = NULL;
 static uid_t nm_uid = -1;
 static uid_t nm_uid = -1;
 static gid_t nm_gid = -1;
 static gid_t nm_gid = -1;
 
 
-struct configuration executor_cfg = {.size=0, .confdetails=NULL};
+struct configuration CFG = {.size=0, .sections=NULL};
+struct section executor_cfg = {.size=0, .kv_pairs=NULL};
 
 
 char *concatenate(char *concat_pattern, char *return_path_name,
 char *concatenate(char *concat_pattern, char *return_path_name,
    int numArgs, ...);
    int numArgs, ...);
@@ -103,18 +104,25 @@ void set_nm_uid(uid_t user, gid_t group) {
 }
 }
 
 
 //function used to load the configurations present in the secure config
 //function used to load the configurations present in the secure config
-void read_executor_config(const char* file_name) {
-    read_config(file_name, &executor_cfg);
+void read_executor_config(const char *file_name) {
+  const struct section *tmp = NULL;
+  int ret = read_config(file_name, &CFG);
+  if (ret == 0) {
+    tmp = get_configuration_section("", &CFG);
+    if (tmp != NULL) {
+      executor_cfg = *tmp;
+    }
+  }
 }
 }
 
 
 //function used to free executor configuration data
 //function used to free executor configuration data
 void free_executor_configurations() {
 void free_executor_configurations() {
-    free_configurations(&executor_cfg);
+    free_configuration(&CFG);
 }
 }
 
 
 //Lookup nodemanager group from container executor configuration.
 //Lookup nodemanager group from container executor configuration.
 char *get_nodemanager_group() {
 char *get_nodemanager_group() {
-    return get_value(NM_GROUP_KEY, &executor_cfg);
+    return get_section_value(NM_GROUP_KEY, &executor_cfg);
 }
 }
 
 
 int check_executor_permissions(char *executable_file) {
 int check_executor_permissions(char *executable_file) {
@@ -431,8 +439,8 @@ int change_user(uid_t user, gid_t group) {
 }
 }
 
 
 int is_feature_enabled(const char* feature_key, int default_value,
 int is_feature_enabled(const char* feature_key, int default_value,
-                              struct configuration *cfg) {
-    char *enabled_str = get_value(feature_key, cfg);
+                              struct section *cfg) {
+    char *enabled_str = get_section_value(feature_key, cfg);
     int enabled = default_value;
     int enabled = default_value;
 
 
     if (enabled_str != NULL) {
     if (enabled_str != NULL) {
@@ -753,7 +761,7 @@ static struct passwd* get_user_info(const char* user) {
 }
 }
 
 
 int is_whitelisted(const char *user) {
 int is_whitelisted(const char *user) {
-  char **whitelist = get_values(ALLOWED_SYSTEM_USERS_KEY, &executor_cfg);
+  char **whitelist = get_section_values(ALLOWED_SYSTEM_USERS_KEY, &executor_cfg);
   char **users = whitelist;
   char **users = whitelist;
   if (whitelist != NULL) {
   if (whitelist != NULL) {
     for(; *users; ++users) {
     for(; *users; ++users) {
@@ -781,7 +789,7 @@ struct passwd* check_user(const char *user) {
     fflush(LOGFILE);
     fflush(LOGFILE);
     return NULL;
     return NULL;
   }
   }
-  char *min_uid_str = get_value(MIN_USERID_KEY, &executor_cfg);
+  char *min_uid_str = get_section_value(MIN_USERID_KEY, &executor_cfg);
   int min_uid = DEFAULT_MIN_USERID;
   int min_uid = DEFAULT_MIN_USERID;
   if (min_uid_str != NULL) {
   if (min_uid_str != NULL) {
     char *end_ptr = NULL;
     char *end_ptr = NULL;
@@ -808,7 +816,7 @@ struct passwd* check_user(const char *user) {
     free(user_info);
     free(user_info);
     return NULL;
     return NULL;
   }
   }
-  char **banned_users = get_values(BANNED_USERS_KEY, &executor_cfg);
+  char **banned_users = get_section_values(BANNED_USERS_KEY, &executor_cfg);
   banned_users = banned_users == NULL ?
   banned_users = banned_users == NULL ?
     (char**) DEFAULT_BANNED_USERS : banned_users;
     (char**) DEFAULT_BANNED_USERS : banned_users;
   char **banned_user = banned_users;
   char **banned_user = banned_users;
@@ -1194,7 +1202,6 @@ char** tokenize_docker_command(const char *input, int *split_counter) {
   char *line = (char *)calloc(strlen(input) + 1, sizeof(char));
   char *line = (char *)calloc(strlen(input) + 1, sizeof(char));
   char **linesplit = (char **) malloc(sizeof(char *));
   char **linesplit = (char **) malloc(sizeof(char *));
   char *p = NULL;
   char *p = NULL;
-  int c = 0;
   *split_counter = 0;
   *split_counter = 0;
   strncpy(line, input, strlen(input));
   strncpy(line, input, strlen(input));
 
 
@@ -1408,12 +1415,12 @@ char* parse_docker_command_file(const char* command_file) {
 
 
 int run_docker(const char *command_file) {
 int run_docker(const char *command_file) {
   char* docker_command = parse_docker_command_file(command_file);
   char* docker_command = parse_docker_command_file(command_file);
-  char* docker_binary = get_value(DOCKER_BINARY_KEY, &executor_cfg);
+  char* docker_binary = get_section_value(DOCKER_BINARY_KEY, &executor_cfg);
   docker_binary = check_docker_binary(docker_binary);
   docker_binary = check_docker_binary(docker_binary);
 
 
   char* docker_command_with_binary = calloc(sizeof(char), EXECUTOR_PATH_MAX);
   char* docker_command_with_binary = calloc(sizeof(char), EXECUTOR_PATH_MAX);
   snprintf(docker_command_with_binary, EXECUTOR_PATH_MAX, "%s %s", docker_binary, docker_command);
   snprintf(docker_command_with_binary, EXECUTOR_PATH_MAX, "%s %s", docker_binary, docker_command);
-  char **args = extract_values_delim(docker_command_with_binary, " ");
+  char **args = split_delimiter(docker_command_with_binary, " ");
 
 
   int exit_code = -1;
   int exit_code = -1;
   if (execvp(docker_binary, args) != 0) {
   if (execvp(docker_binary, args) != 0) {
@@ -1574,7 +1581,7 @@ int launch_docker_container_as_user(const char * user, const char *app_id,
   uid_t prev_uid = geteuid();
   uid_t prev_uid = geteuid();
 
 
   char *docker_command = parse_docker_command_file(command_file);
   char *docker_command = parse_docker_command_file(command_file);
-  char *docker_binary = get_value(DOCKER_BINARY_KEY, &executor_cfg);
+  char *docker_binary = get_section_value(DOCKER_BINARY_KEY, &executor_cfg);
   docker_binary = check_docker_binary(docker_binary);
   docker_binary = check_docker_binary(docker_binary);
 
 
   fprintf(LOGFILE, "Creating script paths...\n");
   fprintf(LOGFILE, "Creating script paths...\n");

+ 1 - 51
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.h

@@ -35,51 +35,6 @@ enum command {
   LIST_AS_USER = 5
   LIST_AS_USER = 5
 };
 };
 
 
-enum errorcodes {
-  INVALID_ARGUMENT_NUMBER = 1,
-  //INVALID_USER_NAME 2
-  INVALID_COMMAND_PROVIDED = 3,
-  // SUPER_USER_NOT_ALLOWED_TO_RUN_TASKS (NOT USED) 4
-  INVALID_NM_ROOT_DIRS = 5,
-  SETUID_OPER_FAILED, //6
-  UNABLE_TO_EXECUTE_CONTAINER_SCRIPT, //7
-  UNABLE_TO_SIGNAL_CONTAINER, //8
-  INVALID_CONTAINER_PID, //9
-  // ERROR_RESOLVING_FILE_PATH (NOT_USED) 10
-  // RELATIVE_PATH_COMPONENTS_IN_FILE_PATH (NOT USED) 11
-  // UNABLE_TO_STAT_FILE (NOT USED) 12
-  // FILE_NOT_OWNED_BY_ROOT (NOT USED) 13
-  // PREPARE_CONTAINER_DIRECTORIES_FAILED (NOT USED) 14
-  // INITIALIZE_CONTAINER_FAILED (NOT USED) 15
-  // PREPARE_CONTAINER_LOGS_FAILED (NOT USED) 16
-  // INVALID_LOG_DIR (NOT USED) 17
-  OUT_OF_MEMORY = 18,
-  // INITIALIZE_DISTCACHEFILE_FAILED (NOT USED) 19
-  INITIALIZE_USER_FAILED = 20,
-  PATH_TO_DELETE_IS_NULL, //21
-  INVALID_CONTAINER_EXEC_PERMISSIONS, //22
-  // PREPARE_JOB_LOGS_FAILED (NOT USED) 23
-  INVALID_CONFIG_FILE = 24,
-  SETSID_OPER_FAILED = 25,
-  WRITE_PIDFILE_FAILED = 26,
-  WRITE_CGROUP_FAILED = 27,
-  TRAFFIC_CONTROL_EXECUTION_FAILED = 28,
-  DOCKER_RUN_FAILED = 29,
-  ERROR_OPENING_DOCKER_FILE = 30,
-  ERROR_READING_DOCKER_FILE = 31,
-  FEATURE_DISABLED = 32,
-  COULD_NOT_CREATE_SCRIPT_COPY = 33,
-  COULD_NOT_CREATE_CREDENTIALS_FILE = 34,
-  COULD_NOT_CREATE_WORK_DIRECTORIES = 35,
-  COULD_NOT_CREATE_APP_LOG_DIRECTORIES = 36,
-  COULD_NOT_CREATE_TMP_DIRECTORIES = 37,
-  ERROR_CREATE_CONTAINER_DIRECTORIES_ARGUMENTS = 38,
-  ERROR_SANITIZING_DOCKER_COMMAND = 39,
-  DOCKER_IMAGE_INVALID = 40,
-  DOCKER_CONTAINER_NAME_INVALID = 41,
-  ERROR_COMPILING_REGEX = 42
-};
-
 enum operations {
 enum operations {
   CHECK_SETUP = 1,
   CHECK_SETUP = 1,
   MOUNT_CGROUPS = 2,
   MOUNT_CGROUPS = 2,
@@ -111,11 +66,6 @@ enum operations {
 
 
 extern struct passwd *user_detail;
 extern struct passwd *user_detail;
 
 
-// the log file for messages
-extern FILE *LOGFILE;
-// the log file for error messages
-extern FILE *ERRORFILE;
-
 // get the executable's filename
 // get the executable's filename
 char* get_executable(char *argv0);
 char* get_executable(char *argv0);
 
 
@@ -276,7 +226,7 @@ int create_validate_dir(const char* npath, mode_t perm, const char* path,
 
 
 /** Check if a feature is enabled in the specified configuration. */
 /** Check if a feature is enabled in the specified configuration. */
 int is_feature_enabled(const char* feature_key, int default_value,
 int is_feature_enabled(const char* feature_key, int default_value,
-                              struct configuration *cfg);
+                              struct section *cfg);
 
 
 /** Check if tc (traffic control) support is enabled in configuration. */
 /** Check if tc (traffic control) support is enabled in configuration. */
 int is_tc_support_enabled();
 int is_tc_support_enabled();

+ 1 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/get_executable.c

@@ -31,6 +31,7 @@
 #include "config.h"
 #include "config.h"
 #include "configuration.h"
 #include "configuration.h"
 #include "container-executor.h"
 #include "container-executor.h"
+#include "util.h"
 
 
 #include <errno.h>
 #include <errno.h>
 #include <stdio.h>
 #include <stdio.h>

+ 9 - 8
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/main.c

@@ -19,6 +19,7 @@
 #include "config.h"
 #include "config.h"
 #include "configuration.h"
 #include "configuration.h"
 #include "container-executor.h"
 #include "container-executor.h"
+#include "util.h"
 
 
 #include <errno.h>
 #include <errno.h>
 #include <grp.h>
 #include <grp.h>
@@ -420,7 +421,7 @@ static int validate_run_as_user_commands(int argc, char **argv, int *operation)
 
 
       cmd_input.resources_key = resources_key;
       cmd_input.resources_key = resources_key;
       cmd_input.resources_value = resources_value;
       cmd_input.resources_value = resources_value;
-      cmd_input.resources_values = extract_values(resources_value);
+      cmd_input.resources_values = split(resources_value);
       *operation = RUN_AS_USER_LAUNCH_DOCKER_CONTAINER;
       *operation = RUN_AS_USER_LAUNCH_DOCKER_CONTAINER;
       return 0;
       return 0;
    } else {
    } else {
@@ -471,7 +472,7 @@ static int validate_run_as_user_commands(int argc, char **argv, int *operation)
 
 
     cmd_input.resources_key = resources_key;
     cmd_input.resources_key = resources_key;
     cmd_input.resources_value = resources_value;
     cmd_input.resources_value = resources_value;
-    cmd_input.resources_values = extract_values(resources_value);
+    cmd_input.resources_values = split(resources_value);
     *operation = RUN_AS_USER_LAUNCH_CONTAINER;
     *operation = RUN_AS_USER_LAUNCH_CONTAINER;
     return 0;
     return 0;
 
 
@@ -565,8 +566,8 @@ int main(int argc, char **argv) {
     exit_code = initialize_app(cmd_input.yarn_user_name,
     exit_code = initialize_app(cmd_input.yarn_user_name,
                             cmd_input.app_id,
                             cmd_input.app_id,
                             cmd_input.cred_file,
                             cmd_input.cred_file,
-                            extract_values(cmd_input.local_dirs),
-                            extract_values(cmd_input.log_dirs),
+                            split(cmd_input.local_dirs),
+                            split(cmd_input.log_dirs),
                             argv + optind);
                             argv + optind);
     break;
     break;
   case RUN_AS_USER_LAUNCH_DOCKER_CONTAINER:
   case RUN_AS_USER_LAUNCH_DOCKER_CONTAINER:
@@ -591,8 +592,8 @@ int main(int argc, char **argv) {
                       cmd_input.script_file,
                       cmd_input.script_file,
                       cmd_input.cred_file,
                       cmd_input.cred_file,
                       cmd_input.pid_file,
                       cmd_input.pid_file,
-                      extract_values(cmd_input.local_dirs),
-                      extract_values(cmd_input.log_dirs),
+                      split(cmd_input.local_dirs),
+                      split(cmd_input.log_dirs),
                       cmd_input.docker_command_file,
                       cmd_input.docker_command_file,
                       cmd_input.resources_key,
                       cmd_input.resources_key,
                       cmd_input.resources_values);
                       cmd_input.resources_values);
@@ -619,8 +620,8 @@ int main(int argc, char **argv) {
                     cmd_input.script_file,
                     cmd_input.script_file,
                     cmd_input.cred_file,
                     cmd_input.cred_file,
                     cmd_input.pid_file,
                     cmd_input.pid_file,
-                    extract_values(cmd_input.local_dirs),
-                    extract_values(cmd_input.log_dirs),
+                    split(cmd_input.local_dirs),
+                    split(cmd_input.log_dirs),
                     cmd_input.resources_key,
                     cmd_input.resources_key,
                     cmd_input.resources_values);
                     cmd_input.resources_values);
     free(cmd_input.resources_key);
     free(cmd_input.resources_key);

+ 134 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/util.c

@@ -0,0 +1,134 @@
+/**
+ * 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.
+ */
+
+#include "util.h"
+#include <stddef.h>
+#include <stdlib.h>
+#include <string.h>
+#include <ctype.h>
+
+char** split_delimiter(char *value, const char *delim) {
+  char **return_values = NULL;
+  char *temp_tok = NULL;
+  char *tempstr = NULL;
+  int size = 0;
+  int per_alloc_size = 10;
+  int return_values_size = per_alloc_size;
+  int failed = 0;
+
+  //first allocate any array of 10
+  if(value != NULL) {
+    return_values = (char **) malloc(sizeof(char *) * return_values_size);
+    if (!return_values) {
+      fprintf(ERRORFILE, "Allocation error for return_values in %s.\n",
+              __func__);
+      failed = 1;
+      goto cleanup;
+    }
+    memset(return_values, 0, sizeof(char *) * return_values_size);
+
+    temp_tok = strtok_r(value, delim, &tempstr);
+    while (temp_tok != NULL) {
+      temp_tok = strdup(temp_tok);
+      if (NULL == temp_tok) {
+        fprintf(ERRORFILE, "Allocation error in %s.\n", __func__);
+        failed = 1;
+        goto cleanup;
+      }
+
+      return_values[size++] = temp_tok;
+
+      // Make sure returned values has enough space for the trailing NULL.
+      if (size >= return_values_size - 1) {
+        return_values_size += per_alloc_size;
+        return_values = (char **) realloc(return_values,(sizeof(char *) *
+          return_values_size));
+
+        // Make sure new added memory are filled with NULL
+        for (int i = size; i < return_values_size; i++) {
+          return_values[i] = NULL;
+        }
+      }
+      temp_tok = strtok_r(NULL, delim, &tempstr);
+    }
+  }
+
+  // Put trailing NULL to indicate values terminates.
+  if (return_values != NULL) {
+    return_values[size] = NULL;
+  }
+
+cleanup:
+  if (failed) {
+    free_values(return_values);
+    return NULL;
+  }
+
+  return return_values;
+}
+
+/**
+ * Extracts array of values from the '%' separated list of values.
+ */
+char** split(char *value) {
+  return split_delimiter(value, "%");
+}
+
+// free an entry set of values
+void free_values(char** values) {
+  if (values != NULL) {
+    int idx = 0;
+    while (values[idx]) {
+      free(values[idx]);
+      idx++;
+    }
+    free(values);
+  }
+}
+
+/**
+ * Trim whitespace from beginning and end.
+*/
+char* trim(const char* input) {
+    const char *val_begin;
+    const char *val_end;
+    char *ret;
+
+    if (input == NULL) {
+      return NULL;
+    }
+
+    val_begin = input;
+    val_end = input + strlen(input);
+
+    while (val_begin < val_end && isspace(*val_begin))
+      val_begin++;
+    while (val_end > val_begin && isspace(*(val_end - 1)))
+      val_end--;
+
+    ret = (char *) malloc(
+            sizeof(char) * (val_end - val_begin + 1));
+    if (ret == NULL) {
+      fprintf(ERRORFILE, "Allocation error\n");
+      exit(OUT_OF_MEMORY);
+    }
+
+    strncpy(ret, val_begin, val_end - val_begin);
+    ret[val_end - val_begin] = '\0';
+    return ret;
+}

+ 115 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/util.h

@@ -0,0 +1,115 @@
+/**
+ * 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.
+ */
+
+#ifndef __YARN_POSIX_CONTAINER_EXECUTOR_UTIL_H__
+#define __YARN_POSIX_CONTAINER_EXECUTOR_UTIL_H__
+
+#include <stdio.h>
+
+enum errorcodes {
+  INVALID_ARGUMENT_NUMBER = 1,
+  //INVALID_USER_NAME 2
+  INVALID_COMMAND_PROVIDED = 3,
+  // SUPER_USER_NOT_ALLOWED_TO_RUN_TASKS (NOT USED) 4
+  INVALID_NM_ROOT_DIRS = 5,
+  SETUID_OPER_FAILED, //6
+  UNABLE_TO_EXECUTE_CONTAINER_SCRIPT, //7
+  UNABLE_TO_SIGNAL_CONTAINER, //8
+  INVALID_CONTAINER_PID, //9
+  // ERROR_RESOLVING_FILE_PATH (NOT_USED) 10
+  // RELATIVE_PATH_COMPONENTS_IN_FILE_PATH (NOT USED) 11
+  // UNABLE_TO_STAT_FILE (NOT USED) 12
+  // FILE_NOT_OWNED_BY_ROOT (NOT USED) 13
+  // PREPARE_CONTAINER_DIRECTORIES_FAILED (NOT USED) 14
+  // INITIALIZE_CONTAINER_FAILED (NOT USED) 15
+  // PREPARE_CONTAINER_LOGS_FAILED (NOT USED) 16
+  // INVALID_LOG_DIR (NOT USED) 17
+  OUT_OF_MEMORY = 18,
+  // INITIALIZE_DISTCACHEFILE_FAILED (NOT USED) 19
+  INITIALIZE_USER_FAILED = 20,
+  PATH_TO_DELETE_IS_NULL, //21
+  INVALID_CONTAINER_EXEC_PERMISSIONS, //22
+  // PREPARE_JOB_LOGS_FAILED (NOT USED) 23
+  INVALID_CONFIG_FILE = 24,
+  SETSID_OPER_FAILED = 25,
+  WRITE_PIDFILE_FAILED = 26,
+  WRITE_CGROUP_FAILED = 27,
+  TRAFFIC_CONTROL_EXECUTION_FAILED = 28,
+  DOCKER_RUN_FAILED = 29,
+  ERROR_OPENING_DOCKER_FILE = 30,
+  ERROR_READING_DOCKER_FILE = 31,
+  FEATURE_DISABLED = 32,
+  COULD_NOT_CREATE_SCRIPT_COPY = 33,
+  COULD_NOT_CREATE_CREDENTIALS_FILE = 34,
+  COULD_NOT_CREATE_WORK_DIRECTORIES = 35,
+  COULD_NOT_CREATE_APP_LOG_DIRECTORIES = 36,
+  COULD_NOT_CREATE_TMP_DIRECTORIES = 37,
+  ERROR_CREATE_CONTAINER_DIRECTORIES_ARGUMENTS = 38,
+  ERROR_SANITIZING_DOCKER_COMMAND = 39,
+  DOCKER_IMAGE_INVALID = 40,
+  DOCKER_CONTAINER_NAME_INVALID = 41,
+  ERROR_COMPILING_REGEX = 42
+};
+
+
+// the log file for messages
+extern FILE *LOGFILE;
+// the log file for error messages
+extern FILE *ERRORFILE;
+/**
+ * Function to split the given string using '%' as the separator. It's
+ * up to the caller to free the memory for the returned array. Use the
+ * free_values function to free the allocated memory.
+ *
+ * @param str the string to split
+ *
+ * @return an array of strings
+ */
+char** split(char *str);
+
+/**
+ * Function to split the given string using the delimiter specified. It's
+ * up to the caller to free the memory for the returned array. Use the
+ * free_values function to free the allocated memory.
+ *
+ * @param str the string to split
+ * @param delimiter the delimiter to use
+ *
+ * @return an array of strings
+ */
+char** split_delimiter(char *value, const char *delimiter);
+
+/**
+ * Function to free an array of strings.
+ *
+ * @param values the array to free
+ *
+ */
+void free_values(char **values);
+
+/**
+ * Trim whitespace from beginning and end. The returned string has to be freed
+ * by the caller.
+ *
+ * @param input    Input string that needs to be trimmed
+ *
+ * @return the trimmed string allocated with malloc
+*/
+char* trim(const char *input);
+
+#endif

+ 31 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/resources/test/test-configurations/configuration-1.cfg

@@ -0,0 +1,31 @@
+# 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.
+[section-1]
+key1=value1
+split-key=val1,val2,val3
+perc-key=perc-val1%perc-val2
+# some comment
+
+[split-section]
+key3=value3
+
+[section-2]
+key1=value2
+
+key2=value2
+
+[split-section]
+key4=value4

+ 28 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/resources/test/test-configurations/configuration-2.cfg

@@ -0,0 +1,28 @@
+# 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.
+
+# Test mixed mode config file
+# Initial few lines are in the key=value format
+# and then the sections start
+
+key1=value1
+key2=value2
+
+
+[section-1]
+key3=value3
+key1=value4
+

+ 25 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/resources/test/test-configurations/old-config.cfg

@@ -0,0 +1,25 @@
+# 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.
+yarn.nodemanager.linux-container-executor.group=yarn
+banned.users=root,testuser1,testuser2#comma separated list of users who can not run applications
+min.user.id=1000
+allowed.system.users=nobody,daemon
+feature.docker.enabled=1
+feature.tc.enabled=0
+docker.binary=/usr/bin/docker
+yarn.local.dirs=/var/run/yarn%/tmp/mydir
+test.key=#no value for this key
+# test.key2=0

+ 9 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test-container-executor.c

@@ -18,6 +18,7 @@
 #include "configuration.h"
 #include "configuration.h"
 #include "container-executor.h"
 #include "container-executor.h"
 #include "utils/string-utils.h"
 #include "utils/string-utils.h"
+#include "util.h"
 
 
 #include <inttypes.h>
 #include <inttypes.h>
 #include <errno.h>
 #include <errno.h>
@@ -404,7 +405,7 @@ void test_delete_app() {
 }
 }
 
 
 void validate_feature_enabled_value(int expected_value, const char* key,
 void validate_feature_enabled_value(int expected_value, const char* key,
-    int default_value, struct configuration *cfg) {
+    int default_value, struct section *cfg) {
   int value = is_feature_enabled(key, default_value, cfg);
   int value = is_feature_enabled(key, default_value, cfg);
 
 
   if (value != expected_value) {
   if (value != expected_value) {
@@ -419,7 +420,8 @@ void test_is_feature_enabled() {
   FILE *file = fopen(filename, "w");
   FILE *file = fopen(filename, "w");
   int disabled = 0;
   int disabled = 0;
   int enabled = 1;
   int enabled = 1;
-  struct configuration cfg = {.size=0, .confdetails=NULL};
+  struct configuration exec_cfg = {.size=0, .sections=NULL};
+  struct section cfg = {.size=0, .kv_pairs=NULL};
 
 
   if (file == NULL) {
   if (file == NULL) {
     printf("FAIL: Could not open configuration file: %s\n", filename);
     printf("FAIL: Could not open configuration file: %s\n", filename);
@@ -433,7 +435,8 @@ void test_is_feature_enabled() {
   fprintf(file, "feature.name5.enabled=-1\n");
   fprintf(file, "feature.name5.enabled=-1\n");
   fprintf(file, "feature.name6.enabled=2\n");
   fprintf(file, "feature.name6.enabled=2\n");
   fclose(file);
   fclose(file);
-  read_config(filename, &cfg);
+  read_config(filename, &exec_cfg);
+  cfg = *(get_configuration_section("", &exec_cfg));
 
 
   validate_feature_enabled_value(disabled, "feature.name1.enabled",
   validate_feature_enabled_value(disabled, "feature.name1.enabled",
       disabled, &cfg);
       disabled, &cfg);
@@ -449,7 +452,7 @@ void test_is_feature_enabled() {
           disabled, &cfg);
           disabled, &cfg);
 
 
 
 
-  free_configurations(&cfg);
+  free_configuration(&exec_cfg);
 }
 }
 
 
 void test_delete_user() {
 void test_delete_user() {
@@ -1345,8 +1348,8 @@ int main(int argc, char **argv) {
 
 
   read_executor_config(TEST_ROOT "/test.cfg");
   read_executor_config(TEST_ROOT "/test.cfg");
 
 
-  local_dirs = extract_values(strdup(NM_LOCAL_DIRS));
-  log_dirs = extract_values(strdup(NM_LOG_DIRS));
+  local_dirs = split(strdup(NM_LOCAL_DIRS));
+  log_dirs = split(strdup(NM_LOG_DIRS));
 
 
   create_nm_roots(local_dirs);
   create_nm_roots(local_dirs);
 
 

+ 432 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test_configuration.cc

@@ -0,0 +1,432 @@
+/**
+ * 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.
+ */
+
+#include <gtest/gtest.h>
+#include <fstream>
+
+extern "C" {
+#include "util.h"
+#include "configuration.h"
+#include "configuration.c"
+}
+
+
+namespace ContainerExecutor {
+  class TestConfiguration : public ::testing::Test {
+  protected:
+    virtual void SetUp() {
+      new_config_format_file = "test-configurations/configuration-1.cfg";
+      old_config_format_file = "test-configurations/old-config.cfg";
+      mixed_config_format_file = "test-configurations/configuration-2.cfg";
+      loadConfigurations();
+      return;
+    }
+
+    void loadConfigurations() {
+      int ret = 0;
+      ret = read_config(new_config_format_file.c_str(), &new_config_format);
+      ASSERT_EQ(0, ret);
+      ret = read_config(old_config_format_file.c_str(), &old_config_format);
+      ASSERT_EQ(0, ret);
+      ret = read_config(mixed_config_format_file.c_str(),
+                        &mixed_config_format);
+      ASSERT_EQ(0, ret);
+    }
+
+    virtual void TearDown() {
+      free_configuration(&new_config_format);
+      free_configuration(&old_config_format);
+      return;
+    }
+
+    std::string new_config_format_file;
+    std::string old_config_format_file;
+    std::string mixed_config_format_file;
+    struct configuration new_config_format;
+    struct configuration old_config_format;
+    struct configuration mixed_config_format;
+  };
+
+
+  TEST_F(TestConfiguration, test_get_configuration_values_delimiter) {
+    char **split_values;
+    split_values = get_configuration_values_delimiter(NULL, "", &old_config_format, "%");
+    ASSERT_EQ(NULL, split_values);
+    split_values = get_configuration_values_delimiter("yarn.local.dirs", NULL,
+                      &old_config_format, "%");
+    ASSERT_EQ(NULL, split_values);
+    split_values = get_configuration_values_delimiter("yarn.local.dirs", "",
+                      NULL, "%");
+    ASSERT_EQ(NULL, split_values);
+    split_values = get_configuration_values_delimiter("yarn.local.dirs", "",
+                      &old_config_format, NULL);
+    ASSERT_EQ(NULL, split_values);
+    split_values = get_configuration_values_delimiter("yarn.local.dirs", "abcd",
+                                                      &old_config_format, "%");
+    ASSERT_EQ(NULL, split_values);
+    split_values = get_configuration_values_delimiter("yarn.local.dirs", "",
+                      &old_config_format, "%");
+    ASSERT_STREQ("/var/run/yarn", split_values[0]);
+    ASSERT_STREQ("/tmp/mydir", split_values[1]);
+    ASSERT_EQ(NULL, split_values[2]);
+    free(split_values);
+    split_values = get_configuration_values_delimiter("allowed.system.users",
+                      "", &old_config_format, "%");
+    ASSERT_STREQ("nobody,daemon", split_values[0]);
+    ASSERT_EQ(NULL, split_values[1]);
+    free(split_values);
+  }
+
+  TEST_F(TestConfiguration, test_get_configuration_values) {
+    char **split_values;
+    split_values = get_configuration_values(NULL, "", &old_config_format);
+    ASSERT_EQ(NULL, split_values);
+    split_values = get_configuration_values("yarn.local.dirs", NULL, &old_config_format);
+    ASSERT_EQ(NULL, split_values);
+    split_values = get_configuration_values("yarn.local.dirs", "", NULL);
+    ASSERT_EQ(NULL, split_values);
+    split_values = get_configuration_values("yarn.local.dirs", "abcd", &old_config_format);
+    ASSERT_EQ(NULL, split_values);
+    split_values = get_configuration_values("yarn.local.dirs", "", &old_config_format);
+    ASSERT_STREQ("/var/run/yarn%/tmp/mydir", split_values[0]);
+    ASSERT_EQ(NULL, split_values[1]);
+    free(split_values);
+    split_values = get_configuration_values("allowed.system.users", "",
+                      &old_config_format);
+    ASSERT_STREQ("nobody", split_values[0]);
+    ASSERT_STREQ("daemon", split_values[1]);
+    ASSERT_EQ(NULL, split_values[2]);
+    free(split_values);
+  }
+
+  TEST_F(TestConfiguration, test_get_configuration_value) {
+    std::string key_value_array[5][2] = {
+        {"yarn.nodemanager.linux-container-executor.group", "yarn"},
+        {"min.user.id", "1000"},
+        {"allowed.system.users", "nobody,daemon"},
+        {"feature.docker.enabled", "1"},
+        {"yarn.local.dirs", "/var/run/yarn%/tmp/mydir"}
+    };
+    char *value;
+    value = get_configuration_value(NULL, "", &old_config_format);
+    ASSERT_EQ(NULL, value);
+    value = get_configuration_value("yarn.local.dirs", NULL, &old_config_format);
+    ASSERT_EQ(NULL, value);
+    value = get_configuration_value("yarn.local.dirs", "", NULL);
+    ASSERT_EQ(NULL, value);
+
+    for (int i = 0; i < 5; ++i) {
+      value = get_configuration_value(key_value_array[i][0].c_str(),
+                "", &old_config_format);
+      ASSERT_STREQ(key_value_array[i][1].c_str(), value);
+      free(value);
+    }
+    value = get_configuration_value("test.key", "", &old_config_format);
+    ASSERT_EQ(NULL, value);
+    value = get_configuration_value("test.key2", "", &old_config_format);
+    ASSERT_EQ(NULL, value);
+    value = get_configuration_value("feature.tc.enabled", "abcd", &old_config_format);
+    ASSERT_EQ(NULL, value);
+  }
+
+  TEST_F(TestConfiguration, test_no_sections_format) {
+    const struct section *executor_cfg = get_configuration_section("", &old_config_format);
+    char *value = NULL;
+    value = get_section_value("yarn.nodemanager.linux-container-executor.group", executor_cfg);
+    ASSERT_STREQ("yarn", value);
+    value = get_section_value("feature.docker.enabled", executor_cfg);
+    ASSERT_STREQ("1", value);
+    value = get_section_value("feature.tc.enabled", executor_cfg);
+    ASSERT_STREQ("0", value);
+    value = get_section_value("min.user.id", executor_cfg);
+    ASSERT_STREQ("1000", value);
+    value = get_section_value("docker.binary", executor_cfg);
+    ASSERT_STREQ("/usr/bin/docker", value);
+    char **list = get_section_values("allowed.system.users", executor_cfg);
+    ASSERT_STREQ("nobody", list[0]);
+    ASSERT_STREQ("daemon", list[1]);
+    list = get_section_values("banned.users", executor_cfg);
+    ASSERT_STREQ("root", list[0]);
+    ASSERT_STREQ("testuser1", list[1]);
+    ASSERT_STREQ("testuser2", list[2]);
+  }
+
+  TEST_F(TestConfiguration, test_get_section_values_delimiter) {
+    const struct section *section;
+    char *value;
+    char **split_values;
+    section = get_configuration_section("section-1", &new_config_format);
+    value = get_section_value("key1", section);
+    ASSERT_STREQ("value1", value);
+    free(value);
+    value = get_section_value("key2", section);
+    ASSERT_EQ(NULL, value);
+    split_values = get_section_values_delimiter(NULL, section, "%");
+    ASSERT_EQ(NULL, split_values);
+    split_values = get_section_values_delimiter("split-key", NULL, "%");
+    ASSERT_EQ(NULL, split_values);
+    split_values = get_section_values_delimiter("split-key", section, NULL);
+    ASSERT_EQ(NULL, split_values);
+    split_values = get_section_values_delimiter("split-key", section, "%");
+    ASSERT_FALSE(split_values == NULL);
+    ASSERT_STREQ("val1,val2,val3", split_values[0]);
+    ASSERT_TRUE(split_values[1] == NULL);
+    free_values(split_values);
+    split_values = get_section_values_delimiter("perc-key", section, "%");
+    ASSERT_FALSE(split_values == NULL);
+    ASSERT_STREQ("perc-val1", split_values[0]);
+    ASSERT_STREQ("perc-val2", split_values[1]);
+    ASSERT_TRUE(split_values[2] == NULL);
+  }
+
+  TEST_F(TestConfiguration, test_get_section_values) {
+    const struct section *section;
+    char *value;
+    char **split_values;
+    section = get_configuration_section("section-1", &new_config_format);
+    value = get_section_value(NULL, section);
+    ASSERT_EQ(NULL, value);
+    value = get_section_value("key1", NULL);
+    ASSERT_EQ(NULL, value);
+    value = get_section_value("key1", section);
+    ASSERT_STREQ("value1", value);
+    free(value);
+    value = get_section_value("key2", section);
+    ASSERT_EQ(NULL, value);
+    split_values = get_section_values("split-key", section);
+    ASSERT_FALSE(split_values == NULL);
+    ASSERT_STREQ("val1", split_values[0]);
+    ASSERT_STREQ("val2", split_values[1]);
+    ASSERT_STREQ("val3", split_values[2]);
+    ASSERT_TRUE(split_values[3] == NULL);
+    free_values(split_values);
+    split_values = get_section_values("perc-key", section);
+    ASSERT_FALSE(split_values == NULL);
+    ASSERT_STREQ("perc-val1%perc-val2", split_values[0]);
+    ASSERT_TRUE(split_values[1] == NULL);
+    free_values(split_values);
+    section = get_configuration_section("section-2", &new_config_format);
+    value = get_section_value("key1", section);
+    ASSERT_STREQ("value2", value);
+    free(value);
+    value = get_section_value("key2", section);
+    ASSERT_STREQ("value2", value);
+    free(value);
+  }
+
+  TEST_F(TestConfiguration, test_split_section) {
+    const struct section *section;
+    char *value;
+    section = get_configuration_section("split-section", &new_config_format);
+    value = get_section_value(NULL, section);
+    ASSERT_EQ(NULL, value);
+    value = get_section_value("key3", NULL);
+    ASSERT_EQ(NULL, value);
+    value = get_section_value("key3", section);
+    ASSERT_STREQ("value3", value);
+    free(value);
+    value = get_section_value("key4", section);
+    ASSERT_STREQ("value4", value);
+
+  }
+
+  TEST_F(TestConfiguration, test_get_configuration_section) {
+    const struct section *section;
+    ASSERT_EQ(3, new_config_format.size);
+    section = get_configuration_section(NULL, &new_config_format);
+    ASSERT_EQ(NULL, section);
+    section = get_configuration_section("section-1", NULL);
+    ASSERT_EQ(NULL, section);
+    section = get_configuration_section("section-1", &new_config_format);
+    ASSERT_FALSE(section == NULL);
+    ASSERT_STREQ("section-1", section->name);
+    ASSERT_EQ(3, section->size);
+    ASSERT_FALSE(NULL == section->kv_pairs);
+    section = get_configuration_section("section-2", &new_config_format);
+    ASSERT_FALSE(section == NULL);
+    ASSERT_STREQ("section-2", section->name);
+    ASSERT_EQ(2, section->size);
+    ASSERT_FALSE(NULL == section->kv_pairs);
+    section = get_configuration_section("section-3", &new_config_format);
+    ASSERT_TRUE(section == NULL);
+  }
+
+  TEST_F(TestConfiguration, test_read_config) {
+    struct configuration config;
+    int ret = 0;
+
+    ret = read_config(NULL, &config);
+    ASSERT_EQ(INVALID_CONFIG_FILE, ret);
+    ret = read_config("bad-config-file", &config);
+    ASSERT_EQ(INVALID_CONFIG_FILE, ret);
+    ret = read_config(new_config_format_file.c_str(), &config);
+    ASSERT_EQ(0, ret);
+    ASSERT_EQ(3, config.size);
+    ASSERT_STREQ("section-1", config.sections[0]->name);
+    ASSERT_STREQ("split-section", config.sections[1]->name);
+    ASSERT_STREQ("section-2", config.sections[2]->name);
+    free_configuration(&config);
+    ret = read_config(old_config_format_file.c_str(), &config);
+    ASSERT_EQ(0, ret);
+    ASSERT_EQ(1, config.size);
+    ASSERT_STREQ("", config.sections[0]->name);
+    free_configuration(&config);
+  }
+
+  TEST_F(TestConfiguration, test_get_kv_key) {
+    int ret = 0;
+    char buff[1024];
+    ret = get_kv_key(NULL, buff, 1024);
+    ASSERT_EQ(-EINVAL, ret);
+    ret = get_kv_key("key1234", buff, 1024);
+    ASSERT_EQ(-EINVAL, ret);
+    ret = get_kv_key("key=abcd", NULL, 1024);
+    ASSERT_EQ(-ENAMETOOLONG, ret);
+    ret = get_kv_key("key=abcd", buff, 1);
+    ASSERT_EQ(-ENAMETOOLONG, ret);
+    ret = get_kv_key("key=abcd", buff, 1024);
+    ASSERT_EQ(0, ret);
+    ASSERT_STREQ("key", buff);
+  }
+
+  TEST_F(TestConfiguration, test_get_kv_value) {
+    int ret = 0;
+    char buff[1024];
+    ret = get_kv_value(NULL, buff, 1024);
+    ASSERT_EQ(-EINVAL, ret);
+    ret = get_kv_value("key1234", buff, 1024);
+    ASSERT_EQ(-EINVAL, ret);
+    ret = get_kv_value("key=abcd", NULL, 1024);
+    ASSERT_EQ(-ENAMETOOLONG, ret);
+    ret = get_kv_value("key=abcd", buff, 1);
+    ASSERT_EQ(-ENAMETOOLONG, ret);
+    ret = get_kv_value("key=abcd", buff, 1024);
+    ASSERT_EQ(0, ret);
+    ASSERT_STREQ("abcd", buff);
+  }
+
+  TEST_F(TestConfiguration, test_single_section_high_key_count) {
+    std::string section_name = "section-1";
+    std::string sample_file_name = "large-section.cfg";
+    std::ofstream sample_file;
+    sample_file.open(sample_file_name.c_str());
+    sample_file << "[" << section_name << "]" << std::endl;
+    for(int i = 0; i < MAX_SIZE + 2; ++i) {
+      sample_file << "key" << i << "=" << "value" << i << std::endl;
+    }
+    struct configuration cfg;
+    int ret = read_config(sample_file_name.c_str(), &cfg);
+    ASSERT_EQ(0, ret);
+    ASSERT_EQ(1, cfg.size);
+    const struct section *section1 = get_configuration_section(section_name.c_str(), &cfg);
+    ASSERT_EQ(MAX_SIZE + 2, section1->size);
+    ASSERT_STREQ(section_name.c_str(), section1->name);
+    for(int i = 0; i < MAX_SIZE + 2; ++i) {
+      std::ostringstream oss;
+      oss << "key" << i;
+      const char *value = get_section_value(oss.str().c_str(), section1);
+      oss.str("");
+      oss << "value" << i;
+      ASSERT_STREQ(oss.str().c_str(), value);
+    }
+    remove(sample_file_name.c_str());
+    free_configuration(&cfg);
+  }
+
+  TEST_F(TestConfiguration, test_multiple_sections) {
+    std::string sample_file_name = "multiple-sections.cfg";
+    std::ofstream sample_file;
+    sample_file.open(sample_file_name.c_str());
+    for(int i = 0; i < MAX_SIZE + 2; ++i) {
+      sample_file << "[section-" << i << "]" << std::endl;
+      sample_file << "key" << i << "=" << "value" << i << std::endl;
+    }
+    struct configuration cfg;
+    int ret = read_config(sample_file_name.c_str(), &cfg);
+    ASSERT_EQ(0, ret);
+    ASSERT_EQ(MAX_SIZE + 2, cfg.size);
+    for(int i = 0; i < MAX_SIZE + 2; ++i) {
+      std::ostringstream oss;
+      oss << "section-" << i;
+      const struct section *section = get_configuration_section(oss.str().c_str(), &cfg);
+      ASSERT_EQ(1, section->size);
+      ASSERT_STREQ(oss.str().c_str(), section->name);
+      oss.str("");
+      oss << "key" << i;
+      const char *value = get_section_value(oss.str().c_str(), section);
+      oss.str("");
+      oss << "value" << i;
+      ASSERT_STREQ(oss.str().c_str(), value);
+    }
+    remove(sample_file_name.c_str());
+    free_configuration(&cfg);
+  }
+
+  TEST_F(TestConfiguration, test_section_start_line) {
+    const char *section_start_line = "[abcd]";
+    const char *non_section_lines[] = {
+        "[abcd", "abcd]", "key=value", "#abcd"
+    };
+    int ret = is_section_start_line(section_start_line);
+    ASSERT_EQ(1, ret);
+    int length = sizeof(non_section_lines) / sizeof(*non_section_lines);
+    for( int i = 0; i < length; ++i) {
+      ret = is_section_start_line(non_section_lines[i]);
+      ASSERT_EQ(0, ret);
+    }
+    ret = is_section_start_line(NULL);
+    ASSERT_EQ(0, ret);
+  }
+
+  TEST_F(TestConfiguration, test_comment_line) {
+    const char *comment_line = "#[abcd]";
+    const char *non_comment_lines[] = {
+        "[abcd", "abcd]", "key=value", "[abcd]"
+    };
+    int ret = is_comment_line(comment_line);
+    ASSERT_EQ(1, ret);
+    int length = sizeof(non_comment_lines) / sizeof(*non_comment_lines);
+    for( int i = 0; i < length; ++i) {
+      ret = is_comment_line(non_comment_lines[i]);
+      ASSERT_EQ(0, ret);
+    }
+    ret = is_comment_line(NULL);
+    ASSERT_EQ(0, ret);
+  }
+
+  TEST_F(TestConfiguration, test_mixed_config_format) {
+    const struct section *executor_cfg =
+        get_configuration_section("", &mixed_config_format);
+    char *value = NULL;
+    value = get_section_value("key1", executor_cfg);
+    ASSERT_STREQ("value1", value);
+    value = get_section_value("key2", executor_cfg);
+    ASSERT_STREQ("value2", value);
+    ASSERT_EQ(2, executor_cfg->size);
+    executor_cfg = get_configuration_section("section-1",
+                                             &mixed_config_format);
+    value = get_section_value("key3", executor_cfg);
+    ASSERT_STREQ("value3", value);
+    value = get_section_value("key1", executor_cfg);
+    ASSERT_STREQ("value4", value);
+    ASSERT_EQ(2, executor_cfg->size);
+    ASSERT_EQ(2, mixed_config_format.size);
+    ASSERT_STREQ("", mixed_config_format.sections[0]->name);
+    ASSERT_STREQ("section-1", mixed_config_format.sections[1]->name);
+  }
+}

+ 29 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test_main.cc

@@ -0,0 +1,29 @@
+/**
+ * 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.
+ */
+
+#include <gtest/gtest.h>
+#include <main/native/container-executor/impl/util.h>
+#include <cstdio>
+
+FILE* ERRORFILE = stderr;
+FILE* LOGFILE = stdout;
+
+int main(int argc, char **argv) {
+    testing::InitGoogleTest(&argc, argv);
+    return RUN_ALL_TESTS();
+}

+ 138 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test_util.cc

@@ -0,0 +1,138 @@
+/**
+ * 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.
+ */
+
+#include <gtest/gtest.h>
+#include <sstream>
+
+extern "C" {
+#include "util.h"
+}
+
+namespace ContainerExecutor {
+
+  class TestUtil : public ::testing::Test {
+  protected:
+    virtual void SetUp() {
+    }
+
+    virtual void TearDown() {
+    }
+  };
+
+  TEST_F(TestUtil, test_split_delimiter) {
+    std::string str = "1,2,3,4,5,6,7,8,9,10,11";
+    char *split_string = (char *) calloc(str.length() + 1, sizeof(char));
+    strncpy(split_string, str.c_str(), str.length());
+    char **splits = split_delimiter(split_string, ",");
+    ASSERT_TRUE(splits != NULL);
+    int count = 0;
+    while(splits[count] != NULL) {
+      ++count;
+    }
+    ASSERT_EQ(11, count);
+    for(int i = 1; i < count; ++i) {
+      std::ostringstream oss;
+      oss << i;
+      ASSERT_STREQ(oss.str().c_str(), splits[i-1]);
+    }
+    ASSERT_EQ(NULL, splits[count]);
+    free_values(splits);
+
+    split_string = (char *) calloc(str.length() + 1, sizeof(char));
+    strncpy(split_string, str.c_str(), str.length());
+    splits = split_delimiter(split_string, "%");
+    ASSERT_TRUE(splits != NULL);
+    ASSERT_TRUE(splits[1] == NULL);
+    ASSERT_STREQ(str.c_str(), splits[0]);
+    free_values(splits);
+
+    splits = split_delimiter(NULL, ",");
+    ASSERT_EQ(NULL, splits);
+    return;
+  }
+
+  TEST_F(TestUtil, test_split) {
+    std::string str = "1%2%3%4%5%6%7%8%9%10%11";
+    char *split_string = (char *) calloc(str.length() + 1, sizeof(char));
+    strncpy(split_string, str.c_str(), str.length());
+    char **splits = split(split_string);
+    int count = 0;
+    while(splits[count] != NULL) {
+      ++count;
+    }
+    ASSERT_EQ(11, count);
+    for(int i = 1; i < count; ++i) {
+      std::ostringstream oss;
+      oss << i;
+      ASSERT_STREQ(oss.str().c_str(), splits[i-1]);
+    }
+    ASSERT_EQ(NULL, splits[count]);
+    free_values(splits);
+
+    str = "1,2,3,4,5,6,7,8,9,10,11";
+    split_string = (char *) calloc(str.length() + 1, sizeof(char));
+    strncpy(split_string, str.c_str(), str.length());
+    splits = split(split_string);
+    ASSERT_TRUE(splits != NULL);
+    ASSERT_TRUE(splits[1] == NULL);
+    ASSERT_STREQ(str.c_str(), splits[0]);
+    return;
+  }
+
+  TEST_F(TestUtil, test_trim) {
+    char* trimmed = NULL;
+
+    // Check NULL input
+    ASSERT_EQ(NULL, trim(NULL));
+
+    // Check empty input
+    trimmed = trim("");
+    ASSERT_STREQ("", trimmed);
+    free(trimmed);
+
+    // Check single space input
+    trimmed = trim(" ");
+    ASSERT_STREQ("", trimmed);
+    free(trimmed);
+
+    // Check multi space input
+    trimmed = trim("   ");
+    ASSERT_STREQ("", trimmed);
+    free(trimmed);
+
+    // Check both side trim input
+    trimmed = trim(" foo ");
+    ASSERT_STREQ("foo", trimmed);
+    free(trimmed);
+
+    // Check left side trim input
+    trimmed = trim("foo   ");
+    ASSERT_STREQ("foo", trimmed);
+    free(trimmed);
+
+    // Check right side trim input
+    trimmed = trim("   foo");
+    ASSERT_STREQ("foo", trimmed);
+    free(trimmed);
+
+    // Check no trim input
+    trimmed = trim("foo");
+    ASSERT_STREQ("foo", trimmed);
+    free(trimmed);
+  }
+}

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

@@ -310,13 +310,13 @@ public abstract class BaseContainerManagerTest {
         new HashSet<>(finalStates);
         new HashSet<>(finalStates);
     int timeoutSecs = 0;
     int timeoutSecs = 0;
     do {
     do {
-      Thread.sleep(2000);
+      Thread.sleep(1000);
       containerStatus =
       containerStatus =
           containerManager.getContainerStatuses(request)
           containerManager.getContainerStatuses(request)
               .getContainerStatuses().get(0);
               .getContainerStatuses().get(0);
       LOG.info("Waiting for container to get into one of states " + fStates
       LOG.info("Waiting for container to get into one of states " + fStates
           + ". Current state is " + containerStatus.getState());
           + ". Current state is " + containerStatus.getState());
-      timeoutSecs += 2;
+      timeoutSecs += 1;
     } while (!fStates.contains(containerStatus.getState())
     } while (!fStates.contains(containerStatus.getState())
         && timeoutSecs < timeOutMax);
         && timeoutSecs < timeOutMax);
     LOG.info("Container state is " + containerStatus.getState());
     LOG.info("Container state is " + containerStatus.getState());
@@ -371,7 +371,7 @@ public abstract class BaseContainerManagerTest {
         .containermanager.container.ContainerState currentState = null;
         .containermanager.container.ContainerState currentState = null;
     int timeoutSecs = 0;
     int timeoutSecs = 0;
     do {
     do {
-      Thread.sleep(2000);
+      Thread.sleep(1000);
       container =
       container =
           containerManager.getContext().getContainers().get(containerID);
           containerManager.getContext().getContainers().get(containerID);
       if (container != null) {
       if (container != null) {
@@ -381,9 +381,9 @@ public abstract class BaseContainerManagerTest {
         LOG.info("Waiting for NM container to get into one of the following " +
         LOG.info("Waiting for NM container to get into one of the following " +
             "states: " + finalStates + ". Current state is " + currentState);
             "states: " + finalStates + ". Current state is " + currentState);
       }
       }
-      timeoutSecs += 2;
+      timeoutSecs += 1;
     } while (!finalStates.contains(currentState)
     } while (!finalStates.contains(currentState)
-        && timeoutSecs++ < timeOutMax);
+        && timeoutSecs < timeOutMax);
     LOG.info("Container state is " + currentState);
     LOG.info("Container state is " + currentState);
     Assert.assertTrue("ContainerState is not correct (timedout)",
     Assert.assertTrue("ContainerState is not correct (timedout)",
         finalStates.contains(currentState));
         finalStates.contains(currentState));

+ 3 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ActiveStandbyElectorBasedElectorService.java

@@ -31,6 +31,7 @@ import org.apache.hadoop.ha.ServiceFailedException;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.ZKUtil;
 import org.apache.hadoop.util.ZKUtil;
+import org.apache.hadoop.util.curator.ZKCuratorManager;
 import org.apache.hadoop.yarn.conf.HAUtil;
 import org.apache.hadoop.yarn.conf.HAUtil;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
@@ -96,8 +97,8 @@ public class ActiveStandbyElectorBasedElectorService extends AbstractService
     zkSessionTimeout = conf.getLong(YarnConfiguration.RM_ZK_TIMEOUT_MS,
     zkSessionTimeout = conf.getLong(YarnConfiguration.RM_ZK_TIMEOUT_MS,
         YarnConfiguration.DEFAULT_RM_ZK_TIMEOUT_MS);
         YarnConfiguration.DEFAULT_RM_ZK_TIMEOUT_MS);
 
 
-    List<ACL> zkAcls = RMZKUtils.getZKAcls(conf);
-    List<ZKUtil.ZKAuthInfo> zkAuths = RMZKUtils.getZKAuths(conf);
+    List<ACL> zkAcls = ZKCuratorManager.getZKAcls(conf);
+    List<ZKUtil.ZKAuthInfo> zkAuths = ZKCuratorManager.getZKAuths(conf);
 
 
     int maxRetryNum =
     int maxRetryNum =
         conf.getInt(YarnConfiguration.RM_HA_FC_ELECTOR_ZK_RETRIES_KEY, conf
         conf.getInt(YarnConfiguration.RM_HA_FC_ELECTOR_ZK_RETRIES_KEY, conf

+ 0 - 81
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMZKUtils.java

@@ -1,81 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.server.resourcemanager;
-
-import java.io.IOException;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.util.ZKUtil;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.zookeeper.data.ACL;
-
-import java.util.Collections;
-import java.util.List;
-
-/**
- * Helper class that provides utility methods specific to ZK operations
- */
-@InterfaceAudience.Private
-public class RMZKUtils {
-  private static final Log LOG = LogFactory.getLog(RMZKUtils.class);
-
-  /**
-   * Utility method to fetch the ZK ACLs from the configuration.
-   *
-   * @throws java.io.IOException if the Zookeeper ACLs configuration file
-   * cannot be read
-   */
-  public static List<ACL> getZKAcls(Configuration conf) throws IOException {
-    // Parse authentication from configuration.
-    String zkAclConf =
-        conf.get(YarnConfiguration.RM_ZK_ACL,
-            YarnConfiguration.DEFAULT_RM_ZK_ACL);
-    try {
-      zkAclConf = ZKUtil.resolveConfIndirection(zkAclConf);
-      return ZKUtil.parseACLs(zkAclConf);
-    } catch (IOException | ZKUtil.BadAclFormatException e) {
-      LOG.error("Couldn't read ACLs based on " + YarnConfiguration.RM_ZK_ACL);
-      throw e;
-    }
-  }
-
-  /**
-   * Utility method to fetch ZK auth info from the configuration.
-   *
-   * @throws java.io.IOException if the Zookeeper ACLs configuration file
-   * cannot be read
-   */
-  public static List<ZKUtil.ZKAuthInfo> getZKAuths(Configuration conf)
-      throws IOException {
-    String zkAuthConf = conf.get(YarnConfiguration.RM_ZK_AUTH);
-    try {
-      zkAuthConf = ZKUtil.resolveConfIndirection(zkAuthConf);
-      if (zkAuthConf != null) {
-        return ZKUtil.parseAuth(zkAuthConf);
-      } else {
-        return Collections.emptyList();
-      }
-    } catch (IOException | ZKUtil.BadAuthFormatException e) {
-      LOG.error("Couldn't read Auth based on " + YarnConfiguration.RM_ZK_AUTH);
-      throw e;
-    }
-  }
-}

+ 43 - 49
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java

@@ -22,8 +22,6 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.curator.framework.AuthInfo;
 import org.apache.curator.framework.AuthInfo;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFramework;
-import org.apache.curator.framework.CuratorFrameworkFactory;
-import org.apache.curator.retry.RetryNTimes;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.ha.HAServiceProtocol;
 import org.apache.hadoop.ha.HAServiceProtocol;
@@ -46,7 +44,7 @@ import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.ShutdownHookManager;
 import org.apache.hadoop.util.ShutdownHookManager;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.VersionInfo;
 import org.apache.hadoop.util.VersionInfo;
-import org.apache.hadoop.util.ZKUtil;
+import org.apache.hadoop.util.curator.ZKCuratorManager;
 import org.apache.hadoop.yarn.YarnUncaughtExceptionHandler;
 import org.apache.hadoop.yarn.YarnUncaughtExceptionHandler;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -92,7 +90,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAlloca
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEventType;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.PreemptableResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 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.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
@@ -192,7 +189,7 @@ public class ResourceManager extends CompositeService implements Recoverable {
   protected ResourceTrackerService resourceTracker;
   protected ResourceTrackerService resourceTracker;
   private JvmMetrics jvmMetrics;
   private JvmMetrics jvmMetrics;
   private boolean curatorEnabled = false;
   private boolean curatorEnabled = false;
-  private CuratorFramework curator;
+  private ZKCuratorManager zkManager;
   private final String zkRootNodePassword =
   private final String zkRootNodePassword =
       Long.toString(new SecureRandom().nextLong());
       Long.toString(new SecureRandom().nextLong());
   private boolean recoveryEnabled;
   private boolean recoveryEnabled;
@@ -345,7 +342,7 @@ public class ResourceManager extends CompositeService implements Recoverable {
         conf.getBoolean(YarnConfiguration.CURATOR_LEADER_ELECTOR,
         conf.getBoolean(YarnConfiguration.CURATOR_LEADER_ELECTOR,
             YarnConfiguration.DEFAULT_CURATOR_LEADER_ELECTOR_ENABLED);
             YarnConfiguration.DEFAULT_CURATOR_LEADER_ELECTOR_ENABLED);
     if (curatorEnabled) {
     if (curatorEnabled) {
-      this.curator = createAndStartCurator(conf);
+      this.zkManager = createAndStartZKManager(conf);
       elector = new CuratorBasedElectorService(this);
       elector = new CuratorBasedElectorService(this);
     } else {
     } else {
       elector = new ActiveStandbyElectorBasedElectorService(this);
       elector = new ActiveStandbyElectorBasedElectorService(this);
@@ -353,50 +350,49 @@ public class ResourceManager extends CompositeService implements Recoverable {
     return elector;
     return elector;
   }
   }
 
 
-  public CuratorFramework createAndStartCurator(Configuration conf)
+  /**
+   * Create and ZooKeeper Curator manager.
+   * @param config Configuration for the ZooKeeper curator.
+   * @return New ZooKeeper Curator manager.
+   * @throws IOException If it cannot create the manager.
+   */
+  public ZKCuratorManager createAndStartZKManager(Configuration config)
       throws IOException {
       throws IOException {
-    String zkHostPort = conf.get(YarnConfiguration.RM_ZK_ADDRESS);
-    if (zkHostPort == null) {
-      throw new YarnRuntimeException(
-          YarnConfiguration.RM_ZK_ADDRESS + " is not configured.");
-    }
-    int numRetries = conf.getInt(YarnConfiguration.RM_ZK_NUM_RETRIES,
-        YarnConfiguration.DEFAULT_ZK_RM_NUM_RETRIES);
-    int zkSessionTimeout = conf.getInt(YarnConfiguration.RM_ZK_TIMEOUT_MS,
-        YarnConfiguration.DEFAULT_RM_ZK_TIMEOUT_MS);
-    int zkRetryInterval = conf.getInt(YarnConfiguration.RM_ZK_RETRY_INTERVAL_MS,
-        YarnConfiguration.DEFAULT_RM_ZK_RETRY_INTERVAL_MS);
-
-    // set up zk auths
-    List<ZKUtil.ZKAuthInfo> zkAuths = RMZKUtils.getZKAuths(conf);
+    ZKCuratorManager manager = new ZKCuratorManager(config);
+
+    // Get authentication
     List<AuthInfo> authInfos = new ArrayList<>();
     List<AuthInfo> authInfos = new ArrayList<>();
-    for (ZKUtil.ZKAuthInfo zkAuth : zkAuths) {
-      authInfos.add(new AuthInfo(zkAuth.getScheme(), zkAuth.getAuth()));
+    if (HAUtil.isHAEnabled(config) && HAUtil.getConfValueForRMInstance(
+        YarnConfiguration.ZK_RM_STATE_STORE_ROOT_NODE_ACL, config) == null) {
+      String zkRootNodeUsername = HAUtil.getConfValueForRMInstance(
+          YarnConfiguration.RM_ADDRESS,
+          YarnConfiguration.DEFAULT_RM_ADDRESS, config);
+      String defaultFencingAuth =
+          zkRootNodeUsername + ":" + zkRootNodePassword;
+      byte[] defaultFencingAuthData =
+          defaultFencingAuth.getBytes(Charset.forName("UTF-8"));
+      String scheme = new DigestAuthenticationProvider().getScheme();
+      AuthInfo authInfo = new AuthInfo(scheme, defaultFencingAuthData);
+      authInfos.add(authInfo);
     }
     }
 
 
-    if (HAUtil.isHAEnabled(conf) && HAUtil.getConfValueForRMInstance(
-        YarnConfiguration.ZK_RM_STATE_STORE_ROOT_NODE_ACL, conf) == null) {
-      String zkRootNodeUsername = HAUtil
-          .getConfValueForRMInstance(YarnConfiguration.RM_ADDRESS,
-              YarnConfiguration.DEFAULT_RM_ADDRESS, conf);
-      byte[] defaultFencingAuth =
-          (zkRootNodeUsername + ":" + zkRootNodePassword)
-              .getBytes(Charset.forName("UTF-8"));
-      authInfos.add(new AuthInfo(new DigestAuthenticationProvider().getScheme(),
-          defaultFencingAuth));
-    }
+    manager.start(authInfos);
+    return manager;
+  }
 
 
-    CuratorFramework client =  CuratorFrameworkFactory.builder()
-        .connectString(zkHostPort)
-        .sessionTimeoutMs(zkSessionTimeout)
-        .retryPolicy(new RetryNTimes(numRetries, zkRetryInterval))
-        .authorization(authInfos).build();
-    client.start();
-    return client;
+  /**
+   * Get the ZooKeeper Curator manager.
+   * @return ZooKeeper Curator manager.
+   */
+  public ZKCuratorManager getZKManager() {
+    return this.zkManager;
   }
   }
 
 
   public CuratorFramework getCurator() {
   public CuratorFramework getCurator() {
-    return this.curator;
+    if (this.zkManager == null) {
+      return null;
+    }
+    return this.zkManager.getCurator();
   }
   }
 
 
   public String getZkRootNodePassword() {
   public String getZkRootNodePassword() {
@@ -701,8 +697,7 @@ public class ResourceManager extends CompositeService implements Recoverable {
         }
         }
       }
       }
 
 
-      // creating monitors that handle preemption
-      createPolicyMonitors();
+      createSchedulerMonitors();
 
 
       masterService = createApplicationMasterService();
       masterService = createApplicationMasterService();
       addService(masterService) ;
       addService(masterService) ;
@@ -803,9 +798,8 @@ public class ResourceManager extends CompositeService implements Recoverable {
 
 
     }
     }
 
 
-    protected void createPolicyMonitors() {
-      if (scheduler instanceof PreemptableResourceScheduler
-          && conf.getBoolean(YarnConfiguration.RM_SCHEDULER_ENABLE_MONITORS,
+    protected void createSchedulerMonitors() {
+      if (conf.getBoolean(YarnConfiguration.RM_SCHEDULER_ENABLE_MONITORS,
           YarnConfiguration.DEFAULT_RM_SCHEDULER_ENABLE_MONITORS)) {
           YarnConfiguration.DEFAULT_RM_SCHEDULER_ENABLE_MONITORS)) {
         LOG.info("Loading policy monitors");
         LOG.info("Loading policy monitors");
         List<SchedulingEditPolicy> policies = conf.getInstances(
         List<SchedulingEditPolicy> policies = conf.getInstances(
@@ -1264,8 +1258,8 @@ public class ResourceManager extends CompositeService implements Recoverable {
       configurationProvider.close();
       configurationProvider.close();
     }
     }
     super.serviceStop();
     super.serviceStop();
-    if (curator != null) {
-      curator.close();
+    if (zkManager != null) {
+      zkManager.close();
     }
     }
     transitionToStandby(false);
     transitionToStandby(false);
     rmContext.setHAServiceState(HAServiceState.STOPPING);
     rmContext.setHAServiceState(HAServiceState.STOPPING);

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/SchedulingEditPolicy.java

@@ -19,12 +19,12 @@ package org.apache.hadoop.yarn.server.resourcemanager.monitor;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.PreemptableResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 
 
 public interface SchedulingEditPolicy {
 public interface SchedulingEditPolicy {
 
 
   void init(Configuration config, RMContext context,
   void init(Configuration config, RMContext context,
-      PreemptableResourceScheduler scheduler);
+      ResourceScheduler scheduler);
 
 
   /**
   /**
    * This method is invoked at regular intervals. Internally the policy is
    * This method is invoked at regular intervals. Internally the policy is

+ 1 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/SchedulingMonitor.java

@@ -29,7 +29,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.PreemptableResourceScheduler;
 
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.annotations.VisibleForTesting;
 
 
@@ -59,8 +58,7 @@ public class SchedulingMonitor extends AbstractService {
   }
   }
 
 
   public void serviceInit(Configuration conf) throws Exception {
   public void serviceInit(Configuration conf) throws Exception {
-    scheduleEditPolicy.init(conf, rmContext,
-        (PreemptableResourceScheduler) rmContext.getScheduler());
+    scheduleEditPolicy.init(conf, rmContext, rmContext.getScheduler());
     this.monitorInterval = scheduleEditPolicy.getMonitoringInterval();
     this.monitorInterval = scheduleEditPolicy.getMonitoringInterval();
     super.serviceInit(conf);
     super.serviceInit(conf);
   }
   }

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.java

@@ -32,7 +32,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingEditPolicy;
 import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingEditPolicy;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.PreemptableResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
 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.capacity.CapacitySchedulerConfiguration;
@@ -150,7 +150,7 @@ public class ProportionalCapacityPreemptionPolicy
   }
   }
 
 
   public void init(Configuration config, RMContext context,
   public void init(Configuration config, RMContext context,
-      PreemptableResourceScheduler sched) {
+      ResourceScheduler sched) {
     LOG.info("Preemption monitor:" + this.getClass().getCanonicalName());
     LOG.info("Preemption monitor:" + this.getClass().getCanonicalName());
     assert null == scheduler : "Unexpected duplicate call to init";
     assert null == scheduler : "Unexpected duplicate call to init";
     if (!(sched instanceof CapacityScheduler)) {
     if (!(sched instanceof CapacityScheduler)) {

+ 5 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/invariants/InvariantsChecker.java

@@ -20,7 +20,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.monitor.invariants;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingEditPolicy;
 import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingEditPolicy;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.PreemptableResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.slf4j.Logger;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
 
 
@@ -39,16 +39,16 @@ public abstract class InvariantsChecker implements SchedulingEditPolicy {
 
 
   private Configuration conf;
   private Configuration conf;
   private RMContext context;
   private RMContext context;
-  private PreemptableResourceScheduler scheduler;
+  private ResourceScheduler scheduler;
   private boolean throwOnInvariantViolation;
   private boolean throwOnInvariantViolation;
   private long monitoringInterval;
   private long monitoringInterval;
 
 
   @Override
   @Override
   public void init(Configuration config, RMContext rmContext,
   public void init(Configuration config, RMContext rmContext,
-      PreemptableResourceScheduler preemptableResourceScheduler) {
+      ResourceScheduler scheduler) {
     this.conf = config;
     this.conf = config;
     this.context = rmContext;
     this.context = rmContext;
-    this.scheduler = preemptableResourceScheduler;
+    this.scheduler = scheduler;
     this.throwOnInvariantViolation =
     this.throwOnInvariantViolation =
         conf.getBoolean(InvariantsChecker.THROW_ON_VIOLATION, false);
         conf.getBoolean(InvariantsChecker.THROW_ON_VIOLATION, false);
     this.monitoringInterval =
     this.monitoringInterval =
@@ -89,7 +89,7 @@ public abstract class InvariantsChecker implements SchedulingEditPolicy {
     return context;
     return context;
   }
   }
 
 
-  public PreemptableResourceScheduler getScheduler() {
+  public ResourceScheduler getScheduler() {
     return scheduler;
     return scheduler;
   }
   }
 
 

+ 3 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/invariants/MetricsInvariantChecker.java

@@ -27,8 +27,8 @@ import org.apache.hadoop.metrics2.impl.MetricsCollectorImpl;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.metrics2.source.JvmMetrics;
 import org.apache.hadoop.metrics2.source.JvmMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.PreemptableResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.slf4j.Logger;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
 
 
@@ -42,7 +42,6 @@ import java.io.IOException;
 import java.util.HashMap;
 import java.util.HashMap;
 import java.util.List;
 import java.util.List;
 import java.util.Map;
 import java.util.Map;
-import java.util.stream.Collectors;
 
 
 /**
 /**
  * This policy checks at every invocation that a given set of invariants
  * This policy checks at every invocation that a given set of invariants
@@ -78,9 +77,9 @@ public class MetricsInvariantChecker extends InvariantsChecker {
 
 
   @Override
   @Override
   public void init(Configuration config, RMContext rmContext,
   public void init(Configuration config, RMContext rmContext,
-      PreemptableResourceScheduler preemptableResourceScheduler) {
+      ResourceScheduler scheduler) {
 
 
-    super.init(config, rmContext, preemptableResourceScheduler);
+    super.init(config, rmContext, scheduler);
 
 
     this.metricsSystem = DefaultMetricsSystem.instance();
     this.metricsSystem = DefaultMetricsSystem.instance();
     this.queueMetrics =
     this.queueMetrics =

+ 17 - 21
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/ZKRMStateStore.java

@@ -31,6 +31,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.security.token.delegation.DelegationKey;
 import org.apache.hadoop.security.token.delegation.DelegationKey;
 import org.apache.hadoop.util.ZKUtil;
 import org.apache.hadoop.util.ZKUtil;
+import org.apache.hadoop.util.curator.ZKCuratorManager;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.ReservationId;
@@ -46,7 +47,6 @@ import org.apache.hadoop.yarn.proto.YarnProtos.ReservationAllocationStateProto;
 import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
 import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
 import org.apache.hadoop.yarn.server.records.Version;
 import org.apache.hadoop.yarn.server.records.Version;
 import org.apache.hadoop.yarn.server.records.impl.pb.VersionPBImpl;
 import org.apache.hadoop.yarn.server.records.impl.pb.VersionPBImpl;
-import org.apache.hadoop.yarn.server.resourcemanager.RMZKUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.AMRMTokenSecretManagerState;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.AMRMTokenSecretManagerState;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.ApplicationAttemptStateData;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.ApplicationAttemptStateData;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.ApplicationStateData;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.ApplicationStateData;
@@ -201,8 +201,8 @@ public class ZKRMStateStore extends RMStateStore {
   private final String zkRootNodeAuthScheme =
   private final String zkRootNodeAuthScheme =
       new DigestAuthenticationProvider().getScheme();
       new DigestAuthenticationProvider().getScheme();
 
 
-  @VisibleForTesting
-  protected CuratorFramework curatorFramework;
+  /** Manager for the ZooKeeper connection. */
+  private ZKCuratorManager zkManager;
 
 
   /*
   /*
    * Indicates different app attempt state store operations.
    * Indicates different app attempt state store operations.
@@ -298,12 +298,11 @@ public class ZKRMStateStore extends RMStateStore {
       appIdNodeSplitIndex = YarnConfiguration.DEFAULT_ZK_APPID_NODE_SPLIT_INDEX;
       appIdNodeSplitIndex = YarnConfiguration.DEFAULT_ZK_APPID_NODE_SPLIT_INDEX;
     }
     }
 
 
-    zkAcl = RMZKUtils.getZKAcls(conf);
+    zkAcl = ZKCuratorManager.getZKAcls(conf);
 
 
     if (HAUtil.isHAEnabled(conf)) {
     if (HAUtil.isHAEnabled(conf)) {
       String zkRootNodeAclConf = HAUtil.getConfValueForRMInstance
       String zkRootNodeAclConf = HAUtil.getConfValueForRMInstance
           (YarnConfiguration.ZK_RM_STATE_STORE_ROOT_NODE_ACL, conf);
           (YarnConfiguration.ZK_RM_STATE_STORE_ROOT_NODE_ACL, conf);
-
       if (zkRootNodeAclConf != null) {
       if (zkRootNodeAclConf != null) {
         zkRootNodeAclConf = ZKUtil.resolveConfIndirection(zkRootNodeAclConf);
         zkRootNodeAclConf = ZKUtil.resolveConfIndirection(zkRootNodeAclConf);
 
 
@@ -330,10 +329,9 @@ public class ZKRMStateStore extends RMStateStore {
     amrmTokenSecretManagerRoot =
     amrmTokenSecretManagerRoot =
         getNodePath(zkRootNodePath, AMRMTOKEN_SECRET_MANAGER_ROOT);
         getNodePath(zkRootNodePath, AMRMTOKEN_SECRET_MANAGER_ROOT);
     reservationRoot = getNodePath(zkRootNodePath, RESERVATION_SYSTEM_ROOT);
     reservationRoot = getNodePath(zkRootNodePath, RESERVATION_SYSTEM_ROOT);
-    curatorFramework = resourceManager.getCurator();
-
-    if (curatorFramework == null) {
-      curatorFramework = resourceManager.createAndStartCurator(conf);
+    zkManager = resourceManager.getZKManager();
+    if (zkManager == null) {
+      zkManager = resourceManager.createAndStartZKManager(conf);
     }
     }
   }
   }
 
 
@@ -382,6 +380,7 @@ public class ZKRMStateStore extends RMStateStore {
       logRootNodeAcls("Before setting ACLs'\n");
       logRootNodeAcls("Before setting ACLs'\n");
     }
     }
 
 
+    CuratorFramework curatorFramework = zkManager.getCurator();
     if (HAUtil.isHAEnabled(getConfig())) {
     if (HAUtil.isHAEnabled(getConfig())) {
       curatorFramework.setACL().withACL(zkRootNodeAcl).forPath(zkRootNodePath);
       curatorFramework.setACL().withACL(zkRootNodeAcl).forPath(zkRootNodePath);
     } else {
     } else {
@@ -401,6 +400,7 @@ public class ZKRMStateStore extends RMStateStore {
     }
     }
 
 
     if (!HAUtil.isHAEnabled(getConfig())) {
     if (!HAUtil.isHAEnabled(getConfig())) {
+      CuratorFramework curatorFramework = zkManager.getCurator();
       IOUtils.closeStream(curatorFramework);
       IOUtils.closeStream(curatorFramework);
     }
     }
   }
   }
@@ -936,6 +936,7 @@ public class ZKRMStateStore extends RMStateStore {
       }
       }
       safeDelete(appIdRemovePath);
       safeDelete(appIdRemovePath);
     } else {
     } else {
+      CuratorFramework curatorFramework = zkManager.getCurator();
       curatorFramework.delete().deletingChildrenIfNeeded().
       curatorFramework.delete().deletingChildrenIfNeeded().
           forPath(appIdRemovePath);
           forPath(appIdRemovePath);
     }
     }
@@ -1236,38 +1237,32 @@ public class ZKRMStateStore extends RMStateStore {
 
 
   @VisibleForTesting
   @VisibleForTesting
   byte[] getData(final String path) throws Exception {
   byte[] getData(final String path) throws Exception {
-    return curatorFramework.getData().forPath(path);
+    return zkManager.getData(path);
   }
   }
 
 
   @VisibleForTesting
   @VisibleForTesting
   List<ACL> getACL(final String path) throws Exception {
   List<ACL> getACL(final String path) throws Exception {
-    return curatorFramework.getACL().forPath(path);
+    return zkManager.getACL(path);
   }
   }
 
 
   @VisibleForTesting
   @VisibleForTesting
   List<String> getChildren(final String path) throws Exception {
   List<String> getChildren(final String path) throws Exception {
-    return curatorFramework.getChildren().forPath(path);
+    return zkManager.getChildren(path);
   }
   }
 
 
   @VisibleForTesting
   @VisibleForTesting
   boolean exists(final String path) throws Exception {
   boolean exists(final String path) throws Exception {
-    return curatorFramework.checkExists().forPath(path) != null;
+    return zkManager.exists(path);
   }
   }
 
 
   @VisibleForTesting
   @VisibleForTesting
   void create(final String path) throws Exception {
   void create(final String path) throws Exception {
-    if (!exists(path)) {
-      curatorFramework.create()
-          .withMode(CreateMode.PERSISTENT).withACL(zkAcl)
-          .forPath(path, null);
-    }
+    zkManager.create(path, zkAcl);
   }
   }
 
 
   @VisibleForTesting
   @VisibleForTesting
   void delete(final String path) throws Exception {
   void delete(final String path) throws Exception {
-    if (exists(path)) {
-      curatorFramework.delete().deletingChildrenIfNeeded().forPath(path);
-    }
+    zkManager.delete(path);
   }
   }
 
 
   private void safeCreate(String path, byte[] data, List<ACL> acl,
   private void safeCreate(String path, byte[] data, List<ACL> acl,
@@ -1310,6 +1305,7 @@ public class ZKRMStateStore extends RMStateStore {
     private CuratorTransactionFinal transactionFinal;
     private CuratorTransactionFinal transactionFinal;
 
 
     SafeTransaction() throws Exception {
     SafeTransaction() throws Exception {
+      CuratorFramework curatorFramework = zkManager.getCurator();
       CuratorTransaction transaction = curatorFramework.inTransaction();
       CuratorTransaction transaction = curatorFramework.inTransaction();
       transactionFinal = transaction.create()
       transactionFinal = transaction.create()
           .withMode(CreateMode.PERSISTENT).withACL(zkAcl)
           .withMode(CreateMode.PERSISTENT).withACL(zkAcl)

+ 13 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/ReservationInputValidator.java

@@ -129,11 +129,12 @@ public class ReservationInputValidator {
               Resources.multiply(rr.getCapability(), rr.getConcurrency()));
               Resources.multiply(rr.getCapability(), rr.getConcurrency()));
     }
     }
     // verify the allocation is possible (skip for ANY)
     // verify the allocation is possible (skip for ANY)
-    if (contract.getDeadline() - contract.getArrival() < minDuration
+    long duration = contract.getDeadline() - contract.getArrival();
+    if (duration < minDuration
         && type != ReservationRequestInterpreter.R_ANY) {
         && type != ReservationRequestInterpreter.R_ANY) {
       message =
       message =
           "The time difference ("
           "The time difference ("
-              + (contract.getDeadline() - contract.getArrival())
+              + (duration)
               + ") between arrival (" + contract.getArrival() + ") "
               + ") between arrival (" + contract.getArrival() + ") "
               + "and deadline (" + contract.getDeadline() + ") must "
               + "and deadline (" + contract.getDeadline() + ") must "
               + " be greater or equal to the minimum resource duration ("
               + " be greater or equal to the minimum resource duration ("
@@ -158,15 +159,22 @@ public class ReservationInputValidator {
     // check that the recurrence is a positive long value.
     // check that the recurrence is a positive long value.
     String recurrenceExpression = contract.getRecurrenceExpression();
     String recurrenceExpression = contract.getRecurrenceExpression();
     try {
     try {
-      Long recurrence = Long.parseLong(recurrenceExpression);
+      long recurrence = Long.parseLong(recurrenceExpression);
       if (recurrence < 0) {
       if (recurrence < 0) {
         message = "Negative Period : " + recurrenceExpression + ". Please try"
         message = "Negative Period : " + recurrenceExpression + ". Please try"
-            + " again with a non-negative long value as period";
+            + " again with a non-negative long value as period.";
+        throw RPCUtil.getRemoteException(message);
+      }
+      // verify duration is less than recurrence for periodic reservations
+      if (recurrence > 0 && duration > recurrence) {
+        message = "Duration of the requested reservation: " + duration
+            + " is greater than the recurrence: " + recurrence
+            + ". Please try again with a smaller duration.";
         throw RPCUtil.getRemoteException(message);
         throw RPCUtil.getRemoteException(message);
       }
       }
     } catch (NumberFormatException e) {
     } catch (NumberFormatException e) {
       message = "Invalid period " + recurrenceExpression + ". Please try"
       message = "Invalid period " + recurrenceExpression + ". Please try"
-          + " again with a non-negative long value as period";
+          + " again with a non-negative long value as period.";
       throw RPCUtil.getRemoteException(message);
       throw RPCUtil.getRemoteException(message);
     }
     }
   }
   }

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

@@ -1315,7 +1315,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
     // AFTER the initial saving on app-attempt-start
     // AFTER the initial saving on app-attempt-start
     // These fields can be visible from outside only after they are saved in
     // These fields can be visible from outside only after they are saved in
     // StateStore
     // StateStore
-    String diags = null;
+    BoundedAppender diags = new BoundedAppender(diagnostics.limit);
 
 
     // don't leave the tracking URL pointing to a non-existent AM
     // don't leave the tracking URL pointing to a non-existent AM
     if (conf.getBoolean(YarnConfiguration.APPLICATION_HISTORY_ENABLED,
     if (conf.getBoolean(YarnConfiguration.APPLICATION_HISTORY_ENABLED,
@@ -1329,15 +1329,15 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
     int exitStatus = ContainerExitStatus.INVALID;
     int exitStatus = ContainerExitStatus.INVALID;
     switch (event.getType()) {
     switch (event.getType()) {
     case LAUNCH_FAILED:
     case LAUNCH_FAILED:
-      diags = event.getDiagnosticMsg();
+      diags.append(event.getDiagnosticMsg());
       break;
       break;
     case REGISTERED:
     case REGISTERED:
-      diags = getUnexpectedAMRegisteredDiagnostics();
+      diags.append(getUnexpectedAMRegisteredDiagnostics());
       break;
       break;
     case UNREGISTERED:
     case UNREGISTERED:
       RMAppAttemptUnregistrationEvent unregisterEvent =
       RMAppAttemptUnregistrationEvent unregisterEvent =
           (RMAppAttemptUnregistrationEvent) event;
           (RMAppAttemptUnregistrationEvent) event;
-      diags = unregisterEvent.getDiagnosticMsg();
+      diags.append(unregisterEvent.getDiagnosticMsg());
       // reset finalTrackingUrl to url sent by am
       // reset finalTrackingUrl to url sent by am
       finalTrackingUrl = sanitizeTrackingUrl(unregisterEvent.getFinalTrackingUrl());
       finalTrackingUrl = sanitizeTrackingUrl(unregisterEvent.getFinalTrackingUrl());
       finalStatus = unregisterEvent.getFinalApplicationStatus();
       finalStatus = unregisterEvent.getFinalApplicationStatus();
@@ -1345,16 +1345,16 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
     case CONTAINER_FINISHED:
     case CONTAINER_FINISHED:
       RMAppAttemptContainerFinishedEvent finishEvent =
       RMAppAttemptContainerFinishedEvent finishEvent =
           (RMAppAttemptContainerFinishedEvent) event;
           (RMAppAttemptContainerFinishedEvent) event;
-      diags = getAMContainerCrashedDiagnostics(finishEvent);
+      diags.append(getAMContainerCrashedDiagnostics(finishEvent));
       exitStatus = finishEvent.getContainerStatus().getExitStatus();
       exitStatus = finishEvent.getContainerStatus().getExitStatus();
       break;
       break;
     case KILL:
     case KILL:
       break;
       break;
     case FAIL:
     case FAIL:
-      diags = event.getDiagnosticMsg();
+      diags.append(event.getDiagnosticMsg());
       break;
       break;
     case EXPIRE:
     case EXPIRE:
-      diags = getAMExpiredDiagnostics(event);
+      diags.append(getAMExpiredDiagnostics(event));
       break;
       break;
     default:
     default:
       break;
       break;
@@ -1368,7 +1368,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
         ApplicationAttemptStateData.newInstance(
         ApplicationAttemptStateData.newInstance(
             applicationAttemptId,  getMasterContainer(),
             applicationAttemptId,  getMasterContainer(),
             rmStore.getCredentialsFromAppAttempt(this),
             rmStore.getCredentialsFromAppAttempt(this),
-            startTime, stateToBeStored, finalTrackingUrl, diags,
+            startTime, stateToBeStored, finalTrackingUrl, diags.toString(),
             finalStatus, exitStatus,
             finalStatus, exitStatus,
           getFinishTime(), resUsage.getMemorySeconds(),
           getFinishTime(), resUsage.getMemorySeconds(),
           resUsage.getVcoreSeconds(),
           resUsage.getVcoreSeconds(),

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

@@ -327,6 +327,10 @@ public class CapacitySchedulerQueueManager implements SchedulerQueueManager<
                 + "it is not yet in stopped state. Current State : "
                 + "it is not yet in stopped state. Current State : "
                 + oldQueue.getState());
                 + oldQueue.getState());
           }
           }
+        } else if (oldQueue instanceof ParentQueue
+            && newQueue instanceof LeafQueue) {
+          LOG.info("Converting the parent queue: " + oldQueue.getQueuePath()
+              + " to leaf queue.");
         }
         }
       }
       }
     }
     }

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

@@ -18,6 +18,14 @@
 
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
@@ -34,7 +42,6 @@ import org.apache.hadoop.yarn.api.records.QueueInfo;
 import org.apache.hadoop.yarn.api.records.QueueState;
 import org.apache.hadoop.yarn.api.records.QueueState;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.security.AccessType;
 import org.apache.hadoop.yarn.security.AccessType;
@@ -45,7 +52,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerStat
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedContainerChangeRequest;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivitiesLogger;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivitiesLogger;
@@ -62,14 +68,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.Placeme
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.PlacementSetUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.PlacementSetUtils;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-
 @Private
 @Private
 @Evolving
 @Evolving
 public class ParentQueue extends AbstractCSQueue {
 public class ParentQueue extends AbstractCSQueue {
@@ -315,18 +313,21 @@ public class ParentQueue extends AbstractCSQueue {
 
 
         // Check if the child-queue already exists
         // Check if the child-queue already exists
         if (childQueue != null) {
         if (childQueue != null) {
-          // Check if the child-queue has been converted into parent queue.
-          // The CS has already checked to ensure that this child-queue is in
-          // STOPPED state.
-          if (childQueue instanceof LeafQueue
-              && newChildQueue instanceof ParentQueue) {
-            // We would convert this LeafQueue to ParentQueue, consider this
-            // as the combination of DELETE then ADD.
+          // Check if the child-queue has been converted into parent queue or
+          // parent Queue has been converted to child queue. The CS has already
+          // checked to ensure that this child-queue is in STOPPED state if
+          // Child queue has been converted to ParentQueue.
+          if ((childQueue instanceof LeafQueue
+              && newChildQueue instanceof ParentQueue)
+              || (childQueue instanceof ParentQueue
+                  && newChildQueue instanceof LeafQueue)) {
+            // We would convert this LeafQueue to ParentQueue, or vice versa.
+            // consider this as the combination of DELETE then ADD.
             newChildQueue.setParent(this);
             newChildQueue.setParent(this);
             currentChildQueues.put(newChildQueueName, newChildQueue);
             currentChildQueues.put(newChildQueueName, newChildQueue);
             // inform CapacitySchedulerQueueManager
             // inform CapacitySchedulerQueueManager
-            CapacitySchedulerQueueManager queueManager = this.csContext
-                .getCapacitySchedulerQueueManager();
+            CapacitySchedulerQueueManager queueManager =
+                this.csContext.getCapacitySchedulerQueueManager();
             queueManager.addQueue(newChildQueueName, newChildQueue);
             queueManager.addQueue(newChildQueueName, newChildQueue);
             continue;
             continue;
           }
           }

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

@@ -266,7 +266,7 @@ public class AllocationFileLoaderService extends AbstractService {
     Map<FSQueueType, Set<String>> configuredQueues = new HashMap<>();
     Map<FSQueueType, Set<String>> configuredQueues = new HashMap<>();
 
 
     for (FSQueueType queueType : FSQueueType.values()) {
     for (FSQueueType queueType : FSQueueType.values()) {
-      configuredQueues.put(queueType, new HashSet<String>());
+      configuredQueues.put(queueType, new HashSet<>());
     }
     }
 
 
     // Read and parse the allocations file.
     // Read and parse the allocations file.
@@ -280,7 +280,7 @@ public class AllocationFileLoaderService extends AbstractService {
       throw new AllocationConfigurationException("Bad fair scheduler config " +
       throw new AllocationConfigurationException("Bad fair scheduler config " +
           "file: top-level element not <allocations>");
           "file: top-level element not <allocations>");
     NodeList elements = root.getChildNodes();
     NodeList elements = root.getChildNodes();
-    List<Element> queueElements = new ArrayList<Element>();
+    List<Element> queueElements = new ArrayList<>();
     Element placementPolicyElement = null;
     Element placementPolicyElement = null;
     for (int i = 0; i < elements.getLength(); i++) {
     for (int i = 0; i < elements.getLength(); i++) {
       Node node = elements.item(i);
       Node node = elements.item(i);
@@ -294,8 +294,9 @@ public class AllocationFileLoaderService extends AbstractService {
           NodeList fields = element.getChildNodes();
           NodeList fields = element.getChildNodes();
           for (int j = 0; j < fields.getLength(); j++) {
           for (int j = 0; j < fields.getLength(); j++) {
             Node fieldNode = fields.item(j);
             Node fieldNode = fields.item(j);
-            if (!(fieldNode instanceof Element))
+            if (!(fieldNode instanceof Element)) {
               continue;
               continue;
+            }
             Element field = (Element) fieldNode;
             Element field = (Element) fieldNode;
             if ("maxRunningApps".equals(field.getTagName())) {
             if ("maxRunningApps".equals(field.getTagName())) {
               String text = ((Text)field.getFirstChild()).getData().trim();
               String text = ((Text)field.getFirstChild()).getData().trim();
@@ -490,8 +491,9 @@ public class AllocationFileLoaderService extends AbstractService {
 
 
     for (int j = 0; j < fields.getLength(); j++) {
     for (int j = 0; j < fields.getLength(); j++) {
       Node fieldNode = fields.item(j);
       Node fieldNode = fields.item(j);
-      if (!(fieldNode instanceof Element))
+      if (!(fieldNode instanceof Element)) {
         continue;
         continue;
+      }
       Element field = (Element) fieldNode;
       Element field = (Element) fieldNode;
       if ("minResources".equals(field.getTagName())) {
       if ("minResources".equals(field.getTagName())) {
         String text = ((Text)field.getFirstChild()).getData().trim();
         String text = ((Text)field.getFirstChild()).getData().trim();

+ 3 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/RMHATestBase.java

@@ -21,6 +21,7 @@ package org.apache.hadoop.yarn.server.resourcemanager;
 import java.io.IOException;
 import java.io.IOException;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.ha.ClientBaseWithFixes;
 import org.apache.hadoop.ha.ClientBaseWithFixes;
 import org.apache.hadoop.ha.HAServiceProtocol;
 import org.apache.hadoop.ha.HAServiceProtocol;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
@@ -61,8 +62,8 @@ public abstract class RMHATestBase extends ClientBaseWithFixes{
     configuration.setBoolean(YarnConfiguration.RECOVERY_ENABLED, true);
     configuration.setBoolean(YarnConfiguration.RECOVERY_ENABLED, true);
     configuration.set(YarnConfiguration.RM_STORE,
     configuration.set(YarnConfiguration.RM_STORE,
         ZKRMStateStore.class.getName());
         ZKRMStateStore.class.getName());
-    configuration.set(YarnConfiguration.RM_ZK_ADDRESS, hostPort);
-    configuration.setInt(YarnConfiguration.RM_ZK_TIMEOUT_MS, ZK_TIMEOUT_MS);
+    configuration.set(CommonConfigurationKeys.ZK_ADDRESS, hostPort);
+    configuration.setInt(CommonConfigurationKeys.ZK_TIMEOUT_MS, ZK_TIMEOUT_MS);
     configuration.setBoolean(YarnConfiguration.AUTO_FAILOVER_ENABLED, false);
     configuration.setBoolean(YarnConfiguration.AUTO_FAILOVER_ENABLED, false);
     configuration.set(YarnConfiguration.RM_CLUSTER_ID, "test-yarn-cluster");
     configuration.set(YarnConfiguration.RM_CLUSTER_ID, "test-yarn-cluster");
     int base = 100;
     int base = 100;

+ 93 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestReservationInputValidator.java

@@ -303,6 +303,7 @@ public class TestReservationInputValidator {
 
 
   @Test
   @Test
   public void testSubmitReservationInvalidRecurrenceExpression() {
   public void testSubmitReservationInvalidRecurrenceExpression() {
+    // first check recurrence expression
     ReservationSubmissionRequest request =
     ReservationSubmissionRequest request =
         createSimpleReservationSubmissionRequest(1, 1, 1, 5, 3, "123abc");
         createSimpleReservationSubmissionRequest(1, 1, 1, 5, 3, "123abc");
     plan = null;
     plan = null;
@@ -318,6 +319,23 @@ public class TestReservationInputValidator {
           .startsWith("Invalid period "));
           .startsWith("Invalid period "));
       LOG.info(message);
       LOG.info(message);
     }
     }
+
+    // now check duration
+    request =
+        createSimpleReservationSubmissionRequest(1, 1, 1, 50, 3, "10");
+    plan = null;
+    try {
+      plan =
+          rrValidator.validateReservationSubmissionRequest(rSystem, request,
+              ReservationSystemTestUtil.getNewReservationId());
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertNull(plan);
+      String message = e.getMessage();
+      Assert.assertTrue(message
+          .startsWith("Duration of the requested reservation:"));
+      LOG.info(message);
+    }
   }
   }
 
 
   @Test
   @Test
@@ -499,6 +517,73 @@ public class TestReservationInputValidator {
     }
     }
   }
   }
 
 
+  @Test
+  public void testUpdateReservationValidRecurrenceExpression() {
+    ReservationUpdateRequest request =
+        createSimpleReservationUpdateRequest(1, 1, 1, 5, 3, "600000");
+    plan = null;
+    try {
+      plan =
+          rrValidator.validateReservationUpdateRequest(rSystem, request);
+    } catch (YarnException e) {
+      Assert.fail(e.getMessage());
+    }
+    Assert.assertNotNull(plan);
+  }
+
+  @Test
+  public void testUpdateReservationNegativeRecurrenceExpression() {
+    ReservationUpdateRequest request =
+        createSimpleReservationUpdateRequest(1, 1, 1, 5, 3, "-1234");
+    plan = null;
+    try {
+      plan =
+          rrValidator.validateReservationUpdateRequest(rSystem, request);
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertNull(plan);
+      String message = e.getMessage();
+      Assert.assertTrue(message
+          .startsWith("Negative Period : "));
+      LOG.info(message);
+    }
+  }
+
+  @Test
+  public void testUpdateReservationInvalidRecurrenceExpression() {
+    // first check recurrence expression
+    ReservationUpdateRequest request =
+        createSimpleReservationUpdateRequest(1, 1, 1, 5, 3, "123abc");
+    plan = null;
+    try {
+      plan =
+          rrValidator.validateReservationUpdateRequest(rSystem, request);
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertNull(plan);
+      String message = e.getMessage();
+      Assert.assertTrue(message
+          .startsWith("Invalid period "));
+      LOG.info(message);
+    }
+
+    // now check duration
+    request =
+        createSimpleReservationUpdateRequest(1, 1, 1, 50, 3, "10");
+    plan = null;
+    try {
+      plan =
+          rrValidator.validateReservationUpdateRequest(rSystem, request);
+      Assert.fail();
+    } catch (YarnException e) {
+      Assert.assertNull(plan);
+      String message = e.getMessage();
+      Assert.assertTrue(message
+          .startsWith("Duration of the requested reservation:"));
+      LOG.info(message);
+    }
+  }
+
   @Test
   @Test
   public void testDeleteReservationNormal() {
   public void testDeleteReservationNormal() {
     ReservationDeleteRequest request = new ReservationDeleteRequestPBImpl();
     ReservationDeleteRequest request = new ReservationDeleteRequestPBImpl();
@@ -710,11 +795,19 @@ public class TestReservationInputValidator {
   private ReservationUpdateRequest createSimpleReservationUpdateRequest(
   private ReservationUpdateRequest createSimpleReservationUpdateRequest(
       int numRequests, int numContainers, long arrival, long deadline,
       int numRequests, int numContainers, long arrival, long deadline,
       long duration) {
       long duration) {
+    return createSimpleReservationUpdateRequest(numRequests, numContainers,
+        arrival, deadline, duration, "0");
+  }
+
+  private ReservationUpdateRequest createSimpleReservationUpdateRequest(
+      int numRequests, int numContainers, long arrival, long deadline,
+      long duration, String recurrence) {
     // create a request with a single atomic ask
     // create a request with a single atomic ask
     ReservationUpdateRequest request = new ReservationUpdateRequestPBImpl();
     ReservationUpdateRequest request = new ReservationUpdateRequestPBImpl();
     ReservationDefinition rDef = new ReservationDefinitionPBImpl();
     ReservationDefinition rDef = new ReservationDefinitionPBImpl();
     rDef.setArrival(arrival);
     rDef.setArrival(arrival);
     rDef.setDeadline(deadline);
     rDef.setDeadline(deadline);
+    rDef.setRecurrenceExpression(recurrence);
     if (numRequests > 0) {
     if (numRequests > 0) {
       ReservationRequests reqs = new ReservationRequestsPBImpl();
       ReservationRequests reqs = new ReservationRequestsPBImpl();
       rDef.setReservationRequests(reqs);
       rDef.setReservationRequests(reqs);

+ 113 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
 
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 import static org.junit.Assert.fail;
@@ -42,7 +43,6 @@ import java.util.Set;
 import java.util.concurrent.BrokenBarrierException;
 import java.util.concurrent.BrokenBarrierException;
 import java.util.concurrent.CyclicBarrier;
 import java.util.concurrent.CyclicBarrier;
 
 
-import com.google.common.base.Supplier;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
@@ -167,6 +167,7 @@ import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
 import org.mockito.Mockito;
 import org.mockito.Mockito;
 
 
+import com.google.common.base.Supplier;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Sets;
 import com.google.common.collect.Sets;
@@ -665,6 +666,36 @@ public class TestCapacityScheduler {
     return conf;
     return conf;
   }
   }
 
 
+  /**
+   * @param conf, to be modified
+   * @return, CS configuration which has deleted all childred of queue(b)
+   *           root
+   *          /     \
+   *        a        b
+   *       / \
+   *      a1  a2
+   */
+  private CapacitySchedulerConfiguration setupQueueConfWithOutChildrenOfB(
+      CapacitySchedulerConfiguration conf) {
+
+    // Define top-level queues
+    conf.setQueues(CapacitySchedulerConfiguration.ROOT,
+        new String[] {"a","b"});
+
+    conf.setCapacity(A, A_CAPACITY);
+    conf.setCapacity(B, B_CAPACITY);
+
+    // Define 2nd-level queues
+    conf.setQueues(A, new String[] {"a1","a2"});
+    conf.setCapacity(A1, A1_CAPACITY);
+    conf.setUserLimitFactor(A1, 100.0f);
+    conf.setCapacity(A2, A2_CAPACITY);
+    conf.setUserLimitFactor(A2, 100.0f);
+
+    LOG.info("Setup top-level queues a and b (without children)");
+    return conf;
+  }
+
   /**
   /**
    * @param conf, to be modified
    * @param conf, to be modified
    * @return, CS configuration which has deleted a queue(b1)
    * @return, CS configuration which has deleted a queue(b1)
@@ -4643,6 +4674,10 @@ public class TestCapacityScheduler {
     try {
     try {
       cs.reinitialize(conf, mockContext);
       cs.reinitialize(conf, mockContext);
     } catch (IOException e) {
     } catch (IOException e) {
+      LOG.error(
+          "Expected to NOT throw exception when refresh queue tries to delete"
+              + " a queue WITHOUT running apps",
+          e);
       fail("Expected to NOT throw exception when refresh queue tries to delete"
       fail("Expected to NOT throw exception when refresh queue tries to delete"
           + " a queue WITHOUT running apps");
           + " a queue WITHOUT running apps");
     }
     }
@@ -4712,6 +4747,83 @@ public class TestCapacityScheduler {
     cs.stop();
     cs.stop();
   }
   }
 
 
+  /**
+   * Test for all child queue deletion and thus making parent queue a child.
+   * @throws Exception
+   */
+  @Test
+  public void testRefreshQueuesWithAllChildQueuesDeleted() throws Exception {
+    CapacityScheduler cs = new CapacityScheduler();
+    CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
+    RMContextImpl rmContext = new RMContextImpl(null, null, null, null, null,
+        null, new RMContainerTokenSecretManager(conf),
+        new NMTokenSecretManagerInRM(conf),
+        new ClientToAMTokenSecretManagerInRM(), null);
+    setupQueueConfiguration(conf);
+    cs.setConf(new YarnConfiguration());
+    cs.setRMContext(resourceManager.getRMContext());
+    cs.init(conf);
+    cs.start();
+    cs.reinitialize(conf, rmContext);
+    checkQueueCapacities(cs, A_CAPACITY, B_CAPACITY);
+
+    // test delete all leaf queues when there is no application running.
+    Map<String, CSQueue> queues =
+        cs.getCapacitySchedulerQueueManager().getQueues();
+
+    CSQueue bQueue = Mockito.spy((LeafQueue) queues.get("b1"));
+    when(bQueue.getState()).thenReturn(QueueState.RUNNING)
+        .thenReturn(QueueState.STOPPED);
+    queues.put("b1", bQueue);
+
+    bQueue = Mockito.spy((LeafQueue) queues.get("b2"));
+    when(bQueue.getState()).thenReturn(QueueState.STOPPED);
+    queues.put("b2", bQueue);
+
+    bQueue = Mockito.spy((LeafQueue) queues.get("b3"));
+    when(bQueue.getState()).thenReturn(QueueState.STOPPED);
+    queues.put("b3", bQueue);
+
+    conf = new CapacitySchedulerConfiguration();
+    setupQueueConfWithOutChildrenOfB(conf);
+
+    // test convert parent queue to leaf queue(root.b) when there is no
+    // application running.
+    try {
+      cs.reinitialize(conf, mockContext);
+      fail("Expected to throw exception when refresh queue tries to make parent"
+          + " queue a child queue when one of its children is still running.");
+    } catch (IOException e) {
+      //do not do anything, expected exception
+    }
+
+    // test delete leaf queues(root.b.b1,b2,b3) when there is no application
+    // running.
+    try {
+      cs.reinitialize(conf, mockContext);
+    } catch (IOException e) {
+      e.printStackTrace();
+      fail("Expected to NOT throw exception when refresh queue tries to delete"
+          + " all children of a parent queue(without running apps).");
+    }
+    CSQueue rootQueue = cs.getRootQueue();
+    CSQueue queueB = findQueue(rootQueue, B);
+    assertNotNull("Parent Queue B should not be deleted", queueB);
+    Assert.assertTrue("As Queue'B children are not deleted",
+        queueB instanceof LeafQueue);
+
+    String message =
+        "Refresh needs to support delete of all children of Parent queue.";
+    assertNull(message,
+        cs.getCapacitySchedulerQueueManager().getQueues().get("b3"));
+    assertNull(message,
+        cs.getCapacitySchedulerQueueManager().getQueues().get("b1"));
+    assertNull(message,
+        cs.getCapacitySchedulerQueueManager().getQueues().get("b2"));
+
+    cs.stop();
+  }
+
   /**
   /**
    * Test if we can convert a leaf queue to a parent queue
    * Test if we can convert a leaf queue to a parent queue
    * @throws Exception
    * @throws Exception

Some files were not shown because too many files changed in this diff