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);
       dumpGenerator.writeStartObject();
       dumpGenerator.writeFieldName("property");
-      appendJSONProperty(dumpGenerator, config, propertyName);
+      appendJSONProperty(dumpGenerator, config, propertyName,
+          new ConfigRedactor(config));
       dumpGenerator.writeEndObject();
       dumpGenerator.flush();
     }
@@ -3186,11 +3187,11 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
     dumpGenerator.writeFieldName("properties");
     dumpGenerator.writeStartArray();
     dumpGenerator.flush();
+    ConfigRedactor redactor = new ConfigRedactor(config);
     synchronized (config) {
       for (Map.Entry<Object,Object> item: config.getProps().entrySet()) {
-        appendJSONProperty(dumpGenerator,
-            config,
-            item.getKey().toString());
+        appendJSONProperty(dumpGenerator, config, item.getKey().toString(),
+            redactor);
       }
     }
     dumpGenerator.writeEndArray();
@@ -3208,12 +3209,14 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
    * @throws IOException
    */
   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
     if(!Strings.isNullOrEmpty(name) && jsonGen != null) {
       jsonGen.writeStartObject();
       jsonGen.writeStringField("key", name);
-      jsonGen.writeStringField("value", config.get(name));
+      jsonGen.writeStringField("value",
+          redactor.redact(name, config.get(name)));
       jsonGen.writeBooleanField("isFinal",
           config.finalParameters.contains(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 {
     try {
       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.
   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)
                                         throws IOException {
     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 (srcq.length() == dstq.length()) {
           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.
       Path root = new Path("/");
       return new FileStatus(length, isDir, blockReplication, blockSize,
-          modTime, root.makeQualified(this));
+          modTime, this.makeQualified(root));
     }
     String pathName = parentPath.toUri().getPath();
     FTPFile[] ftpFiles = client.listFiles(pathName);
@@ -546,7 +546,7 @@ public class FTPFileSystem extends FileSystem {
     String group = ftpFile.getGroup();
     Path filePath = new Path(parentPath, ftpFile.getName());
     return new FileStatus(length, isDir, blockReplication, blockSize, modTime,
-        accessTime, permission, user, group, filePath.makeQualified(this));
+        accessTime, permission, user, group, this.makeQualified(filePath));
   }
 
   @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
     public Reader(FileSystem fs, Path file, 
                   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 {
         groups = provider.getGroups(user);
       } 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()) {
         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>
 
 <!-- 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>
   <name>fs.azure.secure.mode</name>
   <value>false</value>
@@ -2682,11 +2694,16 @@
     <value>ClientCredential</value>
     <description>
       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,
       fs.adl.oauth2.credential, and fs.adl.oauth2.refresh.url.
       The RefreshToken type requires property fs.adl.oauth2.client.id and
       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.
     </description>
   </property>
@@ -2723,6 +2740,36 @@
     </description>
   </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-->
 
   <property>
@@ -2771,4 +2818,50 @@
       This determines the number of open file handles.
     </description>
   </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>

+ 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("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
     // - org.apache.hadoop.hdfs.web.ADLConfKeys
     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.hadoop.conf.Configuration.IntegerRanges;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.net.NetUtils;
@@ -82,6 +83,11 @@ public class TestConfiguration extends TestCase {
   /** Four apostrophes. */
   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
   protected void setUp() throws Exception {
     super.setUp();
@@ -90,6 +96,9 @@ public class TestConfiguration extends TestCase {
   
   @Override
   protected void tearDown() throws Exception {
+    if(out != null) {
+      out.close();
+    }
     super.tearDown();
     new File(CONFIG).delete();
     new File(CONFIG2).delete();
@@ -878,8 +887,6 @@ public class TestConfiguration extends TestCase {
     new File(new File(relConfig).getParent()).delete();
   }
 
-  BufferedWriter out;
-	
   public void testIntegerRanges() {
     Configuration conf = new Configuration();
     conf.set("first", "-100");
@@ -1787,8 +1794,41 @@ public class TestConfiguration extends TestCase {
       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() {
     Configuration conf = new Configuration();
     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
   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();
     assertEquals(home, fsHome);
   }
@@ -229,7 +229,7 @@ public class TestLocalFileSystem {
     Path path = new Path(TEST_ROOT_DIR, "foo%bar");
     writeFile(fileSys, path, 1);
     FileStatus status = fileSys.getFileStatus(path);
-    assertEquals(path.makeQualified(fileSys), status.getPath());
+    assertEquals(fileSys.makeQualified(path), status.getPath());
     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
             .submit(getFromDataNodeCallable);
         futures.add(firstRequest);
+        Future<ByteBuffer> future = null;
         try {
-          Future<ByteBuffer> future = hedgedService.poll(
+          future = hedgedService.poll(
               conf.getHedgedReadThresholdMillis(), TimeUnit.MILLISECONDS);
           if (future != null) {
             ByteBuffer result = future.get();
@@ -1142,16 +1143,18 @@ public class DFSInputStream extends FSInputStream
           }
           DFSClient.LOG.debug("Waited {}ms to read from {}; spawning hedged "
               + "read", conf.getHedgedReadThresholdMillis(), chosenNode.info);
-          // Ignore this node on next go around.
-          ignored.add(chosenNode.info);
           dfsClient.getHedgedReadMetrics().incHedgedReadOps();
           // continue; no need to refresh block locations
         } catch (ExecutionException e) {
-          // Ignore
+          futures.remove(future);
         } catch (InterruptedException e) {
           throw new InterruptedIOException(
               "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 {
         // We are starting up a 'hedged' read. We have a read already
         // 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 =
       "dfs.namenode.posix.acl.inheritance.enabled";
   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 boolean DFS_NAMENODE_XATTRS_ENABLED_DEFAULT = true;
   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.blockManager = blockManager;
 
-    // TODO: Enables DFSNetworkTopology by default after more stress
-    // testings/validations.
     this.useDfsNetworkTopology = conf.getBoolean(
         DFSConfigKeys.DFS_USE_DFS_NETWORK_TOPOLOGY_KEY,
         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.HashSet;
 import java.util.List;
-import java.util.PriorityQueue;
+import java.util.NavigableSet;
 import java.util.Set;
 import java.util.SortedMap;
 import java.util.TreeMap;
+import java.util.TreeSet;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
@@ -87,11 +88,15 @@ public class LeaseManager {
   // Mapping: leaseHolder -> Lease
   private final SortedMap<String, Lease> leases = new TreeMap<>();
   // Set of: Lease
-  private final PriorityQueue<Lease> sortedLeases = new PriorityQueue<>(512,
+  private final NavigableSet<Lease> sortedLeases = new TreeSet<>(
       new Comparator<Lease>() {
         @Override
         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
@@ -528,9 +533,10 @@ public class LeaseManager {
 
     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");
 
       final List<Long> removing = new ArrayList<>();

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

@@ -459,7 +459,7 @@
 
   <property>
     <name>dfs.namenode.posix.acl.inheritance.enabled</name>
-    <value>false</value>
+    <value>true</value>
     <description>
       Set to true to enable POSIX style ACL inheritance. When it is enabled
       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`
 
-    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,
     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,

+ 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() {
     conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY, true);
+    conf.setBoolean(
+        DFSConfigKeys.DFS_NAMENODE_POSIX_ACL_INHERITANCE_ENABLED_KEY, false);
   }
 
   @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 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
@@ -72,6 +74,9 @@ public class TestPread {
   boolean simulatedStorage;
   boolean isHedgedRead;
 
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestPread.class.getName());
+
   @Before
   public void setup() {
     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>
    * 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[] {
       aclEntry(ACCESS, USER, "foo", ALL),
       aclEntry(ACCESS, GROUP, READ_EXECUTE) }, returned);
-    assertPermission(filePath, (short)010640);
+    assertPermission(filePath, (short)010660);
     assertAclFeature(filePath, true);
   }
 
@@ -1003,7 +1003,7 @@ public abstract class FSAclBaseTest {
       aclEntry(DEFAULT, GROUP, READ_EXECUTE),
       aclEntry(DEFAULT, MASK, ALL),
       aclEntry(DEFAULT, OTHER, NONE) }, returned);
-    assertPermission(dirPath, (short)010750);
+    assertPermission(dirPath, (short)010770);
     assertAclFeature(dirPath, true);
   }
 
@@ -1120,7 +1120,7 @@ public abstract class FSAclBaseTest {
     s = fs.getAclStatus(filePath);
     returned = s.getEntries().toArray(new AclEntry[0]);
     assertArrayEquals(expected, returned);
-    assertPermission(filePath, (short)010640);
+    assertPermission(filePath, (short)010660);
     assertAclFeature(filePath, true);
   }
 
@@ -1149,7 +1149,7 @@ public abstract class FSAclBaseTest {
     s = fs.getAclStatus(subdirPath);
     returned = s.getEntries().toArray(new AclEntry[0]);
     assertArrayEquals(expected, returned);
-    assertPermission(subdirPath, (short)010750);
+    assertPermission(subdirPath, (short)010770);
     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, OTHER, READ_EXECUTE) };
 
+    short permExpected = (short)010775;
+
     AclEntry[] fileReturned = fs.getAclStatus(filePath).getEntries()
       .toArray(new AclEntry[0]);
     Assert.assertArrayEquals(fileExpected, fileReturned);
     AclEntry[] subdirReturned = fs.getAclStatus(subdirPath).getEntries()
       .toArray(new AclEntry[0]);
     Assert.assertArrayEquals(subdirExpected, subdirReturned);
-    assertPermission(fs, subdirPath, (short)010755);
+    assertPermission(fs, subdirPath, permExpected);
 
     restart(fs, persistNamespace);
 
@@ -154,7 +156,7 @@ public class TestFSImageWithAcl {
     subdirReturned = fs.getAclStatus(subdirPath).getEntries()
       .toArray(new AclEntry[0]);
     Assert.assertArrayEquals(subdirExpected, subdirReturned);
-    assertPermission(fs, subdirPath, (short)010755);
+    assertPermission(fs, subdirPath, permExpected);
 
     aclSpec = Lists.newArrayList(aclEntry(DEFAULT, USER, "foo", READ_WRITE));
     fs.modifyAclEntries(dirPath, aclSpec);
@@ -165,7 +167,7 @@ public class TestFSImageWithAcl {
     subdirReturned = fs.getAclStatus(subdirPath).getEntries()
       .toArray(new AclEntry[0]);
     Assert.assertArrayEquals(subdirExpected, subdirReturned);
-    assertPermission(fs, subdirPath, (short)010755);
+    assertPermission(fs, subdirPath, permExpected);
 
     restart(fs, persistNamespace);
 
@@ -175,7 +177,7 @@ public class TestFSImageWithAcl {
     subdirReturned = fs.getAclStatus(subdirPath).getEntries()
       .toArray(new AclEntry[0]);
     Assert.assertArrayEquals(subdirExpected, subdirReturned);
-    assertPermission(fs, subdirPath, (short)010755);
+    assertPermission(fs, subdirPath, permExpected);
 
     fs.removeAcl(dirPath);
 
@@ -185,7 +187,7 @@ public class TestFSImageWithAcl {
     subdirReturned = fs.getAclStatus(subdirPath).getEntries()
       .toArray(new AclEntry[0]);
     Assert.assertArrayEquals(subdirExpected, subdirReturned);
-    assertPermission(fs, subdirPath, (short)010755);
+    assertPermission(fs, subdirPath, permExpected);
 
     restart(fs, persistNamespace);
 
@@ -195,7 +197,7 @@ public class TestFSImageWithAcl {
     subdirReturned = fs.getAclStatus(subdirPath).getEntries()
       .toArray(new AclEntry[0]);
     Assert.assertArrayEquals(subdirExpected, subdirReturned);
-    assertPermission(fs, subdirPath, (short)010755);
+    assertPermission(fs, subdirPath, permExpected);
   }
 
   @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 cleanupProgress;
   private boolean isUber = false;
+  private boolean finishJobWhenReducersDone;
+  private boolean completingJob = false;
 
   private Credentials jobCredentials;
   private Token<JobTokenIdentifier> jobToken;
@@ -717,6 +719,9 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
     this.maxFetchFailuresNotifications = conf.getInt(
         MRJobConfig.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() {
@@ -2021,7 +2026,9 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
                 TimeUnit.MILLISECONDS);
         return JobStateInternal.FAIL_WAIT;
       }
-      
+
+      checkReadyForCompletionWhenAllReducersDone(job);
+
       return job.checkReadyForCommit();
     }
 
@@ -2052,6 +2059,32 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
       }
       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

+ 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);
 
     // 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
     job.tasks.putAll(spiedTasks);
 
@@ -641,6 +621,82 @@ public class TestJobImpl {
     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 {
     TestJobImpl t = new TestJobImpl();
     t.testJobNoTasks();
@@ -1021,6 +1077,37 @@ public class TestJobImpl {
     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
       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 DEFAULT_JOB_ACL_MODIFY_JOB = " ";
-  
+
   public static final String JOB_RUNNING_MAP_LIMIT =
       "mapreduce.job.running.map.limit";
   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_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>
 </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>
   <name>mapreduce.job.token.tracking.ids.enabled</name>
   <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) {
       try {
         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());
         } else if ("-seed".equals(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);
 
-    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");
 
     // 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);
     }
   }
-  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, 
                                 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)
   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
   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() {
     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 {
     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);
       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);
     Path inFile = new Path(testdir, "nullin/blah");
     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");
 
   /**

+ 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);
     }
   }
-  @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)
   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);
 
     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[] 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);
 
     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[] 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.
    */
   private String normalizePath(String path) {
-    return (new Path(path)).makeQualified(this.localFileSystem)
+    return this.localFileSystem.makeQualified(new Path(path))
         .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");
   private static final String OUTPUT_ROOT_DIR = "/tmp/" +
     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");
 
   @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;
     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 &&
         (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_PARTSIZE_LABEL, partSize);
     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);
     Path outputPath = new Path(dest, archiveName);
     FileOutputFormat.setOutputPath(conf, outputPath);

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

@@ -110,7 +110,7 @@
     <dependency>
       <groupId>com.microsoft.azure</groupId>
       <artifactId>azure-data-lake-store-sdk</artifactId>
-      <version>2.1.4</version>
+      <version>2.2.1</version>
     </dependency>
     <!--  ENDS HERE-->
     <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 =
       "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 =
       "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.oauth2.AccessTokenProvider;
 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 org.apache.commons.lang.StringUtils;
@@ -254,6 +256,12 @@ public class AdlFileSystem extends FileSystem {
     case ClientCredential:
       tokenProvider = getConfCredentialBasedTokenProvider(conf);
       break;
+    case MSI:
+      tokenProvider = getMsiBasedTokenProvider(conf);
+      break;
+    case DeviceCode:
+      tokenProvider = getDeviceCodeTokenProvider(conf);
+      break;
     case Custom:
     default:
       AzureADTokenProvider azureADTokenProvider = getCustomAccessTokenProvider(
@@ -280,6 +288,19 @@ public class AdlFileSystem extends FileSystem {
     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
   AccessTokenProvider getTokenProvider() {
     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 {
   RefreshToken,
   ClientCredential,
+  MSI,
+  DeviceCode,
   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
 
 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
-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.
-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 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
 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)
 4.  Add "Owner" role
 
@@ -153,6 +157,84 @@ Add the following properties to your `core-site.xml`
 </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
 
 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.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.hadoop.conf.Configuration;
 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;
 import static org.apache.hadoop.fs.adl.AdlConfKeys
     .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.junit.Assert.assertEquals;
 
@@ -97,6 +102,41 @@ public class TestAzureADTokenProvider {
     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
   public void testCustomCredTokenProvider()
       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
     final FileSystem inputFs = ioPath.getFileSystem(conf);
-    ioPath = ioPath.makeQualified(inputFs);
+    ioPath = inputFs.makeQualified(ioPath);
     boolean succeeded = false;
     try {
       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
    */
   long validateFileNameFormat(Path path) throws FileNotFoundException {
-    path = path.makeQualified(this);
+    path = this.makeQualified(path);
     boolean valid = true;
     long fileSize = 0;
     if (!path.toUri().getScheme().equals(getUri().getScheme())) {
@@ -329,4 +329,10 @@ class PseudoLocalFs extends FileSystem {
     throw new UnsupportedOperationException("SetWorkingDirectory "
         + "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 {
       final Configuration conf = new Configuration();
       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) {
       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 {
     final Configuration conf = new Configuration();
     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);
     final byte[] b = new byte[BLOCK];
     for (int i = 0; i < NFILES; ++i) {
@@ -71,8 +71,8 @@ public class TestFileQueue {
   public static void cleanup() throws IOException {
     final Configuration conf = new Configuration();
     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);
   }
 

+ 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
     path = new Path("myPsedoFile.1237");
-    path = path.makeQualified(pfs);
+    path = pfs.makeQualified(path);
     validateGetFileStatus(pfs, path, true);
     validateCreate(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 {
     conf = new Configuration();
     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);
     try {
       if (fileSystem != null) {
-        fileSystem.delete(new Path(cleanupPath).makeQualified(fileSystem),
+        fileSystem.delete(fileSystem.makeQualified(new Path(cleanupPath)),
                           true);
       }
     } 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
    */
   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);
       FileStatus status;
 
-      final Path qualifiedPath = path.makeQualified(fs);
+      final Path qualifiedPath = fs.makeQualified(path);
       status = fs.getFileStatus(qualifiedPath);
       //now see what block location info comes back.
       //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 {
     //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());
     StringBuilder listing = lsToString(parentDirListing);
     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 {
     final Configuration conf = new Configuration();
     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");
 
 
@@ -132,7 +132,7 @@ public class TestHistograms {
     final FileSystem lfs = FileSystem.getLocal(conf);
 
     for (String arg : args) {
-      Path filePath = new Path(arg).makeQualified(lfs);
+      Path filePath = lfs.makeQualified(new Path(arg));
       String fileName = filePath.getName();
       if (fileName.startsWith("input")) {
         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.IOException;
 import java.net.URI;
-import java.net.URISyntaxException;
 import java.net.URLEncoder;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
-import java.util.regex.Pattern;
 import java.util.TreeMap;
 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.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.mapreduce.MRConfig;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.filecache.DistributedCache;
 import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 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.JobID;
 import org.apache.hadoop.mapred.KeyValueTextInputFormat;
-import org.apache.hadoop.mapred.OutputFormat;
 import org.apache.hadoop.mapred.RunningJob;
 import org.apache.hadoop.mapred.SequenceFileAsTextInputFormat;
 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.aggregate.ValueAggregatorCombiner;
 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.InputWriter;
 import org.apache.hadoop.streaming.io.OutputReader;
@@ -297,7 +295,10 @@ public class StreamJob implements Tool {
           try {
             Path path = new Path(file);
             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) {
               fileList.append(',');
             }
@@ -313,7 +314,6 @@ public class StreamJob implements Tool {
           tmpFiles = tmpFiles + "," + fileList;
         }
         config_.set("tmpfiles", tmpFiles);
-        validate(packageFiles_);
       }
 
       String fsName = cmdLine.getOptionValue("dfs");
@@ -391,14 +391,13 @@ public class StreamJob implements Tool {
     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.Unstable;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.ha.ActiveStandbyElector;
 import org.apache.hadoop.http.HttpConfig;
 import org.apache.hadoop.net.NetUtils;
@@ -87,7 +88,17 @@ public class YarnConfiguration extends Configuration {
     });
     Configuration.addDeprecations(new DeprecationDelta[] {
         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."
           + "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
   ////////////////////////////////

+ 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);
     configurationPropsToSkipCompare
         .add(YarnConfiguration.ROUTER_RMADMIN_ADDRESS);
+    configurationPropsToSkipCompare
+        .add(YarnConfiguration.ROUTER_WEBAPP_DEFAULT_INTERCEPTOR_CLASS);
 
     // Federation policies configs to be ignored
     configurationPropsToSkipCompare
@@ -121,6 +123,15 @@ public class TestYarnConfigurationFields extends TestConfigurationFieldsBase {
     configurationPropsToSkipCompare
         .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
     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>
   </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>
     <description>Full path of the ZooKeeper znode where RM state will be
     stored. This must be supplied when using
@@ -485,22 +460,6 @@
     <value>/rmstore</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>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>
     <description>
       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>
   </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>
     <description>URI pointing to the location of the FileSystem path where
     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>
                 </configuration>
               </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>
           </plugin>
         </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)
 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
 # path in case the OS we're compiling on doesn't have
 # a hook in get_executable. We'll use this define
@@ -80,12 +83,20 @@ endfunction()
 include_directories(
     ${CMAKE_CURRENT_SOURCE_DIR}
     ${CMAKE_BINARY_DIR}
+    ${GTEST_SRC_DIR}/include
     main/native/container-executor
     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)
 
+add_library(gtest ${GTEST_SRC_DIR}/gtest-all.cc)
+set_target_properties(gtest PROPERTIES COMPILE_FLAGS "-w")
+
 add_library(container
+    main/native/container-executor/impl/util.c
     main/native/container-executor/impl/configuration.c
     main/native/container-executor/impl/container-executor.c
     main/native/container-executor/impl/get_executable.c
@@ -95,9 +106,11 @@ add_library(container
 add_executable(container-executor
     main/native/container-executor/impl/main.c
 )
+
 target_link_libraries(container-executor
     container
 )
+
 output_directory(container-executor target/usr/local/bin)
 
 add_executable(test-container-executor
@@ -107,3 +120,12 @@ target_link_libraries(test-container-executor
     container ${EXTRA_LIBS}
 )
 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 "configuration.h"
-#include "container-executor.h"
+#include "util.h"
 
+#define __STDC_FORMAT_MACROS
 #include <inttypes.h>
 #include <errno.h>
 #include <unistd.h>
-#include <stdio.h>
 #include <stdlib.h>
 #include <string.h>
 #include <sys/stat.h>
-#include <sys/types.h>
-#include <limits.h>
-#include <ctype.h>
 
 #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
-void free_configurations(struct configuration *cfg) {
+void free_configuration(struct configuration *cfg) {
   int i = 0;
   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;
 }
@@ -65,13 +85,13 @@ static int is_only_root_writable(const char *file) {
   }
   if (file_stat.st_uid != 0) {
     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;
   }
   if ((file_stat.st_mode & (S_IWGRP | S_IWOTH)) != 0) {
     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 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)
  */
-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;
-  char buffer[EXECUTOR_PATH_MAX*2 + 1];
+  char buffer[EXECUTOR_PATH_MAX * 2 + 1];
 
   if (file_name[0] == '/') {
     real_fname = file_name;
@@ -96,7 +116,7 @@ char *resolve_config_path(const char* file_name, const char *root) {
 #ifdef HAVE_CANONICALIZE_FILE_NAME
   char * ret = (real_fname == NULL) ? NULL : canonicalize_file_name(real_fname);
 #else
-  char * ret = (real_fname == NULL) ? NULL : realpath(real_fname, NULL);
+  char *ret = (real_fname == NULL) ? NULL : realpath(real_fname, NULL);
 #endif
 #ifdef DEBUG
   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.
  * 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
-  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 {
     if (!is_only_root_writable(dir)) {
       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);
     }
-
-    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 *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);
   }
-  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);
-      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);
 
   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.
  *
  */
-char * get_value(const char* key, struct configuration *cfg) {
+char *get_section_value(const char *key, const struct section *section) {
   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;
@@ -311,61 +572,80 @@ char * get_value(const char* key, struct configuration *cfg) {
  * Function to return an array of values for a key.
  * 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
  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)
     return -EINVAL;
 
-  char *split = strchr(input, '=');
+  const char *split = strchr(input, '=');
 
   if (split == NULL)
     return -EINVAL;
 
-  int key_len = split - input;
+  unsigned long key_len = split - input;
 
   if (out_len < (key_len + 1) || out == NULL)
     return -ENAMETOOLONG;
@@ -400,13 +680,13 @@ int get_kv_value(const char *input, char *out, size_t out_len) {
   if (input == NULL)
     return -EINVAL;
 
-  char *split = strchr(input, '=');
+  const char *split = strchr(input, '=');
 
   if (split == NULL)
     return -EINVAL;
 
   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)
     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.
  */
 
+#ifndef __YARN_CONTAINER_EXECUTOR_CONFIG_H__
+#define __YARN_CONTAINER_EXECUTOR_CONFIG_H__
+
 #ifdef __FreeBSD__
 #define _WITH_GETLINE
 #endif
@@ -23,62 +26,160 @@
 #include <stddef.h>
 
 /** Define a platform-independent constant instead of using PATH_MAX */
-
 #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 *value;
 };
 
+struct section {
+  int size;
+  char *name;
+  struct kv_pair **kv_pairs;
+};
+
 struct configuration {
   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);
 
-// 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'
@@ -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);
 
-/**
- * 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 "container-executor.h"
 #include "utils/string-utils.h"
+#include "util.h"
+#include "config.h"
 
 #include <inttypes.h>
 #include <libgen.h>
@@ -43,8 +45,6 @@
 #include <getopt.h>
 #include <regex.h>
 
-#include "config.h"
-
 #ifndef HAVE_FCHMODAT
 #include "compat/fchmodat.h"
 #endif
@@ -92,7 +92,8 @@ FILE* ERRORFILE = NULL;
 static uid_t nm_uid = -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,
    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
-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
 void free_executor_configurations() {
-    free_configurations(&executor_cfg);
+    free_configuration(&CFG);
 }
 
 //Lookup nodemanager group from container executor configuration.
 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) {
@@ -431,8 +439,8 @@ int change_user(uid_t user, gid_t group) {
 }
 
 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;
 
     if (enabled_str != NULL) {
@@ -753,7 +761,7 @@ static struct passwd* get_user_info(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;
   if (whitelist != NULL) {
     for(; *users; ++users) {
@@ -781,7 +789,7 @@ struct passwd* check_user(const char *user) {
     fflush(LOGFILE);
     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;
   if (min_uid_str != NULL) {
     char *end_ptr = NULL;
@@ -808,7 +816,7 @@ struct passwd* check_user(const char *user) {
     free(user_info);
     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 ?
     (char**) DEFAULT_BANNED_USERS : 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 **linesplit = (char **) malloc(sizeof(char *));
   char *p = NULL;
-  int c = 0;
   *split_counter = 0;
   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) {
   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);
 
   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);
-  char **args = extract_values_delim(docker_command_with_binary, " ");
+  char **args = split_delimiter(docker_command_with_binary, " ");
 
   int exit_code = -1;
   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();
 
   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);
 
   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
 };
 
-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 {
   CHECK_SETUP = 1,
   MOUNT_CGROUPS = 2,
@@ -111,11 +66,6 @@ enum operations {
 
 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
 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. */
 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. */
 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 "configuration.h"
 #include "container-executor.h"
+#include "util.h"
 
 #include <errno.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 "configuration.h"
 #include "container-executor.h"
+#include "util.h"
 
 #include <errno.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_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;
       return 0;
    } 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_value = resources_value;
-    cmd_input.resources_values = extract_values(resources_value);
+    cmd_input.resources_values = split(resources_value);
     *operation = RUN_AS_USER_LAUNCH_CONTAINER;
     return 0;
 
@@ -565,8 +566,8 @@ int main(int argc, char **argv) {
     exit_code = initialize_app(cmd_input.yarn_user_name,
                             cmd_input.app_id,
                             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);
     break;
   case RUN_AS_USER_LAUNCH_DOCKER_CONTAINER:
@@ -591,8 +592,8 @@ int main(int argc, char **argv) {
                       cmd_input.script_file,
                       cmd_input.cred_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.resources_key,
                       cmd_input.resources_values);
@@ -619,8 +620,8 @@ int main(int argc, char **argv) {
                     cmd_input.script_file,
                     cmd_input.cred_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_values);
     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 "container-executor.h"
 #include "utils/string-utils.h"
+#include "util.h"
 
 #include <inttypes.h>
 #include <errno.h>
@@ -404,7 +405,7 @@ void test_delete_app() {
 }
 
 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);
 
   if (value != expected_value) {
@@ -419,7 +420,8 @@ void test_is_feature_enabled() {
   FILE *file = fopen(filename, "w");
   int disabled = 0;
   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) {
     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.name6.enabled=2\n");
   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",
       disabled, &cfg);
@@ -449,7 +452,7 @@ void test_is_feature_enabled() {
           disabled, &cfg);
 
 
-  free_configurations(&cfg);
+  free_configuration(&exec_cfg);
 }
 
 void test_delete_user() {
@@ -1345,8 +1348,8 @@ int main(int argc, char **argv) {
 
   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);
 

+ 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);
     int timeoutSecs = 0;
     do {
-      Thread.sleep(2000);
+      Thread.sleep(1000);
       containerStatus =
           containerManager.getContainerStatuses(request)
               .getContainerStatuses().get(0);
       LOG.info("Waiting for container to get into one of states " + fStates
           + ". Current state is " + containerStatus.getState());
-      timeoutSecs += 2;
+      timeoutSecs += 1;
     } while (!fStates.contains(containerStatus.getState())
         && timeoutSecs < timeOutMax);
     LOG.info("Container state is " + containerStatus.getState());
@@ -371,7 +371,7 @@ public abstract class BaseContainerManagerTest {
         .containermanager.container.ContainerState currentState = null;
     int timeoutSecs = 0;
     do {
-      Thread.sleep(2000);
+      Thread.sleep(1000);
       container =
           containerManager.getContext().getContainers().get(containerID);
       if (container != null) {
@@ -381,9 +381,9 @@ public abstract class BaseContainerManagerTest {
         LOG.info("Waiting for NM container to get into one of the following " +
             "states: " + finalStates + ". Current state is " + currentState);
       }
-      timeoutSecs += 2;
+      timeoutSecs += 1;
     } while (!finalStates.contains(currentState)
-        && timeoutSecs++ < timeOutMax);
+        && timeoutSecs < timeOutMax);
     LOG.info("Container state is " + currentState);
     Assert.assertTrue("ContainerState is not correct (timedout)",
         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.util.StringUtils;
 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.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
@@ -96,8 +97,8 @@ public class ActiveStandbyElectorBasedElectorService extends AbstractService
     zkSessionTimeout = conf.getLong(YarnConfiguration.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 =
         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.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.Private;
 import org.apache.hadoop.conf.Configuration;
 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.StringUtils;
 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.api.records.ApplicationAttemptId;
 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.RMNodeEvent;
 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.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
@@ -192,7 +189,7 @@ public class ResourceManager extends CompositeService implements Recoverable {
   protected ResourceTrackerService resourceTracker;
   private JvmMetrics jvmMetrics;
   private boolean curatorEnabled = false;
-  private CuratorFramework curator;
+  private ZKCuratorManager zkManager;
   private final String zkRootNodePassword =
       Long.toString(new SecureRandom().nextLong());
   private boolean recoveryEnabled;
@@ -345,7 +342,7 @@ public class ResourceManager extends CompositeService implements Recoverable {
         conf.getBoolean(YarnConfiguration.CURATOR_LEADER_ELECTOR,
             YarnConfiguration.DEFAULT_CURATOR_LEADER_ELECTOR_ENABLED);
     if (curatorEnabled) {
-      this.curator = createAndStartCurator(conf);
+      this.zkManager = createAndStartZKManager(conf);
       elector = new CuratorBasedElectorService(this);
     } else {
       elector = new ActiveStandbyElectorBasedElectorService(this);
@@ -353,50 +350,49 @@ public class ResourceManager extends CompositeService implements Recoverable {
     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 {
-    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<>();
-    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() {
-    return this.curator;
+    if (this.zkManager == null) {
+      return null;
+    }
+    return this.zkManager.getCurator();
   }
 
   public String getZkRootNodePassword() {
@@ -701,8 +697,7 @@ public class ResourceManager extends CompositeService implements Recoverable {
         }
       }
 
-      // creating monitors that handle preemption
-      createPolicyMonitors();
+      createSchedulerMonitors();
 
       masterService = createApplicationMasterService();
       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)) {
         LOG.info("Loading policy monitors");
         List<SchedulingEditPolicy> policies = conf.getInstances(
@@ -1264,8 +1258,8 @@ public class ResourceManager extends CompositeService implements Recoverable {
       configurationProvider.close();
     }
     super.serviceStop();
-    if (curator != null) {
-      curator.close();
+    if (zkManager != null) {
+      zkManager.close();
     }
     transitionToStandby(false);
     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.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 {
 
   void init(Configuration config, RMContext context,
-      PreemptableResourceScheduler scheduler);
+      ResourceScheduler scheduler);
 
   /**
    * 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.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.PreemptableResourceScheduler;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -59,8 +58,7 @@ public class SchedulingMonitor extends AbstractService {
   }
 
   public void serviceInit(Configuration conf) throws Exception {
-    scheduleEditPolicy.init(conf, rmContext,
-        (PreemptableResourceScheduler) rmContext.getScheduler());
+    scheduleEditPolicy.init(conf, rmContext, rmContext.getScheduler());
     this.monitorInterval = scheduleEditPolicy.getMonitoringInterval();
     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.nodelabels.RMNodeLabelsManager;
 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.CapacityScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
@@ -150,7 +150,7 @@ public class ProportionalCapacityPreemptionPolicy
   }
 
   public void init(Configuration config, RMContext context,
-      PreemptableResourceScheduler sched) {
+      ResourceScheduler sched) {
     LOG.info("Preemption monitor:" + this.getClass().getCanonicalName());
     assert null == scheduler : "Unexpected duplicate call to init";
     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.yarn.server.resourcemanager.RMContext;
 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.LoggerFactory;
 
@@ -39,16 +39,16 @@ public abstract class InvariantsChecker implements SchedulingEditPolicy {
 
   private Configuration conf;
   private RMContext context;
-  private PreemptableResourceScheduler scheduler;
+  private ResourceScheduler scheduler;
   private boolean throwOnInvariantViolation;
   private long monitoringInterval;
 
   @Override
   public void init(Configuration config, RMContext rmContext,
-      PreemptableResourceScheduler preemptableResourceScheduler) {
+      ResourceScheduler scheduler) {
     this.conf = config;
     this.context = rmContext;
-    this.scheduler = preemptableResourceScheduler;
+    this.scheduler = scheduler;
     this.throwOnInvariantViolation =
         conf.getBoolean(InvariantsChecker.THROW_ON_VIOLATION, false);
     this.monitoringInterval =
@@ -89,7 +89,7 @@ public abstract class InvariantsChecker implements SchedulingEditPolicy {
     return context;
   }
 
-  public PreemptableResourceScheduler getScheduler() {
+  public ResourceScheduler getScheduler() {
     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.source.JvmMetrics;
 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.ResourceScheduler;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -42,7 +42,6 @@ import java.io.IOException;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.stream.Collectors;
 
 /**
  * This policy checks at every invocation that a given set of invariants
@@ -78,9 +77,9 @@ public class MetricsInvariantChecker extends InvariantsChecker {
 
   @Override
   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.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.security.token.delegation.DelegationKey;
 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.ApplicationId;
 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.server.records.Version;
 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.ApplicationAttemptStateData;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.ApplicationStateData;
@@ -201,8 +201,8 @@ public class ZKRMStateStore extends RMStateStore {
   private final String zkRootNodeAuthScheme =
       new DigestAuthenticationProvider().getScheme();
 
-  @VisibleForTesting
-  protected CuratorFramework curatorFramework;
+  /** Manager for the ZooKeeper connection. */
+  private ZKCuratorManager zkManager;
 
   /*
    * Indicates different app attempt state store operations.
@@ -298,12 +298,11 @@ public class ZKRMStateStore extends RMStateStore {
       appIdNodeSplitIndex = YarnConfiguration.DEFAULT_ZK_APPID_NODE_SPLIT_INDEX;
     }
 
-    zkAcl = RMZKUtils.getZKAcls(conf);
+    zkAcl = ZKCuratorManager.getZKAcls(conf);
 
     if (HAUtil.isHAEnabled(conf)) {
       String zkRootNodeAclConf = HAUtil.getConfValueForRMInstance
           (YarnConfiguration.ZK_RM_STATE_STORE_ROOT_NODE_ACL, conf);
-
       if (zkRootNodeAclConf != null) {
         zkRootNodeAclConf = ZKUtil.resolveConfIndirection(zkRootNodeAclConf);
 
@@ -330,10 +329,9 @@ public class ZKRMStateStore extends RMStateStore {
     amrmTokenSecretManagerRoot =
         getNodePath(zkRootNodePath, AMRMTOKEN_SECRET_MANAGER_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");
     }
 
+    CuratorFramework curatorFramework = zkManager.getCurator();
     if (HAUtil.isHAEnabled(getConfig())) {
       curatorFramework.setACL().withACL(zkRootNodeAcl).forPath(zkRootNodePath);
     } else {
@@ -401,6 +400,7 @@ public class ZKRMStateStore extends RMStateStore {
     }
 
     if (!HAUtil.isHAEnabled(getConfig())) {
+      CuratorFramework curatorFramework = zkManager.getCurator();
       IOUtils.closeStream(curatorFramework);
     }
   }
@@ -936,6 +936,7 @@ public class ZKRMStateStore extends RMStateStore {
       }
       safeDelete(appIdRemovePath);
     } else {
+      CuratorFramework curatorFramework = zkManager.getCurator();
       curatorFramework.delete().deletingChildrenIfNeeded().
           forPath(appIdRemovePath);
     }
@@ -1236,38 +1237,32 @@ public class ZKRMStateStore extends RMStateStore {
 
   @VisibleForTesting
   byte[] getData(final String path) throws Exception {
-    return curatorFramework.getData().forPath(path);
+    return zkManager.getData(path);
   }
 
   @VisibleForTesting
   List<ACL> getACL(final String path) throws Exception {
-    return curatorFramework.getACL().forPath(path);
+    return zkManager.getACL(path);
   }
 
   @VisibleForTesting
   List<String> getChildren(final String path) throws Exception {
-    return curatorFramework.getChildren().forPath(path);
+    return zkManager.getChildren(path);
   }
 
   @VisibleForTesting
   boolean exists(final String path) throws Exception {
-    return curatorFramework.checkExists().forPath(path) != null;
+    return zkManager.exists(path);
   }
 
   @VisibleForTesting
   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
   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,
@@ -1310,6 +1305,7 @@ public class ZKRMStateStore extends RMStateStore {
     private CuratorTransactionFinal transactionFinal;
 
     SafeTransaction() throws Exception {
+      CuratorFramework curatorFramework = zkManager.getCurator();
       CuratorTransaction transaction = curatorFramework.inTransaction();
       transactionFinal = transaction.create()
           .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()));
     }
     // 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) {
       message =
           "The time difference ("
-              + (contract.getDeadline() - contract.getArrival())
+              + (duration)
               + ") between arrival (" + contract.getArrival() + ") "
               + "and deadline (" + contract.getDeadline() + ") must "
               + " 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.
     String recurrenceExpression = contract.getRecurrenceExpression();
     try {
-      Long recurrence = Long.parseLong(recurrenceExpression);
+      long recurrence = Long.parseLong(recurrenceExpression);
       if (recurrence < 0) {
         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);
       }
     } catch (NumberFormatException e) {
       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);
     }
   }

+ 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
     // These fields can be visible from outside only after they are saved in
     // StateStore
-    String diags = null;
+    BoundedAppender diags = new BoundedAppender(diagnostics.limit);
 
     // don't leave the tracking URL pointing to a non-existent AM
     if (conf.getBoolean(YarnConfiguration.APPLICATION_HISTORY_ENABLED,
@@ -1329,15 +1329,15 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
     int exitStatus = ContainerExitStatus.INVALID;
     switch (event.getType()) {
     case LAUNCH_FAILED:
-      diags = event.getDiagnosticMsg();
+      diags.append(event.getDiagnosticMsg());
       break;
     case REGISTERED:
-      diags = getUnexpectedAMRegisteredDiagnostics();
+      diags.append(getUnexpectedAMRegisteredDiagnostics());
       break;
     case UNREGISTERED:
       RMAppAttemptUnregistrationEvent unregisterEvent =
           (RMAppAttemptUnregistrationEvent) event;
-      diags = unregisterEvent.getDiagnosticMsg();
+      diags.append(unregisterEvent.getDiagnosticMsg());
       // reset finalTrackingUrl to url sent by am
       finalTrackingUrl = sanitizeTrackingUrl(unregisterEvent.getFinalTrackingUrl());
       finalStatus = unregisterEvent.getFinalApplicationStatus();
@@ -1345,16 +1345,16 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
     case CONTAINER_FINISHED:
       RMAppAttemptContainerFinishedEvent finishEvent =
           (RMAppAttemptContainerFinishedEvent) event;
-      diags = getAMContainerCrashedDiagnostics(finishEvent);
+      diags.append(getAMContainerCrashedDiagnostics(finishEvent));
       exitStatus = finishEvent.getContainerStatus().getExitStatus();
       break;
     case KILL:
       break;
     case FAIL:
-      diags = event.getDiagnosticMsg();
+      diags.append(event.getDiagnosticMsg());
       break;
     case EXPIRE:
-      diags = getAMExpiredDiagnostics(event);
+      diags.append(getAMExpiredDiagnostics(event));
       break;
     default:
       break;
@@ -1368,7 +1368,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
         ApplicationAttemptStateData.newInstance(
             applicationAttemptId,  getMasterContainer(),
             rmStore.getCredentialsFromAppAttempt(this),
-            startTime, stateToBeStored, finalTrackingUrl, diags,
+            startTime, stateToBeStored, finalTrackingUrl, diags.toString(),
             finalStatus, exitStatus,
           getFinishTime(), resUsage.getMemorySeconds(),
           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 : "
                 + 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;
 
+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.logging.Log;
 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.QueueUserACLInfo;
 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.factory.providers.RecordFactoryProvider;
 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.NodeType;
 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.SchedulerUtils;
 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.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
 @Evolving
 public class ParentQueue extends AbstractCSQueue {
@@ -315,18 +313,21 @@ public class ParentQueue extends AbstractCSQueue {
 
         // Check if the child-queue already exists
         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);
             currentChildQueues.put(newChildQueueName, newChildQueue);
             // inform CapacitySchedulerQueueManager
-            CapacitySchedulerQueueManager queueManager = this.csContext
-                .getCapacitySchedulerQueueManager();
+            CapacitySchedulerQueueManager queueManager =
+                this.csContext.getCapacitySchedulerQueueManager();
             queueManager.addQueue(newChildQueueName, newChildQueue);
             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<>();
 
     for (FSQueueType queueType : FSQueueType.values()) {
-      configuredQueues.put(queueType, new HashSet<String>());
+      configuredQueues.put(queueType, new HashSet<>());
     }
 
     // Read and parse the allocations file.
@@ -280,7 +280,7 @@ public class AllocationFileLoaderService extends AbstractService {
       throw new AllocationConfigurationException("Bad fair scheduler config " +
           "file: top-level element not <allocations>");
     NodeList elements = root.getChildNodes();
-    List<Element> queueElements = new ArrayList<Element>();
+    List<Element> queueElements = new ArrayList<>();
     Element placementPolicyElement = null;
     for (int i = 0; i < elements.getLength(); i++) {
       Node node = elements.item(i);
@@ -294,8 +294,9 @@ public class AllocationFileLoaderService extends AbstractService {
           NodeList fields = element.getChildNodes();
           for (int j = 0; j < fields.getLength(); j++) {
             Node fieldNode = fields.item(j);
-            if (!(fieldNode instanceof Element))
+            if (!(fieldNode instanceof Element)) {
               continue;
+            }
             Element field = (Element) fieldNode;
             if ("maxRunningApps".equals(field.getTagName())) {
               String text = ((Text)field.getFirstChild()).getData().trim();
@@ -490,8 +491,9 @@ public class AllocationFileLoaderService extends AbstractService {
 
     for (int j = 0; j < fields.getLength(); j++) {
       Node fieldNode = fields.item(j);
-      if (!(fieldNode instanceof Element))
+      if (!(fieldNode instanceof Element)) {
         continue;
+      }
       Element field = (Element) fieldNode;
       if ("minResources".equals(field.getTagName())) {
         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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.ha.ClientBaseWithFixes;
 import org.apache.hadoop.ha.HAServiceProtocol;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
@@ -61,8 +62,8 @@ public abstract class RMHATestBase extends ClientBaseWithFixes{
     configuration.setBoolean(YarnConfiguration.RECOVERY_ENABLED, true);
     configuration.set(YarnConfiguration.RM_STORE,
         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.set(YarnConfiguration.RM_CLUSTER_ID, "test-yarn-cluster");
     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
   public void testSubmitReservationInvalidRecurrenceExpression() {
+    // first check recurrence expression
     ReservationSubmissionRequest request =
         createSimpleReservationSubmissionRequest(1, 1, 1, 5, 3, "123abc");
     plan = null;
@@ -318,6 +319,23 @@ public class TestReservationInputValidator {
           .startsWith("Invalid period "));
       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
@@ -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
   public void testDeleteReservationNormal() {
     ReservationDeleteRequest request = new ReservationDeleteRequestPBImpl();
@@ -710,11 +795,19 @@ public class TestReservationInputValidator {
   private ReservationUpdateRequest createSimpleReservationUpdateRequest(
       int numRequests, int numContainers, long arrival, long deadline,
       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
     ReservationUpdateRequest request = new ReservationUpdateRequestPBImpl();
     ReservationDefinition rDef = new ReservationDefinitionPBImpl();
     rDef.setArrival(arrival);
     rDef.setDeadline(deadline);
+    rDef.setRecurrenceExpression(recurrence);
     if (numRequests > 0) {
       ReservationRequests reqs = new ReservationRequestsPBImpl();
       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.assertFalse;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -42,7 +43,6 @@ import java.util.Set;
 import java.util.concurrent.BrokenBarrierException;
 import java.util.concurrent.CyclicBarrier;
 
-import com.google.common.base.Supplier;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -167,6 +167,7 @@ import org.junit.Before;
 import org.junit.Test;
 import org.mockito.Mockito;
 
+import com.google.common.base.Supplier;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Sets;
@@ -665,6 +666,36 @@ public class TestCapacityScheduler {
     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
    * @return, CS configuration which has deleted a queue(b1)
@@ -4643,6 +4674,10 @@ public class TestCapacityScheduler {
     try {
       cs.reinitialize(conf, mockContext);
     } 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"
           + " a queue WITHOUT running apps");
     }
@@ -4712,6 +4747,83 @@ public class TestCapacityScheduler {
     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
    * @throws Exception

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