Bladeren bron

Merge branch 'trunk' into HDFS-7240

 Conflicts:
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetConf.java
Anu Engineer 7 jaren geleden
bovenliggende
commit
7c58cfea8f
99 gewijzigde bestanden met toevoegingen van 4190 en 694 verwijderingen
  1. 30 22
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
  2. 5 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java
  3. 5 4
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/DecayRpcScheduler.java
  4. 2 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java
  5. 3 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/log/Log4Json.java
  6. 9 5
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/MetricsJsonBuilder.java
  7. 4 4
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsSystemImpl.java
  8. 5 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticator.java
  9. 9 4
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/HttpExceptionUtils.java
  10. 2 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ReflectionUtils.java
  11. 7 3
      hadoop-common-project/hadoop-common/src/site/markdown/Compatibility.md
  12. 18 0
      hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md
  13. 9 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java
  14. 10 10
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/loadGenerator/LoadGenerator.java
  15. 12 2
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/mount/MountdBase.java
  16. 8 2
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/Nfs3Base.java
  17. 3 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
  18. 38 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
  19. 17 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
  20. 2 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java
  21. 8 2
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3.java
  22. 5 0
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3Utils.java
  23. 5 2
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/PrivilegedNfsGatewayStarter.java
  24. 11 14
      hadoop-hdfs-project/hadoop-hdfs/src/main/bin/start-dfs.sh
  25. 12 15
      hadoop-hdfs-project/hadoop-hdfs/src/main/bin/stop-dfs.sh
  26. 83 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
  27. 4 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
  28. 7 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SlowDiskTracker.java
  29. 6 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SlowPeerTracker.java
  30. 6 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
  31. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ShortCircuitRegistry.java
  32. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/checker/AbstractFuture.java
  33. 9 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java
  34. 3 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedReconstructor.java
  35. 2 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
  36. 8 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/NodePlan.java
  37. 114 45
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  38. 24 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystemLock.java
  39. 19 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
  40. 10 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/NameNodeMetrics.java
  41. 21 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
  42. 30 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetConf.java
  43. 20 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
  44. 2 0
      hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
  45. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingInvalidateBlock.java
  46. 4 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSlowDiskTracker.java
  47. 6 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSlowPeerTracker.java
  48. 16 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLogger.java
  49. 667 25
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLoggerWithCommands.java
  50. 46 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystemLock.java
  51. 7 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java
  52. 169 6
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestGetConf.java
  53. 8 8
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java
  54. 134 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java
  55. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
  56. 10 1
      hadoop-project/pom.xml
  57. 10 4
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
  58. 58 2
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
  59. 100 4
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java
  60. 20 5
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LocalMetadataStore.java
  61. 17 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStore.java
  62. 15 0
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/NullMetadataStore.java
  63. 14 3
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3Guard.java
  64. 430 109
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java
  65. 157 3
      hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3guard.md
  66. 10 0
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java
  67. 4 4
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardListConsistency.java
  68. 22 15
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardWriteBack.java
  69. 0 18
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
  70. 134 4
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java
  71. 164 3
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolDynamoDB.java
  72. 129 13
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolLocal.java
  73. 121 0
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestS3GuardCLI.java
  74. 1 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/AMRMClientAsyncImpl.java
  75. 11 30
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
  76. 5 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerEventType.java
  77. 160 15
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
  78. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerState.java
  79. 86 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/UpdateContainerTokenEvent.java
  80. 47 67
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/ContainerScheduler.java
  81. 25 21
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/UpdateContainerSchedulerEvent.java
  82. 1 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestEventFlow.java
  83. 2 0
      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. 221 8
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
  85. 0 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/TestContainer.java
  86. 101 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/TestContainerSchedulerQueuing.java
  87. 10 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
  88. 130 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockMemoryRMStateStore.java
  89. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
  90. 0 32
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRMMemoryStateStore.java
  91. 3 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationCleanup.java
  92. 3 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestContainerResourceUsage.java
  93. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHA.java
  94. 30 17
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
  95. 37 14
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java
  96. 10 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java
  97. 185 14
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java
  98. 31 25
      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
  99. 3 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestRMDelegationTokens.java

+ 30 - 22
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java

@@ -81,6 +81,7 @@ import javax.xml.transform.stream.StreamResult;
 
 import com.google.common.base.Charsets;
 import org.apache.commons.collections.map.UnmodifiableMap;
+import org.apache.commons.io.FilenameUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
@@ -2811,6 +2812,7 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
       String confName = null;
       String confValue = null;
       String confInclude = null;
+      String confTag = null;
       boolean confFinal = false;
       boolean fallbackAllowed = false;
       boolean fallbackEntered = false;
@@ -2825,6 +2827,7 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
             confName = null;
             confValue = null;
             confFinal = false;
+            confTag = null;
             confSource.clear();
 
             // First test for short format configuration
@@ -2843,9 +2846,8 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
                 confSource.add(StringInterner.weakIntern(
                     reader.getAttributeValue(i)));
               } else if ("tag".equals(propertyAttr)) {
-                //Read tags and put them in propertyTagsMap
-                readTagFromConfig(reader.getAttributeValue(i), confName,
-                    confValue, confSource);
+                confTag = StringInterner
+                    .weakIntern(reader.getAttributeValue(i));
               }
             }
             break;
@@ -2937,9 +2939,7 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
             break;
           case "tag":
             if (token.length() > 0) {
-              //Read tags and put them in propertyTagsMap
-              readTagFromConfig(token.toString(), confName,
-                  confValue, confSource);
+              confTag = StringInterner.weakIntern(token.toString());
             }
             break;
           case "include":
@@ -2956,6 +2956,11 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
               break;
             }
             confSource.add(name);
+            //Read tags and put them in propertyTagsMap
+            if (confTag != null) {
+              readTagFromConfig(confTag, confName, confValue, confSource);
+            }
+
             DeprecatedKeyInfo keyInfo =
                 deprecations.getDeprecatedKeyMap().get(confName);
             if (keyInfo != null) {
@@ -3001,21 +3006,24 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
         if (confSource.size() > 0) {
           for (String source : confSource) {
             PropertyTag tag1 = this.getPropertyTag(tagStr,
-                source.split("-")[0]);
-            if (propertyTagsMap.containsKey(tag1)) {
-              propertyTagsMap.get(tag1)
-                  .setProperty(confName, confValue);
-            } else {
-              Properties props = new Properties();
-              props.setProperty(confName, confValue);
-              propertyTagsMap.put(tag1, props);
+                FilenameUtils.getName(source).split("-")[0]);
+            if (tag1 != null) {
+              //Handle property with no/null value
+              if (confValue == null) {
+                confValue = "";
+              }
+              if (propertyTagsMap.containsKey(tag1)) {
+                propertyTagsMap.get(tag1).setProperty(confName, confValue);
+              } else {
+                Properties props = new Properties();
+                props.setProperty(confName, confValue);
+                propertyTagsMap.put(tag1, props);
+              }
             }
           }
         } else {
-          //If no source is set try to find tag in CorePropertyTag
-          if (propertyTagsMap
-              .containsKey(CorePropertyTag.valueOf(tagStr)
-              )) {
+          // If no source is set try to find tag in CorePropertyTag
+          if (propertyTagsMap.containsKey(CorePropertyTag.valueOf(tagStr))) {
             propertyTagsMap.get(CorePropertyTag.valueOf(tagStr))
                 .setProperty(confName, confValue);
           } else {
@@ -3025,11 +3033,11 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
                 props);
           }
         }
-      } catch (IllegalArgumentException iae) {
-        //Log the invalid tag and continue to parse rest of the
-        // properties.
+      } catch (Exception ex) {
+        // Log the invalid tag and continue to parse rest of the properties.
         LOG.info("Invalid tag '" + tagStr + "' found for "
-            + "property:" + confName, iae);
+            + "property:" + confName + " Source:" + Arrays
+            .toString(confSource.toArray()), ex);
       }
 
     }

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

@@ -77,6 +77,7 @@ import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension;
 import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.CryptoExtension;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Strings;
@@ -133,6 +134,9 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
 
   private final ValueQueue<EncryptedKeyVersion> encKeyVersionQueue;
 
+  private static final ObjectWriter WRITER =
+      new ObjectMapper().writerWithDefaultPrettyPrinter();
+
   private class EncryptedQueueRefiller implements
     ValueQueue.QueueRefiller<EncryptedKeyVersion> {
 
@@ -226,8 +230,7 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
   private static void writeJson(Object obj, OutputStream os)
       throws IOException {
     Writer writer = new OutputStreamWriter(os, StandardCharsets.UTF_8);
-    ObjectMapper jsonMapper = new ObjectMapper();
-    jsonMapper.writerWithDefaultPrettyPrinter().writeValue(writer, obj);
+    WRITER.writeValue(writer, obj);
   }
 
   /**

+ 5 - 4
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/DecayRpcScheduler.java

@@ -36,6 +36,7 @@ import java.util.concurrent.atomic.AtomicReference;
 import javax.management.ObjectName;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
 import com.google.common.base.Preconditions;
 import com.google.common.util.concurrent.AtomicDoubleArray;
 import org.apache.commons.lang.exception.ExceptionUtils;
@@ -128,6 +129,8 @@ public class DecayRpcScheduler implements RpcScheduler,
   public static final Logger LOG =
       LoggerFactory.getLogger(DecayRpcScheduler.class);
 
+  private static final ObjectWriter WRITER = new ObjectMapper().writer();
+
   // Track the decayed and raw (no decay) number of calls for each schedulable
   // identity from all previous decay windows: idx 0 for decayed call count and
   // idx 1 for the raw call count
@@ -909,8 +912,7 @@ public class DecayRpcScheduler implements RpcScheduler,
       return "{}";
     } else {
       try {
-        ObjectMapper om = new ObjectMapper();
-        return om.writeValueAsString(decisions);
+        return WRITER.writeValueAsString(decisions);
       } catch (Exception e) {
         return "Error: " + e.getMessage();
       }
@@ -919,8 +921,7 @@ public class DecayRpcScheduler implements RpcScheduler,
 
   public String getCallVolumeSummary() {
     try {
-      ObjectMapper om = new ObjectMapper();
-      return om.writeValueAsString(getDecayedCallCounts());
+      return WRITER.writeValueAsString(getDecayedCallCounts());
     } catch (Exception e) {
       return "Error: " + e.getMessage();
     }

+ 2 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java

@@ -231,7 +231,7 @@ public class WritableRpcEngine implements RpcEngine {
       throws Throwable {
       long startTime = 0;
       if (LOG.isDebugEnabled()) {
-        startTime = Time.now();
+        startTime = Time.monotonicNow();
       }
 
       // if Tracing is on then start a new span for this rpc.
@@ -251,7 +251,7 @@ public class WritableRpcEngine implements RpcEngine {
         if (traceScope != null) traceScope.close();
       }
       if (LOG.isDebugEnabled()) {
-        long callTime = Time.now() - startTime;
+        long callTime = Time.monotonicNow() - startTime;
         LOG.debug("Call: " + method.getName() + " " + callTime);
       }
       return value.get();

+ 3 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/log/Log4Json.java

@@ -24,6 +24,7 @@ import com.fasterxml.jackson.core.JsonGenerator;
 import com.fasterxml.jackson.databind.JsonNode;
 import com.fasterxml.jackson.databind.MappingJsonFactory;
 import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectReader;
 import com.fasterxml.jackson.databind.node.ContainerNode;
 import org.apache.log4j.Layout;
 import org.apache.log4j.helpers.ISO8601DateFormat;
@@ -105,6 +106,7 @@ public class Log4Json extends Layout {
    * configuration it must be done in a static intializer block.
    */
   private static final JsonFactory factory = new MappingJsonFactory();
+  private static final ObjectReader READER = new ObjectMapper(factory).reader();
   public static final String DATE = "date";
   public static final String EXCEPTION_CLASS = "exceptionclass";
   public static final String LEVEL = "level";
@@ -252,8 +254,7 @@ public class Log4Json extends Layout {
    * @throws IOException on any parsing problems
    */
   public static ContainerNode parse(String json) throws IOException {
-    ObjectMapper mapper = new ObjectMapper(factory);
-    JsonNode jsonNode = mapper.readTree(json);
+    JsonNode jsonNode = READER.readTree(json);
     if (!(jsonNode instanceof ContainerNode)) {
       throw new IOException("Wrong JSON data: " + json);
     }

+ 9 - 5
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/MetricsJsonBuilder.java

@@ -18,17 +18,18 @@
 
 package org.apache.hadoop.metrics2;
 
-import java.io.IOException;
-import java.util.LinkedHashMap;
-import java.util.Map;
-
 import org.apache.commons.lang.exception.ExceptionUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.codehaus.jackson.map.ObjectMapper;
+import org.codehaus.jackson.map.ObjectWriter;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.IOException;
+import java.util.LinkedHashMap;
+import java.util.Map;
+
 /**
  * Build a JSON dump of the metrics.
  *
@@ -44,6 +45,9 @@ public class MetricsJsonBuilder extends MetricsRecordBuilder {
   private final MetricsCollector parent;
   private Map<String, Object> innerMetrics = new LinkedHashMap<>();
 
+  private static final ObjectWriter WRITER =
+      new ObjectMapper().writer();
+
   /**
    * Build an instance.
    * @param parent parent collector. Unused in this instance; only used for
@@ -116,7 +120,7 @@ public class MetricsJsonBuilder extends MetricsRecordBuilder {
   @Override
   public String toString() {
     try {
-      return new ObjectMapper().writeValueAsString(innerMetrics);
+      return WRITER.writeValueAsString(innerMetrics);
     } catch (IOException e) {
       LOG.warn("Failed to dump to Json.", e);
       return ExceptionUtils.getStackTrace(e);

+ 4 - 4
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsSystemImpl.java

@@ -415,10 +415,10 @@ public class MetricsSystemImpl extends MetricsSystem implements MetricsSource {
 
   private void snapshotMetrics(MetricsSourceAdapter sa,
                                MetricsBufferBuilder bufferBuilder) {
-    long startTime = Time.now();
+    long startTime = Time.monotonicNow();
     bufferBuilder.add(sa.name(), sa.getMetrics(collector, true));
     collector.clear();
-    snapshotStat.add(Time.now() - startTime);
+    snapshotStat.add(Time.monotonicNow() - startTime);
     LOG.debug("Snapshotted source "+ sa.name());
   }
 
@@ -431,7 +431,7 @@ public class MetricsSystemImpl extends MetricsSystem implements MetricsSource {
   synchronized void publishMetrics(MetricsBuffer buffer, boolean immediate) {
     int dropped = 0;
     for (MetricsSinkAdapter sa : sinks.values()) {
-      long startTime = Time.now();
+      long startTime = Time.monotonicNow();
       boolean result;
       if (immediate) {
         result = sa.putMetricsImmediate(buffer); 
@@ -439,7 +439,7 @@ public class MetricsSystemImpl extends MetricsSystem implements MetricsSource {
         result = sa.putMetrics(buffer, logicalTime);
       }
       dropped += result ? 0 : 1;
-      publishStat.add(Time.now() - startTime);
+      publishStat.add(Time.monotonicNow() - startTime);
     }
     droppedPubAll.incr(dropped);
   }

+ 5 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticator.java

@@ -18,6 +18,7 @@
 package org.apache.hadoop.security.token.delegation.web;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectReader;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.security.SecurityUtil;
@@ -54,6 +55,9 @@ public abstract class DelegationTokenAuthenticator implements Authenticator {
   private static final String CONTENT_TYPE = "Content-Type";
   private static final String APPLICATION_JSON_MIME = "application/json";
 
+  private static final ObjectReader READER =
+      new ObjectMapper().readerFor(Map.class);
+
   private static final String HTTP_GET = "GET";
   private static final String HTTP_PUT = "PUT";
 
@@ -316,8 +320,7 @@ public abstract class DelegationTokenAuthenticator implements Authenticator {
         if (contentType != null &&
             contentType.contains(APPLICATION_JSON_MIME)) {
           try {
-            ObjectMapper mapper = new ObjectMapper();
-            ret = mapper.readValue(conn.getInputStream(), Map.class);
+            ret = READER.readValue(conn.getInputStream());
           } catch (Exception ex) {
             throw new AuthenticationException(String.format(
                 "'%s' did not handle the '%s' delegation token operation: %s",

+ 9 - 4
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/HttpExceptionUtils.java

@@ -18,6 +18,8 @@
 package org.apache.hadoop.util;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectReader;
+import com.fasterxml.jackson.databind.ObjectWriter;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 
@@ -54,6 +56,11 @@ public class HttpExceptionUtils {
 
   private static final String ENTER = System.getProperty("line.separator");
 
+  private static final ObjectReader READER =
+      new ObjectMapper().readerFor(Map.class);
+  private static final ObjectWriter WRITER =
+      new ObjectMapper().writerWithDefaultPrettyPrinter();
+
   /**
    * Creates a HTTP servlet response serializing the exception in it as JSON.
    *
@@ -74,9 +81,8 @@ public class HttpExceptionUtils {
     json.put(ERROR_CLASSNAME_JSON, ex.getClass().getName());
     Map<String, Object> jsonResponse = new LinkedHashMap<String, Object>();
     jsonResponse.put(ERROR_JSON, json);
-    ObjectMapper jsonMapper = new ObjectMapper();
     Writer writer = response.getWriter();
-    jsonMapper.writerWithDefaultPrettyPrinter().writeValue(writer, jsonResponse);
+    WRITER.writeValue(writer, jsonResponse);
     writer.flush();
   }
 
@@ -144,8 +150,7 @@ public class HttpExceptionUtils {
       InputStream es = null;
       try {
         es = conn.getErrorStream();
-        ObjectMapper mapper = new ObjectMapper();
-        Map json = mapper.readValue(es, Map.class);
+        Map json = READER.readValue(es);
         json = (Map) json.get(ERROR_JSON);
         String exClass = (String) json.get(ERROR_CLASSNAME_JSON);
         String exMsg = (String) json.get(ERROR_MESSAGE_JSON);

+ 2 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ReflectionUtils.java

@@ -212,7 +212,7 @@ public class ReflectionUtils {
     boolean dumpStack = false;
     if (log.isInfoEnabled()) {
       synchronized (ReflectionUtils.class) {
-        long now = Time.now();
+        long now = Time.monotonicNow();
         if (now - previousLogTime >= minInterval * 1000) {
           previousLogTime = now;
           dumpStack = true;
@@ -241,7 +241,7 @@ public class ReflectionUtils {
     boolean dumpStack = false;
     if (log.isInfoEnabled()) {
       synchronized (ReflectionUtils.class) {
-        long now = Time.now();
+        long now = Time.monotonicNow();
         if (now - previousLogTime >= minInterval * 1000) {
           previousLogTime = now;
           dumpStack = true;

+ 7 - 3
hadoop-common-project/hadoop-common/src/site/markdown/Compatibility.md

@@ -191,9 +191,13 @@ dependencies is part of the Hadoop ABI.
 
 The minimum required versions of the native components on which Hadoop depends
 at compile time and/or runtime SHALL be considered
-[Stable](./InterfaceClassification.html#Stable). Changes to the minimum
-required versions MUST NOT increase between minor releases within a major
-version.
+[Evolving](./InterfaceClassification.html#Evolving). Changes to the minimum
+required versions SHOULD NOT increase between minor releases within a major
+version, though updates because of security issues, license issues, or other
+reasons may occur. When the native components on which Hadoop depends must
+be updated between minor releases within a major release, where possible the
+changes SHOULD only change the minor versions of the components without
+changing the major versions.
 
 ### Wire Protocols
 

+ 18 - 0
hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md

@@ -268,6 +268,24 @@ The server-side metrics for a journal from the JournalNode's perspective. Each m
 | `Syncs3600s90thPercentileLatencyMicros` | The 90th percentile of sync latency in microseconds (1 hour granularity) |
 | `Syncs3600s95thPercentileLatencyMicros` | The 95th percentile of sync latency in microseconds (1 hour granularity) |
 | `Syncs3600s99thPercentileLatencyMicros` | The 99th percentile of sync latency in microseconds (1 hour granularity) |
+| `NumTransactionsBatchedInSync60sNumOps` | Number of times transactions were batched in sync operation (1 minute granularity) |
+| `NumTransactionsBatchedInSync60s50thPercentileLatencyMicros` | The 50th percentile of transactions batched in sync count (1 minute granularity) |
+| `NumTransactionsBatchedInSync60s75thPercentileLatencyMicros` | The 75th percentile of transactions batched in sync count (1 minute granularity) |
+| `NumTransactionsBatchedInSync60s90thPercentileLatencyMicros` | The 90th percentile of transactions batched in sync count (1 minute granularity) |
+| `NumTransactionsBatchedInSync60s95thPercentileLatencyMicros` | The 95th percentile of transactions batched in sync count (1 minute granularity) |
+| `NumTransactionsBatchedInSync60s99thPercentileLatencyMicros` | The 99th percentile of transactions batched in sync count (1 minute granularity) |
+| `NumTransactionsBatchedInSync300sNumOps` | Number of times transactions were batched in sync operation (5 minutes granularity) |
+| `NumTransactionsBatchedInSync300s50thPercentileLatencyMicros` | The 50th percentile of transactions batched in sync count (5 minutes granularity) |
+| `NumTransactionsBatchedInSync300s75thPercentileLatencyMicros` | The 75th percentile of transactions batched in sync count (5 minutes granularity) |
+| `NumTransactionsBatchedInSync300s90thPercentileLatencyMicros` | The 90th percentile of transactions batched in sync count (5 minutes granularity) |
+| `NumTransactionsBatchedInSync300s95thPercentileLatencyMicros` | The 95th percentile of transactions batched in sync count (5 minutes granularity) |
+| `NumTransactionsBatchedInSync300s99thPercentileLatencyMicros` | The 99th percentile of transactions batched in sync count (5 minutes granularity) |
+| `NumTransactionsBatchedInSync3600sNumOps` | Number of times transactions were batched in sync operation (1 hour granularity) |
+| `NumTransactionsBatchedInSync3600s50thPercentileLatencyMicros` | The 50th percentile of transactions batched in sync count (1 hour granularity) |
+| `NumTransactionsBatchedInSync3600s75thPercentileLatencyMicros` | The 75th percentile of transactions batched in sync count (1 hour granularity) |
+| `NumTransactionsBatchedInSync3600s90thPercentileLatencyMicros` | The 90th percentile of transactions batched in sync count (1 hour granularity) |
+| `NumTransactionsBatchedInSync3600s95thPercentileLatencyMicros` | The 95th percentile of transactions batched in sync count (1 hour granularity) |
+| `NumTransactionsBatchedInSync3600s99thPercentileLatencyMicros` | The 99th percentile of transactions batched in sync count (1 hour granularity) |
 | `BatchesWritten` | Total number of batches written since startup |
 | `TxnsWritten` | Total number of transactions written since startup |
 | `BytesWritten` | Total number of bytes written since startup |

+ 9 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java

@@ -77,6 +77,8 @@ public class TestConfiguration {
   private Configuration conf;
   final static String CONFIG = new File("./test-config-TestConfiguration.xml").getAbsolutePath();
   final static String CONFIG2 = new File("./test-config2-TestConfiguration.xml").getAbsolutePath();
+  final static String CONFIG_CORE = new File("./core-site.xml")
+      .getAbsolutePath();
   final static String CONFIG_FOR_ENUM = new File("./test-config-enum-TestConfiguration.xml").getAbsolutePath();
   final static String CONFIG_FOR_URI = "file://"
       + new File("./test-config-uri-TestConfiguration.xml").getAbsolutePath();
@@ -114,6 +116,7 @@ public class TestConfiguration {
     new File(new URI(CONFIG_FOR_URI)).delete();
     new File(CONFIG_MULTI_BYTE).delete();
     new File(CONFIG_MULTI_BYTE_SAVED).delete();
+    new File(CONFIG_CORE).delete();
   }
 
   private void startConfig() throws IOException{
@@ -2248,14 +2251,14 @@ public class TestConfiguration {
   @Test
   public void testGetAllPropertiesByTags() throws Exception {
 
-    out = new BufferedWriter(new FileWriter(CONFIG));
+    out = new BufferedWriter(new FileWriter(CONFIG_CORE));
     startConfig();
     appendPropertyByTag("dfs.cblock.trace.io", "false", "DEBUG");
     appendPropertyByTag("dfs.replication", "1", "PERFORMANCE,REQUIRED");
     appendPropertyByTag("dfs.namenode.logging.level", "INFO", "CLIENT,DEBUG");
     endConfig();
 
-    Path fileResource = new Path(CONFIG);
+    Path fileResource = new Path(CONFIG_CORE);
     conf.addResource(fileResource);
     conf.getProps();
 
@@ -2266,6 +2269,10 @@ public class TestConfiguration {
     tagList.add(CorePropertyTag.CLIENT);
 
     Properties properties = conf.getAllPropertiesByTags(tagList);
+    String[] sources = conf.getPropertySources("dfs.replication");
+    assertTrue(sources.length == 1);
+    assertTrue(Arrays.toString(sources).contains("core-site.xml"));
+
     assertEq(3, properties.size());
     assertEq(true, properties.containsKey("dfs.namenode.logging.level"));
     assertEq(true, properties.containsKey("dfs.replication"));

+ 10 - 10
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/loadGenerator/LoadGenerator.java

@@ -277,9 +277,9 @@ public class LoadGenerator extends Configured implements Tool {
      * the entire file */
     private void read() throws IOException {
       String fileName = files.get(r.nextInt(files.size()));
-      long startTime = Time.now();
+      long startTimestamp = Time.monotonicNow();
       InputStream in = fc.open(new Path(fileName));
-      executionTime[OPEN] += (Time.now()-startTime);
+      executionTime[OPEN] += (Time.monotonicNow() - startTimestamp);
       totalNumOfOps[OPEN]++;
       while (in.read(buffer) != -1) {}
       in.close();
@@ -299,9 +299,9 @@ public class LoadGenerator extends Configured implements Tool {
       double fileSize = 0;
       while ((fileSize = r.nextGaussian()+2)<=0) {}
       genFile(file, (long)(fileSize*BLOCK_SIZE));
-      long startTime = Time.now();
+      long startTimestamp = Time.monotonicNow();
       fc.delete(file, true);
-      executionTime[DELETE] += (Time.now()-startTime);
+      executionTime[DELETE] += (Time.monotonicNow() - startTimestamp);
       totalNumOfOps[DELETE]++;
     }
     
@@ -310,9 +310,9 @@ public class LoadGenerator extends Configured implements Tool {
      */
     private void list() throws IOException {
       String dirName = dirs.get(r.nextInt(dirs.size()));
-      long startTime = Time.now();
+      long startTimestamp = Time.monotonicNow();
       fc.listStatus(new Path(dirName));
-      executionTime[LIST] += (Time.now()-startTime);
+      executionTime[LIST] += (Time.monotonicNow() - startTimestamp);
       totalNumOfOps[LIST]++;
     }
 
@@ -320,14 +320,14 @@ public class LoadGenerator extends Configured implements Tool {
      * The file is filled with 'a'.
      */
     private void genFile(Path file, long fileSize) throws IOException {
-      long startTime = Time.now();
+      long startTimestamp = Time.monotonicNow();
       FSDataOutputStream out = null;
       try {
         out = fc.create(file,
             EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE),
             CreateOpts.createParent(), CreateOpts.bufferSize(4096),
             CreateOpts.repFac((short) 3));
-        executionTime[CREATE] += (Time.now() - startTime);
+        executionTime[CREATE] += (Time.monotonicNow() - startTimestamp);
         numOfOps[CREATE]++;
 
         long i = fileSize;
@@ -337,8 +337,8 @@ public class LoadGenerator extends Configured implements Tool {
           i -= s;
         }
 
-        startTime = Time.now();
-        executionTime[WRITE_CLOSE] += (Time.now() - startTime);
+        startTimestamp = Time.monotonicNow();
+        executionTime[WRITE_CLOSE] += (Time.monotonicNow() - startTimestamp);
         numOfOps[WRITE_CLOSE]++;
       } finally {
         IOUtils.cleanupWithLogger(LOG, out);

+ 12 - 2
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/mount/MountdBase.java

@@ -109,6 +109,17 @@ abstract public class MountdBase {
     }
   }
 
+  public void stop() {
+    if (udpBoundPort > 0) {
+      rpcProgram.unregister(PortmapMapping.TRANSPORT_UDP, udpBoundPort);
+      udpBoundPort = 0;
+    }
+    if (tcpBoundPort > 0) {
+      rpcProgram.unregister(PortmapMapping.TRANSPORT_TCP, tcpBoundPort);
+      tcpBoundPort = 0;
+    }
+  }
+
   /**
    * Priority of the mountd shutdown hook.
    */
@@ -117,8 +128,7 @@ abstract public class MountdBase {
   private class Unregister implements Runnable {
     @Override
     public synchronized void run() {
-      rpcProgram.unregister(PortmapMapping.TRANSPORT_UDP, udpBoundPort);
-      rpcProgram.unregister(PortmapMapping.TRANSPORT_TCP, tcpBoundPort);
+      stop();
     }
   }
 

+ 8 - 2
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/Nfs3Base.java

@@ -78,6 +78,13 @@ public abstract class Nfs3Base {
     nfsBoundPort = tcpServer.getBoundPort();
   }
 
+  public void stop() {
+    if (nfsBoundPort > 0) {
+      rpcProgram.unregister(PortmapMapping.TRANSPORT_TCP, nfsBoundPort);
+      nfsBoundPort = 0;
+    }
+    rpcProgram.stopDaemons();
+  }
   /**
    * Priority of the nfsd shutdown hook.
    */
@@ -86,8 +93,7 @@ public abstract class Nfs3Base {
   private class NfsShutdownHook implements Runnable {
     @Override
     public synchronized void run() {
-      rpcProgram.unregister(PortmapMapping.TRANSPORT_TCP, nfsBoundPort);
-      rpcProgram.stopDaemons();
+      stop();
     }
   }
 }

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

@@ -21,6 +21,7 @@ import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Maps;
 import com.google.common.primitives.SignedBytes;
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.crypto.key.KeyProvider;
 import org.apache.hadoop.fs.BlockLocation;
@@ -94,6 +95,7 @@ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_DATA_TRANSF
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX;
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_NAMESERVICES;
 
+@InterfaceAudience.Private
 public class DFSUtilClient {
   public static final byte[] EMPTY_BYTES = {};
   private static final Logger LOG = LoggerFactory.getLogger(
@@ -406,7 +408,7 @@ public class DFSUtilClient {
    * @param keys list of keys in the order of preference
    * @return value of the key or default if a key was not found in configuration
    */
-  private static String getConfValue(String defaultValue, String keySuffix,
+  public static String getConfValue(String defaultValue, String keySuffix,
       Configuration conf, String... keys) {
     String value = null;
     for (String key : keys) {

+ 38 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java

@@ -26,6 +26,7 @@ import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.ContentSummary.Builder;
 import org.apache.hadoop.fs.FileChecksum;
 import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.MD5MD5CRC32CastagnoliFileChecksum;
 import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
 import org.apache.hadoop.fs.MD5MD5CRC32GzipFileChecksum;
@@ -65,6 +66,8 @@ import java.util.Map;
 
 class JsonUtilClient {
   static final DatanodeInfo[] EMPTY_DATANODE_INFO_ARRAY = {};
+  static final String UNSUPPPORTED_EXCEPTION_STR =
+      UnsupportedOperationException.class.getName();
 
   /** Convert a Json map to a RemoteException. */
   static RemoteException toRemoteException(final Map<?, ?> json) {
@@ -72,6 +75,9 @@ class JsonUtilClient {
         RemoteException.class.getSimpleName());
     final String message = (String)m.get("message");
     final String javaClassName = (String)m.get("javaClassName");
+    if (UNSUPPPORTED_EXCEPTION_STR.equals(javaClassName)) {
+      throw new UnsupportedOperationException(message);
+    }
     return new RemoteException(javaClassName, message);
   }
 
@@ -644,4 +650,36 @@ class JsonUtilClient {
     }
   }
 
+  /*
+   * The parameters which have default value -1 are required fields according
+   * to hdfs.proto.
+   * The default values for optional fields are taken from
+   * hdfs.proto#FsServerDefaultsProto.
+   */
+  public static FsServerDefaults toFsServerDefaults(final Map<?, ?> json) {
+    if (json == null) {
+      return null;
+    }
+    Map<?, ?> m =
+        (Map<?, ?>) json.get(FsServerDefaults.class.getSimpleName());
+    long blockSize =  getLong(m, "blockSize", -1);
+    int bytesPerChecksum = getInt(m, "bytesPerChecksum", -1);
+    int writePacketSize = getInt(m, "writePacketSize", -1);
+    short replication = (short) getInt(m, "replication", -1);
+    int fileBufferSize = getInt(m, "fileBufferSize", -1);
+    boolean encryptDataTransfer = m.containsKey("encryptDataTransfer")
+        ? (Boolean) m.get("encryptDataTransfer")
+        : false;
+    long trashInterval = getLong(m, "trashInterval", 0);
+    DataChecksum.Type type =
+        DataChecksum.Type.valueOf(getInt(m, "checksumType", 1));
+    String keyProviderUri = (String) m.get("keyProviderUri");
+    byte storagepolicyId = m.containsKey("defaultStoragePolicyId")
+        ? ((Number) m.get("defaultStoragePolicyId")).byteValue()
+        : HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
+    return new FsServerDefaults(blockSize, bytesPerChecksum,
+        writePacketSize, replication, fileBufferSize,
+        encryptDataTransfer, trashInterval, type, keyProviderUri,
+        storagepolicyId);
+  }
 }

+ 17 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java

@@ -65,6 +65,7 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FSInputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.GlobalStorageStatistics;
 import org.apache.hadoop.fs.GlobalStorageStatistics.StorageStatisticsProvider;
 import org.apache.hadoop.fs.StorageStatistics;
@@ -1766,6 +1767,22 @@ public class WebHdfsFileSystem extends FileSystem
     new FsPathRunner(op, src).run();
   }
 
+  /*
+   * Caller of this method should handle UnsupportedOperationException in case
+   * when new client is talking to old namenode that don't support
+   * FsServerDefaults call.
+   */
+  @Override
+  public FsServerDefaults getServerDefaults() throws IOException {
+    final HttpOpParam.Op op = GetOpParam.Op.GETSERVERDEFAULTS;
+    return new FsPathResponseRunner<FsServerDefaults>(op, null) {
+      @Override
+      FsServerDefaults decodeResponse(Map<?, ?> json) throws IOException {
+        return JsonUtilClient.toFsServerDefaults(json);
+      }
+    }.run();
+  }
+
   @VisibleForTesting
   InetSocketAddress[] getResolvedNNAddr() {
     return nnAddrs;

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java

@@ -46,7 +46,8 @@ public class GetOpParam extends HttpOpParam<GetOpParam.Op> {
     NULL(false, HttpURLConnection.HTTP_NOT_IMPLEMENTED),
 
     CHECKACCESS(false, HttpURLConnection.HTTP_OK),
-    LISTSTATUS_BATCH(false, HttpURLConnection.HTTP_OK);
+    LISTSTATUS_BATCH(false, HttpURLConnection.HTTP_OK),
+    GETSERVERDEFAULTS(false, HttpURLConnection.HTTP_OK);
 
     final boolean redirect;
     final int expectedHttpResponseCode;

+ 8 - 2
hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3.java

@@ -57,7 +57,7 @@ public class Nfs3 extends Nfs3Base {
     start(register);
   }
   
-  static void startService(String[] args,
+  static Nfs3 startService(String[] args,
       DatagramSocket registrationSocket) throws IOException {
     StringUtils.startupShutdownMessage(Nfs3.class, args, LOG);
     NfsConfiguration conf = new NfsConfiguration();
@@ -67,8 +67,14 @@ public class Nfs3 extends Nfs3Base {
     final Nfs3 nfsServer = new Nfs3(conf, registrationSocket,
         allowInsecurePorts);
     nfsServer.startServiceInternal(true);
+    return nfsServer;
   }
-  
+
+  public void stop() {
+    super.stop();
+    mountd.stop();
+  }
+
   public static void main(String[] args) throws IOException {
     startService(args, null);
   }

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3Utils.java

@@ -88,6 +88,8 @@ public class Nfs3Utils {
   /**
    * HDFS directory size is always zero. Try to return something meaningful
    * here. Assume each child take 32bytes.
+   * @param childNum number of children of the directory
+   * @return total size of the directory
    */
   public static long getDirSize(int childNum) {
     return (childNum + 2) * 32;
@@ -122,6 +124,9 @@ public class Nfs3Utils {
 
   /**
    * Send a write response to the netty network socket channel
+   * @param channel channel to which the buffer needs to be written
+   * @param out xdr object to be written to the channel
+   * @param xid transaction identifier
    */
   public static void writeChannel(Channel channel, XDR out, int xid) {
     if (channel == null) {

+ 5 - 2
hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/PrivilegedNfsGatewayStarter.java

@@ -40,6 +40,7 @@ public class PrivilegedNfsGatewayStarter implements Daemon {
   static final Log LOG = LogFactory.getLog(PrivilegedNfsGatewayStarter.class);
   private String[] args = null;
   private DatagramSocket registrationSocket = null;
+  private Nfs3 nfs3Server = null;
 
   @Override
   public void init(DaemonContext context) throws Exception {
@@ -68,12 +69,14 @@ public class PrivilegedNfsGatewayStarter implements Daemon {
 
   @Override
   public void start() throws Exception {
-    Nfs3.startService(args, registrationSocket);
+    nfs3Server = Nfs3.startService(args, registrationSocket);
   }
 
   @Override
   public void stop() throws Exception {
-    // Nothing to do.
+    if (nfs3Server != null) {
+      nfs3Server.stop();
+    }
   }
 
   @Override

+ 11 - 14
hadoop-hdfs-project/hadoop-hdfs/src/main/bin/start-dfs.sh

@@ -144,22 +144,19 @@ fi
 #---------------------------------------------------------
 # quorumjournal nodes (if any)
 
-SHARED_EDITS_DIR=$("${HADOOP_HDFS_HOME}/bin/hdfs" getconf -confKey dfs.namenode.shared.edits.dir 2>&-)
+JOURNAL_NODES=$("${HADOOP_HDFS_HOME}/bin/hdfs" getconf -journalNodes 2>&-)
 
-case "${SHARED_EDITS_DIR}" in
-  qjournal://*)
-    JOURNAL_NODES=$(echo "${SHARED_EDITS_DIR}" | sed 's,qjournal://\([^/]*\)/.*,\1,g; s/;/ /g; s/:[0-9]*//g')
-    echo "Starting journal nodes [${JOURNAL_NODES}]"
+if [[ "{$JOURNAL_NODES-}" != $'\n' ]]; then
+  echo "Starting journal nodes [${JOURNAL_NODES}]"
 
-    hadoop_uservar_su hdfs journalnode "${HADOOP_HDFS_HOME}/bin/hdfs" \
-      --workers \
-      --config "${HADOOP_CONF_DIR}" \
-      --hostnames "${JOURNAL_NODES}" \
-      --daemon start \
-      journalnode
-    (( HADOOP_JUMBO_RETCOUNTER=HADOOP_JUMBO_RETCOUNTER + $? ))
-  ;;
-esac
+  hadoop_uservar_su hdfs journalnode "${HADOOP_HDFS_HOME}/bin/hdfs" \
+    --workers \
+    --config "${HADOOP_CONF_DIR}" \
+    --hostnames "${JOURNAL_NODES}" \
+    --daemon start \
+    journalnode
+   (( HADOOP_JUMBO_RETCOUNTER=HADOOP_JUMBO_RETCOUNTER + $? ))
+fi
 
 #---------------------------------------------------------
 # ZK Failover controllers, if auto-HA is enabled

+ 12 - 15
hadoop-hdfs-project/hadoop-hdfs/src/main/bin/stop-dfs.sh

@@ -100,21 +100,18 @@ fi
 #---------------------------------------------------------
 # quorumjournal nodes (if any)
 
-SHARED_EDITS_DIR=$("${HADOOP_HDFS_HOME}/bin/hdfs" getconf -confKey dfs.namenode.shared.edits.dir 2>&-)
-
-case "${SHARED_EDITS_DIR}" in
-  qjournal://*)
-    JOURNAL_NODES=$(echo "${SHARED_EDITS_DIR}" | sed 's,qjournal://\([^/]*\)/.*,\1,g; s/;/ /g; s/:[0-9]*//g')
-    echo "Stopping journal nodes [${JOURNAL_NODES}]"
-
-    hadoop_uservar_su hdfs journalnode "${HADOOP_HDFS_HOME}/bin/hdfs" \
-      --workers \
-      --config "${HADOOP_CONF_DIR}" \
-      --hostnames "${JOURNAL_NODES}" \
-      --daemon stop \
-      journalnode
-  ;;
-esac
+JOURNAL_NODES=$("${HADOOP_HDFS_HOME}/bin/hdfs" getconf -journalNodes 2>&-)
+
+if [[ "{$JOURNAL_NODES-}" != $'\n' ]]; then
+  echo "Stopping journal nodes [${JOURNAL_NODES}]"
+
+  hadoop_uservar_su hdfs journalnode "${HADOOP_HDFS_HOME}/bin/hdfs" \
+    --workers \
+    --config "${HADOOP_CONF_DIR}" \
+    --hostnames "${JOURNAL_NODES}" \
+    --daemon stop \
+    journalnode
+fi
 
 #---------------------------------------------------------
 # ZK Failover controllers, if auto-HA is enabled

+ 83 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java

@@ -31,6 +31,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LIFELINE_RPC_ADD
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMESERVICE_ID;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SERVER_HTTPS_KEYPASSWORD_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SERVER_HTTPS_KEYSTORE_PASSWORD_KEY;
@@ -44,6 +45,8 @@ import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.URI;
 import java.net.URISyntaxException;
+import java.net.UnknownHostException;
+
 import java.security.SecureRandom;
 import java.util.Arrays;
 import java.util.Collection;
@@ -74,6 +77,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
+import org.apache.hadoop.hdfs.server.common.Util;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.http.HttpConfig;
 import org.apache.hadoop.http.HttpServer2;
@@ -453,6 +457,85 @@ public class DFSUtil {
     return principals;
   }
 
+  /**
+   * Returns list of Journalnode addresses from the configuration.
+   *
+   * @param conf configuration
+   * @return list of journalnode host names
+   * @throws URISyntaxException
+   * @throws IOException
+   */
+  public static Set<String> getJournalNodeAddresses(
+      Configuration conf) throws URISyntaxException, IOException {
+    Set<String> journalNodeList = new HashSet<>();
+    String journalsUri = "";
+    try {
+      journalsUri = conf.get(DFS_NAMENODE_SHARED_EDITS_DIR_KEY);
+      if (journalsUri == null) {
+        Collection<String> nameserviceIds = DFSUtilClient.
+            getNameServiceIds(conf);
+        for (String nsId : nameserviceIds) {
+          journalsUri = DFSUtilClient.getConfValue(
+              null, nsId, conf, DFS_NAMENODE_SHARED_EDITS_DIR_KEY);
+          if (journalsUri == null) {
+            Collection<String> nnIds = DFSUtilClient.getNameNodeIds(conf, nsId);
+            for (String nnId : nnIds) {
+              String suffix = DFSUtilClient.concatSuffixes(nsId, nnId);
+              journalsUri = DFSUtilClient.getConfValue(
+                  null, suffix, conf, DFS_NAMENODE_SHARED_EDITS_DIR_KEY);
+              if (journalsUri == null ||
+                  !journalsUri.startsWith("qjournal://")) {
+                return journalNodeList;
+              } else {
+                LOG.warn(DFS_NAMENODE_SHARED_EDITS_DIR_KEY +" is to be " +
+                    "configured as nameservice" +
+                    " specific key(append it with nameserviceId), no need" +
+                    " to append it with namenodeId");
+                URI uri = new URI(journalsUri);
+                List<InetSocketAddress> socketAddresses = Util.
+                    getAddressesList(uri);
+                for (InetSocketAddress is : socketAddresses) {
+                  journalNodeList.add(is.getHostName());
+                }
+              }
+            }
+          } else if (!journalsUri.startsWith("qjournal://")) {
+            return journalNodeList;
+          } else {
+            URI uri = new URI(journalsUri);
+            List<InetSocketAddress> socketAddresses = Util.
+                getAddressesList(uri);
+            for (InetSocketAddress is : socketAddresses) {
+              journalNodeList.add(is.getHostName());
+            }
+          }
+        }
+      } else {
+        if (!journalsUri.startsWith("qjournal://")) {
+          return journalNodeList;
+        } else {
+          URI uri = new URI(journalsUri);
+          List<InetSocketAddress> socketAddresses = Util.getAddressesList(uri);
+          for (InetSocketAddress is : socketAddresses) {
+            journalNodeList.add(is.getHostName());
+          }
+        }
+      }
+    } catch(UnknownHostException e) {
+      LOG.error("The conf property " + DFS_NAMENODE_SHARED_EDITS_DIR_KEY
+          + " is not properly set with correct journal node hostnames");
+      throw new UnknownHostException(journalsUri);
+    } catch(URISyntaxException e)  {
+      LOG.error("The conf property " + DFS_NAMENODE_SHARED_EDITS_DIR_KEY
+          + "is not set properly with correct journal node uri");
+      throw new URISyntaxException(journalsUri, "The conf property " +
+          DFS_NAMENODE_SHARED_EDITS_DIR_KEY + "is not" +
+          " properly set with correct journal node uri");
+    }
+
+    return journalNodeList;
+  }
+
   /**
    * Returns list of InetSocketAddress corresponding to  backup node rpc 
    * addresses from the configuration.

+ 4 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java

@@ -24,6 +24,8 @@ import org.apache.hadoop.hdfs.protocol.BlockType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLog;
 
 import java.io.IOException;
 
@@ -31,8 +33,8 @@ import static org.apache.hadoop.hdfs.protocol.BlockType.STRIPED;
 
 /**
  * BlockIdManager allocates the generation stamps and the block ID. The
- * {@see FSNamesystem} is responsible for persisting the allocations in the
- * {@see EditLog}.
+ * {@link FSNamesystem} is responsible for persisting the allocations in the
+ * {@link FSEditLog}.
  */
 public class BlockIdManager {
   /**

+ 7 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SlowDiskTracker.java

@@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonIgnore;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.core.JsonProcessingException;
 import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
@@ -67,6 +68,11 @@ public class SlowDiskTracker {
    */
   private final Timer timer;
 
+  /**
+   * ObjectWriter to convert JSON reports to String.
+   */
+  private static final ObjectWriter WRITER = new ObjectMapper().writer();
+
   /**
    * Number of disks to include in JSON report per operation. We will return
    * disks with the highest latency.
@@ -254,12 +260,11 @@ public class SlowDiskTracker {
    *         serialization failed.
    */
   public String getSlowDiskReportAsJsonString() {
-    ObjectMapper objectMapper = new ObjectMapper();
     try {
       if (slowDisksReport.isEmpty()) {
         return null;
       }
-      return objectMapper.writeValueAsString(slowDisksReport);
+      return WRITER.writeValueAsString(slowDisksReport);
     } catch (JsonProcessingException e) {
       // Failed to serialize. Don't log the exception call stack.
       LOG.debug("Failed to serialize statistics" + e);

+ 6 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SlowPeerTracker.java

@@ -21,6 +21,7 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.core.JsonProcessingException;
 import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.primitives.Ints;
@@ -69,6 +70,10 @@ public class SlowPeerTracker {
    */
   private final Timer timer;
 
+  /**
+   * ObjectWriter to convert JSON reports to String.
+   */
+  private static final ObjectWriter WRITER = new ObjectMapper().writer();
   /**
    * Number of nodes to include in JSON report. We will return nodes with
    * the highest number of votes from peers.
@@ -188,9 +193,8 @@ public class SlowPeerTracker {
   public String getJson() {
     Collection<ReportForJson> validReports = getJsonReports(
         MAX_NODES_TO_REPORT);
-    ObjectMapper objectMapper = new ObjectMapper();
     try {
-      return objectMapper.writeValueAsString(validReports);
+      return WRITER.writeValueAsString(validReports);
     } catch (JsonProcessingException e) {
       // Failed to serialize. Don't log the exception call stack.
       LOG.debug("Failed to serialize statistics" + e);

+ 6 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java

@@ -1128,7 +1128,7 @@ public class DataNode extends ReconfigurableBase
   /**
    * Shutdown disk balancer.
    */
-  private  void shutdownDiskBalancer() {
+  private void shutdownDiskBalancer() {
     if (this.diskBalancer != null) {
       this.diskBalancer.shutdown();
       this.diskBalancer = null;
@@ -2137,7 +2137,11 @@ public class DataNode extends ReconfigurableBase
       ipcServer.stop();
     }
 
-    if (blockPoolManager != null) {
+    if (ecWorker != null) {
+      ecWorker.shutDown();
+    }
+
+    if(blockPoolManager != null) {
       try {
         this.blockPoolManager.shutDownAll(bposArray);
       } catch (InterruptedException ie) {

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ShortCircuitRegistry.java

@@ -44,6 +44,7 @@ import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.SlotId;
 import org.apache.hadoop.io.nativeio.SharedFileDescriptorFactory;
 import org.apache.hadoop.net.unix.DomainSocket;
 import org.apache.hadoop.net.unix.DomainSocketWatcher;
+import org.apache.hadoop.hdfs.shortcircuit.DfsClientShmManager;
 
 import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/checker/AbstractFuture.java

@@ -40,6 +40,7 @@ import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
+import java.util.concurrent.ThreadPoolExecutor.CallerRunsPolicy;
 import java.util.concurrent.locks.LockSupport;
 import java.util.logging.Level;
 import java.util.logging.Logger;

+ 9 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java

@@ -27,6 +27,8 @@ import org.apache.hadoop.util.Daemon;
 import org.slf4j.Logger;
 
 import java.util.Collection;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ExecutorCompletionService;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.SynchronousQueue;
 import java.util.concurrent.ThreadPoolExecutor;
@@ -149,7 +151,12 @@ public final class ErasureCodingWorker {
     return conf;
   }
 
-  ThreadPoolExecutor getStripedReadPool() {
-    return stripedReadPool;
+  CompletionService<Void> createReadService() {
+    return new ExecutorCompletionService<>(stripedReadPool);
+  }
+
+  public void shutDown() {
+    stripedReconstructionPool.shutdown();
+    stripedReadPool.shutdown();
   }
 }

+ 3 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedReconstructor.java

@@ -39,8 +39,6 @@ import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
 import java.util.BitSet;
 import java.util.concurrent.CompletionService;
-import java.util.concurrent.ExecutorCompletionService;
-import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.atomic.AtomicLong;
 
 /**
@@ -110,7 +108,7 @@ abstract class StripedReconstructor {
   // position in striped internal block
   private long positionInBlock;
   private StripedReader stripedReader;
-  private ThreadPoolExecutor stripedReadPool;
+  private ErasureCodingWorker erasureCodingWorker;
   private final CachingStrategy cachingStrategy;
   private long maxTargetLength = 0L;
   private final BitSet liveBitSet;
@@ -122,7 +120,7 @@ abstract class StripedReconstructor {
 
   StripedReconstructor(ErasureCodingWorker worker,
       StripedReconstructionInfo stripedReconInfo) {
-    this.stripedReadPool = worker.getStripedReadPool();
+    this.erasureCodingWorker = worker;
     this.datanode = worker.getDatanode();
     this.conf = worker.getConf();
     this.ecPolicy = stripedReconInfo.getEcPolicy();
@@ -225,7 +223,7 @@ abstract class StripedReconstructor {
   }
 
   CompletionService<Void> createReadService() {
-    return new ExecutorCompletionService<>(stripedReadPool);
+    return erasureCodingWorker.createReadService();
   }
 
   ExtendedBlock getBlockGroup() {

+ 2 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java

@@ -315,7 +315,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     deletingBlock = new HashMap<String, Set<Long>>();
 
     for (int idx = 0; idx < storage.getNumStorageDirs(); idx++) {
-      addVolume(dataLocations, storage.getStorageDir(idx));
+      addVolume(storage.getStorageDir(idx));
     }
     setupAsyncLazyPersistThreads();
 
@@ -413,8 +413,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     }
   }
 
-  private void addVolume(Collection<StorageLocation> dataLocations,
-      Storage.StorageDirectory sd) throws IOException {
+  private void addVolume(Storage.StorageDirectory sd) throws IOException {
     final StorageLocation storageLocation = sd.getStorageLocation();
 
     // If IOException raises from FsVolumeImpl() or getVolumeMap(), there is

+ 8 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/NodePlan.java

@@ -18,8 +18,9 @@
 package org.apache.hadoop.hdfs.server.diskbalancer.planner;
 
 import com.fasterxml.jackson.annotation.JsonTypeInfo;
-import com.fasterxml.jackson.databind.JavaType;
 import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectReader;
+import com.fasterxml.jackson.databind.ObjectWriter;
 import com.google.common.base.Preconditions;
 
 import java.io.IOException;
@@ -38,6 +39,10 @@ public class NodePlan {
   private int port;
   private long timeStamp;
 
+  private static final ObjectMapper MAPPER = new ObjectMapper();
+  private static final ObjectReader READER = MAPPER.readerFor(NodePlan.class);
+  private static final ObjectWriter WRITER = MAPPER.writerFor(
+      MAPPER.constructType(NodePlan.class));
   /**
    * returns timestamp when this plan was created.
    *
@@ -153,8 +158,7 @@ public class NodePlan {
    * @throws IOException
    */
   public static NodePlan parseJson(String json) throws IOException {
-    ObjectMapper mapper = new ObjectMapper();
-    return mapper.readValue(json, NodePlan.class);
+    return READER.readValue(json);
   }
 
   /**
@@ -164,10 +168,7 @@ public class NodePlan {
    * @throws IOException
    */
   public String toJson() throws IOException {
-    ObjectMapper mapper = new ObjectMapper();
-    JavaType planType = mapper.constructType(NodePlan.class);
-    return mapper.writerFor(planType)
-        .writeValueAsString(this);
+    return WRITER.writeValueAsString(this);
   }
 
   /**

+ 114 - 45
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -87,6 +87,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PERMISSIONS_SUPERUSERGROU
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PERMISSIONS_SUPERUSERGROUP_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_KEY;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import static org.apache.hadoop.hdfs.server.namenode.FSDirStatAndListingOp.*;
 
 import org.apache.hadoop.hdfs.protocol.ReplicatedBlockStats;
@@ -351,7 +352,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         && !auditLoggers.isEmpty();
   }
 
-  private void logAuditEvent(boolean succeeded, String cmd, String src)
+  void logAuditEvent(boolean succeeded, String cmd, String src)
       throws IOException {
     logAuditEvent(succeeded, cmd, src, null, null);
   }
@@ -1095,7 +1096,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       if (!success) {
         fsImage.close();
       }
-      writeUnlock("loadFSImage");
+      writeUnlock("loadFSImage", true);
     }
     imageLoadComplete();
   }
@@ -1586,6 +1587,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   public void writeUnlock(String opName) {
     this.fsLock.writeUnlock(opName);
   }
+
+  public void writeUnlock(String opName, boolean suppressWriteLockReport) {
+    this.fsLock.writeUnlock(opName, suppressWriteLockReport);
+  }
+
   @Override
   public boolean hasWriteLock() {
     return this.fsLock.isWriteLockedByCurrentThread();
@@ -1710,7 +1716,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * Dump all metadata into specified file
    */
   void metaSave(String filename) throws IOException {
-    checkSuperuserPrivilege();
+    String operationName = "metaSave";
+    checkSuperuserPrivilege(operationName);
     checkOperation(OperationCategory.UNCHECKED);
     writeLock();
     try {
@@ -1722,8 +1729,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       out.flush();
       out.close();
     } finally {
-      writeUnlock("metaSave");
+      writeUnlock(operationName);
     }
+    logAuditEvent(true, operationName, null);
   }
 
   private void metaSave(PrintWriter out) {
@@ -1774,7 +1782,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     return sw.toString();
   }
 
-  FsServerDefaults getServerDefaults() throws StandbyException {
+  @VisibleForTesting
+  public FsServerDefaults getServerDefaults() throws StandbyException {
     checkOperation(OperationCategory.READ);
     return serverDefaults;
   }
@@ -3118,7 +3127,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       requireEffectiveLayoutVersionForFeature(Feature.QUOTA_BY_STORAGE_TYPE);
     }
     checkOperation(OperationCategory.WRITE);
-    final String operationName = "setQuota";
+    final String operationName = getQuotaCommand(nsQuota, ssQuota);
     writeLock();
     boolean success = false;
     try {
@@ -4202,30 +4211,34 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     }
   }
 
-  DatanodeInfo[] datanodeReport(final DatanodeReportType type
-      ) throws AccessControlException, StandbyException {
-    checkSuperuserPrivilege();
+  DatanodeInfo[] datanodeReport(final DatanodeReportType type)
+      throws IOException {
+    String operationName = "datanodeReport";
+    DatanodeInfo[] arr;
+    checkSuperuserPrivilege(operationName);
     checkOperation(OperationCategory.UNCHECKED);
     readLock();
     try {
       checkOperation(OperationCategory.UNCHECKED);
       final DatanodeManager dm = getBlockManager().getDatanodeManager();      
       final List<DatanodeDescriptor> results = dm.getDatanodeListForReport(type);
-
-      DatanodeInfo[] arr = new DatanodeInfo[results.size()];
+      arr = new DatanodeInfo[results.size()];
       for (int i=0; i<arr.length; i++) {
         arr[i] = new DatanodeInfoBuilder().setFrom(results.get(i))
             .build();
       }
-      return arr;
     } finally {
-      readUnlock("datanodeReport");
+      readUnlock(operationName);
     }
+    logAuditEvent(true, operationName, null);
+    return arr;
   }
 
   DatanodeStorageReport[] getDatanodeStorageReport(final DatanodeReportType type
-      ) throws AccessControlException, StandbyException {
-    checkSuperuserPrivilege();
+      ) throws IOException {
+    String operationName = "getDatanodeStorageReport";
+    DatanodeStorageReport[] reports;
+    checkSuperuserPrivilege(operationName);
     checkOperation(OperationCategory.UNCHECKED);
     readLock();
     try {
@@ -4233,17 +4246,18 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       final DatanodeManager dm = getBlockManager().getDatanodeManager();      
       final List<DatanodeDescriptor> datanodes = dm.getDatanodeListForReport(type);
 
-      DatanodeStorageReport[] reports = new DatanodeStorageReport[datanodes.size()];
+      reports = new DatanodeStorageReport[datanodes.size()];
       for (int i = 0; i < reports.length; i++) {
         final DatanodeDescriptor d = datanodes.get(i);
         reports[i] = new DatanodeStorageReport(
             new DatanodeInfoBuilder().setFrom(d).build(),
             d.getStorageReports());
       }
-      return reports;
     } finally {
       readUnlock("getDatanodeStorageReport");
     }
+    logAuditEvent(true, operationName, null);
+    return reports;
   }
 
   /**
@@ -4253,8 +4267,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    */
   boolean saveNamespace(final long timeWindow, final long txGap)
       throws IOException {
+    String operationName = "saveNamespace";
     checkOperation(OperationCategory.UNCHECKED);
-    checkSuperuserPrivilege();
+    checkSuperuserPrivilege(operationName);
 
     boolean saved = false;
     cpLock();  // Block if a checkpointing is in progress on standby.
@@ -4268,12 +4283,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       }
       saved = getFSImage().saveNamespace(timeWindow, txGap, this);
     } finally {
-      readUnlock("saveNamespace");
+      readUnlock(operationName);
       cpUnlock();
     }
     if (saved) {
       LOG.info("New namespace image has been created");
     }
+    logAuditEvent(true, operationName, null);
     return saved;
   }
   
@@ -4283,9 +4299,10 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * 
    * @throws AccessControlException if superuser privilege is violated.
    */
-  boolean restoreFailedStorage(String arg) throws AccessControlException,
-      StandbyException {
-    checkSuperuserPrivilege();
+  boolean restoreFailedStorage(String arg) throws IOException {
+    String operationName = getFailedStorageCommand(arg);
+    boolean val = false;
+    checkSuperuserPrivilege(operationName);
     checkOperation(OperationCategory.UNCHECKED);
     cpLock();  // Block if a checkpointing is in progress on standby.
     writeLock();
@@ -4293,17 +4310,18 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       checkOperation(OperationCategory.UNCHECKED);
       
       // if it is disabled - enable it and vice versa.
-      if(arg.equals("check"))
-        return getFSImage().getStorage().getRestoreFailedStorage();
-      
-      boolean val = arg.equals("true");  // false if not
-      getFSImage().getStorage().setRestoreFailedStorage(val);
-      
-      return val;
+      if(arg.equals("check")) {
+        val = getFSImage().getStorage().getRestoreFailedStorage();
+      } else {
+        val = arg.equals("true");  // false if not
+        getFSImage().getStorage().setRestoreFailedStorage(val);
+      }
     } finally {
-      writeUnlock("restoreFailedStorage");
+      writeUnlock(operationName);
       cpUnlock();
     }
+    logAuditEvent(true, operationName, null);
+    return val;
   }
 
   Date getStartTime() {
@@ -4311,7 +4329,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
     
   void finalizeUpgrade() throws IOException {
-    checkSuperuserPrivilege();
+    String operationName = "finalizeUpgrade";
+    checkSuperuserPrivilege(operationName);
     checkOperation(OperationCategory.UNCHECKED);
     cpLock();  // Block if a checkpointing is in progress on standby.
     writeLock();
@@ -4319,26 +4338,33 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       checkOperation(OperationCategory.UNCHECKED);
       getFSImage().finalizeUpgrade(this.isHaEnabled() && inActiveState());
     } finally {
-      writeUnlock("finalizeUpgrade");
+      writeUnlock(operationName);
       cpUnlock();
     }
+    logAuditEvent(true, operationName, null);
   }
 
   void refreshNodes() throws IOException {
+    String operationName = "refreshNodes";
     checkOperation(OperationCategory.UNCHECKED);
-    checkSuperuserPrivilege();
+    checkSuperuserPrivilege(operationName);
     getBlockManager().getDatanodeManager().refreshNodes(new HdfsConfiguration());
+    logAuditEvent(true, operationName, null);
   }
 
   void setBalancerBandwidth(long bandwidth) throws IOException {
+    String operationName = "setBalancerBandwidth";
     checkOperation(OperationCategory.UNCHECKED);
-    checkSuperuserPrivilege();
+    checkSuperuserPrivilege(operationName);
     getBlockManager().getDatanodeManager().setBalancerBandwidth(bandwidth);
+    logAuditEvent(true, operationName, null);
   }
 
   boolean setSafeMode(SafeModeAction action) throws IOException {
+    String operationName = action.toString().toLowerCase();
+    boolean error = false;
     if (action != SafeModeAction.SAFEMODE_GET) {
-      checkSuperuserPrivilege();
+      checkSuperuserPrivilege(operationName);
       switch(action) {
       case SAFEMODE_LEAVE: // leave safe mode
         leaveSafeMode(false);
@@ -4351,8 +4377,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         break;
       default:
         LOG.error("Unexpected safe mode action");
+        error = true;
       }
     }
+    if (!error) {
+      logAuditEvent(true, operationName, null);
+    }
     return isInSafeMode();
   }
 
@@ -4488,7 +4518,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   CheckpointSignature rollEditLog() throws IOException {
-    checkSuperuserPrivilege();
+    String operationName = "rollEditLog";
+    CheckpointSignature result = null;
+    checkSuperuserPrivilege(operationName);
     checkOperation(OperationCategory.JOURNAL);
     writeLock();
     try {
@@ -4497,10 +4529,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       if (Server.isRpcInvocation()) {
         LOG.info("Roll Edit Log from " + Server.getRemoteAddress());
       }
-      return getFSImage().rollEditLog(getEffectiveLayoutVersion());
+      result = getFSImage().rollEditLog(getEffectiveLayoutVersion());
     } finally {
-      writeUnlock("rollEditLog");
+      writeUnlock(operationName);
     }
+    logAuditEvent(true, operationName, null);
+    return result;
   }
 
   NamenodeCommand startCheckpoint(NamenodeRegistration backupNode,
@@ -6189,11 +6223,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     checkOperation(OperationCategory.WRITE);
     final String operationName = "allowSnapshot";
     boolean success = false;
+    checkSuperuserPrivilege(operationName);
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot allow snapshot for " + path);
-      checkSuperuserPrivilege();
       FSDirSnapshotOp.allowSnapshot(dir, snapshotManager, path);
       success = true;
     } finally {
@@ -6207,12 +6241,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   void disallowSnapshot(String path) throws IOException {
     checkOperation(OperationCategory.WRITE);
     final String operationName = "disallowSnapshot";
+    checkSuperuserPrivilege(operationName);
     boolean success = false;
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot disallow snapshot for " + path);
-      checkSuperuserPrivilege();
       FSDirSnapshotOp.disallowSnapshot(dir, snapshotManager, path);
       success = true;
     } finally {
@@ -6405,7 +6439,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   RollingUpgradeInfo queryRollingUpgrade() throws IOException {
-    checkSuperuserPrivilege();
+    final String operationName = "queryRollingUpgrade";
+    checkSuperuserPrivilege(operationName);
     checkOperation(OperationCategory.READ);
     readLock();
     try {
@@ -6415,15 +6450,16 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       Preconditions.checkNotNull(rollingUpgradeInfo);
       boolean hasRollbackImage = this.getFSImage().hasRollbackFSImage();
       rollingUpgradeInfo.setCreatedRollbackImages(hasRollbackImage);
-      return rollingUpgradeInfo;
     } finally {
-      readUnlock("queryRollingUpgrade");
+      readUnlock(operationName);
     }
+    logAuditEvent(true, operationName, null, null, null);
+    return rollingUpgradeInfo;
   }
 
   RollingUpgradeInfo startRollingUpgrade() throws IOException {
     final String operationName = "startRollingUpgrade";
-    checkSuperuserPrivilege();
+    checkSuperuserPrivilege(operationName);
     checkOperation(OperationCategory.WRITE);
     writeLock();
     try {
@@ -6614,7 +6650,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
   RollingUpgradeInfo finalizeRollingUpgrade() throws IOException {
     final String operationName = "finalizeRollingUpgrade";
-    checkSuperuserPrivilege();
+    checkSuperuserPrivilege(operationName);
     checkOperation(OperationCategory.WRITE);
     writeLock();
     try {
@@ -7739,5 +7775,38 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         .size();
   }
 
+  void checkSuperuserPrivilege(String operationName)
+      throws IOException {
+    try {
+      checkSuperuserPrivilege();
+    } catch (AccessControlException ace) {
+      logAuditEvent(false, operationName, null);
+      throw ace;
+    }
+  }
+
+  String getQuotaCommand(long nsQuota, long dsQuota) {
+    if (nsQuota == HdfsConstants.QUOTA_RESET
+        && dsQuota == HdfsConstants.QUOTA_DONT_SET) {
+      return "clearQuota";
+    } else if (nsQuota == HdfsConstants.QUOTA_DONT_SET
+        && dsQuota == HdfsConstants.QUOTA_RESET) {
+      return "clearSpaceQuota";
+    } else if (dsQuota == HdfsConstants.QUOTA_DONT_SET) {
+      return "setQuota";
+    } else {
+      return "setSpaceQuota";
+    }
+  }
+
+  String getFailedStorageCommand(String mode) {
+    if(mode.equals("check")) {
+      return "checkRestoreFailedStorage";
+    } else if (mode.equals("true")) {
+      return "enableRestoreFailedStorage";
+    } else {
+      return "disableRestoreFailedStorage";
+    }
+  }
 }
 

+ 24 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystemLock.java

@@ -207,13 +207,34 @@ class FSNamesystemLock {
     }
   }
 
+  /**
+   * Unlocks FSNameSystem write lock. This internally calls {@link
+   * FSNamesystemLock#writeUnlock(String, boolean)}
+   */
   public void writeUnlock() {
-    writeUnlock(OP_NAME_OTHER);
+    writeUnlock(OP_NAME_OTHER, false);
   }
 
+  /**
+   * Unlocks FSNameSystem write lock. This internally calls {@link
+   * FSNamesystemLock#writeUnlock(String, boolean)}
+   *
+   * @param opName Operation name.
+   */
   public void writeUnlock(String opName) {
-    final boolean needReport = coarseLock.getWriteHoldCount() == 1 &&
-        coarseLock.isWriteLockedByCurrentThread();
+    writeUnlock(opName, false);
+  }
+
+  /**
+   * Unlocks FSNameSystem write lock.
+   *
+   * @param opName Operation name
+   * @param suppressWriteLockReport When false, event of write lock being held
+   * for long time will be logged in logs and metrics.
+   */
+  public void writeUnlock(String opName, boolean suppressWriteLockReport) {
+    final boolean needReport = !suppressWriteLockReport && coarseLock
+        .getWriteHoldCount() == 1 && coarseLock.isWriteLockedByCurrentThread();
     final long currentTimeNanos = timer.monotonicNowNanos();
     final long writeLockIntervalNanos =
         currentTimeNanos - writeLockHeldTimeStampNanos;

+ 19 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java

@@ -1624,6 +1624,7 @@ public class NameNodeRpcServer implements NamenodeProtocols {
     if (this.serviceRpcServer != null) {
       this.serviceRpcServer.refreshServiceAcl(new Configuration(), new HDFSPolicyProvider());
     }
+    namesystem.logAuditEvent(true, "refreshServiceAcl", null);
   }
 
   @Override // RefreshAuthorizationPolicyProtocol
@@ -1631,17 +1632,19 @@ public class NameNodeRpcServer implements NamenodeProtocols {
     LOG.info("Refreshing all user-to-groups mappings. Requested by user: " +
         getRemoteUser().getShortUserName());
     Groups.getUserToGroupsMappingService().refresh();
+    namesystem.logAuditEvent(true, "refreshUserToGroupsMappings", null);
   }
 
   @Override // RefreshAuthorizationPolicyProtocol
-  public void refreshSuperUserGroupsConfiguration() {
+  public void refreshSuperUserGroupsConfiguration() throws IOException {
     LOG.info("Refreshing SuperUser proxy group mapping list ");
 
     ProxyUsers.refreshSuperUserGroupsConfiguration();
+    namesystem.logAuditEvent(true, "refreshSuperUserGroupsConfiguration", null);
   }
 
   @Override // RefreshCallQueueProtocol
-  public void refreshCallQueue() {
+  public void refreshCallQueue() throws IOException {
     LOG.info("Refreshing call queue.");
 
     Configuration conf = new Configuration();
@@ -1649,6 +1652,7 @@ public class NameNodeRpcServer implements NamenodeProtocols {
     if (this.serviceRpcServer != null) {
       serviceRpcServer.refreshCallQueue(conf);
     }
+    namesystem.logAuditEvent(true, "refreshCallQueue", null);
   }
 
   @Override // GenericRefreshProtocol
@@ -2412,22 +2416,30 @@ public class NameNodeRpcServer implements NamenodeProtocols {
   @Override // ReconfigurationProtocol
   public void startReconfiguration() throws IOException {
     checkNNStartup();
-    namesystem.checkSuperuserPrivilege();
+    String operationName = "startNamenodeReconfiguration";
+    namesystem.checkSuperuserPrivilege(operationName);
     nn.startReconfigurationTask();
+    namesystem.logAuditEvent(true, operationName, null);
   }
 
   @Override // ReconfigurationProtocol
   public ReconfigurationTaskStatus getReconfigurationStatus()
       throws IOException {
     checkNNStartup();
-    namesystem.checkSuperuserPrivilege();
-    return nn.getReconfigurationTaskStatus();
+    String operationName = "getNamenodeReconfigurationStatus";
+    namesystem.checkSuperuserPrivilege(operationName);
+    ReconfigurationTaskStatus status = nn.getReconfigurationTaskStatus();
+    namesystem.logAuditEvent(true, operationName, null);
+    return status;
   }
 
   @Override // ReconfigurationProtocol
   public List<String> listReconfigurableProperties() throws IOException {
     checkNNStartup();
-    namesystem.checkSuperuserPrivilege();
-    return Lists.newArrayList(nn.getReconfigurableProperties());
+    String operationName = "listNamenodeReconfigurableProperties";
+    namesystem.checkSuperuserPrivilege(operationName);
+    List<String> result = Lists.newArrayList(nn.getReconfigurableProperties());
+    namesystem.logAuditEvent(true, operationName, null);
+    return result;
   }
 }

+ 10 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/NameNodeMetrics.java

@@ -115,6 +115,8 @@ public class NameNodeMetrics {
   final MutableQuantiles[] syncsQuantiles;
   @Metric("Journal transactions batched in sync")
   MutableCounterLong transactionsBatchedInSync;
+  @Metric("Journal transactions batched in sync")
+  final MutableQuantiles[] numTransactionsBatchedInSync;
   @Metric("Number of blockReports from individual storages")
   MutableRate storageBlockReport;
   final MutableQuantiles[] storageBlockReportQuantiles;
@@ -148,6 +150,7 @@ public class NameNodeMetrics {
     
     final int len = intervals.length;
     syncsQuantiles = new MutableQuantiles[len];
+    numTransactionsBatchedInSync = new MutableQuantiles[len];
     storageBlockReportQuantiles = new MutableQuantiles[len];
     cacheReportQuantiles = new MutableQuantiles[len];
     generateEDEKTimeQuantiles = new MutableQuantiles[len];
@@ -159,6 +162,10 @@ public class NameNodeMetrics {
       syncsQuantiles[i] = registry.newQuantiles(
           "syncs" + interval + "s",
           "Journal syncs", "ops", "latency", interval);
+      numTransactionsBatchedInSync[i] = registry.newQuantiles(
+          "numTransactionsBatchedInSync" + interval + "s",
+          "Number of Transactions batched in sync", "ops",
+          "count", interval);
       storageBlockReportQuantiles[i] = registry.newQuantiles(
           "storageBlockReport" + interval + "s",
           "Storage block report", "ops", "latency", interval);
@@ -304,6 +311,9 @@ public class NameNodeMetrics {
 
   public void incrTransactionsBatchedInSync(long count) {
     transactionsBatchedInSync.incr(count);
+    for (MutableQuantiles q : numTransactionsBatchedInSync) {
+      q.add(count);
+    }
   }
 
   public void incSuccessfulReReplications() {

+ 21 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java

@@ -57,6 +57,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.fs.permission.AclStatus;
@@ -115,6 +116,7 @@ public class NamenodeWebHdfsMethods {
   private Principal userPrincipal;
   private String remoteAddr;
 
+  private static volatile String serverDefaultsResponse = null;
   private @Context ServletContext context;
   private @Context HttpServletResponse response;
 
@@ -1121,11 +1123,30 @@ public class NamenodeWebHdfsMethods {
       final String js = JsonUtil.toJsonString(storagePolicy);
       return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
     }
+    case GETSERVERDEFAULTS: {
+      // Since none of the server defaults values are hot reloaded, we can
+      // cache the output of serverDefaults.
+      if (serverDefaultsResponse == null) {
+        FsServerDefaults serverDefaults = np.getServerDefaults();
+        serverDefaultsResponse = JsonUtil.toJsonString(serverDefaults);
+      }
+      return Response.ok(serverDefaultsResponse)
+          .type(MediaType.APPLICATION_JSON).build();
+    }
     default:
       throw new UnsupportedOperationException(op + " is not supported");
     }
   }
 
+  /*
+   * This is used only and only for testing.
+   * Please don't use it otherwise.
+   */
+  @VisibleForTesting
+  public static void resetServerDefaultsResponse() {
+    serverDefaultsResponse = null;
+  }
+
   private static String getTrashRoot(String fullPath,
       Configuration conf) throws IOException {
     FileSystem fs = FileSystem.get(conf != null ? conf : new Configuration());

+ 30 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetConf.java

@@ -20,12 +20,14 @@ package org.apache.hadoop.hdfs.tools;
 import java.io.IOException;
 import java.io.PrintStream;
 import java.net.InetSocketAddress;
+import java.net.URISyntaxException;
 import java.security.PrivilegedExceptionAction;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Collection;
+import java.util.Set;
 
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.conf.Configuration;
@@ -71,6 +73,7 @@ public class GetConf extends Configured implements Tool {
     SECONDARY("-secondaryNameNodes", 
         "gets list of secondary namenodes in the cluster."),
     BACKUP("-backupNodes", "gets list of backup nodes in the cluster."),
+    JOURNALNODE("-journalNodes", "gets list of journal nodes in the cluster."),
     INCLUDE_FILE("-includeFile",
         "gets the include file path that defines the datanodes " +
         "that can join the cluster."),
@@ -93,6 +96,8 @@ public class GetConf extends Configured implements Tool {
           new SecondaryNameNodesCommandHandler());
       map.put(StringUtils.toLowerCase(BACKUP.getName()),
           new BackupNodesCommandHandler());
+      map.put(StringUtils.toLowerCase(JOURNALNODE.getName()),
+          new JournalNodeCommandHandler());
       map.put(StringUtils.toLowerCase(INCLUDE_FILE.getName()),
           new CommandHandler(DFSConfigKeys.DFS_HOSTS));
       map.put(StringUtils.toLowerCase(EXCLUDE_FILE.getName()),
@@ -213,7 +218,19 @@ public class GetConf extends Configured implements Tool {
       return 0;
     }
   }
-  
+
+  /**
+   * Handler for {@linke Command#JOURNALNODE}.
+   */
+  static class JournalNodeCommandHandler extends CommandHandler {
+    @Override
+    public int doWorkInternal(GetConf tool, String[] args)
+        throws URISyntaxException, IOException {
+      tool.printSet(DFSUtil.getJournalNodeAddresses(tool.getConf()));
+      return 0;
+    }
+  }
+
   /**
    * Handler for {@link Command#SECONDARY}
    */
@@ -326,6 +343,18 @@ public class GetConf extends Configured implements Tool {
     printOut(buffer.toString());
   }
 
+  void printSet(Set<String> journalnodes) {
+    StringBuilder buffer = new StringBuilder();
+
+    for (String journalnode : journalnodes) {
+      if (buffer.length() > 0) {
+        buffer.append(" ");
+      }
+      buffer.append(journalnode);
+    }
+    printOut(buffer.toString());
+  }
+
   private void printUsage() {
     printError(USAGE);
   }

+ 20 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java

@@ -21,6 +21,7 @@ import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FileChecksum;
 import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.XAttr;
@@ -470,4 +471,23 @@ public class JsonUtil {
   public static String toJsonString(BlockStoragePolicy storagePolicy) {
     return toJsonString(BlockStoragePolicy.class, toJsonMap(storagePolicy));
   }
+
+  public static String toJsonString(FsServerDefaults serverDefaults) {
+    return toJsonString(FsServerDefaults.class, toJsonMap(serverDefaults));
+  }
+
+  private static Object toJsonMap(FsServerDefaults serverDefaults) {
+    final Map<String, Object> m = new HashMap<String, Object>();
+    m.put("blockSize", serverDefaults.getBlockSize());
+    m.put("bytesPerChecksum", serverDefaults.getBytesPerChecksum());
+    m.put("writePacketSize", serverDefaults.getWritePacketSize());
+    m.put("replication", serverDefaults.getReplication());
+    m.put("fileBufferSize", serverDefaults.getFileBufferSize());
+    m.put("encryptDataTransfer", serverDefaults.getEncryptDataTransfer());
+    m.put("trashInterval", serverDefaults.getTrashInterval());
+    m.put("checksumType", serverDefaults.getChecksumType().id);
+    m.put("keyProviderUri", serverDefaults.getKeyProviderUri());
+    m.put("defaultStoragePolicyId", serverDefaults.getDefaultStoragePolicyId());
+    return m;
+  }
 }

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md

@@ -116,6 +116,7 @@ Usage:
        hdfs getconf -namenodes
        hdfs getconf -secondaryNameNodes
        hdfs getconf -backupNodes
+       hdfs getconf -journalNodes
        hdfs getconf -includeFile
        hdfs getconf -excludeFile
        hdfs getconf -nnRpcAddresses
@@ -126,6 +127,7 @@ Usage:
 | `-namenodes` | gets list of namenodes in the cluster. |
 | `-secondaryNameNodes` | gets list of secondary namenodes in the cluster. |
 | `-backupNodes` | gets list of backup nodes in the cluster. |
+| `-journalNodes` | gets list of journal nodes in the cluster. |
 | `-includeFile` | gets the include file path that defines the datanodes that can join the cluster. |
 | `-excludeFile` | gets the exclude file path that defines the datanodes that need to decommissioned. |
 | `-nnRpcAddresses` | gets the namenode rpc addresses |

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingInvalidateBlock.java

@@ -172,7 +172,7 @@ public class TestPendingInvalidateBlock {
     Assert.assertEquals(0L, cluster.getNamesystem().getPendingDeletionBlocks());
     // restart DataNodes
     for (int i = 0; i < REPLICATION; i++) {
-      cluster.restartDataNode(dnprops[i], true);
+      cluster.restartDataNode(dnprops[i]);
     }
     cluster.waitActive();
 

+ 4 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSlowDiskTracker.java

@@ -25,6 +25,7 @@ import static org.junit.Assert.assertTrue;
 
 import com.fasterxml.jackson.core.type.TypeReference;
 import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectReader;
 import com.google.common.collect.ImmutableMap;
 import org.apache.hadoop.conf.Configuration;
 import static org.apache.hadoop.hdfs.DFSConfigKeys
@@ -76,6 +77,8 @@ public class TestSlowDiskTracker {
   private FakeTimer timer;
   private long reportValidityMs;
   private static final long OUTLIERS_REPORT_INTERVAL = 1000;
+  private static final ObjectReader READER = new ObjectMapper().readerFor(
+          new TypeReference<ArrayList<DiskLatency>>() {});
 
   static {
     conf = new HdfsConfiguration();
@@ -416,8 +419,7 @@ public class TestSlowDiskTracker {
 
   private ArrayList<DiskLatency> getAndDeserializeJson(
       final String json) throws IOException {
-    return (new ObjectMapper()).readValue(json,
-        new TypeReference<ArrayList<DiskLatency>>() {});
+    return READER.readValue(json);
   }
 
   private void addSlowDiskForTesting(String dnID, String disk,

+ 6 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSlowPeerTracker.java

@@ -20,9 +20,11 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
 
 import com.fasterxml.jackson.core.type.TypeReference;
 import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectReader;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
-import org.apache.hadoop.hdfs.server.blockmanagement.SlowPeerTracker.ReportForJson;
+import org.apache.hadoop.hdfs.server.blockmanagement.SlowPeerTracker
+    .ReportForJson;
 import org.apache.hadoop.util.FakeTimer;
 import org.junit.Before;
 import org.junit.Rule;
@@ -39,7 +41,6 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 
-
 /**
  * Tests for {@link SlowPeerTracker}.
  */
@@ -57,6 +58,8 @@ public class TestSlowPeerTracker {
   private SlowPeerTracker tracker;
   private FakeTimer timer;
   private long reportValidityMs;
+  private static final ObjectReader READER =
+      new ObjectMapper().readerFor(new TypeReference<Set<ReportForJson>>() {});
 
   @Before
   public void setup() {
@@ -220,7 +223,6 @@ public class TestSlowPeerTracker {
       throws IOException {
     final String json = tracker.getJson();
     LOG.info("Got JSON: {}", json);
-    return (new ObjectMapper()).readValue(
-        json, new TypeReference<Set<ReportForJson>>() {});
+    return READER.readValue(json);
   }
 }

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

@@ -152,7 +152,6 @@ public class TestAuditLogger {
     conf.set(DFS_NAMENODE_AUDIT_LOGGERS_KEY,
         DummyAuditLogger.class.getName());
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
-    
     GetOpParam.Op op = GetOpParam.Op.GETFILESTATUS;
     try {
       cluster.waitClusterUp();
@@ -168,7 +167,8 @@ public class TestAuditLogger {
       conn.connect();
       assertEquals(200, conn.getResponseCode());
       conn.disconnect();
-      assertEquals(1, DummyAuditLogger.logCount);
+      assertEquals("getfileinfo", DummyAuditLogger.lastCommand);
+      DummyAuditLogger.resetLogCount();
       assertEquals("127.0.0.1", DummyAuditLogger.remoteAddr);
       
       // non-trusted proxied request
@@ -178,7 +178,9 @@ public class TestAuditLogger {
       conn.connect();
       assertEquals(200, conn.getResponseCode());
       conn.disconnect();
-      assertEquals(2, DummyAuditLogger.logCount);
+      assertEquals("getfileinfo", DummyAuditLogger.lastCommand);
+      assertTrue(DummyAuditLogger.logCount == 1);
+      DummyAuditLogger.resetLogCount();
       assertEquals("127.0.0.1", DummyAuditLogger.remoteAddr);
       
       // trusted proxied request
@@ -190,7 +192,8 @@ public class TestAuditLogger {
       conn.connect();
       assertEquals(200, conn.getResponseCode());
       conn.disconnect();
-      assertEquals(3, DummyAuditLogger.logCount);
+      assertEquals("getfileinfo", DummyAuditLogger.lastCommand);
+      assertTrue(DummyAuditLogger.logCount == 1);
       assertEquals("1.1.1.1", DummyAuditLogger.remoteAddr);
     } finally {
       cluster.shutdown();
@@ -547,6 +550,7 @@ public class TestAuditLogger {
     static int unsuccessfulCount;
     static short foundPermission;
     static String remoteAddr;
+    private static String lastCommand;
     
     public void initialize(Configuration conf) {
       initialized = true;
@@ -565,11 +569,16 @@ public class TestAuditLogger {
       if (!succeeded) {
         unsuccessfulCount++;
       }
+      lastCommand = cmd;
       if (stat != null) {
         foundPermission = stat.getPermission().toShort();
       }
     }
 
+    public static String getLastCommand() {
+      return lastCommand;
+    }
+
   }
 
   public static class BrokenAuditLogger implements AuditLogger {
@@ -581,7 +590,9 @@ public class TestAuditLogger {
     public void logAuditEvent(boolean succeeded, String userName,
         InetAddress addr, String cmd, String src, String dst,
         FileStatus stat) {
-      throw new RuntimeException("uh oh");
+      if (!cmd.equals("datanodeReport")) {
+        throw new RuntimeException("uh oh");
+      }
     }
 
   }

+ 667 - 25
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLoggerWithCommands.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BatchedRemoteIterator;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -31,24 +32,30 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
 import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
 
-import org.junit.BeforeClass;
-import org.junit.AfterClass;
-import org.junit.Ignore;
+import org.junit.After;
+import static org.junit.Assert.assertEquals;
+import org.junit.Before;
 import org.junit.Test;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
-import org.mockito.Mock;
 import org.mockito.Mockito;
 import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.when;
 
 public class TestAuditLoggerWithCommands {
 
@@ -65,13 +72,15 @@ public class TestAuditLoggerWithCommands {
   static UserGroupInformation user2;
   private static NamenodeProtocols proto;
 
-  @BeforeClass
-  public static void initialize() throws Exception {
+  @Before
+  public void initialize() throws Exception {
     // start a cluster
     conf = new HdfsConfiguration();
     conf.setBoolean(DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY, true);
     conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY,true);
     conf.setBoolean(DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY, true);
+    conf.setBoolean(
+        CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION, true);
     cluster =
         new MiniDFSCluster.Builder(conf).numDataNodes(NUM_DATA_NODES).build();
     cluster.waitActive();
@@ -88,8 +97,9 @@ public class TestAuditLoggerWithCommands {
     fs = cluster.getFileSystem();
   }
 
-  @AfterClass
-  public static void tearDown() throws Exception {
+  @After
+  public void tearDown() throws Exception {
+    Server.getCurCall().set(null);
     fs.close();
     fs2.close();
     fileSys.close();
@@ -126,22 +136,29 @@ public class TestAuditLoggerWithCommands {
     Path path = new Path("/testdir/testdir1");
     fs.mkdirs(path);
     fileSys = DFSTestUtil.getFileSystemAs(user1, conf);
-    try {
-      ((DistributedFileSystem)fileSys).setQuota(path, 10l, 10l);
-      fail("The operation should have failed with AccessControlException");
-    } catch (AccessControlException ace) {
-    }
-    String acePattern =
-        ".*allowed=false.*ugi=theDoctor.*cmd=setQuota.*";
-    int length = verifyAuditLogs(acePattern);
+    verifySetQuota(path, HdfsConstants.QUOTA_RESET,
+        HdfsConstants.QUOTA_DONT_SET);
+    verifySetQuota(path, HdfsConstants.QUOTA_DONT_SET,
+        HdfsConstants.QUOTA_RESET);
+    verifySetQuota(path, HdfsConstants.QUOTA_DONT_SET,
+        HdfsConstants.BYTES_IN_INTEGER);
+    verifySetQuota(path, HdfsConstants.BYTES_IN_INTEGER,
+        HdfsConstants.BYTES_IN_INTEGER);
     fileSys.close();
+  }
+
+  private void verifySetQuota(Path path, long nsQuota, long ssQuota)
+      throws IOException {
+    String operationName = cluster.getNamesystem().getQuotaCommand(
+        nsQuota, ssQuota);
+    String acePattern =
+        ".*allowed=false.*ugi=theDoctor.*cmd=.*" + operationName + ".*";
     try {
-      ((DistributedFileSystem)fileSys).setQuota(path, 10l, 10l);
-      fail("The operation should have failed with IOException");
-    } catch (IOException ace) {
+      ((DistributedFileSystem) fileSys).setQuota(path, nsQuota, ssQuota);
+      fail("The operation should have failed");
+    } catch (AccessControlException ace) {
     }
-    assertTrue("Unexpected log from getContentSummary",
-        length == auditlog.getOutput().split("\n").length);
+    verifyAuditLogs(acePattern);
   }
 
   @Test
@@ -180,7 +197,7 @@ public class TestAuditLoggerWithCommands {
         ".*allowed=false.*ugi=theDoctor.*cmd=renameSnapshot.*";
     fs.mkdirs(srcDir);
     fileSys = DFSTestUtil.getFileSystemAs(user1, conf);
-    cluster.getNamesystem().allowSnapshot(srcDir.toString());
+    ((DistributedFileSystem)fs).allowSnapshot(srcDir);
     try {
       fileSys.createSnapshot(srcDir);
       fail("The operation should have failed with AccessControlException");
@@ -215,7 +232,7 @@ public class TestAuditLoggerWithCommands {
     Path s1;
     fs.mkdirs(srcDir);
     fileSys = DFSTestUtil.getFileSystemAs(user1, conf);
-    cluster.getNamesystem().allowSnapshot(srcDir.toString());
+    ((DistributedFileSystem)fs).allowSnapshot(srcDir);
     try {
       s1 = fs.createSnapshot(srcDir);
       fileSys.deleteSnapshot(srcDir, s1.getName());
@@ -236,13 +253,66 @@ public class TestAuditLoggerWithCommands {
         length+1 == auditlog.getOutput().split("\n").length);
   }
 
+  @Test
+  public void testAllowSnapshot() throws Exception {
+    Path srcDir = new Path(System.getProperty("user.dir"), "/src");
+    fs.mkdirs(srcDir);
+    String pattern =
+        ".*allowed=true.*ugi=" +
+            System.getProperty("user.name")+".*cmd=allowSnapshot.*";
+    try {
+      ((DistributedFileSystem)fs).allowSnapshot(srcDir);
+      verifyAuditLogs(pattern);
+    } catch (Exception e) {
+      fail("The operation should not have failed with Exception");
+    }
+    fileSys = DFSTestUtil.getFileSystemAs(user1, conf);
+    try {
+      ((DistributedFileSystem)fileSys).allowSnapshot(srcDir);
+      fail("The operation should have failed with AccessControlException");
+    } catch (AccessControlException ace) {
+    }
+    pattern =
+        ".*allowed=false.*ugi=theDoctor.*cmd=allowSnapshot.*";
+    verifyAuditLogs(pattern);
+    fs.delete(srcDir, true);
+    fileSys.close();
+  }
+
+  @Test
+  public void testDisallowSnapshot() throws Exception {
+    Path srcDir = new Path(System.getProperty("user.dir"), "/src");
+    fs.mkdirs(srcDir);
+    cluster.getNamesystem().allowSnapshot(srcDir.toString());
+    String pattern =
+        ".*allowed=true.*ugi=" +
+            System.getProperty("user.name")+".*cmd=disallowSnapshot.*";
+    try {
+      ((DistributedFileSystem)fs).disallowSnapshot(srcDir);
+      verifyAuditLogs(pattern);
+    } catch (Exception e) {
+      fail("The operation should not have failed with Exception");
+    }
+    cluster.getNamesystem().allowSnapshot(srcDir.toString());
+    fileSys = DFSTestUtil.getFileSystemAs(user1, conf);
+    try {
+      ((DistributedFileSystem)fileSys).disallowSnapshot(srcDir);
+      fail("The operation should have failed with AccessControlException");
+    } catch (AccessControlException ace) {
+      pattern =
+          ".*allowed=false.*ugi=theDoctor.*cmd=disallowSnapshot.*";
+      verifyAuditLogs(pattern);
+    }
+    fileSys.close();
+  }
+
   @Test
   public void testAddCacheDirective() throws Exception {
     removeExistingCachePools(null);
     proto.addCachePool(new CachePoolInfo("pool1").
         setMode(new FsPermission((short) 0)));
     CacheDirectiveInfo alpha = new CacheDirectiveInfo.Builder().
-        setPath(new Path("/alpha")).
+        setPath(new Path(System.getProperty("user.dir"), "/alpha")).
         setPool("pool1").
         build();
     fileSys = DFSTestUtil.getFileSystemAs(user1, conf);
@@ -618,6 +688,579 @@ public class TestAuditLoggerWithCommands {
     return verifyAuditLogs(".*allowed=" + allowed + pattern);
   }
 
+  @Test
+  public void testMetaSave() throws Exception {
+    String aceMetaSave =
+        ".*allowed=true.*cmd=metaSave.*";
+    try {
+      ((DistributedFileSystem)fs).metaSave("test.log");
+      verifyAuditLogs(aceMetaSave);
+    } catch (Exception e) {
+      fail("The operation should not have failed with Exception");
+    }
+    try {
+      ((DistributedFileSystem)fileSys).metaSave("test.log");
+      fail("The operation should have failed with AccessControlException");
+    } catch (IOException ace) {
+      GenericTestUtils.assertExceptionContains("Access denied", ace);
+      aceMetaSave =
+          ".*allowed=false.*cmd=metaSave.*";
+      verifyAuditLogs(aceMetaSave);
+    }
+  }
+
+  @Test
+  public void testStartReconfiguration() throws Exception {
+    String auditLogString =
+        ".*allowed=true.*cmd=startNamenodeReconfiguration.*";
+    FSNamesystem fsNamesystem = spy(cluster.getNamesystem());
+    when(fsNamesystem.isExternalInvocation()).thenReturn(true);
+    Server.Call call = spy(new Server.Call(
+        1, 1, null, null, RPC.RpcKind.RPC_BUILTIN, new byte[] {1, 2, 3}));
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser(System.getProperty("user.name")));
+    Server.getCurCall().set(call);
+    try {
+      ((NameNodeRpcServer) cluster.getNameNodeRpc()).startReconfiguration();
+      verifyAuditLogs(auditLogString);
+    } catch (Exception e) {
+      fail("StartConfiguration should have passed!");
+    }
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser("theDoctor"));
+    try {
+      ((NameNodeRpcServer)cluster.getNameNodeRpc()).startReconfiguration();
+      fail(
+          "startNameNodeReconfiguration should throw AccessControlException!");
+    } catch (AccessControlException ace) {
+      auditLogString =
+          ".*allowed=false.*cmd=startNamenodeReconfiguration.*";
+      verifyAuditLogs(auditLogString);
+    }
+  }
+
+  @Test
+  public void testGetReconfigurationStatus() throws Exception {
+    String auditLogString =
+        ".*allowed=true.*cmd=getNamenodeReconfigurationStatus.*";
+    FSNamesystem fsNamesystem = spy(cluster.getNamesystem());
+    when(fsNamesystem.isExternalInvocation()).thenReturn(true);
+    Server.Call call = spy(new Server.Call(
+        1, 1, null, null, RPC.RpcKind.RPC_BUILTIN, new byte[] {1, 2, 3}));
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser(System.getProperty("user.name")));
+    Server.getCurCall().set(call);
+    try {
+      ((NameNodeRpcServer) cluster.getNameNodeRpc()).getReconfigurationStatus();
+      verifyAuditLogs(auditLogString);
+    } catch (Exception e) {
+      fail("getNamenodeReconfigurationStatus " +
+          " threw Exception!");
+    }
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser("theDoctor"));
+    try {
+      ((NameNodeRpcServer)cluster.getNameNodeRpc()).getReconfigurationStatus();
+      fail("getNamenodeReconfigurationStatus " +
+          " did not throw AccessControlException!");
+    } catch (AccessControlException ace) {
+      auditLogString =
+          ".*allowed=false.*cmd=getNamenodeReconfigurationStatus.*";
+      verifyAuditLogs(auditLogString);
+    }
+  }
+
+  @Test
+  public void testListReconfigurableProperties() throws Exception {
+    String auditLogString =
+        ".*allowed=true.*cmd=listNamenodeReconfigurableProperties.*";
+    FSNamesystem fsNamesystem = spy(cluster.getNamesystem());
+    when(fsNamesystem.isExternalInvocation()).thenReturn(true);
+    Server.Call call = spy(new Server.Call(
+        1, 1, null, null, RPC.RpcKind.RPC_BUILTIN, new byte[] {1, 2, 3}));
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser(System.getProperty("user.name")));
+    Server.getCurCall().set(call);
+    try {
+      ((NameNodeRpcServer) cluster.getNameNodeRpc()).
+          listReconfigurableProperties();
+      verifyAuditLogs(auditLogString);
+    } catch (Exception e) {
+      fail("listReconfigurableProperties " +
+          " threw Exception!");
+    }
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser("theDoctor"));
+    try {
+      ((NameNodeRpcServer)cluster.getNameNodeRpc()).
+          listReconfigurableProperties();
+      fail("getNamenodeReconfigurationStatus " +
+          " did not throw AccessControlException!");
+    } catch (AccessControlException ace) {
+      auditLogString =
+          ".*allowed=false.*cmd=listNamenodeReconfigurableProperties.*";
+      verifyAuditLogs(auditLogString);
+    }
+  }
+
+  @Test
+  public void testRefreshUserToGroupsMappings() throws Exception {
+    String auditLogString =
+        ".*allowed=true.*cmd=refreshUserToGroupsMappings.*";
+    FSNamesystem fsNamesystem = spy(cluster.getNamesystem());
+    when(fsNamesystem.isExternalInvocation()).thenReturn(true);
+    Server.Call call = spy(new Server.Call(
+        1, 1, null, null, RPC.RpcKind.RPC_BUILTIN, new byte[] {1, 2, 3}));
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser(System.getProperty("user.name")));
+    Server.getCurCall().set(call);
+    ((NameNodeRpcServer)cluster.getNameNodeRpc()).
+        refreshUserToGroupsMappings();
+    verifyAuditLogs(auditLogString);
+  }
+
+  @Test
+  public void testRefreshSuperUserGroupsConfiguration() throws Exception {
+    String auditLogString =
+        ".*allowed=true.*cmd=refreshSuperUserGroupsConfiguration.*";
+    FSNamesystem fsNamesystem = spy(cluster.getNamesystem());
+    when(fsNamesystem.isExternalInvocation()).thenReturn(true);
+    Server.Call call = spy(new Server.Call(
+        1, 1, null, null, RPC.RpcKind.RPC_BUILTIN, new byte[] {1, 2, 3}));
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser(System.getProperty("user.name")));
+    Server.getCurCall().set(call);
+    try {
+      ((NameNodeRpcServer) cluster.getNameNodeRpc()).
+          refreshSuperUserGroupsConfiguration();
+      verifyAuditLogs(auditLogString);
+    } catch (Exception e) {
+      fail(" The operation threw an exception");
+    }
+  }
+
+  @Test
+  public void testRefreshQueue() throws Exception {
+    String auditLogString =
+        ".*allowed=true.*cmd=refreshCallQueue.*";
+    FSNamesystem fsNamesystem = spy(cluster.getNamesystem());
+    when(fsNamesystem.isExternalInvocation()).thenReturn(true);
+    Server.Call call = spy(new Server.Call(
+        1, 1, null, null, RPC.RpcKind.RPC_BUILTIN, new byte[] {1, 2, 3}));
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser(System.getProperty("user.name")));
+    Server.getCurCall().set(call);
+    try {
+      ((NameNodeRpcServer) cluster.getNameNodeRpc()).refreshCallQueue();
+      verifyAuditLogs(auditLogString);
+    } catch (Exception e) {
+      fail(" The operation threw an exception");
+    }
+  }
+
+  @Test
+  public void testRefreshServiceAcl() throws Exception {
+    String auditLogString =
+        ".*allowed=true.*cmd=refreshServiceAcl.*";
+    FSNamesystem fsNamesystem = spy(cluster.getNamesystem());
+    when(fsNamesystem.isExternalInvocation()).thenReturn(true);
+    Server.Call call = spy(new Server.Call(
+        1, 1, null, null, RPC.RpcKind.RPC_BUILTIN, new byte[] {1, 2, 3}));
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser(System.getProperty("user.name")));
+    Server.getCurCall().set(call);
+    try {
+      ((NameNodeRpcServer) cluster.getNameNodeRpc()).refreshServiceAcl();
+      verifyAuditLogs(auditLogString);
+    } catch (Exception e) {
+      fail(" The operation threw an exception" + e);
+    }
+  }
+
+  @Test
+  public void testFinalizeRollingUpgrade() throws Exception {
+    String auditLogString =
+        ".*allowed=true.*cmd=finalizeRollingUpgrade.*";
+    FSNamesystem fsNamesystem = spy(cluster.getNamesystem());
+    when(fsNamesystem.isExternalInvocation()).thenReturn(true);
+    Server.Call call = spy(new Server.Call(
+        1, 1, null, null, RPC.RpcKind.RPC_BUILTIN, new byte[] {1, 2, 3}));
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser(System.getProperty("user.name")));
+    Server.getCurCall().set(call);
+    fsNamesystem.setRollingUpgradeInfo(false, System.currentTimeMillis());
+    try {
+      fsNamesystem.finalizeRollingUpgrade();
+      verifyAuditLogs(auditLogString);
+    } catch (Exception e) {
+      fail("finalizeRollingUpgrade threw Exception");
+    }
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser("theDoctor"));
+    try {
+      fsNamesystem.finalizeRollingUpgrade();
+      fail("finalizeRollingUpgrade should throw AccessControlException!");
+    } catch (AccessControlException ace) {
+      auditLogString =
+          ".*allowed=false.*cmd=finalizeRollingUpgrade.*";
+      verifyAuditLogs(auditLogString);
+    }
+  }
+
+  @Test
+  public void testQueryRollingUpgrade() throws Exception {
+    String auditLogString =
+        ".*allowed=true.*cmd=queryRollingUpgrade.*";
+    FSNamesystem fsNamesystem = spy(cluster.getNamesystem());
+    when(fsNamesystem.isExternalInvocation()).thenReturn(true);
+    Server.Call call = spy(new Server.Call(
+        1, 1, null, null, RPC.RpcKind.RPC_BUILTIN, new byte[] {1, 2, 3}));
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser(System.getProperty("user.name")));
+    Server.getCurCall().set(call);
+    fsNamesystem.setRollingUpgradeInfo(false, System.currentTimeMillis());
+    try {
+      fsNamesystem.queryRollingUpgrade();
+      verifyAuditLogs(auditLogString);
+    } catch (Exception e) {
+      fail("queryRollingUpgrade threw Exception");
+    }
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser("theDoctor"));
+    try {
+      fsNamesystem.queryRollingUpgrade();
+      fail("queryRollingUpgrade should have thrown an AccessControlException!");
+    } catch (AccessControlException ace) {
+      auditLogString =
+          ".*allowed=false.*cmd=queryRollingUpgrade.*";
+      verifyAuditLogs(auditLogString);
+    }
+  }
+
+  @Test
+  public void testRollEditLog() throws Exception {
+    String auditLogString =
+        ".*allowed=true.*cmd=rollEditLog.*";
+    FSNamesystem fsNamesystem = spy(cluster.getNamesystem());
+    when(fsNamesystem.isExternalInvocation()).thenReturn(true);
+    Server.Call call = spy(new Server.Call(
+        1, 1, null, null, RPC.RpcKind.RPC_BUILTIN, new byte[] {1, 2, 3}));
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser(System.getProperty("user.name")));
+    Server.getCurCall().set(call);
+    try {
+      fsNamesystem.rollEditLog();
+    } catch (Exception e) {
+      fail("rollEditLog threw Exception");
+    }
+    verifyAuditLogs(auditLogString);
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser("theDoctor"));
+    try {
+      fsNamesystem.rollEditLog();
+      fail("rollEditLog should have thrown an AccessControlException!");
+    } catch (AccessControlException ace) {
+      auditLogString =
+          ".*allowed=false.*cmd=rollEditLog.*";
+      verifyAuditLogs(auditLogString);
+    }
+  }
+
+  @Test
+  public void testSetSafeMode() throws Exception {
+    FSNamesystem fsNamesystem = spy(cluster.getNamesystem());
+    when(fsNamesystem.isExternalInvocation()).thenReturn(true);
+    Server.Call call = spy(new Server.Call(
+        1, 1, null, null, RPC.RpcKind.RPC_BUILTIN, new byte[] {1, 2, 3}));
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser(System.getProperty("user.name")));
+    Server.getCurCall().set(call);
+    verifySuccessfulSetSafeMode(fsNamesystem,
+        HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
+    verifySuccessfulSetSafeMode(fsNamesystem,
+        HdfsConstants.SafeModeAction.SAFEMODE_GET);
+    verifySuccessfulSetSafeMode(fsNamesystem,
+        HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
+    verifySuccessfulSetSafeMode(fsNamesystem,
+        HdfsConstants.SafeModeAction.SAFEMODE_FORCE_EXIT);
+    String auditLogString;
+    auditLogString =
+        ".*allowed=true.*cmd=safemode_get.*";
+    fsNamesystem.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_GET);
+    verifyAuditLogs(auditLogString);
+    auditLogString =
+        ".*allowed=true.*cmd=safemode_leave.*";
+    fsNamesystem.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
+    verifyAuditLogs(auditLogString);
+    auditLogString =
+        ".*allowed=true.*cmd=safemode_force_exit.*";
+    fsNamesystem.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_FORCE_EXIT);
+    verifyAuditLogs(auditLogString);
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser("theDoctor"));
+    verifySafeModeAction(fsNamesystem,
+        HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
+    verifySafeModeAction(fsNamesystem,
+        HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
+    verifySafeModeAction(fsNamesystem,
+        HdfsConstants.SafeModeAction.SAFEMODE_FORCE_EXIT);
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser(System.getProperty("user.name")));
+  }
+
+  @Test
+  public void testSetBalancerBandwidth() throws Exception {
+    String auditLogString =
+        ".*allowed=true.*cmd=setBalancerBandwidth.*";
+    FSNamesystem fsNamesystem = spy(cluster.getNamesystem());
+    when(fsNamesystem.isExternalInvocation()).thenReturn(true);
+    Server.Call call = spy(new Server.Call(
+        1, 1, null, null, RPC.RpcKind.RPC_BUILTIN, new byte[] {1, 2, 3}));
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser(System.getProperty("user.name")));
+    Server.getCurCall().set(call);
+    try {
+      fsNamesystem.setBalancerBandwidth(10);
+      verifyAuditLogs(auditLogString);
+    } catch (Exception e) {
+      fail("setBalancerBandwidth threw exception!");
+    }
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser("theDoctor"));
+    try {
+      fsNamesystem.setBalancerBandwidth(10);
+      fail(
+          "setBalancerBandwidth should have thrown AccessControlException!");
+    } catch (AccessControlException ace) {
+      auditLogString =
+          ".*allowed=false.*cmd=setBalancerBandwidth.*";
+      verifyAuditLogs(auditLogString);
+    }
+  }
+
+  @Test
+  public void testRefreshNodes() throws Exception {
+    String auditLogString =
+        ".*allowed=true.*cmd=refreshNodes.*";
+    FSNamesystem fsNamesystem = spy(cluster.getNamesystem());
+    when(fsNamesystem.isExternalInvocation()).thenReturn(true);
+    Server.Call call = spy(new Server.Call(
+        1, 1, null, null, RPC.RpcKind.RPC_BUILTIN, new byte[] {1, 2, 3}));
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser(System.getProperty("user.name")));
+    Server.getCurCall().set(call);
+    try {
+      fsNamesystem.refreshNodes();
+      verifyAuditLogs(auditLogString);
+    } catch (Exception e) {
+      fail("refreshNodes threw exception!");
+    }
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser("theDoctor"));
+    try {
+      fsNamesystem.refreshNodes();
+      fail(
+          "refreshNodes should have thrown an AccessControlException!");
+    } catch (AccessControlException ace) {
+      auditLogString =
+          ".*allowed=false.*cmd=refreshNodes.*";
+      verifyAuditLogs(auditLogString);
+    }
+  }
+
+  @Test
+  public void testFinalizeUpgrade() throws Exception {
+    String auditLogString =
+        ".*allowed=true.*cmd=finalizeUpgrade.*";
+    FSNamesystem fsNamesystem = spy(cluster.getNamesystem());
+    when(fsNamesystem.isExternalInvocation()).thenReturn(true);
+    Server.Call call = spy(new Server.Call(
+        1, 1, null, null, RPC.RpcKind.RPC_BUILTIN, new byte[] {1, 2, 3}));
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser(System.getProperty("user.name")));
+    Server.getCurCall().set(call);
+    try {
+      fsNamesystem.finalizeUpgrade();
+      verifyAuditLogs(auditLogString);
+    } catch (Exception e) {
+      fail("finalizeUpgrade threw Exception");
+    }
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser("theDoctor"));
+    try {
+      fsNamesystem.finalizeUpgrade();
+      fail("finalizeUpgrade should have thrown an AccessControlException!");
+    } catch (AccessControlException ace) {
+      auditLogString =
+          ".*allowed=false.*cmd=finalizeUpgrade.*";
+      verifyAuditLogs(auditLogString);
+    }
+  }
+
+  @Test
+  public void testSaveNamespace() throws Exception {
+    String auditLogString =
+        ".*allowed=true.*cmd=saveNamespace.*";
+    FSNamesystem fsNamesystem = spy(cluster.getNamesystem());
+    when(fsNamesystem.isExternalInvocation()).thenReturn(true);
+    Server.Call call = spy(new Server.Call(
+        1, 1, null, null, RPC.RpcKind.RPC_BUILTIN, new byte[] {1, 2, 3}));
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser(System.getProperty("user.name")));
+    Server.getCurCall().set(call);
+    fsNamesystem.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
+    try {
+      fsNamesystem.saveNamespace(10, 10);
+      verifyAuditLogs(auditLogString);
+    } catch (Exception e) {
+      fail("saveNamespace threw Exception");
+    }
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser("theDoctor"));
+    try {
+      fsNamesystem.saveNamespace(10, 10);
+      fail("saveNamespace should have thrown an AccessControlException!");
+    } catch (AccessControlException ace) {
+      auditLogString =
+          ".*allowed=false.*cmd=saveNamespace.*";
+      verifyAuditLogs(auditLogString);
+    }
+  }
+
+  @Test
+  public void testDatanodeReport() throws Exception {
+    String auditLogString =
+        ".*allowed=true.*cmd=datanodeReport.*";
+    FSNamesystem fsNamesystem = spy(cluster.getNamesystem());
+    when(fsNamesystem.isExternalInvocation()).thenReturn(true);
+    Server.Call call = spy(new Server.Call(
+        1, 1, null, null, RPC.RpcKind.RPC_BUILTIN, new byte[] {1, 2, 3}));
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser(System.getProperty("user.name")));
+    Server.getCurCall().set(call);
+    try {
+      fsNamesystem.datanodeReport(HdfsConstants.DatanodeReportType.ALL);
+      verifyAuditLogs(auditLogString);
+    } catch (Exception e) {
+      fail("datanodeReport threw Exception");
+    }
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser("theDoctor"));
+    try {
+      fsNamesystem.datanodeReport(HdfsConstants.DatanodeReportType.ALL);
+      fail(
+          "datanodeReport should have thrown an AccessControlException!");
+    } catch (AccessControlException ace) {
+      auditLogString =
+          ".*allowed=false.*cmd=datanodeReport.*";
+      verifyAuditLogs(auditLogString);
+    }
+  }
+
+  @Test
+  public void testRestoreFailedStorage() throws Exception {
+    FSNamesystem fsNamesystem = spy(cluster.getNamesystem());
+    when(fsNamesystem.isExternalInvocation()).thenReturn(true);
+    Server.Call call = spy(new Server.Call(
+        1, 1, null, null, RPC.RpcKind.RPC_BUILTIN, new byte[] {1, 2, 3}));
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser(System.getProperty("user.name")));
+    Server.getCurCall().set(call);
+    verifyAuditRestoreFailedStorage(fsNamesystem, "check");
+    verifyAuditRestoreFailedStorage(fsNamesystem, "true");
+    verifyAuditRestoreFailedStorage(fsNamesystem, "false");
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser("theDoctor"));
+    verifyAuditRestoreFailedStorageACE(fsNamesystem, "check");
+    verifyAuditRestoreFailedStorageACE(fsNamesystem, "true");
+    verifyAuditRestoreFailedStorageACE(fsNamesystem, "false");
+  }
+
+  @Test
+  public void testGetDatanodeStorageReport() throws Exception {
+    FSNamesystem fsNamesystem = spy(cluster.getNamesystem());
+    when(fsNamesystem.isExternalInvocation()).thenReturn(true);
+    Server.Call call = spy(new Server.Call(
+        1, 1, null, null, RPC.RpcKind.RPC_BUILTIN, new byte[] {1, 2, 3}));
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser(System.getProperty("user.name")));
+    Server.getCurCall().set(call);
+    DatanodeStorageReport[] reports  = fsNamesystem.getDatanodeStorageReport(
+        HdfsConstants.DatanodeReportType.ALL);
+    String auditLogString =
+        ".*allowed=true.*cmd=" + "getDatanodeStorageReport" + ".*";
+    verifyAuditLogs(auditLogString);
+    when(call.getRemoteUser()).thenReturn(
+        UserGroupInformation.createRemoteUser("theDoctor"));
+    auditLogString =
+        ".*allowed=false.*cmd=" + "getDatanodeStorageReport" + ".*";
+    try {
+      fsNamesystem.getDatanodeStorageReport(
+          HdfsConstants.DatanodeReportType.ALL);
+      fail("Should have thrown an AccessControlException!");
+    } catch (AccessControlException ace) {
+      verifyAuditLogs(auditLogString);
+    }
+  }
+
+  private void verifyAuditRestoreFailedStorageACE(
+      FSNamesystem fsNamesystem, String arg) throws IOException {
+    String operationName = fsNamesystem.getFailedStorageCommand(arg);
+    try {
+      fsNamesystem.restoreFailedStorage(arg);
+      fail(
+          "RestoreFailedStorage should have thrown AccessControlException!");
+    } catch (IOException ace) {
+      assertEquals("Unexpected Exception!",
+          ace.getClass(), AccessControlException.class);
+      String auditLogString =
+          ".*allowed=false.*cmd=" + operationName + ".*";
+      verifyAuditLogs(auditLogString);
+    }
+  }
+
+  private void verifyAuditRestoreFailedStorage(
+      FSNamesystem fsNamesystem, String arg) throws IOException {
+    String operationName = fsNamesystem.getFailedStorageCommand(arg);
+    String auditLogString =
+        ".*allowed=true.*cmd=" + operationName + ".*";
+    try {
+      fsNamesystem.restoreFailedStorage(arg);
+      verifyAuditLogs(auditLogString);
+    } catch (Exception e) {
+      fail(
+          "The operation should not have failed with Exception");
+    }
+  }
+
+  private void verifySuccessfulSetSafeMode(FSNamesystem fsNamesystem,
+      HdfsConstants.SafeModeAction safeModeAction) throws IOException {
+    String operationName = safeModeAction.toString().toLowerCase();
+    String auditLogString =
+        ".*allowed=true.*cmd=" + operationName +".*";
+    try {
+      fsNamesystem.setSafeMode(safeModeAction);
+      verifyAuditLogs(auditLogString);
+    } catch (Exception e) {
+      fail("The operation should not have failed with Exception");
+    }
+  }
+
+  private void verifySafeModeAction(
+      FSNamesystem fsNamesystem, HdfsConstants.SafeModeAction safeModeAction)
+      throws IOException {
+    String operationName = safeModeAction.toString().toLowerCase();
+    String auditLogString;
+    try {
+      fsNamesystem.setSafeMode(safeModeAction);
+      fail("setSafeMode should have thrown an AccessControlException!");
+    } catch (AccessControlException ace) {
+      auditLogString =
+          ".*allowed=false.*cmd=" + operationName +".*";
+      verifyAuditLogs(auditLogString);
+    }
+  }
+
   private int verifyAuditLogs(String pattern) {
     int length = auditlog.getOutput().split("\n").length;
     String lastAudit = auditlog.getOutput().split("\n")[length - 1];
@@ -633,4 +1276,3 @@ public class TestAuditLoggerWithCommands {
     }
   }
 }
-

+ 46 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystemLock.java

@@ -38,6 +38,7 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import java.util.regex.Pattern;
+import org.slf4j.LoggerFactory;
 
 import static org.junit.Assert.*;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_FSLOCK_FAIR_KEY;
@@ -347,7 +348,7 @@ public class TestFSNamesystemLock {
 
     fsLock.writeLock();
     timer.advance(1);
-    fsLock.writeUnlock("baz");
+    fsLock.writeUnlock("baz", false);
 
     MetricsRecordBuilder rb = MetricsAsserts.mockMetricsRecordBuilder();
     rates.snapshot(rb, true);
@@ -360,4 +361,48 @@ public class TestFSNamesystemLock {
     assertCounter("FSNWriteLockBazNanosNumOps", 1L, rb);
   }
 
+  /**
+   * Test to suppress FSNameSystem write lock report when it is held for long
+   * time.
+   */
+  @Test(timeout = 45000)
+  public void testFSWriteLockReportSuppressed() throws Exception {
+    final long writeLockReportingThreshold = 1L;
+    final long writeLockSuppressWarningInterval = 10L;
+    Configuration conf = new Configuration();
+    conf.setLong(
+        DFSConfigKeys.DFS_NAMENODE_WRITE_LOCK_REPORTING_THRESHOLD_MS_KEY,
+        writeLockReportingThreshold);
+    conf.setTimeDuration(DFSConfigKeys.DFS_LOCK_SUPPRESS_WARNING_INTERVAL_KEY,
+        writeLockSuppressWarningInterval, TimeUnit.MILLISECONDS);
+
+    final FakeTimer timer = new FakeTimer();
+    final FSNamesystemLock fsnLock = new FSNamesystemLock(conf, null, timer);
+    timer.advance(writeLockSuppressWarningInterval);
+
+    LogCapturer logs = LogCapturer.captureLogs(FSNamesystem.LOG);
+    GenericTestUtils
+        .setLogLevel(LoggerFactory.getLogger(FSNamesystem.class.getName()),
+            org.slf4j.event.Level.INFO);
+
+    // Should trigger the write lock report
+    fsnLock.writeLock();
+    timer.advance(writeLockReportingThreshold + 100);
+    fsnLock.writeUnlock();
+    assertTrue(logs.getOutput().contains(
+        "FSNamesystem write lock held for"));
+
+    logs.clearOutput();
+
+    // Suppress report if the write lock is held for a long time
+    fsnLock.writeLock();
+    timer.advance(writeLockReportingThreshold + 100);
+    fsnLock.writeUnlock("testFSWriteLockReportSuppressed", true);
+    assertFalse(logs.getOutput().contains(GenericTestUtils.getMethodName()));
+    assertFalse(logs.getOutput().contains(
+        "Number of suppressed write-lock reports:"));
+    assertFalse(logs.getOutput().contains(
+        "FSNamesystem write lock held for"));
+  }
+
 }

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

@@ -556,13 +556,18 @@ public class TestNameNodeMXBean {
       assertEquals("Unexpected num windows", 3, windows.size());
       for (Map<String, List<Map<String, Object>>> window : windows) {
         final List<Map<String, Object>> ops = window.get("ops");
-        assertEquals("Unexpected num ops", 3, ops.size());
+        assertEquals("Unexpected num ops", 4, ops.size());
         for (Map<String, Object> op: ops) {
+          if (op.get("opType").equals("datanodeReport")) {
+            continue;
+          }
           final long count = Long.parseLong(op.get("totalCount").toString());
           final String opType = op.get("opType").toString();
           final int expected;
           if (opType.equals(TopConf.ALL_CMDS)) {
-            expected = 2*NUM_OPS;
+            expected = 2 * NUM_OPS + 2;
+          } else if (opType.equals("datanodeReport")) {
+            expected = 2;
           } else {
             expected = NUM_OPS;
           }

+ 169 - 6
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestGetConf.java

@@ -23,7 +23,10 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_BACKUP_ADDRESS_K
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMESERVICES;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX;
+
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
@@ -33,10 +36,14 @@ import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.PrintStream;
 import java.net.InetSocketAddress;
+import java.net.URISyntaxException;
+import java.net.UnknownHostException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
+import java.util.HashSet;
+import java.util.Set;
 import java.util.StringTokenizer;
 
 import org.apache.hadoop.fs.FileSystem;
@@ -58,7 +65,7 @@ import com.google.common.base.Joiner;
  */
 public class TestGetConf {
   enum TestType {
-    NAMENODE, BACKUP, SECONDARY, NNRPCADDRESSES
+    NAMENODE, BACKUP, SECONDARY, NNRPCADDRESSES, JOURNALNODE
   }
   FileSystem localFileSys; 
   /** Setup federation nameServiceIds in the configuration */
@@ -96,9 +103,10 @@ public class TestGetConf {
    * Add namenodes to the static resolution list to avoid going
    * through DNS which can be really slow in some configurations.
    */
-  private void setupStaticHostResolution(int nameServiceIdCount) {
+  private void setupStaticHostResolution(int nameServiceIdCount,
+                                         String hostname) {
     for (int i = 0; i < nameServiceIdCount; i++) {
-      NetUtils.addStaticResolution("nn" + i, "localhost");
+      NetUtils.addStaticResolution(hostname + i, "localhost");
     }
   }
 
@@ -173,6 +181,8 @@ public class TestGetConf {
     case NNRPCADDRESSES:
       args[0] = Command.NNRPCADDRESSES.getName();
       break;
+    case JOURNALNODE:
+      args[0] = Command.JOURNALNODE.getName();
     }
     return runTool(conf, args, success);
   }
@@ -321,7 +331,7 @@ public class TestGetConf {
     String[] nnAddresses = setupAddress(conf,
         DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY, nsCount, 1000);
     setupAddress(conf, DFS_NAMENODE_RPC_ADDRESS_KEY, nsCount, 1500);
-    setupStaticHostResolution(nsCount);
+    setupStaticHostResolution(nsCount, "nn");
     String[] backupAddresses = setupAddress(conf,
         DFS_NAMENODE_BACKUP_ADDRESS_KEY, nsCount, 2000);
     String[] secondaryAddresses = setupAddress(conf,
@@ -348,7 +358,160 @@ public class TestGetConf {
     verifyAddresses(conf, TestType.SECONDARY, false, secondaryAddresses);
     verifyAddresses(conf, TestType.NNRPCADDRESSES, true, nnAddresses);
   }
-  
+
+  /**
+   * Tests for journal node addresses.
+   * @throws Exception
+   */
+  @Test(timeout=10000)
+  public void testGetJournalNodes() throws Exception {
+
+    final int nsCount = 3;
+    final String journalsBaseUri = "qjournal://jn0:9820;jn1:9820;jn2:9820";
+    setupStaticHostResolution(nsCount, "jn");
+
+    // With out Name service Id
+    HdfsConfiguration conf = new HdfsConfiguration(false);
+    conf.set(DFS_NAMENODE_SHARED_EDITS_DIR_KEY,
+        journalsBaseUri+"/");
+
+    Set<String> expected = new HashSet<>();
+    expected.add("jn0");
+    expected.add("jn1");
+    expected.add("jn2");
+
+    String expected1 = "";
+    StringBuilder buffer = new StringBuilder();
+    for (String val : expected) {
+      if (buffer.length() > 0) {
+        buffer.append(" ");
+      }
+      buffer.append(val);
+    }
+    buffer.append("\n");
+    expected1 = buffer.toString();
+
+    Set<String> actual = DFSUtil.getJournalNodeAddresses(conf);
+    assertEquals(expected.toString(), actual.toString());
+
+    String actual1 = getAddressListFromTool(TestType.JOURNALNODE,
+        conf, true);
+    assertEquals(expected1, actual1);
+    conf.clear();
+
+    //With out Name service Id
+    conf.set(DFS_NAMENODE_SHARED_EDITS_DIR_KEY,
+        journalsBaseUri + "/");
+
+    actual = DFSUtil.getJournalNodeAddresses(conf);
+    assertEquals(expected.toString(), actual.toString());
+
+    actual1 = getAddressListFromTool(TestType.JOURNALNODE,
+        conf, true);
+    assertEquals(expected1, actual1);
+    conf.clear();
+
+
+    //Federation with HA, but suffixed only with Name service Id
+    setupNameServices(conf, nsCount);
+    conf.set(DFS_HA_NAMENODES_KEY_PREFIX +".ns0",
+        "nn0,nn1");
+    conf.set(DFS_HA_NAMENODES_KEY_PREFIX +".ns1",
+        "nn0, nn1");
+    conf.set(DFS_NAMENODE_SHARED_EDITS_DIR_KEY+".ns0",
+        journalsBaseUri + "/ns0");
+    conf.set(DFS_NAMENODE_SHARED_EDITS_DIR_KEY+".ns1",
+        journalsBaseUri + "/ns1");
+
+    actual = DFSUtil.getJournalNodeAddresses(conf);
+    assertEquals(expected.toString(), actual.toString());
+
+    expected1 = getAddressListFromTool(TestType.JOURNALNODE,
+        conf, true);
+    assertEquals(expected1, actual1);
+
+    conf.clear();
+
+
+    // Federation with HA
+    setupNameServices(conf, nsCount);
+    conf.set(DFS_HA_NAMENODES_KEY_PREFIX + ".ns0", "nn0,nn1");
+    conf.set(DFS_HA_NAMENODES_KEY_PREFIX + ".ns1", "nn0, nn1");
+    conf.set(DFS_NAMENODE_SHARED_EDITS_DIR_KEY + ".ns0.nn0",
+        journalsBaseUri + "/ns0");
+    conf.set(DFS_NAMENODE_SHARED_EDITS_DIR_KEY + ".ns0.nn1",
+        journalsBaseUri + "/ns0");
+    conf.set(DFS_NAMENODE_SHARED_EDITS_DIR_KEY + ".ns1.nn2",
+        journalsBaseUri + "/ns1");
+    conf.set(DFS_NAMENODE_SHARED_EDITS_DIR_KEY + ".ns1.nn3",
+        journalsBaseUri + "/ns1");
+
+    actual = DFSUtil.getJournalNodeAddresses(conf);
+    assertEquals(expected.toString(), actual.toString());
+
+    actual1 = getAddressListFromTool(TestType.JOURNALNODE,
+        conf, true);
+    assertEquals(expected1, actual1);
+
+    conf.clear();
+
+    // Name service setup, but no journal node
+    setupNameServices(conf, nsCount);
+
+    expected = new HashSet<>();
+    actual = DFSUtil.getJournalNodeAddresses(conf);
+    assertEquals(expected.toString(), actual.toString());
+
+    actual1 = "\n";
+    expected1 = getAddressListFromTool(TestType.JOURNALNODE,
+        conf, true);
+    assertEquals(expected1, actual1);
+    conf.clear();
+
+    //name node edits dir is present, but set
+    //to location of storage shared directory
+    conf.set(DFS_NAMENODE_SHARED_EDITS_DIR_KEY,
+        "file:///mnt/filer1/dfs/ha-name-dir-shared");
+
+    expected = new HashSet<>();
+    actual = DFSUtil.getJournalNodeAddresses(conf);
+    assertEquals(expected.toString(), actual.toString());
+
+    expected1 = getAddressListFromTool(TestType.JOURNALNODE,
+        conf, true);
+    actual1 = "\n";
+    assertEquals(expected1, actual1);
+    conf.clear();
+  }
+
+  /*
+   ** Test for unknown journal node host exception.
+  */
+  @Test(expected = UnknownHostException.class, timeout = 10000)
+  public void testUnknownJournalNodeHost()
+      throws URISyntaxException, IOException {
+    String journalsBaseUri = "qjournal://jn1:9820;jn2:9820;jn3:9820";
+    HdfsConfiguration conf = new HdfsConfiguration(false);
+    conf.set(DFS_NAMENODE_SHARED_EDITS_DIR_KEY,
+        journalsBaseUri + "/jndata");
+    DFSUtil.getJournalNodeAddresses(conf);
+  }
+
+  /*
+   ** Test for malformed journal node urisyntax exception.
+  */
+  @Test(expected = URISyntaxException.class, timeout = 10000)
+  public void testJournalNodeUriError()
+      throws URISyntaxException, IOException {
+    final int nsCount = 3;
+    String journalsBaseUri = "qjournal://jn0 :9820;jn1:9820;jn2:9820";
+    setupStaticHostResolution(nsCount, "jn");
+    HdfsConfiguration conf = new HdfsConfiguration(false);
+    conf.set(DFS_NAMENODE_SHARED_EDITS_DIR_KEY,
+        journalsBaseUri + "/jndata");
+    DFSUtil.getJournalNodeAddresses(conf);
+  }
+
   @Test(timeout=10000)
   public void testGetSpecificKey() throws Exception {
     HdfsConfiguration conf = new HdfsConfiguration();
@@ -422,7 +585,7 @@ public class TestGetConf {
     setupAddress(conf, DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY, nsCount, 1000);
     setupAddress(conf, DFS_NAMENODE_RPC_ADDRESS_KEY, nsCount, 1500);
     conf.set(DFS_INTERNAL_NAMESERVICES_KEY, "ns1");
-    setupStaticHostResolution(nsCount);
+    setupStaticHostResolution(nsCount, "nn");
 
     String[] includedNN = new String[] {"nn1:1001"};
     verifyAddresses(conf, TestType.NAMENODE, false, includedNN);

+ 8 - 8
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java

@@ -52,6 +52,10 @@ import com.fasterxml.jackson.databind.ObjectReader;
 import com.google.common.collect.Lists;
 
 public class TestJsonUtil {
+
+  private static final ObjectReader READER =
+      new ObjectMapper().readerFor(Map.class);
+
   static FileStatus toFileStatus(HdfsFileStatus f, String parent) {
     return new FileStatus(f.getLen(), f.isDirectory(), f.getReplication(),
         f.getBlockSize(), f.getModificationTime(), f.getAccessTime(),
@@ -76,9 +80,8 @@ public class TestJsonUtil {
     System.out.println("fstatus = " + fstatus);
     final String json = JsonUtil.toJsonString(status, true);
     System.out.println("json    = " + json.replace(",", ",\n  "));
-    ObjectReader reader = new ObjectMapper().readerFor(Map.class);
     final HdfsFileStatus s2 =
-        JsonUtilClient.toFileStatus((Map<?, ?>) reader.readValue(json), true);
+        JsonUtilClient.toFileStatus((Map<?, ?>) READER.readValue(json), true);
     final FileStatus fs2 = toFileStatus(s2, parent);
     System.out.println("s2      = " + s2);
     System.out.println("fs2     = " + fs2);
@@ -164,8 +167,7 @@ public class TestJsonUtil {
   public void testToAclStatus() throws IOException {
     String jsonString =
         "{\"AclStatus\":{\"entries\":[\"user::rwx\",\"user:user1:rw-\",\"group::rw-\",\"other::r-x\"],\"group\":\"supergroup\",\"owner\":\"testuser\",\"stickyBit\":false}}";
-    ObjectReader reader = new ObjectMapper().readerFor(Map.class);
-    Map<?, ?> json = reader.readValue(jsonString);
+    Map<?, ?> json = READER.readValue(jsonString);
 
     List<AclEntry> aclSpec =
         Lists.newArrayList(aclEntry(ACCESS, USER, ALL),
@@ -224,8 +226,7 @@ public class TestJsonUtil {
     String jsonString = 
         "{\"XAttrs\":[{\"name\":\"user.a1\",\"value\":\"0x313233\"}," +
         "{\"name\":\"user.a2\",\"value\":\"0x313131\"}]}";
-    ObjectReader reader = new ObjectMapper().readerFor(Map.class);
-    Map<?, ?> json = reader.readValue(jsonString);
+    Map<?, ?> json = READER.readValue(jsonString);
     XAttr xAttr1 = (new XAttr.Builder()).setNameSpace(XAttr.NameSpace.USER).
         setName("a1").setValue(XAttrCodec.decodeValue("0x313233")).build();
     XAttr xAttr2 = (new XAttr.Builder()).setNameSpace(XAttr.NameSpace.USER).
@@ -250,8 +251,7 @@ public class TestJsonUtil {
     String jsonString = 
         "{\"XAttrs\":[{\"name\":\"user.a1\",\"value\":\"0x313233\"}," +
         "{\"name\":\"user.a2\",\"value\":\"0x313131\"}]}";
-    ObjectReader reader = new ObjectMapper().readerFor(Map.class);
-    Map<?, ?> json = reader.readValue(jsonString);
+    Map<?, ?> json = READER.readValue(jsonString);
 
     // Get xattr: user.a2
     byte[] value = JsonUtilClient.getXAttr(json, "user.a2");

+ 134 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java

@@ -18,6 +18,14 @@
 
 package org.apache.hadoop.hdfs.web;
 
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CHECKSUM_TYPE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_KEY;
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY;
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotEquals;
@@ -53,6 +61,7 @@ import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.StorageType;
@@ -74,7 +83,9 @@ import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper;
 import org.apache.hadoop.hdfs.server.namenode.web.resources.NamenodeWebHdfsMethods;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
@@ -92,11 +103,13 @@ import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.util.DataChecksum;
 import org.apache.log4j.Level;
 import org.codehaus.jettison.json.JSONException;
 import org.codehaus.jettison.json.JSONObject;
 import org.junit.Assert;
 import org.junit.Test;
+import org.mockito.Mockito;
 import org.mockito.internal.util.reflection.Whitebox;
 
 import static org.mockito.Matchers.any;
@@ -1318,4 +1331,125 @@ public class TestWebHDFS {
       }
     }
   }
+
+  /**
+   * Test fsserver defaults response from {@link DistributedFileSystem} and
+   * {@link WebHdfsFileSystem} are the same.
+   * @throws Exception
+   */
+  @Test
+  public void testFsserverDefaults() throws Exception {
+    MiniDFSCluster cluster = null;
+    final Configuration conf = WebHdfsTestUtil.createConf();
+    // Here we override all the default values so that we can verify that it
+    // doesn't pick up the default value.
+    long blockSize = 256*1024*1024;
+    int bytesPerChecksum = 256;
+    int writePacketSize = 128*1024;
+    int replicationFactor = 0;
+    int bufferSize = 1024;
+    boolean encryptDataTransfer = true;
+    long trashInterval = 1;
+    String checksumType = "CRC32";
+    // Setting policy to a special value 7 because BlockManager will
+    // create defaultSuite with policy id 7.
+    byte policyId = (byte) 7;
+
+    conf.setLong(DFS_BLOCK_SIZE_KEY, blockSize);
+    conf.setInt(DFS_BYTES_PER_CHECKSUM_KEY, bytesPerChecksum);
+    conf.setInt(DFS_CLIENT_WRITE_PACKET_SIZE_KEY, writePacketSize);
+    conf.setInt(DFS_REPLICATION_KEY, replicationFactor);
+    conf.setInt(IO_FILE_BUFFER_SIZE_KEY, bufferSize);
+    conf.setBoolean(DFS_ENCRYPT_DATA_TRANSFER_KEY, encryptDataTransfer);
+    conf.setLong(FS_TRASH_INTERVAL_KEY, trashInterval);
+    conf.set(DFS_CHECKSUM_TYPE_KEY, checksumType);
+    FsServerDefaults originalServerDefaults = new FsServerDefaults(blockSize,
+        bytesPerChecksum, writePacketSize, (short)replicationFactor,
+        bufferSize, encryptDataTransfer, trashInterval,
+        DataChecksum.Type.valueOf(checksumType), "", policyId);
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).build();
+      final DistributedFileSystem dfs = cluster.getFileSystem();
+      final WebHdfsFileSystem webfs = WebHdfsTestUtil.getWebHdfsFileSystem(
+          conf, WebHdfsConstants.WEBHDFS_SCHEME);
+      FsServerDefaults dfsServerDefaults = dfs.getServerDefaults();
+      FsServerDefaults webfsServerDefaults = webfs.getServerDefaults();
+      // Verify whether server defaults value that we override is equal to
+      // dfsServerDefaults.
+      compareFsServerDefaults(originalServerDefaults, dfsServerDefaults);
+      // Verify whether dfs serverdefaults is equal to
+      // webhdfsServerDefaults.
+      compareFsServerDefaults(dfsServerDefaults, webfsServerDefaults);
+      webfs.getServerDefaults();
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  private void compareFsServerDefaults(FsServerDefaults serverDefaults1,
+      FsServerDefaults serverDefaults2) throws Exception {
+    Assert.assertEquals("Block size is different",
+        serverDefaults1.getBlockSize(),
+        serverDefaults2.getBlockSize());
+    Assert.assertEquals("Bytes per checksum are different",
+        serverDefaults1.getBytesPerChecksum(),
+        serverDefaults2.getBytesPerChecksum());
+    Assert.assertEquals("Write packet size is different",
+        serverDefaults1.getWritePacketSize(),
+        serverDefaults2.getWritePacketSize());
+    Assert.assertEquals("Default replication is different",
+        serverDefaults1.getReplication(),
+        serverDefaults2.getReplication());
+    Assert.assertEquals("File buffer size are different",
+        serverDefaults1.getFileBufferSize(),
+        serverDefaults2.getFileBufferSize());
+    Assert.assertEquals("Encrypt data transfer key is different",
+        serverDefaults1.getEncryptDataTransfer(),
+        serverDefaults2.getEncryptDataTransfer());
+    Assert.assertEquals("Trash interval is different",
+        serverDefaults1.getTrashInterval(),
+        serverDefaults2.getTrashInterval());
+    Assert.assertEquals("Checksum type is different",
+        serverDefaults1.getChecksumType(),
+        serverDefaults2.getChecksumType());
+    Assert.assertEquals("Key provider uri is different",
+        serverDefaults1.getKeyProviderUri(),
+        serverDefaults2.getKeyProviderUri());
+    Assert.assertEquals("Default storage policy is different",
+        serverDefaults1.getDefaultStoragePolicyId(),
+        serverDefaults2.getDefaultStoragePolicyId());
+  }
+
+  /**
+   * Tests the case when client is upgraded to return {@link FsServerDefaults}
+   * but then namenode is not upgraded.
+   * @throws Exception
+   */
+  @Test
+  public void testFsserverDefaultsBackwardsCompatible() throws Exception {
+    MiniDFSCluster cluster = null;
+    final Configuration conf = WebHdfsTestUtil.createConf();
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).build();
+      final WebHdfsFileSystem webfs = WebHdfsTestUtil.getWebHdfsFileSystem(
+          conf, WebHdfsConstants.WEBHDFS_SCHEME);
+      NamenodeWebHdfsMethods.resetServerDefaultsResponse();
+      FSNamesystem fsnSpy =
+          NameNodeAdapter.spyOnNamesystem(cluster.getNameNode());
+      Mockito.when(fsnSpy.getServerDefaults()).
+          thenThrow(new UnsupportedOperationException());
+      try {
+        webfs.getServerDefaults();
+        Assert.fail("should have thrown UnSupportedOperationException.");
+      } catch (UnsupportedOperationException uoe) {
+       //Expected exception.
+      }
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
 }

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

@@ -1039,7 +1039,7 @@ public interface MRJobConfig {
   boolean DEFAULT_FINISH_JOB_WHEN_REDUCERS_DONE = true;
 
   String MR_AM_STAGING_DIR_ERASURECODING_ENABLED =
-      MR_AM_STAGING_DIR + "erasurecoding.enabled";
+      MR_AM_STAGING_DIR + ".erasurecoding.enabled";
 
   boolean DEFAULT_MR_AM_STAGING_ERASURECODING_ENABLED = false;
 }

+ 10 - 1
hadoop-project/pom.xml

@@ -137,7 +137,7 @@
     <make-maven-plugin.version>1.0-beta-1</make-maven-plugin.version>
     <native-maven-plugin.version>1.0-alpha-8</native-maven-plugin.version>
     <surefire.fork.timeout>900</surefire.fork.timeout>
-    <aws-java-sdk.version>1.11.134</aws-java-sdk.version>
+    <aws-java-sdk.version>1.11.199</aws-java-sdk.version>
     <hsqldb.version>2.3.4</hsqldb.version>
     <!-- the version of Hadoop declared in the version resources; can be overridden
     so that Hadoop 3.x can declare itself a 2.x artifact. -->
@@ -1251,6 +1251,15 @@
           </exclusion>
         </exclusions>
       </dependency>
+      <dependency>
+        <!-- HACK.  Transitive dependency for nimbus-jose-jwt.  Needed for
+             packaging.  Please re-check this version when updating
+             nimbus-jose-jwt.  Please read HADOOP-14903 for more details.
+          -->
+        <groupId>net.minidev</groupId>
+        <artifactId>json-smart</artifactId>
+        <version>2.3</version>
+      </dependency>
       <dependency>
         <groupId>org.skyscreamer</groupId>
         <artifactId>jsonassert</artifactId>

+ 10 - 4
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java

@@ -298,6 +298,10 @@ public class S3AFileSystem extends FileSystem {
       metadataStore = S3Guard.getMetadataStore(this);
       allowAuthoritative = conf.getBoolean(METADATASTORE_AUTHORITATIVE,
           DEFAULT_METADATASTORE_AUTHORITATIVE);
+      if (hasMetadataStore()) {
+        LOG.debug("Using metadata store {}, authoritative={}",
+            getMetadataStore(), allowAuthoritative);
+      }
     } catch (AmazonClientException e) {
       throw translateException("initializing ", new Path(name), e);
     }
@@ -967,7 +971,7 @@ public class S3AFileSystem extends FileSystem {
    * @return the metadata store of this FS instance
    */
   @VisibleForTesting
-  MetadataStore getMetadataStore() {
+  public MetadataStore getMetadataStore() {
     return metadataStore;
   }
 
@@ -2474,9 +2478,11 @@ public class S3AFileSystem extends FileSystem {
     sb.append(", statistics {")
         .append(statistics)
         .append("}");
-    sb.append(", metrics {")
-        .append(instrumentation.dump("{", "=", "} ", true))
-        .append("}");
+    if (instrumentation != null) {
+      sb.append(", metrics {")
+          .append(instrumentation.dump("{", "=", "} ", true))
+          .append("}");
+    }
     sb.append('}');
     return sb.toString();
   }

+ 58 - 2
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java

@@ -736,9 +736,11 @@ public final class S3AUtils {
         // propagate the value, building a new origin field.
         // to track overwrites, the generic key is overwritten even if
         // already matches the new one.
+        String origin = "[" + StringUtils.join(
+            source.getPropertySources(key), ", ") +"]";
         final String generic = FS_S3A_PREFIX + stripped;
-        LOG.debug("Updating {}", generic);
-        dest.set(generic, value, key);
+        LOG.debug("Updating {} from {}", generic, origin);
+        dest.set(generic, value, key + " via " + origin);
       }
     }
     return dest;
@@ -881,4 +883,58 @@ public final class S3AUtils {
     }
   }
 
+  /**
+   * Set a bucket-specific property to a particular value.
+   * If the generic key passed in has an {@code fs.s3a. prefix},
+   * that's stripped off, so that when the the bucket properties are propagated
+   * down to the generic values, that value gets copied down.
+   * @param conf configuration to set
+   * @param bucket bucket name
+   * @param genericKey key; can start with "fs.s3a."
+   * @param value value to set
+   */
+  public static void setBucketOption(Configuration conf, String bucket,
+      String genericKey, String value) {
+    final String baseKey = genericKey.startsWith(FS_S3A_PREFIX) ?
+        genericKey.substring(FS_S3A_PREFIX.length())
+        : genericKey;
+    conf.set(FS_S3A_BUCKET_PREFIX + bucket + '.' + baseKey, value, "S3AUtils");
+  }
+
+  /**
+   * Clear a bucket-specific property.
+   * If the generic key passed in has an {@code fs.s3a. prefix},
+   * that's stripped off, so that when the the bucket properties are propagated
+   * down to the generic values, that value gets copied down.
+   * @param conf configuration to set
+   * @param bucket bucket name
+   * @param genericKey key; can start with "fs.s3a."
+   */
+  public static void clearBucketOption(Configuration conf, String bucket,
+      String genericKey) {
+    final String baseKey = genericKey.startsWith(FS_S3A_PREFIX) ?
+        genericKey.substring(FS_S3A_PREFIX.length())
+        : genericKey;
+    String k = FS_S3A_BUCKET_PREFIX + bucket + '.' + baseKey;
+    LOG.debug("Unset {}", k);
+    conf.unset(k);
+  }
+
+  /**
+   * Get a bucket-specific property.
+   * If the generic key passed in has an {@code fs.s3a. prefix},
+   * that's stripped off.
+   * @param conf configuration to set
+   * @param bucket bucket name
+   * @param genericKey key; can start with "fs.s3a."
+   * @return the bucket option, null if there is none
+   */
+  public static String getBucketOption(Configuration conf, String bucket,
+      String genericKey) {
+    final String baseKey = genericKey.startsWith(FS_S3A_PREFIX) ?
+        genericKey.substring(FS_S3A_PREFIX.length())
+        : genericKey;
+    return conf.get(FS_S3A_BUCKET_PREFIX + bucket + '.' + baseKey);
+  }
+
 }

+ 100 - 4
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java

@@ -29,6 +29,7 @@ import java.util.Date;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.TreeMap;
 import java.util.concurrent.TimeUnit;
 
 import com.amazonaws.AmazonClientException;
@@ -51,6 +52,7 @@ import com.amazonaws.services.dynamodbv2.model.ProvisionedThroughput;
 import com.amazonaws.services.dynamodbv2.model.ProvisionedThroughputDescription;
 import com.amazonaws.services.dynamodbv2.model.ResourceInUseException;
 import com.amazonaws.services.dynamodbv2.model.ResourceNotFoundException;
+import com.amazonaws.services.dynamodbv2.model.TableDescription;
 import com.amazonaws.services.dynamodbv2.model.WriteRequest;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
@@ -184,6 +186,18 @@ public class DynamoDBMetadataStore implements MetadataStore {
    * DynamoDB. Value is {@value} msec. */
   public static final long MIN_RETRY_SLEEP_MSEC = 100;
 
+  @VisibleForTesting
+  static final String DESCRIPTION
+      = "S3Guard metadata store in DynamoDB";
+  @VisibleForTesting
+  static final String READ_CAPACITY = "read-capacity";
+  @VisibleForTesting
+  static final String WRITE_CAPACITY = "write-capacity";
+  @VisibleForTesting
+  static final String STATUS = "status";
+  @VisibleForTesting
+  static final String TABLE = "table";
+
   private static ValueMap deleteTrackingValueMap =
       new ValueMap().withBoolean(":false", false);
 
@@ -788,7 +802,9 @@ public class DynamoDBMetadataStore implements MetadataStore {
     try {
       try {
         LOG.debug("Binding to table {}", tableName);
-        final String status = table.describe().getTableStatus();
+        TableDescription description = table.describe();
+        LOG.debug("Table state: {}", description);
+        final String status = description.getTableStatus();
         switch (status) {
         case "CREATING":
         case "UPDATING":
@@ -824,9 +840,10 @@ public class DynamoDBMetadataStore implements MetadataStore {
 
           createTable(capacity);
         } else {
-          throw new FileNotFoundException("DynamoDB table "
-              + "'" + tableName + "' does not "
-              + "exist in region " + region + "; auto-creation is turned off");
+          throw (FileNotFoundException)new FileNotFoundException(
+              "DynamoDB table '" + tableName + "' does not "
+              + "exist in region " + region + "; auto-creation is turned off")
+              .initCause(rnfe);
         }
       }
 
@@ -1007,4 +1024,83 @@ public class DynamoDBMetadataStore implements MetadataStore {
     Preconditions.checkNotNull(meta.getFileStatus().getPath());
   }
 
+  @Override
+  public Map<String, String> getDiagnostics() throws IOException {
+    Map<String, String> map = new TreeMap<>();
+    if (table != null) {
+      TableDescription desc = getTableDescription(true);
+      map.put("name", desc.getTableName());
+      map.put(STATUS, desc.getTableStatus());
+      map.put("ARN", desc.getTableArn());
+      map.put("size", desc.getTableSizeBytes().toString());
+      map.put(TABLE, desc.toString());
+      ProvisionedThroughputDescription throughput
+          = desc.getProvisionedThroughput();
+      map.put(READ_CAPACITY, throughput.getReadCapacityUnits().toString());
+      map.put(WRITE_CAPACITY, throughput.getWriteCapacityUnits().toString());
+      map.put(TABLE, desc.toString());
+    } else {
+      map.put("name", "DynamoDB Metadata Store");
+      map.put(TABLE, "none");
+      map.put(STATUS, "undefined");
+    }
+    map.put("description", DESCRIPTION);
+    map.put("region", region);
+    if (dataAccessRetryPolicy != null) {
+      map.put("retryPolicy", dataAccessRetryPolicy.toString());
+    }
+    return map;
+  }
+
+  private TableDescription getTableDescription(boolean forceUpdate) {
+    TableDescription desc = table.getDescription();
+    if (desc == null || forceUpdate) {
+      desc = table.describe();
+    }
+    return desc;
+  }
+
+  @Override
+  public void updateParameters(Map<String, String> parameters)
+      throws IOException {
+    Preconditions.checkNotNull(table, "Not initialized");
+    TableDescription desc = getTableDescription(true);
+    ProvisionedThroughputDescription current
+        = desc.getProvisionedThroughput();
+
+    long currentRead = current.getReadCapacityUnits();
+    long newRead = getLongParam(parameters,
+        S3GUARD_DDB_TABLE_CAPACITY_READ_KEY,
+        currentRead);
+    long currentWrite = current.getWriteCapacityUnits();
+    long newWrite = getLongParam(parameters,
+            S3GUARD_DDB_TABLE_CAPACITY_WRITE_KEY,
+            currentWrite);
+
+    ProvisionedThroughput throughput = new ProvisionedThroughput()
+        .withReadCapacityUnits(newRead)
+        .withWriteCapacityUnits(newWrite);
+    if (newRead != currentRead || newWrite != currentWrite) {
+      LOG.info("Current table capacity is read: {}, write: {}",
+          currentRead, currentWrite);
+      LOG.info("Changing capacity of table to read: {}, write: {}",
+          newRead, newWrite);
+      table.updateTable(throughput);
+    } else {
+      LOG.info("Table capacity unchanged at read: {}, write: {}",
+          newRead, newWrite);
+    }
+  }
+
+  private long getLongParam(Map<String, String> parameters,
+      String key,
+      long defVal) {
+    String k = parameters.get(key);
+    if (k != null) {
+      return Long.parseLong(k);
+    } else {
+      return defVal;
+    }
+  }
+
 }

+ 20 - 5
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/LocalMetadataStore.java

@@ -20,6 +20,8 @@ package org.apache.hadoop.fs.s3a.s3guard;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
+
+import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -31,6 +33,7 @@ import org.slf4j.LoggerFactory;
 import java.io.IOException;
 import java.net.URI;
 import java.util.Collection;
+import java.util.HashMap;
 import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.Map;
@@ -100,7 +103,7 @@ public class LocalMetadataStore implements MetadataStore {
   public String toString() {
     final StringBuilder sb = new StringBuilder(
         "LocalMetadataStore{");
-    sb.append(", uriHost='").append(uriHost).append('\'');
+    sb.append("uriHost='").append(uriHost).append('\'');
     sb.append('}');
     return sb.toString();
   }
@@ -153,7 +156,9 @@ public class LocalMetadataStore implements MetadataStore {
         m.setIsEmptyDirectory(isEmptyDirectory(p));
       }
 
-      LOG.debug("get({}) -> {}", path, m == null ? "null" : m.prettyPrint());
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("get({}) -> {}", path, m == null ? "null" : m.prettyPrint());
+      }
       return m;
     }
   }
@@ -424,12 +429,22 @@ public class LocalMetadataStore implements MetadataStore {
     Preconditions.checkArgument(p.isAbsolute(), "Path must be absolute");
     URI uri = p.toUri();
     if (uriHost != null) {
-      Preconditions.checkArgument(!isEmpty(uri.getHost()));
+      Preconditions.checkArgument(StringUtils.isNotEmpty(uri.getHost()));
     }
     return p;
   }
 
-  private static boolean isEmpty(String s) {
-    return (s == null || s.isEmpty());
+  @Override
+  public Map<String, String> getDiagnostics() throws IOException {
+    Map<String, String> map = new HashMap<>();
+    map.put("name", "local://metadata");
+    map.put("uriHost", uriHost);
+    map.put("description", "Local in-VM metadata store for testing");
+    return map;
+  }
+
+  @Override
+  public void updateParameters(Map<String, String> parameters)
+      throws IOException {
   }
 }

+ 17 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/MetadataStore.java

@@ -21,6 +21,7 @@ package org.apache.hadoop.fs.s3a.s3guard;
 import java.io.Closeable;
 import java.io.IOException;
 import java.util.Collection;
+import java.util.Map;
 
 import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -218,4 +219,20 @@ public interface MetadataStore extends Closeable {
    * @throws UnsupportedOperationException if not implemented
    */
   void prune(long modTime) throws IOException, UnsupportedOperationException;
+
+  /**
+   * Get any diagnostics information from a store, as a list of (key, value)
+   * tuples for display. Arbitrary values; no guarantee of stability.
+   * These are for debugging only.
+   * @return a map of strings.
+   * @throws IOException if there is an error
+   */
+  Map<String, String> getDiagnostics() throws IOException;
+
+  /**
+   * Tune/update parameters for an existing table.
+   * @param parameters map of params to change.
+   * @throws IOException if there is an error
+   */
+  void updateParameters(Map<String, String> parameters) throws IOException;
 }

+ 15 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/NullMetadataStore.java

@@ -24,6 +24,8 @@ import org.apache.hadoop.fs.Path;
 
 import java.io.IOException;
 import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
 
 /**
  * A no-op implementation of MetadataStore.  Clients that use this
@@ -101,4 +103,17 @@ public class NullMetadataStore implements MetadataStore {
   public String toString() {
     return "NullMetadataStore";
   }
+
+  @Override
+  public Map<String, String> getDiagnostics() throws IOException {
+    Map<String, String> map = new HashMap<>();
+    map.put("name", "Null Metadata Store");
+    map.put("description", "This is not a real metadata store");
+    return map;
+  }
+
+  @Override
+  public void updateParameters(Map<String, String> parameters)
+      throws IOException {
+  }
 }

+ 14 - 3
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3Guard.java

@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.fs.s3a.s3guard;
 
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.net.URI;
 import java.util.ArrayList;
@@ -95,6 +96,10 @@ public final class S3Guard {
           msClass.getSimpleName(), fs.getScheme());
       msInstance.initialize(fs);
       return msInstance;
+    } catch (FileNotFoundException e) {
+      // Don't log this exception as it means the table doesn't exist yet;
+      // rely on callers to catch and treat specially
+      throw e;
     } catch (RuntimeException | IOException e) {
       String message = "Failed to instantiate metadata store " +
           conf.get(S3_METADATA_STORE_IMPL)
@@ -109,14 +114,20 @@ public final class S3Guard {
     }
   }
 
-  private static Class<? extends MetadataStore> getMetadataStoreClass(
+  static Class<? extends MetadataStore> getMetadataStoreClass(
       Configuration conf) {
     if (conf == null) {
       return NullMetadataStore.class;
     }
+    if (conf.get(S3_METADATA_STORE_IMPL) != null && LOG.isDebugEnabled()) {
+      LOG.debug("Metastore option source {}",
+          conf.getPropertySources(S3_METADATA_STORE_IMPL));
+    }
 
-    return conf.getClass(S3_METADATA_STORE_IMPL, NullMetadataStore.class,
-            MetadataStore.class);
+    Class<? extends MetadataStore> aClass = conf.getClass(
+        S3_METADATA_STORE_IMPL, NullMetadataStore.class,
+        MetadataStore.class);
+    return aClass;
   }
 
 

+ 430 - 109
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java

@@ -36,6 +36,7 @@ import com.google.common.base.Preconditions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.FileStatus;
@@ -43,15 +44,17 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RemoteIterator;
-import org.apache.hadoop.fs.s3a.Constants;
 import org.apache.hadoop.fs.s3a.S3AFileStatus;
 import org.apache.hadoop.fs.s3a.S3AFileSystem;
+import org.apache.hadoop.fs.s3a.S3AUtils;
 import org.apache.hadoop.fs.shell.CommandFormat;
+import org.apache.hadoop.util.ExitUtil;
 import org.apache.hadoop.util.GenericOptionsParser;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 
 import static org.apache.hadoop.fs.s3a.Constants.*;
+import static org.apache.hadoop.service.launcher.LauncherExitCodes.*;
 
 /**
  * CLI to manage S3Guard Metadata Store.
@@ -74,40 +77,46 @@ public abstract class S3GuardTool extends Configured implements Tool {
       "\t" + Init.NAME + " - " + Init.PURPOSE + "\n" +
       "\t" + Destroy.NAME + " - " + Destroy.PURPOSE + "\n" +
       "\t" + Import.NAME + " - " + Import.PURPOSE + "\n" +
+      "\t" + BucketInfo.NAME + " - " + BucketInfo.PURPOSE + "\n" +
       "\t" + Diff.NAME + " - " + Diff.PURPOSE + "\n" +
-      "\t" + Prune.NAME + " - " + Prune.PURPOSE + "\n";
+      "\t" + Prune.NAME + " - " + Prune.PURPOSE + "\n" +
+      "\t" + SetCapacity.NAME + " - " +SetCapacity.PURPOSE + "\n";
   private static final String DATA_IN_S3_IS_PRESERVED
-      = "(all data in S3 is preserved";
+      = "(all data in S3 is preserved)";
 
   abstract public String getUsage();
 
   // Exit codes
-  static final int SUCCESS = 0;
-  static final int INVALID_ARGUMENT = 1;
-  static final int ERROR = 99;
+  static final int SUCCESS = EXIT_SUCCESS;
+  static final int INVALID_ARGUMENT = EXIT_COMMAND_ARGUMENT_ERROR;
+  static final int E_USAGE = EXIT_USAGE;
+  static final int ERROR = EXIT_FAIL;
+  static final int E_BAD_STATE = EXIT_NOT_ACCEPTABLE;
+  static final int E_NOT_FOUND = EXIT_NOT_FOUND;
 
   private S3AFileSystem filesystem;
   private MetadataStore store;
   private final CommandFormat commandFormat;
 
-  private static final String META_FLAG = "meta";
-  private static final String DAYS_FLAG = "days";
-  private static final String HOURS_FLAG = "hours";
-  private static final String MINUTES_FLAG = "minutes";
-  private static final String SECONDS_FLAG = "seconds";
+  public static final String META_FLAG = "meta";
+  public static final String DAYS_FLAG = "days";
+  public static final String HOURS_FLAG = "hours";
+  public static final String MINUTES_FLAG = "minutes";
+  public static final String SECONDS_FLAG = "seconds";
 
-  private static final String REGION_FLAG = "region";
-  private static final String READ_FLAG = "read";
-  private static final String WRITE_FLAG = "write";
+  public static final String REGION_FLAG = "region";
+  public static final String READ_FLAG = "read";
+  public static final String WRITE_FLAG = "write";
 
   /**
    * Constructor a S3Guard tool with HDFS configuration.
    * @param conf Configuration.
+   * @param opts any boolean options to support
    */
-  protected S3GuardTool(Configuration conf) {
+  protected S3GuardTool(Configuration conf, String...opts) {
     super(conf);
 
-    commandFormat = new CommandFormat(0, Integer.MAX_VALUE);
+    commandFormat = new CommandFormat(0, Integer.MAX_VALUE, opts);
     // For metadata store URI
     commandFormat.addOptionWithValue(META_FLAG);
     // DDB region.
@@ -126,10 +135,10 @@ public abstract class S3GuardTool extends Configured implements Tool {
    * {@link Destroy}.
    *
    * @param paths remaining parameters from CLI.
-   * @return false for invalid parameters.
    * @throws IOException on I/O errors.
+   * @throws ExitUtil.ExitException on validation errors
    */
-  boolean parseDynamoDBRegion(List<String> paths) throws IOException {
+  void parseDynamoDBRegion(List<String> paths) throws IOException {
     Configuration conf = getConf();
     String fromCli = getCommandFormat().getOptValue(REGION_FLAG);
     String fromConf = conf.get(S3GUARD_DDB_REGION_KEY);
@@ -137,37 +146,34 @@ public abstract class S3GuardTool extends Configured implements Tool {
 
     if (fromCli != null) {
       if (fromCli.isEmpty()) {
-        System.err.println("No region provided with -" + REGION_FLAG + " flag");
-        return false;
+        throw invalidArgs("No region provided with -" + REGION_FLAG + " flag");
       }
       if (hasS3Path) {
-        System.err.println("Providing both an S3 path and the -" + REGION_FLAG
+        throw invalidArgs("Providing both an S3 path and the"
+            + " -" + REGION_FLAG
             + " flag is not supported. If you need to specify a different "
             + "region than the S3 bucket, configure " + S3GUARD_DDB_REGION_KEY);
-        return false;
       }
       conf.set(S3GUARD_DDB_REGION_KEY, fromCli);
-      return true;
+      return;
     }
 
     if (fromConf != null) {
       if (fromConf.isEmpty()) {
-        System.err.printf("No region provided with config %s, %n",
+        throw invalidArgs("No region provided with config %s",
             S3GUARD_DDB_REGION_KEY);
-        return false;
       }
-      return true;
+      return;
     }
 
     if (hasS3Path) {
       String s3Path = paths.get(0);
       initS3AFileSystem(s3Path);
-      return true;
+      return;
     }
 
-    System.err.println("No region found from -" + REGION_FLAG + " flag, " +
+    throw invalidArgs("No region found from -" + REGION_FLAG + " flag, " +
         "config, or S3 bucket");
-    return false;
   }
 
   /**
@@ -189,7 +195,7 @@ public abstract class S3GuardTool extends Configured implements Tool {
     String metaURI = getCommandFormat().getOptValue(META_FLAG);
     if (metaURI != null && !metaURI.isEmpty()) {
       URI uri = URI.create(metaURI);
-      LOG.info("create metadata store: {}", uri + " scheme: "
+      LOG.info("Create metadata store: {}", uri + " scheme: "
           + uri.getScheme());
       switch (uri.getScheme().toLowerCase(Locale.ENGLISH)) {
       case "local":
@@ -225,35 +231,48 @@ public abstract class S3GuardTool extends Configured implements Tool {
   }
 
   /**
-   * Initialize S3A FileSystem instance.
+   * Create and initialize a new S3A FileSystem instance.
+   * This instance is always created without S3Guard, so allowing
+   * a previously created metastore to be patched in.
+   *
+   * Note: this is a bit convoluted as it needs to also handle the situation
+   * of a per-bucket option in core-site.xml, which isn't easily overridden.
+   * The new config and the setting of the values before any
+   * {@code Configuration.get()} calls are critical.
    *
    * @param path s3a URI
-   * @throws IOException
+   * @throws IOException failure to init filesystem
+   * @throws ExitUtil.ExitException if the FS is not an S3A FS
    */
   void initS3AFileSystem(String path) throws IOException {
-    URI uri;
-    try {
-      uri = new URI(path);
-    } catch (URISyntaxException e) {
-      throw new IOException(e);
-    }
+    URI uri = toUri(path);
     // Make sure that S3AFileSystem does not hold an actual MetadataStore
     // implementation.
-    Configuration conf = getConf();
-    conf.setClass(S3_METADATA_STORE_IMPL, NullMetadataStore.class,
-        MetadataStore.class);
-    FileSystem fs = FileSystem.get(uri, getConf());
+    Configuration conf = new Configuration(getConf());
+    String nullStore = NullMetadataStore.class.getName();
+    conf.set(S3_METADATA_STORE_IMPL, nullStore);
+    String bucket = uri.getHost();
+    S3AUtils.setBucketOption(conf,
+        bucket,
+        S3_METADATA_STORE_IMPL, S3GUARD_METASTORE_NULL);
+    String updatedBucketOption = S3AUtils.getBucketOption(conf, bucket,
+        S3_METADATA_STORE_IMPL);
+    LOG.debug("updated bucket store option {}", updatedBucketOption);
+    Preconditions.checkState(S3GUARD_METASTORE_NULL.equals(updatedBucketOption),
+        "Expected bucket option to be %s but was %s",
+        S3GUARD_METASTORE_NULL, updatedBucketOption);
+
+    FileSystem fs = FileSystem.newInstance(uri, conf);
     if (!(fs instanceof S3AFileSystem)) {
-      throw new IOException(
-          String.format("URI %s is not a S3A file system: %s", uri,
-              fs.getClass().getName()));
+      throw invalidArgs("URI %s is not a S3A file system: %s",
+          uri, fs.getClass().getName());
     }
     filesystem = (S3AFileSystem) fs;
   }
 
   /**
    * Parse CLI arguments and returns the position arguments.
-   * The options are stored in {@link #commandFormat}
+   * The options are stored in {@link #commandFormat}.
    *
    * @param args command line arguments.
    * @return the position arguments from CLI.
@@ -285,11 +304,32 @@ public abstract class S3GuardTool extends Configured implements Tool {
     return commandFormat;
   }
 
+  @Override
+  public final int run(String[] args) throws Exception {
+    return run(args, System.out);
+  }
+
+  /**
+   * Run the tool, capturing the output (if the tool supports that).
+   *
+   * As well as returning an exit code, the implementations can choose to
+   * throw an instance of {@link ExitUtil.ExitException} with their exit
+   * code set to the desired exit value. The exit code of auch an exception
+   * is used for the tool's exit code, and the stack trace only logged at
+   * debug.
+   * @param args argument list
+   * @param out output stream
+   * @return the exit code to return.
+   * @throws Exception on any failure
+   * @throws ExitUtil.ExitException for an alternative clean exit
+   */
+  public abstract int run(String[] args, PrintStream out) throws Exception;
+
   /**
    * Create the metadata store.
    */
   static class Init extends S3GuardTool {
-    private static final String NAME = "init";
+    public static final String NAME = "init";
     public static final String PURPOSE = "initialize metadata repository";
     private static final String USAGE = NAME + " [OPTIONS] [s3a://BUCKET]\n" +
         "\t" + PURPOSE + "\n\n" +
@@ -325,7 +365,7 @@ public abstract class S3GuardTool extends Configured implements Tool {
     }
 
     @Override
-    public int run(String[] args) throws IOException {
+    public int run(String[] args, PrintStream out) throws Exception {
       List<String> paths = parseArgs(args);
 
       String readCap = getCommandFormat().getOptValue(READ_FLAG);
@@ -340,20 +380,92 @@ public abstract class S3GuardTool extends Configured implements Tool {
       }
 
       // Validate parameters.
-      if (!parseDynamoDBRegion(paths)) {
-        System.err.println(USAGE);
-        return INVALID_ARGUMENT;
+      try {
+        parseDynamoDBRegion(paths);
+      } catch (ExitUtil.ExitException e) {
+        errorln(USAGE);
+        throw e;
+      }
+      MetadataStore store = initMetadataStore(true);
+      printStoreDiagnostics(out, store);
+      return SUCCESS;
+    }
+  }
+
+  /**
+   * Change the capacity of the metadata store.
+   */
+  static class SetCapacity extends S3GuardTool {
+    public static final String NAME = "set-capacity";
+    public static final String PURPOSE = "Alter metadata store IO capacity";
+    private static final String USAGE = NAME + " [OPTIONS] [s3a://BUCKET]\n" +
+        "\t" + PURPOSE + "\n\n" +
+        "Common options:\n" +
+        "  -" + META_FLAG + " URL - Metadata repository details " +
+          "(implementation-specific)\n" +
+        "\n" +
+        "Amazon DynamoDB-specific options:\n" +
+        "  -" + READ_FLAG + " UNIT - Provisioned read throughput units\n" +
+        "  -" + WRITE_FLAG + " UNIT - Provisioned write through put units\n" +
+        "\n" +
+        "  URLs for Amazon DynamoDB are of the form dynamodb://TABLE_NAME.\n" +
+        "  Specifying both the -" + REGION_FLAG + " option and an S3A path\n" +
+        "  is not supported.";
+
+    SetCapacity(Configuration conf) {
+      super(conf);
+      // read capacity.
+      getCommandFormat().addOptionWithValue(READ_FLAG);
+      // write capacity.
+      getCommandFormat().addOptionWithValue(WRITE_FLAG);
+    }
+
+    @Override
+    String getName() {
+      return NAME;
+    }
+
+    @Override
+    public String getUsage() {
+      return USAGE;
+    }
+
+    @Override
+    public int run(String[] args, PrintStream out) throws Exception {
+      List<String> paths = parseArgs(args);
+      Map<String, String> options = new HashMap<>();
+
+      String readCap = getCommandFormat().getOptValue(READ_FLAG);
+      if (StringUtils.isNotEmpty(readCap)) {
+        S3GuardTool.println(out, "Read capacity set to %s", readCap);
+        options.put(S3GUARD_DDB_TABLE_CAPACITY_READ_KEY, readCap);
+      }
+      String writeCap = getCommandFormat().getOptValue(WRITE_FLAG);
+      if (StringUtils.isNotEmpty(writeCap)) {
+        S3GuardTool.println(out, "Write capacity set to %s", writeCap);
+        options.put(S3GUARD_DDB_TABLE_CAPACITY_WRITE_KEY, writeCap);
+      }
+
+      // Validate parameters.
+      try {
+        parseDynamoDBRegion(paths);
+      } catch (ExitUtil.ExitException e) {
+        errorln(USAGE);
+        throw e;
       }
-      initMetadataStore(true);
+      MetadataStore store = initMetadataStore(false);
+      store.updateParameters(options);
+      printStoreDiagnostics(out, store);
       return SUCCESS;
     }
   }
 
+
   /**
    * Destroy a metadata store.
    */
   static class Destroy extends S3GuardTool {
-    private static final String NAME = "destroy";
+    public static final String NAME = "destroy";
     public static final String PURPOSE = "destroy Metadata Store data "
         + DATA_IN_S3_IS_PRESERVED;
     private static final String USAGE = NAME + " [OPTIONS] [s3a://BUCKET]\n" +
@@ -383,19 +495,21 @@ public abstract class S3GuardTool extends Configured implements Tool {
       return USAGE;
     }
 
-    public int run(String[] args) throws IOException {
+    public int run(String[] args, PrintStream out) throws Exception {
       List<String> paths = parseArgs(args);
-      if (!parseDynamoDBRegion(paths)) {
-        System.err.println(USAGE);
-        return INVALID_ARGUMENT;
+      try {
+        parseDynamoDBRegion(paths);
+      } catch (ExitUtil.ExitException e) {
+        errorln(USAGE);
+        throw e;
       }
 
       try {
         initMetadataStore(false);
       } catch (FileNotFoundException e) {
         // indication that the table was not found
+        println(out, "Metadata Store does not exist.");
         LOG.debug("Failed to bind to store to be destroyed", e);
-        LOG.info("Metadata Store does not exist.");
         return SUCCESS;
       }
 
@@ -403,7 +517,7 @@ public abstract class S3GuardTool extends Configured implements Tool {
           "Metadata Store is not initialized");
 
       getStore().destroy();
-      LOG.info("Metadata store is deleted.");
+      println(out, "Metadata store is deleted.");
       return SUCCESS;
     }
   }
@@ -412,7 +526,7 @@ public abstract class S3GuardTool extends Configured implements Tool {
    * Import s3 metadata to the metadata store.
    */
   static class Import extends S3GuardTool {
-    private static final String NAME = "import";
+    public static final String NAME = "import";
     public static final String PURPOSE = "import metadata from existing S3 " +
         "data";
     private static final String USAGE = NAME + " [OPTIONS] [s3a://BUCKET]\n" +
@@ -498,21 +612,16 @@ public abstract class S3GuardTool extends Configured implements Tool {
     }
 
     @Override
-    public int run(String[] args) throws IOException {
+    public int run(String[] args, PrintStream out) throws Exception {
       List<String> paths = parseArgs(args);
       if (paths.isEmpty()) {
-        System.err.println(getUsage());
-        return INVALID_ARGUMENT;
+        errorln(getUsage());
+        throw invalidArgs("no arguments");
       }
       String s3Path = paths.get(0);
       initS3AFileSystem(s3Path);
 
-      URI uri;
-      try {
-        uri = new URI(s3Path);
-      } catch (URISyntaxException e) {
-        throw new IOException(e);
-      }
+      URI uri = toUri(s3Path);
       String filePath = uri.getPath();
       if (filePath.isEmpty()) {
         // If they specify a naked S3 URI (e.g. s3a://bucket), we'll consider
@@ -522,7 +631,11 @@ public abstract class S3GuardTool extends Configured implements Tool {
       Path path = new Path(filePath);
       FileStatus status = getFilesystem().getFileStatus(path);
 
-      initMetadataStore(false);
+      try {
+        initMetadataStore(false);
+      } catch (FileNotFoundException e) {
+        throw storeNotFound(e);
+      }
 
       long items = 1;
       if (status.isFile()) {
@@ -532,17 +645,18 @@ public abstract class S3GuardTool extends Configured implements Tool {
         items = importDir(status);
       }
 
-      System.out.printf("Inserted %d items into Metadata Store%n", items);
+      println(out, "Inserted %d items into Metadata Store", items);
 
       return SUCCESS;
     }
+
   }
 
   /**
    * Show diffs between the s3 and metadata store.
    */
   static class Diff extends S3GuardTool {
-    private static final String NAME = "diff";
+    public static final String NAME = "diff";
     public static final String PURPOSE = "report on delta between S3 and " +
         "repository";
     private static final String USAGE = NAME + " [OPTIONS] s3a://BUCKET\n" +
@@ -625,10 +739,10 @@ public abstract class S3GuardTool extends Configured implements Tool {
 
       if (differ(msStatus, s3Status)) {
         if (s3Status != null) {
-          out.printf("%s%s%s%n", S3_PREFIX, SEP, formatFileStatus(s3Status));
+          println(out, "%s%s%s", S3_PREFIX, SEP, formatFileStatus(s3Status));
         }
         if (msStatus != null) {
-          out.printf("%s%s%s%n", MS_PREFIX, SEP, formatFileStatus(msStatus));
+          println(out, "%s%s%s", MS_PREFIX, SEP, formatFileStatus(msStatus));
         }
       }
     }
@@ -705,6 +819,7 @@ public abstract class S3GuardTool extends Configured implements Tool {
       try {
         s3Status = getFilesystem().getFileStatus(qualified);
       } catch (FileNotFoundException e) {
+        /* ignored */
       }
       PathMetadata meta = getStore().get(qualified);
       FileStatus msStatus = (meta != null && !meta.isDeleted()) ?
@@ -717,18 +832,13 @@ public abstract class S3GuardTool extends Configured implements Tool {
       List<String> paths = parseArgs(args);
       if (paths.isEmpty()) {
         out.println(USAGE);
-        return INVALID_ARGUMENT;
+        throw invalidArgs("no arguments");
       }
       String s3Path = paths.get(0);
       initS3AFileSystem(s3Path);
-      initMetadataStore(true);
+      initMetadataStore(false);
 
-      URI uri;
-      try {
-        uri = new URI(s3Path);
-      } catch (URISyntaxException e) {
-        throw new IOException(e);
-      }
+      URI uri = toUri(s3Path);
       Path root;
       if (uri.getPath().isEmpty()) {
         root = new Path("/");
@@ -741,17 +851,13 @@ public abstract class S3GuardTool extends Configured implements Tool {
       return SUCCESS;
     }
 
-    @Override
-    public int run(String[] args) throws IOException {
-      return run(args, System.out);
-    }
   }
 
   /**
    * Prune metadata that has not been modified recently.
    */
   static class Prune extends S3GuardTool {
-    private static final String NAME = "prune";
+    public static final String NAME = "prune";
     public static final String PURPOSE = "truncate older metadata from " +
         "repository "
         + DATA_IN_S3_IS_PRESERVED;;
@@ -803,18 +909,19 @@ public abstract class S3GuardTool extends Configured implements Tool {
       return unit.toMillis(parsed);
     }
 
-    @VisibleForTesting
     public int run(String[] args, PrintStream out) throws
         InterruptedException, IOException {
       List<String> paths = parseArgs(args);
-      if (!parseDynamoDBRegion(paths)) {
-        System.err.println(USAGE);
-        return INVALID_ARGUMENT;
+      try {
+        parseDynamoDBRegion(paths);
+      } catch (ExitUtil.ExitException e) {
+        errorln(USAGE);
+        throw e;
       }
       initMetadataStore(false);
 
       Configuration conf = getConf();
-      long confDelta = conf.getLong(Constants.S3GUARD_CLI_PRUNE_AGE, 0);
+      long confDelta = conf.getLong(S3GUARD_CLI_PRUNE_AGE, 0);
 
       long cliDelta = 0;
       cliDelta += getDeltaComponent(TimeUnit.DAYS, "days");
@@ -823,8 +930,7 @@ public abstract class S3GuardTool extends Configured implements Tool {
       cliDelta += getDeltaComponent(TimeUnit.SECONDS, "seconds");
 
       if (confDelta <= 0 && cliDelta <= 0) {
-        System.err.println(
-            "You must specify a positive age for metadata to prune.");
+        errorln("You must specify a positive age for metadata to prune.");
       }
 
       // A delta provided on the CLI overrides if one is configured
@@ -842,35 +948,235 @@ public abstract class S3GuardTool extends Configured implements Tool {
       return SUCCESS;
     }
 
+  }
+
+  /**
+   * Get info about a bucket and its S3Guard integration status.
+   */
+  static class BucketInfo extends S3GuardTool {
+    public static final String NAME = "bucket-info";
+    public static final String GUARDED_FLAG = "guarded";
+    public static final String UNGUARDED_FLAG = "unguarded";
+    public static final String AUTH_FLAG = "auth";
+    public static final String NONAUTH_FLAG = "nonauth";
+    public static final String ENCRYPTION_FLAG = "encryption";
+
+    public static final String PURPOSE = "provide/check S3Guard information"
+        + " about a specific bucket";
+    private static final String USAGE = NAME + " [OPTIONS] s3a://BUCKET\n"
+        + "\t" + PURPOSE + "\n\n"
+        + "Common options:\n"
+        + "  -" + GUARDED_FLAG + " - Require S3Guard\n"
+        + "  -" + ENCRYPTION_FLAG
+        + " -require {none, sse-s3, sse-kms} - Require encryption policy";
+
+    BucketInfo(Configuration conf) {
+      super(conf, GUARDED_FLAG, UNGUARDED_FLAG, AUTH_FLAG, NONAUTH_FLAG);
+      CommandFormat format = getCommandFormat();
+      format.addOptionWithValue(ENCRYPTION_FLAG);
+    }
+
+    @Override
+    String getName() {
+      return NAME;
+    }
+
     @Override
-    public int run(String[] args) throws InterruptedException, IOException {
-      return run(args, System.out);
+    public String getUsage() {
+      return USAGE;
     }
+
+    public int run(String[] args, PrintStream out)
+        throws InterruptedException, IOException {
+      List<String> paths = parseArgs(args);
+      if (paths.isEmpty()) {
+        errorln(getUsage());
+        throw invalidArgs("No bucket specified");
+      }
+      String s3Path = paths.get(0);
+      S3AFileSystem fs = (S3AFileSystem) FileSystem.newInstance(
+          toUri(s3Path), getConf());
+      setFilesystem(fs);
+      Configuration conf = fs.getConf();
+      URI fsUri = fs.getUri();
+      MetadataStore store = fs.getMetadataStore();
+      println(out, "Filesystem %s", fsUri);
+      println(out, "Location: %s", fs.getBucketLocation());
+      boolean usingS3Guard = !(store instanceof NullMetadataStore);
+      boolean authMode = false;
+      if (usingS3Guard) {
+        out.printf("Filesystem %s is using S3Guard with store %s%n",
+            fsUri, store.toString());
+        printOption(out, "Authoritative S3Guard",
+            METADATASTORE_AUTHORITATIVE, "false");
+        authMode = conf.getBoolean(METADATASTORE_AUTHORITATIVE, false);
+        printStoreDiagnostics(out, store);
+      } else {
+        println(out, "Filesystem %s is not using S3Guard", fsUri);
+      }
+      println(out, "%nS3A Client");
+
+      String endpoint = conf.getTrimmed(ENDPOINT, "");
+      println(out, "\tEndpoint: %s=%s",
+          ENDPOINT,
+          StringUtils.isNotEmpty(endpoint) ? endpoint : "(unset)");
+      String encryption =
+          printOption(out, "\tEncryption", SERVER_SIDE_ENCRYPTION_ALGORITHM,
+              "none");
+      printOption(out, "\tInput seek policy", INPUT_FADVISE, INPUT_FADV_NORMAL);
+
+      CommandFormat commands = getCommandFormat();
+      if (usingS3Guard) {
+        if (commands.getOpt(UNGUARDED_FLAG)) {
+          throw badState("S3Guard is enabled for %s", fsUri);
+        }
+        if (commands.getOpt(AUTH_FLAG) && !authMode) {
+          throw badState("S3Guard is enabled for %s,"
+              + " but not in authoritative mode", fsUri);
+        }
+        if (commands.getOpt(NONAUTH_FLAG) && authMode) {
+          throw badState("S3Guard is enabled in authoritative mode for %s",
+              fsUri);
+        }
+      } else {
+        if (commands.getOpt(GUARDED_FLAG)) {
+          throw badState("S3Guard is not enabled for %s", fsUri);
+        }
+      }
+
+      String desiredEncryption = getCommandFormat()
+          .getOptValue(ENCRYPTION_FLAG);
+      if (StringUtils.isNotEmpty(desiredEncryption)
+          && !desiredEncryption.equalsIgnoreCase(encryption)) {
+        throw badState("Bucket %s: required encryption is %s"
+                    + " but actual encryption is %s",
+                fsUri, desiredEncryption, encryption);
+      }
+
+      out.flush();
+      return SUCCESS;
+    }
+
+    private String printOption(PrintStream out,
+        String description, String key, String defVal) {
+      String t = getFilesystem().getConf().getTrimmed(key, defVal);
+      println(out, "%s: %s=%s", description, key, t);
+      return t;
+    }
+
   }
 
   private static S3GuardTool command;
 
+  /**
+   * Convert a path to a URI, catching any {@code URISyntaxException}
+   * and converting to an invalid args exception.
+   * @param s3Path path to convert to a URI
+   * @return a URI of the path
+   * @throws ExitUtil.ExitException INVALID_ARGUMENT if the URI is invalid
+   */
+  protected static URI toUri(String s3Path) {
+    URI uri;
+    try {
+      uri = new URI(s3Path);
+    } catch (URISyntaxException e) {
+      throw invalidArgs("Not a valid fileystem path: %s", s3Path);
+    }
+    return uri;
+  }
+
   private static void printHelp() {
     if (command == null) {
-      System.err.println("Usage: hadoop " + USAGE);
-      System.err.println("\tperform S3Guard metadata store " +
+      errorln("Usage: hadoop " + USAGE);
+      errorln("\tperform S3Guard metadata store " +
           "administrative commands.");
     } else {
-      System.err.println("Usage: hadoop " + command.getUsage());
+      errorln("Usage: hadoop " + command.getUsage());
     }
+    errorln();
+    errorln(COMMON_USAGE);
+  }
+
+  private static void errorln() {
     System.err.println();
-    System.err.println(COMMON_USAGE);
+  }
+
+  private static void errorln(String x) {
+    System.err.println(x);
+  }
+
+  /**
+   * Print a formatted string followed by a newline to the output stream.
+   * @param out destination
+   * @param format format string
+   * @param args optional arguments
+   */
+  private static void println(PrintStream out, String format, Object... args) {
+    out.println(String.format(format, args));
+  }
+
+  /**
+   * Retrieve and Print store diagnostics.
+   * @param out output stream
+   * @param store store
+   * @throws IOException Failure to retrieve the data.
+   */
+  protected static void printStoreDiagnostics(PrintStream out,
+      MetadataStore store)
+      throws IOException {
+    Map<String, String> diagnostics = store.getDiagnostics();
+    out.println("Metadata Store Diagnostics:");
+    for (Map.Entry<String, String> entry : diagnostics.entrySet()) {
+      println(out, "\t%s=%s", entry.getKey(), entry.getValue());
+    }
+  }
+
+
+  /**
+   * Handle store not found by converting to an exit exception
+   * with specific error code.
+   * @param e exception
+   * @return a new exception to throw
+   */
+  protected static ExitUtil.ExitException storeNotFound(
+      FileNotFoundException e) {
+    return new ExitUtil.ExitException(
+        E_NOT_FOUND, e.toString(), e);
+  }
+
+  /**
+   * Build the exception to raise on invalid arguments.
+   * @param format string format
+   * @param args optional arguments for the string
+   * @return a new exception to throw
+   */
+  protected static ExitUtil.ExitException invalidArgs(
+      String format, Object...args) {
+    return new ExitUtil.ExitException(INVALID_ARGUMENT,
+        String.format(format, args));
+  }
+
+  /**
+   * Build the exception to raise on a bad store/bucket state.
+   * @param format string format
+   * @param args optional arguments for the string
+   * @return a new exception to throw
+   */
+  protected static ExitUtil.ExitException badState(
+      String format, Object...args) {
+    return new ExitUtil.ExitException(E_BAD_STATE,
+        String.format(format, args));
   }
 
   /**
    * Execute the command with the given arguments.
    *
-   * @param args command specific arguments.
    * @param conf Hadoop configuration.
+   * @param args command specific arguments.
    * @return exit code.
    * @throws Exception on I/O errors.
    */
-  public static int run(String[] args, Configuration conf) throws
+  public static int run(Configuration conf, String...args) throws
       Exception {
     /* ToolRunner.run does this too, but we must do it before looking at
     subCommand or instantiating the cmd object below */
@@ -878,9 +1184,10 @@ public abstract class S3GuardTool extends Configured implements Tool {
         .getRemainingArgs();
     if (otherArgs.length == 0) {
       printHelp();
-      return INVALID_ARGUMENT;
+      throw new ExitUtil.ExitException(E_USAGE, "No arguments provided");
     }
     final String subCommand = otherArgs[0];
+    LOG.debug("Executing command {}", subCommand);
     switch (subCommand) {
     case Init.NAME:
       command = new Init(conf);
@@ -891,15 +1198,22 @@ public abstract class S3GuardTool extends Configured implements Tool {
     case Import.NAME:
       command = new Import(conf);
       break;
+    case BucketInfo.NAME:
+      command = new BucketInfo(conf);
+      break;
     case Diff.NAME:
       command = new Diff(conf);
       break;
     case Prune.NAME:
       command = new Prune(conf);
       break;
+    case SetCapacity.NAME:
+      command = new SetCapacity(conf);
+      break;
     default:
       printHelp();
-      return INVALID_ARGUMENT;
+      throw new ExitUtil.ExitException(E_USAGE,
+          "Unknown command " + subCommand);
     }
     return ToolRunner.run(conf, command, otherArgs);
   }
@@ -910,15 +1224,22 @@ public abstract class S3GuardTool extends Configured implements Tool {
    */
   public static void main(String[] args) {
     try {
-      int ret = run(args, new Configuration());
-      System.exit(ret);
+      int ret = run(new Configuration(), args);
+      exit(ret, "");
     } catch (CommandFormat.UnknownOptionException e) {
-      System.err.println(e.getMessage());
+      errorln(e.getMessage());
       printHelp();
-      System.exit(INVALID_ARGUMENT);
+      exit(E_USAGE, e.getMessage());
+    } catch (ExitUtil.ExitException e) {
+      // explicitly raised exit code
+      exit(e.getExitCode(), e.toString());
     } catch (Throwable e) {
       e.printStackTrace(System.err);
-      System.exit(ERROR);
+      exit(ERROR, e.toString());
     }
   }
+
+  protected static void exit(int status, String text) {
+    ExitUtil.terminate(status, text);
+  }
 }

+ 157 - 3
hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3guard.md

@@ -387,7 +387,7 @@ Example 2
 hadoop s3guard init -meta dynamodb://ireland-team -region eu-west-1
 ```
 
-Creates a table "ireland-team" in the same region "s3-eu-west-1.amazonaws.com"
+Creates a table "ireland-team" in the region "eu-west-1.amazonaws.com"
 
 
 ### Import a bucket: `s3guard import`
@@ -421,6 +421,98 @@ Example
 hadoop s3guard diff s3a://ireland-1
 ```
 
+### Display information about a bucket, `s3guard bucket-info`
+
+Prints and optionally checks the s3guard and encryption status of a bucket.
+
+```bash
+hadoop s3guard bucket-info [ -guarded ] [-unguarded] [-auth] [-nonauth] [-encryption ENCRYPTION] s3a://BUCKET
+```
+
+Options
+
+| argument | meaning |
+|-----------|-------------|
+| `-guarded` | Require S3Guard to be enabled |
+| `-unguarded` | Require S3Guard to be disabled |
+| `-auth` | Require the S3Guard mode to be "authoritative" |
+| `-nonauth` | Require the S3Guard mode to be "non-authoritative" |
+| `-encryption <type>` | Require a specific server-side encryption algorithm  |
+
+The server side encryption options are not directly related to S3Guard, but
+it is often convenient to check them at the same time.
+
+Example
+
+```bash
+hadoop s3guard bucket-info -guarded s3a://ireland-1
+```
+
+List the details of bucket `s3a://ireland-1`, mandating that it must have S3Guard enabled
+
+```
+Filesystem s3a://ireland-1
+Location: eu-west-1
+Filesystem s3a://ireland-1 is using S3Guard with store DynamoDBMetadataStore{region=eu-west-1, tableName=ireland-1}
+Authoritative S3Guard: fs.s3a.metadatastore.authoritative=false
+Metadata Store Diagnostics:
+  ARN=arn:aws:dynamodb:eu-west-1:00000000:table/ireland-1
+  description=S3Guard metadata store in DynamoDB
+  name=ireland-1
+  read-capacity=20
+  region=eu-west-1
+  retryPolicy=ExponentialBackoffRetry(maxRetries=9, sleepTime=100 MILLISECONDS)
+  size=12812
+  status=ACTIVE
+  table={AttributeDefinitions: [{AttributeName: child,AttributeType: S},
+    {AttributeName: parent,AttributeType: S}],TableName: ireland-1,
+    KeySchema: [{AttributeName: parent,KeyType: HASH}, {AttributeName: child,KeyType: RANGE}],
+    TableStatus: ACTIVE,
+    CreationDateTime: Fri Aug 25 19:07:25 BST 2017,
+    ProvisionedThroughput: {LastIncreaseDateTime: Tue Aug 29 11:45:18 BST 2017,
+    LastDecreaseDateTime: Wed Aug 30 15:37:51 BST 2017,
+    NumberOfDecreasesToday: 1,
+    ReadCapacityUnits: 20,WriteCapacityUnits: 20},
+    TableSizeBytes: 12812,ItemCount: 91,
+    TableArn: arn:aws:dynamodb:eu-west-1:00000000:table/ireland-1,}
+  write-capacity=20
+
+S3A Client
+  Endpoint: fs.s3a.endpoint=s3-eu-west-1.amazonaws.com
+  Encryption: fs.s3a.server-side-encryption-algorithm=none
+  Input seek policy: fs.s3a.experimental.input.fadvise=normal
+```
+
+This listing includes all the information about the table supplied from
+
+```bash
+hadoop s3guard bucket-info -unguarded -encryption none s3a://landsat-pds
+```
+
+List the S3Guard status of clients of the public `landsat-pds` bucket,
+and verifies that the data is neither tracked with S3Guard nor encrypted.
+
+
+```
+Filesystem s3a://landsat-pdsLocation: us-west-2
+Filesystem s3a://landsat-pds is not using S3Guard
+Endpoint: fs.s3a.endpoints3.amazonaws.com
+Encryption: fs.s3a.server-side-encryption-algorithm=none
+Input seek policy: fs.s3a.experimental.input.fadvise=normal
+```
+
+Note that other clients may have a S3Guard table set up to store metadata
+on this bucket; the checks are all done from the perspective of the configuration
+setttings of the current client.
+
+```bash
+hadoop s3guard bucket-info -guarded -auth s3a://landsat-pds
+```
+
+Require the bucket to be using S3Guard in authoritative mode. This will normally
+fail against this specific bucket.
+
+
 ### Delete a table: `s3guard destroy`
 
 
@@ -449,7 +541,6 @@ hadoop s3guard destroy -meta dynamodb://ireland-team -region eu-west-1
 ```
 
 
-
 ### Clean up a table, `s3guard prune`
 
 Delete all file entries in the MetadataStore table whose object "modification
@@ -460,7 +551,7 @@ hadoop s3guard prune [-days DAYS] [-hours HOURS] [-minutes MINUTES]
     [-seconds SECONDS] [-m URI] ( -region REGION | s3a://BUCKET )
 ```
 
-A time value must be supplied.
+A time value of hours, minutes and/or seconds must be supplied.
 
 1. This does not delete the entries in the bucket itself.
 1. The modification time is effectively the creation time of the objects
@@ -485,6 +576,63 @@ Delete all entries more than 90 minutes old from the table "ireland-team" in
 the region "eu-west-1".
 
 
+### Tune the IO capacity of the DynamoDB Table, `s3guard set-capacity`
+
+Alter the read and/or write capacity of a s3guard table.
+
+```bash
+hadoop s3guard set-capacity [--read UNIT] [--write UNIT] ( -region REGION | s3a://BUCKET )
+```
+
+The `--read` and `--write` units are those of `s3guard init`.
+
+
+Example
+
+```
+hadoop s3guard set-capacity  -read 20 -write 20 s3a://ireland-1
+```
+
+Set the capacity of the table used by bucket `s3a://ireland-1` to 20 read
+and 20 write. (This is a low number, incidentally)
+
+```
+2017-08-30 16:21:26,343 [main] INFO  s3guard.S3GuardTool (S3GuardTool.java:initMetadataStore(229)) - Metadata store DynamoDBMetadataStore{region=eu-west-1, tableName=ireland-1} is initialized.
+2017-08-30 16:21:26,344 [main] INFO  s3guard.DynamoDBMetadataStore (DynamoDBMetadataStore.java:updateParameters(1084)) - Current table capacity is read: 25, write: 25
+2017-08-30 16:21:26,344 [main] INFO  s3guard.DynamoDBMetadataStore (DynamoDBMetadataStore.java:updateParameters(1086)) - Changing capacity of table to read: 20, write: 20
+Metadata Store Diagnostics:
+  ARN=arn:aws:dynamodb:eu-west-1:00000000000:table/ireland-1
+  description=S3Guard metadata store in DynamoDB
+  name=ireland-1
+  read-capacity=25
+  region=eu-west-1
+  retryPolicy=ExponentialBackoffRetry(maxRetries=9, sleepTime=100 MILLISECONDS)
+  size=12812
+  status=UPDATING
+  table={ ... }
+  write-capacity=25
+```
+
+After the update, the table status changes to `UPDATING`; this is a sign that
+the capacity has been changed
+
+Repeating the same command will not change the capacity, as both read and
+write values match that already in use
+
+```
+2017-08-30 16:24:35,337 [main] INFO  s3guard.DynamoDBMetadataStore (DynamoDBMetadataStore.java:updateParameters(1090)) - Table capacity unchanged at read: 20, write: 20
+Metadata Store Diagnostics:
+  ARN=arn:aws:dynamodb:eu-west-1:00000000000:table/ireland-1
+  description=S3Guard metadata store in DynamoDB
+  name=ireland-1
+  read-capacity=20
+  region=eu-west-1
+  retryPolicy=ExponentialBackoffRetry(maxRetries=9, sleepTime=100 MILLISECONDS)
+  size=12812
+  status=ACTIVE
+  table={ ... }
+  write-capacity=20
+```
 
 ## Debugging and Error Handling
 
@@ -607,6 +755,12 @@ or the configuration is preventing S3Guard from finding the table.
 region as the bucket being used.
 1. Create the table if necessary.
 
+### Error `"The level of configured provisioned throughput for the table was exceeded"`
+
+The IO load of clients of the (shared) DynamoDB table was exceeded.
+
+Currently S3Guard doesn't do any throttling and retries here; the way to address
+this is to increase capacity via the AWS console or the `set-capacity` command.
 
 ## Other Topis
 

+ 10 - 0
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java

@@ -558,6 +558,16 @@ public class ITestS3AConfiguration {
     assertOptionEquals(updated, USER_AGENT_PREFIX, "UA-c");
   }
 
+  @Test
+  public void testClearBucketOption() throws Throwable {
+    Configuration config = new Configuration();
+    config.set(USER_AGENT_PREFIX, "base");
+    setBucketOption(config, "bucket", USER_AGENT_PREFIX, "overridden");
+    clearBucketOption(config, "bucket", USER_AGENT_PREFIX);
+    Configuration updated = propagateBucketOptions(config, "c");
+    assertOptionEquals(updated, USER_AGENT_PREFIX, "base");
+  }
+
   @Test
   public void testBucketConfigurationSkipsUnmodifiable() throws Throwable {
     Configuration config = new Configuration(false);

+ 4 - 4
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardListConsistency.java

@@ -435,16 +435,16 @@ public class ITestS3GuardListConsistency extends AbstractS3ATestBase {
     // files to list are delaying visibility
     if (!recursive) {
       // in this case only the top level files are listed
+      verifyFileIsListed(listedFiles, baseTestDir, fileNames);
       assertEquals("Unexpected number of files returned by listFiles() call",
           normalFileNum + delayedFileNum, listedFiles.size());
-      verifyFileIsListed(listedFiles, baseTestDir, fileNames);
     } else {
-      assertEquals("Unexpected number of files returned by listFiles() call",
-          filesAndEmptyDirectories,
-          listedFiles.size());
       for (Path dir : testDirs) {
         verifyFileIsListed(listedFiles, dir, fileNames);
       }
+      assertEquals("Unexpected number of files returned by listFiles() call",
+          filesAndEmptyDirectories,
+          listedFiles.size());
     }
   }
 

+ 22 - 15
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3GuardWriteBack.java

@@ -22,15 +22,17 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
 import org.apache.hadoop.fs.s3a.s3guard.DirListingMetadata;
 import org.junit.Assume;
 import org.junit.Test;
 
-import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.net.URI;
 import java.util.Arrays;
 
+import static org.apache.hadoop.fs.s3a.Constants.*;
+
 /**
  * Test cases that validate S3Guard's behavior for writing things like
  * directory listings back to the MetadataStore.
@@ -66,7 +68,7 @@ public class ITestS3GuardWriteBack extends AbstractS3ATestBase {
     noS3Guard.mkdirs(new Path(directory, "OnS3"));
     // Create a directory on both S3 and metadata store
     Path p = new Path(directory, "OnS3AndMS");
-    assertPathDoesntExist(noWriteBack, p);
+    ContractTestUtils.assertPathDoesNotExist(noWriteBack, "path", p);
     noWriteBack.mkdirs(p);
 
     FileStatus[] fsResults;
@@ -87,7 +89,7 @@ public class ITestS3GuardWriteBack extends AbstractS3ATestBase {
 
     // FS should return both (and will write it back)
     fsResults = yesWriteBack.listStatus(directory);
-    assertEquals("Filesystem enabled S3Guard with write back should have "
+    assertEquals("Filesystem enabled S3Guard with write back should have"
             + " both /OnS3 and /OnS3AndMS: " + Arrays.toString(fsResults),
         2, fsResults.length);
 
@@ -104,7 +106,12 @@ public class ITestS3GuardWriteBack extends AbstractS3ATestBase {
         new Path(directory, "OnS3"));
   }
 
-  /** Create a separate S3AFileSystem instance for testing. */
+  /**
+   * Create a separate S3AFileSystem instance for testing.
+   * There's a bit of complexity as it forces pushes up s3guard options from
+   * the base values to the per-bucket options. This stops explicit bucket
+   * settings in test XML configs from unintentionally breaking tests.
+   */
   private S3AFileSystem createTestFS(URI fsURI, boolean disableS3Guard,
       boolean authoritativeMeta) throws IOException {
     Configuration conf;
@@ -112,12 +119,22 @@ public class ITestS3GuardWriteBack extends AbstractS3ATestBase {
     // Create a FileSystem that is S3-backed only
     conf = createConfiguration();
     S3ATestUtils.disableFilesystemCaching(conf);
+    String host = fsURI.getHost();
     if (disableS3Guard) {
       conf.set(Constants.S3_METADATA_STORE_IMPL,
           Constants.S3GUARD_METASTORE_NULL);
+      S3AUtils.setBucketOption(conf, host,
+          S3_METADATA_STORE_IMPL,
+          S3GUARD_METASTORE_NULL);
     } else {
       S3ATestUtils.maybeEnableS3Guard(conf);
-      conf.setBoolean(Constants.METADATASTORE_AUTHORITATIVE, authoritativeMeta);
+      conf.setBoolean(METADATASTORE_AUTHORITATIVE, authoritativeMeta);
+      S3AUtils.setBucketOption(conf, host,
+          METADATASTORE_AUTHORITATIVE,
+          Boolean.toString(authoritativeMeta));
+      S3AUtils.setBucketOption(conf, host,
+          S3_METADATA_STORE_IMPL,
+          conf.get(S3_METADATA_STORE_IMPL));
     }
     FileSystem fs = FileSystem.get(fsURI, conf);
     return asS3AFS(fs);
@@ -128,14 +145,4 @@ public class ITestS3GuardWriteBack extends AbstractS3ATestBase {
     return (S3AFileSystem)fs;
   }
 
-  private static void assertPathDoesntExist(FileSystem fs, Path p)
-      throws IOException {
-    try {
-      FileStatus s = fs.getFileStatus(p);
-    } catch (FileNotFoundException e) {
-      return;
-    }
-    fail("Path should not exist: " + p);
-  }
-
 }

+ 0 - 18
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java

@@ -696,24 +696,6 @@ public final class S3ATestUtils {
     assertEquals("Permission: " + details, permission, status.getPermission());
   }
 
-  /**
-   * Set a bucket specific property to a particular value.
-   * If the generic key passed in has an {@code fs.s3a. prefix},
-   * that's stripped off, so that when the the bucket properties are propagated
-   * down to the generic values, that value gets copied down.
-   * @param conf configuration to set
-   * @param bucket bucket name
-   * @param genericKey key; can start with "fs.s3a."
-   * @param value value to set
-   */
-  public static void setBucketOption(Configuration conf, String bucket,
-      String genericKey, String value) {
-    final String baseKey = genericKey.startsWith(FS_S3A_PREFIX) ?
-        genericKey.substring(FS_S3A_PREFIX.length())
-        : genericKey;
-    conf.set(FS_S3A_BUCKET_PREFIX + bucket + '.' + baseKey, value);
-  }
-
   /**
    * Assert that a configuration option matches the expected value.
    * @param conf configuration

+ 134 - 4
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java

@@ -18,9 +18,15 @@
 
 package org.apache.hadoop.fs.s3a.s3guard;
 
+import java.io.ByteArrayOutputStream;
+import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.io.PrintStream;
+import java.util.Collection;
+import java.util.concurrent.Callable;
 import java.util.concurrent.TimeUnit;
 
+import org.junit.Assume;
 import org.junit.Test;
 
 import org.apache.hadoop.conf.Configuration;
@@ -31,8 +37,11 @@ import org.apache.hadoop.fs.s3a.Constants;
 import org.apache.hadoop.fs.s3a.S3AFileStatus;
 import org.apache.hadoop.fs.s3a.S3ATestUtils;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.util.ExitUtil;
+import org.apache.hadoop.util.StringUtils;
 
 import static org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.SUCCESS;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 
 /**
  * Common functionality for S3GuardTool test cases.
@@ -40,6 +49,9 @@ import static org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.SUCCESS;
 public abstract class AbstractS3GuardToolTestBase extends AbstractS3ATestBase {
 
   protected static final String OWNER = "hdfs";
+  protected static final String DYNAMODB_TABLE = "dynamodb://ireland-team";
+  protected static final String S3A_THIS_BUCKET_DOES_NOT_EXIST
+      = "s3a://this-bucket-does-not-exist-00000000000";
 
   private MetadataStore ms;
 
@@ -57,6 +69,51 @@ public abstract class AbstractS3GuardToolTestBase extends AbstractS3ATestBase {
     assertEquals(message, SUCCESS, tool.run(args));
   }
 
+  /**
+   * Run a S3GuardTool command from a varags list.
+   * @param conf configuration
+   * @param args argument list
+   * @return the return code
+   * @throws Exception any exception
+   */
+  protected int run(Configuration conf, String... args)
+      throws Exception {
+    return S3GuardTool.run(conf, args);
+  }
+
+  /**
+   * Run a S3GuardTool command from a varags list and the
+   * configuration returned by {@code getConfiguration()}.
+   * @param args argument list
+   * @return the return code
+   * @throws Exception any exception
+   */
+  protected int run(String... args) throws Exception {
+    return S3GuardTool.run(getConfiguration(), args);
+  }
+
+  /**
+   * Run a S3GuardTool command from a varags list, catch any raised
+   * ExitException and verify the status code matches that expected.
+   * @param status expected status code of the exception
+   * @param args argument list
+   * @throws Exception any exception
+   */
+  protected void runToFailure(int status, String... args)
+      throws Exception {
+    ExitUtil.ExitException ex =
+        intercept(ExitUtil.ExitException.class,
+            new Callable<Integer>() {
+              @Override
+              public Integer call() throws Exception {
+                return run(args);
+              }
+            });
+    if (ex.status != status) {
+      throw ex;
+    }
+  }
+
   protected MetadataStore getMetadataStore() {
     return ms;
   }
@@ -134,16 +191,23 @@ public abstract class AbstractS3GuardToolTestBase extends AbstractS3ATestBase {
       Thread.sleep(TimeUnit.SECONDS.toMillis(2));
       createFile(new Path(parent, "fresh"), true, true);
 
-      assertEquals(2, ms.listChildren(parent).getListing().size());
-      expectSuccess("Prune command did not exit successfully - see output", cmd,
-          args);
-      assertEquals(1, ms.listChildren(parent).getListing().size());
+      assertMetastoreListingCount(parent, "Children count before pruning", 2);
+      exec(cmd, args);
+      assertMetastoreListingCount(parent, "Pruned children count", 1);
     } finally {
       getFileSystem().delete(parent, true);
       ms.prune(Long.MAX_VALUE);
     }
   }
 
+  private void assertMetastoreListingCount(Path parent,
+      String message,
+      int expected) throws IOException {
+    Collection<PathMetadata> listing = ms.listChildren(parent).getListing();
+    assertEquals(message +" [" + StringUtils.join(", ", listing) + "]",
+        expected, listing.size());
+  }
+
   @Test
   public void testPruneCommandCLI() throws Exception {
     String testPath = path("testPruneCommandCLI").toString();
@@ -158,4 +222,70 @@ public abstract class AbstractS3GuardToolTestBase extends AbstractS3ATestBase {
     String testPath = path("testPruneCommandConf").toString();
     testPruneCommand(getConfiguration(), "prune", testPath);
   }
+
+  @Test
+  public void testDestroyNoBucket() throws Throwable {
+    intercept(FileNotFoundException.class,
+        new Callable<Integer>() {
+          @Override
+          public Integer call() throws Exception {
+            return run(S3GuardTool.Destroy.NAME,
+                S3A_THIS_BUCKET_DOES_NOT_EXIST);
+          }
+        });
+  }
+
+  /**
+   * Get the test CSV file; assume() that it is not modified (i.e. we haven't
+   * switched to a new storage infrastructure where the bucket is no longer
+   * read only).
+   * @return test file.
+   */
+  protected String getLandsatCSVFile() {
+    String csvFile = getConfiguration()
+        .getTrimmed(KEY_CSVTEST_FILE, DEFAULT_CSVTEST_FILE);
+    Assume.assumeTrue("CSV test file is not the default",
+        DEFAULT_CSVTEST_FILE.equals(csvFile));
+    return csvFile;
+  }
+
+  /**
+   * Execute a command, returning the buffer if the command actually completes.
+   * If an exception is raised the output is logged instead.
+   * @param cmd command
+   * @param buf buffer to use for tool output (not SLF4J output)
+   * @param args argument list
+   * @throws Exception on any failure
+   */
+  public String exec(S3GuardTool cmd, String...args) throws Exception {
+    ByteArrayOutputStream buf = new ByteArrayOutputStream();
+    try {
+      exec(cmd, buf, args);
+      return buf.toString();
+    } catch (AssertionError e) {
+      throw e;
+    } catch (Exception e) {
+      LOG.error("Command {} failed: \n{}", cmd, buf);
+      throw e;
+    }
+  }
+
+  /**
+   * Execute a command, saving the output into the buffer.
+   * @param cmd command
+   * @param buf buffer to use for tool output (not SLF4J output)
+   * @param args argument list
+   * @throws Exception on any failure
+   */
+  protected void exec(S3GuardTool cmd, ByteArrayOutputStream buf, String...args)
+      throws Exception {
+    LOG.info("exec {}", (Object) args);
+    int r = 0;
+    try(PrintStream out =new PrintStream(buf)) {
+      r = cmd.run(args, out);
+      out.flush();
+    }
+    assertEquals("Command " + cmd + " failed\n"+ buf, 0, r);
+  }
+
 }

+ 164 - 3
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolDynamoDB.java

@@ -19,19 +19,28 @@
 package org.apache.hadoop.fs.s3a.s3guard;
 
 import java.io.IOException;
+import java.util.Map;
+import java.util.Objects;
 import java.util.Random;
 import java.util.concurrent.Callable;
+import java.util.concurrent.atomic.AtomicInteger;
 
 import com.amazonaws.services.dynamodbv2.document.DynamoDB;
 import com.amazonaws.services.dynamodbv2.document.Table;
 import com.amazonaws.services.dynamodbv2.model.ResourceNotFoundException;
 import org.junit.Test;
 
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.s3a.Constants;
 import org.apache.hadoop.fs.s3a.S3AFileSystem;
+import org.apache.hadoop.fs.s3a.S3AUtils;
 import org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.Destroy;
 import org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.Init;
 import org.apache.hadoop.test.LambdaTestUtils;
 
+import static org.apache.hadoop.fs.s3a.s3guard.DynamoDBMetadataStore.*;
+import static org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.*;
+
 /**
  * Test S3Guard related CLI commands against DynamoDB.
  */
@@ -78,8 +87,76 @@ public class ITestS3GuardToolDynamoDB extends AbstractS3GuardToolTestBase {
         });
   }
 
+  private static class Capacities {
+    private final long read, write;
+
+    Capacities(long read, long write) {
+      this.read = read;
+      this.write = write;
+    }
+
+    public long getRead() {
+      return read;
+    }
+
+    public long getWrite() {
+      return write;
+    }
+
+    String getReadStr() {
+      return Long.toString(read);
+    }
+
+    String getWriteStr() {
+      return Long.toString(write);
+    }
+
+    void checkEquals(String text, Capacities that) throws Exception {
+      if (!this.equals(that)) {
+        throw new Exception(text + " expected = " + this +"; actual = "+ that);
+      }
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      Capacities that = (Capacities) o;
+      return read == that.read && write == that.write;
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(read, write);
+    }
+
+    @Override
+    public String toString() {
+      final StringBuilder sb = new StringBuilder("Capacities{");
+      sb.append("read=").append(read);
+      sb.append(", write=").append(write);
+      sb.append('}');
+      return sb.toString();
+    }
+  }
+
+  private Capacities getCapacities() throws IOException {
+    Map<String, String> diagnostics = getMetadataStore().getDiagnostics();
+    return getCapacities(diagnostics);
+  }
+
+  private Capacities getCapacities(Map<String, String> diagnostics) {
+    return new Capacities(
+        Long.parseLong(diagnostics.get(DynamoDBMetadataStore.READ_CAPACITY)),
+        Long.parseLong(diagnostics.get(DynamoDBMetadataStore.WRITE_CAPACITY)));
+  }
+
   @Test
-  public void testDynamoDBInitDestroyCycle() throws Exception {
+  public void testDynamoDBInitDestroyCycle() throws Throwable {
     String testTableName = "testDynamoDBInitDestroy" + new Random().nextInt();
     String testS3Url = path(testTableName).toString();
     S3AFileSystem fs = getFileSystem();
@@ -99,11 +176,80 @@ public class ITestS3GuardToolDynamoDB extends AbstractS3GuardToolTestBase {
       assertTrue(String.format("%s does not exist", testTableName),
           exist(db, testTableName));
 
+      Configuration conf = fs.getConf();
+      String bucket = fs.getBucket();
+      // force in a new bucket
+      S3AUtils.setBucketOption(conf, bucket, Constants.S3_METADATA_STORE_IMPL,
+          Constants.S3GUARD_METASTORE_DYNAMO);
+      initCmd = new Init(conf);
+      String initOutput = exec(initCmd,
+          "init", "-meta", "dynamodb://" + testTableName, testS3Url);
+      assertTrue("No Dynamo diagnostics in output " + initOutput,
+          initOutput.contains(DESCRIPTION));
+
+      // run a bucket info command and look for
+      // confirmation that it got the output from DDB diags
+      S3GuardTool.BucketInfo infocmd = new S3GuardTool.BucketInfo(conf);
+      String info = exec(infocmd, S3GuardTool.BucketInfo.NAME,
+          "-" + S3GuardTool.BucketInfo.GUARDED_FLAG,
+          testS3Url);
+      assertTrue("No Dynamo diagnostics in output " + info,
+          info.contains(DESCRIPTION));
+
+      // get the current values to set again
+
+      // play with the set-capacity option
+      Capacities original = getCapacities();
+      String fsURI = getFileSystem().getUri().toString();
+      String capacityOut = exec(newSetCapacity(),
+          S3GuardTool.SetCapacity.NAME,
+          fsURI);
+      LOG.info("Set Capacity output=\n{}", capacityOut);
+      capacityOut = exec(newSetCapacity(),
+          S3GuardTool.SetCapacity.NAME,
+          "-" + READ_FLAG, original.getReadStr(),
+          "-" + WRITE_FLAG, original.getWriteStr(),
+          fsURI);
+      LOG.info("Set Capacity output=\n{}", capacityOut);
+
+      // that call does not change the values
+      original.checkEquals("unchanged", getCapacities());
+
+      // now update the value
+      long readCap = original.getRead();
+      long writeCap = original.getWrite();
+      long rc2 = readCap + 1;
+      long wc2 = writeCap + 1;
+      Capacities desired = new Capacities(rc2, wc2);
+      capacityOut = exec(newSetCapacity(),
+          S3GuardTool.SetCapacity.NAME,
+          "-" + READ_FLAG, Long.toString(rc2),
+          "-" + WRITE_FLAG, Long.toString(wc2),
+          fsURI);
+      LOG.info("Set Capacity output=\n{}", capacityOut);
+
+      // to avoid race conditions, spin for the state change
+      AtomicInteger c = new AtomicInteger(0);
+      LambdaTestUtils.eventually(60000,
+          new LambdaTestUtils.VoidCallable() {
+            @Override
+            public void call() throws Exception {
+                c.incrementAndGet();
+                Map<String, String> diags = getMetadataStore().getDiagnostics();
+                Capacities updated = getCapacities(diags);
+                String tableInfo = String.format("[%02d] table state: %s",
+                    c.intValue(), diags.get(STATUS));
+                LOG.info("{}; capacities {}",
+                    tableInfo, updated);
+                desired.checkEquals(tableInfo, updated);
+            }
+          },
+          new LambdaTestUtils.ProportionalRetryInterval(500, 5000));
+
       // Destroy MetadataStore
       Destroy destroyCmd = new Destroy(fs.getConf());
 
-      expectSuccess("Destroy command did not exit successfully - see output",
-          destroyCmd,
+      String destroyed = exec(destroyCmd,
           "destroy", "-meta", "dynamodb://" + testTableName, testS3Url);
       // Verify it does not exist
       assertFalse(String.format("%s still exists", testTableName),
@@ -131,4 +277,19 @@ public class ITestS3GuardToolDynamoDB extends AbstractS3GuardToolTestBase {
       }
     }
   }
+
+  private S3GuardTool newSetCapacity() {
+    S3GuardTool setCapacity = new S3GuardTool.SetCapacity(
+        getFileSystem().getConf());
+    setCapacity.setStore(getMetadataStore());
+    return setCapacity;
+  }
+
+  @Test
+  public void testDestroyUnknownTable() throws Throwable {
+    run(S3GuardTool.Destroy.NAME,
+        "-region", "us-west-2",
+        "-meta", DYNAMODB_TABLE);
+  }
+
 }

+ 129 - 13
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolLocal.java

@@ -21,11 +21,12 @@ package org.apache.hadoop.fs.s3a.s3guard;
 import java.io.BufferedReader;
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InputStreamReader;
-import java.io.PrintStream;
 import java.util.HashSet;
 import java.util.Set;
+import java.util.concurrent.Callable;
 
 import org.junit.Test;
 
@@ -34,13 +35,16 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.s3a.S3AFileSystem;
 import org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.Diff;
 
-import static org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.SUCCESS;
+import static org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.*;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 
 /**
  * Test S3Guard related CLI commands against a LocalMetadataStore.
  */
 public class ITestS3GuardToolLocal extends AbstractS3GuardToolTestBase {
 
+  private static final String LOCAL_METADATA = "local://metadata";
+
   @Override
   protected MetadataStore newMetadataStore() {
     return new LocalMetadataStore();
@@ -65,10 +69,7 @@ public class ITestS3GuardToolLocal extends AbstractS3GuardToolTestBase {
 
     S3GuardTool.Import cmd = new S3GuardTool.Import(fs.getConf());
     cmd.setStore(ms);
-
-    expectSuccess("Import command did not exit successfully - see output",
-        cmd,
-        "import", parent.toString());
+    exec(cmd, "import", parent.toString());
 
     DirListingMetadata children =
         ms.listChildren(dir);
@@ -80,7 +81,7 @@ public class ITestS3GuardToolLocal extends AbstractS3GuardToolTestBase {
   }
 
   @Test
-  public void testDiffCommand() throws IOException {
+  public void testDiffCommand() throws Exception {
     S3AFileSystem fs = getFileSystem();
     MetadataStore ms = getMetadataStore();
     Set<Path> filesOnS3 = new HashSet<>(); // files on S3.
@@ -108,13 +109,10 @@ public class ITestS3GuardToolLocal extends AbstractS3GuardToolTestBase {
     }
 
     ByteArrayOutputStream buf = new ByteArrayOutputStream();
-    PrintStream out = new PrintStream(buf);
     Diff cmd = new Diff(fs.getConf());
     cmd.setStore(ms);
-    assertEquals("Diff command did not exit successfully - see output", SUCCESS,
-        cmd.run(new String[]{"diff", "-meta", "local://metadata",
-            testPath.toString()}, out));
-    out.close();
+    exec(cmd, buf, "diff", "-meta", LOCAL_METADATA,
+            testPath.toString());
 
     Set<Path> actualOnS3 = new HashSet<>();
     Set<Path> actualOnMS = new HashSet<>();
@@ -140,10 +138,128 @@ public class ITestS3GuardToolLocal extends AbstractS3GuardToolTestBase {
         }
       }
     }
-    String actualOut = out.toString();
+    String actualOut = buf.toString();
     assertEquals("Mismatched metadata store outputs: " + actualOut,
         filesOnMS, actualOnMS);
     assertEquals("Mismatched s3 outputs: " + actualOut, filesOnS3, actualOnS3);
     assertFalse("Diff contained duplicates", duplicates);
   }
+
+  @Test
+  public void testDestroyBucketExistsButNoTable() throws Throwable {
+    run(Destroy.NAME,
+        "-meta", LOCAL_METADATA,
+        getLandsatCSVFile());
+  }
+
+  @Test
+  public void testImportNoFilesystem() throws Throwable {
+    final Import importer =
+        new S3GuardTool.Import(getConfiguration());
+    importer.setStore(getMetadataStore());
+    intercept(IOException.class,
+        new Callable<Integer>() {
+          @Override
+          public Integer call() throws Exception {
+            return importer.run(
+                new String[]{
+                    "import",
+                    "-meta", LOCAL_METADATA,
+                    S3A_THIS_BUCKET_DOES_NOT_EXIST
+                });
+          }
+        });
+  }
+
+  @Test
+  public void testInfoBucketAndRegionNoFS() throws Throwable {
+    intercept(FileNotFoundException.class,
+        new Callable<Integer>() {
+          @Override
+          public Integer call() throws Exception {
+            return run(BucketInfo.NAME, "-meta",
+                LOCAL_METADATA, "-region",
+                "any-region", S3A_THIS_BUCKET_DOES_NOT_EXIST);
+          }
+        });
+  }
+
+  @Test
+  public void testInitNegativeRead() throws Throwable {
+    runToFailure(INVALID_ARGUMENT,
+        Init.NAME, "-meta", LOCAL_METADATA, "-region",
+        "eu-west-1",
+        READ_FLAG, "-10");
+  }
+
+  @Test
+  public void testInit() throws Throwable {
+    run(Init.NAME,
+        "-meta", LOCAL_METADATA,
+        "-region", "us-west-1");
+  }
+
+  @Test
+  public void testInitTwice() throws Throwable {
+    run(Init.NAME,
+        "-meta", LOCAL_METADATA,
+        "-region", "us-west-1");
+    run(Init.NAME,
+        "-meta", LOCAL_METADATA,
+        "-region", "us-west-1");
+  }
+
+  @Test
+  public void testLandsatBucketUnguarded() throws Throwable {
+    run(BucketInfo.NAME,
+        "-" + BucketInfo.UNGUARDED_FLAG,
+        getLandsatCSVFile());
+  }
+
+  @Test
+  public void testLandsatBucketRequireGuarded() throws Throwable {
+    runToFailure(E_BAD_STATE,
+        BucketInfo.NAME,
+        "-" + BucketInfo.GUARDED_FLAG,
+        ITestS3GuardToolLocal.this.getLandsatCSVFile());
+  }
+
+  @Test
+  public void testLandsatBucketRequireUnencrypted() throws Throwable {
+    run(BucketInfo.NAME,
+        "-" + BucketInfo.ENCRYPTION_FLAG, "none",
+        getLandsatCSVFile());
+  }
+
+  @Test
+  public void testLandsatBucketRequireEncrypted() throws Throwable {
+    runToFailure(E_BAD_STATE,
+        BucketInfo.NAME,
+        "-" + BucketInfo.ENCRYPTION_FLAG,
+        "AES256", ITestS3GuardToolLocal.this.getLandsatCSVFile());
+  }
+
+  @Test
+  public void testStoreInfo() throws Throwable {
+    S3GuardTool.BucketInfo cmd = new S3GuardTool.BucketInfo(
+        getFileSystem().getConf());
+    cmd.setStore(getMetadataStore());
+    String output = exec(cmd, cmd.getName(),
+        "-" + S3GuardTool.BucketInfo.GUARDED_FLAG,
+        getFileSystem().getUri().toString());
+    LOG.info("Exec output=\n{}", output);
+  }
+
+  @Test
+  public void testSetCapacity() throws Throwable {
+    S3GuardTool cmd = new S3GuardTool.SetCapacity(getFileSystem().getConf());
+    cmd.setStore(getMetadataStore());
+    String output = exec(cmd, cmd.getName(),
+        "-" + READ_FLAG, "100",
+        "-" + WRITE_FLAG, "100",
+        getFileSystem().getUri().toString());
+    LOG.info("Exec output=\n{}", output);
+  }
+
+
 }

+ 121 - 0
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestS3GuardCLI.java

@@ -0,0 +1,121 @@
+/*
+ * 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.fs.s3a.s3guard;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.s3a.S3ATestConstants;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.apache.hadoop.util.ExitUtil;
+
+import static org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.*;
+
+/**
+ * Test the S3Guard CLI entry point.
+ */
+public class TestS3GuardCLI extends Assert {
+
+  /**
+   * Run a S3GuardTool command from a varags list.
+   * @param args argument list
+   * @return the return code
+   * @throws Exception any exception
+   */
+  protected int run(String... args)
+      throws Exception {
+    Configuration conf = new Configuration(false);
+    return S3GuardTool.run(conf, args);
+  }
+
+  /**
+   * Run a S3GuardTool command from a varags list, catch any raised
+   * ExitException and verify the status code matches that expected.
+   * @param status expected status code of an exception
+   * @param args argument list
+   * @throws Exception any exception
+   */
+  protected void runToFailure(int status, String... args)
+      throws Exception {
+    ExitUtil.ExitException ex =
+        LambdaTestUtils.intercept(ExitUtil.ExitException.class,
+            () -> run(args));
+    if (ex.status != status) {
+      throw ex;
+    }
+  }
+
+  @Test
+  public void testInfoNoArgs() throws Throwable {
+    runToFailure(INVALID_ARGUMENT, BucketInfo.NAME);
+  }
+
+  @Test
+  public void testInfoWrongFilesystem() throws Throwable {
+    runToFailure(INVALID_ARGUMENT,
+        BucketInfo.NAME, "file://");
+  }
+
+  @Test
+  public void testNoCommand() throws Throwable {
+    runToFailure(E_USAGE);
+  }
+
+  @Test
+  public void testUnknownCommand() throws Throwable {
+    runToFailure(E_USAGE, "unknown");
+  }
+
+  @Test
+  public void testPruneNoArgs() throws Throwable {
+    runToFailure(INVALID_ARGUMENT, Prune.NAME);
+  }
+
+  @Test
+  public void testDiffNoArgs() throws Throwable {
+    runToFailure(INVALID_ARGUMENT, Diff.NAME);
+  }
+
+  @Test
+  public void testImportNoArgs() throws Throwable {
+    runToFailure(INVALID_ARGUMENT, Import.NAME);
+  }
+
+  @Test
+  public void testDestroyNoArgs() throws Throwable {
+    runToFailure(INVALID_ARGUMENT, Destroy.NAME);
+  }
+
+  @Test
+  public void testDestroyUnknownTableNoRegion() throws Throwable {
+    runToFailure(INVALID_ARGUMENT, Destroy.NAME,
+        "-meta", "dynamodb://ireland-team");
+  }
+
+  @Test
+  public void testInitBucketAndRegion() throws Throwable {
+    runToFailure(INVALID_ARGUMENT, Init.NAME,
+        "-meta", "dynamodb://ireland-team",
+        "-region", "eu-west-1",
+        S3ATestConstants.DEFAULT_CSVTEST_FILE
+    );
+  }
+
+}

+ 1 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/AMRMClientAsyncImpl.java

@@ -142,11 +142,7 @@ extends AMRMClientAsync<T> {
     handlerThread.interrupt();
     super.serviceStop();
   }
-  
-  public void setHeartbeatInterval(int interval) {
-    heartbeatIntervalMs.set(interval);
-  }
-  
+
   public List<? extends Collection<T>> getMatchingRequests(
                                                    Priority priority, 
                                                    String resourceName, 

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

@@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.server.nodemanager.containermanager;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.protobuf.ByteString;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.UpdateContainerTokenEvent;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
@@ -144,7 +145,6 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.Contai
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler.ContainerScheduler;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler.ContainerSchedulerEventType;
 
-import org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler.UpdateContainerSchedulerEvent;
 import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService;
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService.RecoveredApplicationsState;
@@ -1251,29 +1251,6 @@ public class ContainerManagerImpl extends CompositeService implements
           + " [" + containerTokenIdentifier.getVersion() + "]");
     }
 
-    // Check container state
-    org.apache.hadoop.yarn.server.nodemanager.
-        containermanager.container.ContainerState currentState =
-        container.getContainerState();
-    EnumSet<org.apache.hadoop.yarn.server.nodemanager.containermanager
-        .container.ContainerState> allowedStates = EnumSet.of(
-        org.apache.hadoop.yarn.server.nodemanager.containermanager.container
-            .ContainerState.RUNNING,
-        org.apache.hadoop.yarn.server.nodemanager.containermanager.container
-            .ContainerState.SCHEDULED,
-        org.apache.hadoop.yarn.server.nodemanager.containermanager.container
-            .ContainerState.LOCALIZING,
-        org.apache.hadoop.yarn.server.nodemanager.containermanager.container
-            .ContainerState.REINITIALIZING,
-        org.apache.hadoop.yarn.server.nodemanager.containermanager.container
-            .ContainerState.RELAUNCHING);
-    if (!allowedStates.contains(currentState)) {
-      throw RPCUtil.getRemoteException("Container " + containerId.toString()
-          + " is in " + currentState.name() + " state."
-          + " Resource can only be changed when a container is in"
-          + " RUNNING or SCHEDULED state");
-    }
-
     // Check validity of the target resource.
     Resource currentResource = container.getResource();
     ExecutionType currentExecType =
@@ -1313,11 +1290,11 @@ public class ContainerManagerImpl extends CompositeService implements
     this.readLock.lock();
     try {
       if (!serviceStopped) {
-        // Dispatch message to ContainerScheduler to actually
+        // Dispatch message to Container to actually
         // make the change.
-        dispatcher.getEventHandler().handle(new UpdateContainerSchedulerEvent(
-            container, containerTokenIdentifier, isResourceChange,
-            isExecTypeUpdate, isIncrease));
+        dispatcher.getEventHandler().handle(new UpdateContainerTokenEvent(
+            container.getContainerId(), containerTokenIdentifier,
+            isResourceChange, isExecTypeUpdate, isIncrease));
       } else {
         throw new YarnException(
             "Unable to change container resource as the NodeManager is "
@@ -1816,10 +1793,14 @@ public class ContainerManagerImpl extends CompositeService implements
     if (container == null) {
       throw new YarnException("Specified " + containerId + " does not exist!");
     }
-    if (!container.isRunning() || container.isReInitializing()) {
+    if (!container.isRunning() || container.isReInitializing()
+        || container.getContainerTokenIdentifier().getExecutionType()
+        == ExecutionType.OPPORTUNISTIC) {
       throw new YarnException("Cannot perform " + op + " on [" + containerId
           + "]. Current state is [" + container.getContainerState() + ", " +
-          "isReInitializing=" + container.isReInitializing() + "].");
+          "isReInitializing=" + container.isReInitializing() + "]. Container"
+          + " Execution Type is [" + container.getContainerTokenIdentifier()
+          .getExecutionType() + "].");
     }
     return container;
   }

+ 5 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerEventType.java

@@ -29,6 +29,7 @@ public enum ContainerEventType {
   ROLLBACK_REINIT,
   PAUSE_CONTAINER,
   RESUME_CONTAINER,
+  UPDATE_CONTAINER_TOKEN,
 
   // DownloadManager
   CONTAINER_INITED,
@@ -42,5 +43,8 @@ public enum ContainerEventType {
   CONTAINER_EXITED_WITH_FAILURE,
   CONTAINER_KILLED_ON_REQUEST,
   CONTAINER_PAUSED,
-  CONTAINER_RESUMED
+  CONTAINER_RESUMED,
+
+  // Producer: ContainerScheduler
+  CONTAINER_TOKEN_UPDATED
 }

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

@@ -33,6 +33,8 @@ import java.util.Set;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler.UpdateContainerSchedulerEvent;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -308,8 +310,8 @@ public class ContainerImpl implements Container {
         UPDATE_DIAGNOSTICS_TRANSITION)
     .addTransition(ContainerState.NEW, ContainerState.DONE,
         ContainerEventType.KILL_CONTAINER, new KillOnNewTransition())
-    .addTransition(ContainerState.NEW, ContainerState.DONE,
-            ContainerEventType.PAUSE_CONTAINER, new KillOnPauseTransition())
+    .addTransition(ContainerState.NEW, ContainerState.NEW,
+        ContainerEventType.UPDATE_CONTAINER_TOKEN, new UpdateTransition())
 
     // From LOCALIZING State
     .addTransition(ContainerState.LOCALIZING,
@@ -325,8 +327,9 @@ public class ContainerImpl implements Container {
     .addTransition(ContainerState.LOCALIZING, ContainerState.KILLING,
         ContainerEventType.KILL_CONTAINER,
         new KillBeforeRunningTransition())
-    .addTransition(ContainerState.LOCALIZING, ContainerState.KILLING,
-        ContainerEventType.PAUSE_CONTAINER, new KillOnPauseTransition())
+    .addTransition(ContainerState.LOCALIZING, ContainerState.LOCALIZING,
+        ContainerEventType.UPDATE_CONTAINER_TOKEN, new UpdateTransition())
+
 
     // From LOCALIZATION_FAILED State
     .addTransition(ContainerState.LOCALIZATION_FAILED,
@@ -351,6 +354,9 @@ public class ContainerImpl implements Container {
     .addTransition(ContainerState.LOCALIZATION_FAILED,
         ContainerState.LOCALIZATION_FAILED,
         ContainerEventType.RESOURCE_FAILED)
+    .addTransition(ContainerState.LOCALIZATION_FAILED,
+        ContainerState.LOCALIZATION_FAILED,
+        ContainerEventType.UPDATE_CONTAINER_TOKEN, new UpdateTransition())
 
     // From SCHEDULED State
     .addTransition(ContainerState.SCHEDULED, ContainerState.RUNNING,
@@ -364,6 +370,9 @@ public class ContainerImpl implements Container {
     .addTransition(ContainerState.SCHEDULED, ContainerState.KILLING,
         ContainerEventType.KILL_CONTAINER,
         new KillBeforeRunningTransition())
+    .addTransition(ContainerState.SCHEDULED, ContainerState.SCHEDULED,
+        ContainerEventType.UPDATE_CONTAINER_TOKEN,
+        new NotifyContainerSchedulerOfUpdateTransition())
 
     // From RUNNING State
     .addTransition(ContainerState.RUNNING,
@@ -376,10 +385,16 @@ public class ContainerImpl implements Container {
             ContainerState.EXITED_WITH_FAILURE),
         ContainerEventType.CONTAINER_EXITED_WITH_FAILURE,
         new RetryFailureTransition())
-    .addTransition(ContainerState.RUNNING, ContainerState.REINITIALIZING,
+    .addTransition(ContainerState.RUNNING,
+        EnumSet.of(ContainerState.RUNNING,
+            ContainerState.REINITIALIZING,
+            ContainerState.REINITIALIZING_AWAITING_KILL),
         ContainerEventType.REINITIALIZE_CONTAINER,
         new ReInitializeContainerTransition())
-    .addTransition(ContainerState.RUNNING, ContainerState.REINITIALIZING,
+    .addTransition(ContainerState.RUNNING,
+        EnumSet.of(ContainerState.RUNNING,
+            ContainerState.REINITIALIZING,
+            ContainerState.REINITIALIZING_AWAITING_KILL),
         ContainerEventType.ROLLBACK_REINIT,
         new RollbackContainerTransition())
     .addTransition(ContainerState.RUNNING, ContainerState.RUNNING,
@@ -398,9 +413,16 @@ public class ContainerImpl implements Container {
         ContainerEventType.CONTAINER_KILLED_ON_REQUEST,
         new KilledExternallyTransition())
     .addTransition(ContainerState.RUNNING, ContainerState.PAUSING,
-    ContainerEventType.PAUSE_CONTAINER, new PauseContainerTransition())
+        ContainerEventType.PAUSE_CONTAINER, new PauseContainerTransition())
+    .addTransition(ContainerState.RUNNING, ContainerState.RUNNING,
+        ContainerEventType.UPDATE_CONTAINER_TOKEN,
+        new NotifyContainerSchedulerOfUpdateTransition())
+
 
     // From PAUSING State
+    .addTransition(ContainerState.PAUSING, ContainerState.PAUSING,
+        ContainerEventType.RESOURCE_LOCALIZED,
+        new ResourceLocalizedWhileRunningTransition())
     .addTransition(ContainerState.PAUSING, ContainerState.KILLING,
         ContainerEventType.KILL_CONTAINER, new KillTransition())
     .addTransition(ContainerState.PAUSING, ContainerState.PAUSING,
@@ -420,6 +442,12 @@ public class ContainerImpl implements Container {
     .addTransition(ContainerState.PAUSING, ContainerState.EXITED_WITH_FAILURE,
         ContainerEventType.CONTAINER_KILLED_ON_REQUEST,
         new KilledExternallyTransition())
+    .addTransition(ContainerState.PAUSING, ContainerState.PAUSING,
+        ContainerEventType.RESOURCE_LOCALIZED,
+        new ResourceLocalizedWhileRunningTransition())
+    .addTransition(ContainerState.PAUSING, ContainerState.PAUSING,
+        ContainerEventType.UPDATE_CONTAINER_TOKEN,
+        new NotifyContainerSchedulerOfUpdateTransition())
 
     // From PAUSED State
     .addTransition(ContainerState.PAUSED, ContainerState.KILLING,
@@ -429,6 +457,10 @@ public class ContainerImpl implements Container {
         UPDATE_DIAGNOSTICS_TRANSITION)
     .addTransition(ContainerState.PAUSED, ContainerState.PAUSED,
         ContainerEventType.PAUSE_CONTAINER)
+    // This can happen during re-initialization.
+    .addTransition(ContainerState.PAUSED, ContainerState.PAUSED,
+        ContainerEventType.RESOURCE_LOCALIZED,
+        new ResourceLocalizedWhileRunningTransition())
     .addTransition(ContainerState.PAUSED, ContainerState.RESUMING,
         ContainerEventType.RESUME_CONTAINER, new ResumeContainerTransition())
     // In case something goes wrong then container will exit from the
@@ -444,6 +476,9 @@ public class ContainerImpl implements Container {
         ContainerState.EXITED_WITH_SUCCESS,
         ContainerEventType.CONTAINER_EXITED_WITH_SUCCESS,
         new ExitedWithSuccessTransition(true))
+    .addTransition(ContainerState.PAUSED, ContainerState.PAUSED,
+        ContainerEventType.UPDATE_CONTAINER_TOKEN,
+        new NotifyContainerSchedulerOfUpdateTransition())
 
     // From RESUMING State
     .addTransition(ContainerState.RESUMING, ContainerState.KILLING,
@@ -453,6 +488,10 @@ public class ContainerImpl implements Container {
     .addTransition(ContainerState.RESUMING, ContainerState.RESUMING,
         ContainerEventType.UPDATE_DIAGNOSTICS_MSG,
         UPDATE_DIAGNOSTICS_TRANSITION)
+    // This can happen during re-initialization
+    .addTransition(ContainerState.RESUMING, ContainerState.RESUMING,
+        ContainerEventType.RESOURCE_LOCALIZED,
+        new ResourceLocalizedWhileRunningTransition())
     // In case something goes wrong then container will exit from the
     // RESUMING state
     .addTransition(ContainerState.RESUMING,
@@ -467,6 +506,10 @@ public class ContainerImpl implements Container {
         ContainerState.EXITED_WITH_SUCCESS,
         ContainerEventType.CONTAINER_EXITED_WITH_SUCCESS,
         new ExitedWithSuccessTransition(true))
+    .addTransition(ContainerState.RESUMING, ContainerState.RESUMING,
+        ContainerEventType.UPDATE_CONTAINER_TOKEN,
+        new NotifyContainerSchedulerOfUpdateTransition())
+    // NOTE - We cannot get a PAUSE_CONTAINER while in RESUMING state.
 
     // From REINITIALIZING State
     .addTransition(ContainerState.REINITIALIZING,
@@ -478,7 +521,8 @@ public class ContainerImpl implements Container {
         ContainerEventType.CONTAINER_EXITED_WITH_FAILURE,
         new ExitedWithFailureTransition(true))
     .addTransition(ContainerState.REINITIALIZING,
-        ContainerState.REINITIALIZING,
+        EnumSet.of(ContainerState.REINITIALIZING,
+            ContainerState.REINITIALIZING_AWAITING_KILL),
         ContainerEventType.RESOURCE_LOCALIZED,
         new ResourceLocalizedWhileReInitTransition())
     .addTransition(ContainerState.REINITIALIZING, ContainerState.RUNNING,
@@ -490,12 +534,39 @@ public class ContainerImpl implements Container {
         UPDATE_DIAGNOSTICS_TRANSITION)
     .addTransition(ContainerState.REINITIALIZING, ContainerState.KILLING,
         ContainerEventType.KILL_CONTAINER, new KillTransition())
-    .addTransition(ContainerState.REINITIALIZING, ContainerState.KILLING,
-        ContainerEventType.PAUSE_CONTAINER, new KillOnPauseTransition())
+    .addTransition(ContainerState.REINITIALIZING, ContainerState.PAUSING,
+        ContainerEventType.PAUSE_CONTAINER, new PauseContainerTransition())
     .addTransition(ContainerState.REINITIALIZING,
+        ContainerState.REINITIALIZING,
+        ContainerEventType.UPDATE_CONTAINER_TOKEN,
+        new NotifyContainerSchedulerOfUpdateTransition())
+
+    // from REINITIALIZING_AWAITING_KILL
+    .addTransition(ContainerState.REINITIALIZING_AWAITING_KILL,
+        ContainerState.EXITED_WITH_SUCCESS,
+        ContainerEventType.CONTAINER_EXITED_WITH_SUCCESS,
+        new ExitedWithSuccessTransition(true))
+    .addTransition(ContainerState.REINITIALIZING_AWAITING_KILL,
+        ContainerState.EXITED_WITH_FAILURE,
+        ContainerEventType.CONTAINER_EXITED_WITH_FAILURE,
+        new ExitedWithFailureTransition(true))
+    .addTransition(ContainerState.REINITIALIZING_AWAITING_KILL,
+        ContainerState.REINITIALIZING_AWAITING_KILL,
+        ContainerEventType.UPDATE_DIAGNOSTICS_MSG,
+        UPDATE_DIAGNOSTICS_TRANSITION)
+    .addTransition(ContainerState.REINITIALIZING_AWAITING_KILL,
+        ContainerState.KILLING,
+        ContainerEventType.KILL_CONTAINER, new KillTransition())
+    .addTransition(ContainerState.REINITIALIZING_AWAITING_KILL,
+        ContainerState.SCHEDULED, ContainerEventType.PAUSE_CONTAINER)
+    .addTransition(ContainerState.REINITIALIZING_AWAITING_KILL,
         ContainerState.SCHEDULED,
         ContainerEventType.CONTAINER_KILLED_ON_REQUEST,
         new KilledForReInitializationTransition())
+    .addTransition(ContainerState.REINITIALIZING_AWAITING_KILL,
+        ContainerState.REINITIALIZING_AWAITING_KILL,
+        ContainerEventType.UPDATE_CONTAINER_TOKEN,
+        new NotifyContainerSchedulerOfUpdateTransition())
 
     // From RELAUNCHING State
     .addTransition(ContainerState.RELAUNCHING, ContainerState.RUNNING,
@@ -511,6 +582,10 @@ public class ContainerImpl implements Container {
         ContainerEventType.KILL_CONTAINER, new KillTransition())
     .addTransition(ContainerState.RELAUNCHING, ContainerState.KILLING,
         ContainerEventType.PAUSE_CONTAINER, new KillOnPauseTransition())
+    .addTransition(ContainerState.RELAUNCHING, ContainerState.RELAUNCHING,
+        ContainerEventType.UPDATE_CONTAINER_TOKEN,
+        new NotifyContainerSchedulerOfUpdateTransition())
+
 
     // From CONTAINER_EXITED_WITH_SUCCESS State
     .addTransition(ContainerState.EXITED_WITH_SUCCESS, ContainerState.DONE,
@@ -524,6 +599,10 @@ public class ContainerImpl implements Container {
         ContainerState.EXITED_WITH_SUCCESS,
         EnumSet.of(ContainerEventType.KILL_CONTAINER,
             ContainerEventType.PAUSE_CONTAINER))
+    // No transition - assuming container is on its way to completion
+    .addTransition(ContainerState.EXITED_WITH_SUCCESS,
+        ContainerState.EXITED_WITH_SUCCESS,
+        ContainerEventType.UPDATE_CONTAINER_TOKEN)
 
     // From EXITED_WITH_FAILURE State
     .addTransition(ContainerState.EXITED_WITH_FAILURE, ContainerState.DONE,
@@ -537,6 +616,10 @@ public class ContainerImpl implements Container {
                    ContainerState.EXITED_WITH_FAILURE,
         EnumSet.of(ContainerEventType.KILL_CONTAINER,
             ContainerEventType.PAUSE_CONTAINER))
+    // No transition - assuming container is on its way to completion
+    .addTransition(ContainerState.EXITED_WITH_FAILURE,
+        ContainerState.EXITED_WITH_FAILURE,
+        ContainerEventType.UPDATE_CONTAINER_TOKEN)
 
     // From KILLING State.
     .addTransition(ContainerState.KILLING,
@@ -572,6 +655,9 @@ public class ContainerImpl implements Container {
         ContainerState.KILLING,
         EnumSet.of(ContainerEventType.CONTAINER_LAUNCHED,
             ContainerEventType.PAUSE_CONTAINER))
+    // No transition - assuming container is on its way to completion
+    .addTransition(ContainerState.KILLING, ContainerState.KILLING,
+        ContainerEventType.UPDATE_CONTAINER_TOKEN)
 
     // From CONTAINER_CLEANEDUP_AFTER_KILL State.
     .addTransition(ContainerState.CONTAINER_CLEANEDUP_AFTER_KILL,
@@ -589,6 +675,10 @@ public class ContainerImpl implements Container {
             ContainerEventType.CONTAINER_EXITED_WITH_SUCCESS,
             ContainerEventType.CONTAINER_EXITED_WITH_FAILURE,
             ContainerEventType.PAUSE_CONTAINER))
+    // No transition - assuming container is on its way to completion
+    .addTransition(ContainerState.CONTAINER_CLEANEDUP_AFTER_KILL,
+        ContainerState.CONTAINER_CLEANEDUP_AFTER_KILL,
+        ContainerEventType.UPDATE_CONTAINER_TOKEN)
 
     // From DONE
     .addTransition(ContainerState.DONE, ContainerState.DONE,
@@ -606,6 +696,9 @@ public class ContainerImpl implements Container {
         EnumSet.of(ContainerEventType.RESOURCE_FAILED,
             ContainerEventType.CONTAINER_EXITED_WITH_SUCCESS,
             ContainerEventType.CONTAINER_EXITED_WITH_FAILURE))
+    // No transition - assuming container is on its way to completion
+    .addTransition(ContainerState.DONE, ContainerState.DONE,
+        ContainerEventType.UPDATE_CONTAINER_TOKEN)
 
     // create the topology tables
     .installTopology();
@@ -616,7 +709,6 @@ public class ContainerImpl implements Container {
   public org.apache.hadoop.yarn.api.records.ContainerState getCurrentState() {
     switch (stateMachine.getCurrentState()) {
     case NEW:
-      return org.apache.hadoop.yarn.api.records.ContainerState.NEW;
     case LOCALIZING:
     case LOCALIZATION_FAILED:
     case SCHEDULED:
@@ -626,6 +718,7 @@ public class ContainerImpl implements Container {
     case RUNNING:
     case RELAUNCHING:
     case REINITIALIZING:
+    case REINITIALIZING_AWAITING_KILL:
     case EXITED_WITH_SUCCESS:
     case EXITED_WITH_FAILURE:
     case KILLING:
@@ -929,6 +1022,45 @@ public class ContainerImpl implements Container {
 
   }
 
+  static class UpdateTransition extends ContainerTransition {
+    @Override
+    public void transition(
+        ContainerImpl container, ContainerEvent event) {
+      UpdateContainerTokenEvent updateEvent = (UpdateContainerTokenEvent)event;
+      // Update the container token
+      container.setContainerTokenIdentifier(updateEvent.getUpdatedToken());
+      if (updateEvent.isResourceChange()) {
+        try {
+          // Persist change in the state store.
+          container.context.getNMStateStore().storeContainerResourceChanged(
+              container.containerId,
+              container.getContainerTokenIdentifier().getVersion(),
+              container.getResource());
+        } catch (IOException e) {
+          LOG.warn("Could not store container [" + container.containerId
+              + "] resource change..", e);
+        }
+      }
+    }
+  }
+
+  static class NotifyContainerSchedulerOfUpdateTransition extends
+      UpdateTransition {
+    @Override
+    public void transition(
+        ContainerImpl container, ContainerEvent event) {
+
+      UpdateContainerTokenEvent updateEvent = (UpdateContainerTokenEvent)event;
+      // Save original token
+      ContainerTokenIdentifier originalToken =
+          container.containerTokenIdentifier;
+      super.transition(container, updateEvent);
+      container.dispatcher.getEventHandler().handle(
+          new UpdateContainerSchedulerEvent(container,
+              originalToken, updateEvent));
+    }
+  }
+
   /**
    * State transition when a NEW container receives the INIT_CONTAINER
    * message.
@@ -1074,12 +1206,15 @@ public class ContainerImpl implements Container {
   /**
    * Transition to start the Re-Initialization process.
    */
-  static class ReInitializeContainerTransition extends ContainerTransition {
+  static class ReInitializeContainerTransition implements
+      MultipleArcTransition<ContainerImpl, ContainerEvent, ContainerState> {
 
     @SuppressWarnings("unchecked")
     @Override
-    public void transition(ContainerImpl container, ContainerEvent event) {
+    public ContainerState transition(
+        ContainerImpl container, ContainerEvent event) {
       container.reInitContext = createReInitContext(container, event);
+      boolean resourcesPresent = false;
       try {
         // 'reInitContext.newResourceSet' can be
         // a) current container resourceSet (In case of Restart)
@@ -1101,6 +1236,7 @@ public class ContainerImpl implements Container {
           container.dispatcher.getEventHandler().handle(
               new ContainersLauncherEvent(container,
                   ContainersLauncherEventType.CLEANUP_CONTAINER_FOR_REINIT));
+          resourcesPresent = true;
         }
         container.metrics.reInitingContainer();
         NMAuditLogger.logSuccess(container.user,
@@ -1112,7 +1248,11 @@ public class ContainerImpl implements Container {
             " re-initialization failure..", e);
         container.addDiagnostics("Error re-initializing due to" +
             "[" + e.getMessage() + "]");
+        return ContainerState.RUNNING;
       }
+      return resourcesPresent ?
+          ContainerState.REINITIALIZING_AWAITING_KILL :
+          ContainerState.REINITIALIZING;
     }
 
     protected ReInitializationContext createReInitContext(
@@ -1164,11 +1304,14 @@ public class ContainerImpl implements Container {
    * If all dependencies are met, then restart Container with new bits.
    */
   static class ResourceLocalizedWhileReInitTransition
-      extends ContainerTransition {
+      implements MultipleArcTransition
+      <ContainerImpl, ContainerEvent, ContainerState> {
+
 
     @SuppressWarnings("unchecked")
     @Override
-    public void transition(ContainerImpl container, ContainerEvent event) {
+    public ContainerState transition(
+        ContainerImpl container, ContainerEvent event) {
       ContainerResourceLocalizedEvent rsrcEvent =
           (ContainerResourceLocalizedEvent) event;
       container.reInitContext.newResourceSet.resourceLocalized(
@@ -1180,7 +1323,9 @@ public class ContainerImpl implements Container {
         container.dispatcher.getEventHandler().handle(
             new ContainersLauncherEvent(container,
                 ContainersLauncherEventType.CLEANUP_CONTAINER_FOR_REINIT));
+        return ContainerState.REINITIALIZING_AWAITING_KILL;
       }
+      return ContainerState.REINITIALIZING;
     }
   }
 

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

@@ -20,7 +20,8 @@ package org.apache.hadoop.yarn.server.nodemanager.containermanager.container;
 
 public enum ContainerState {
   NEW, LOCALIZING, LOCALIZATION_FAILED, SCHEDULED, RUNNING, RELAUNCHING,
-  REINITIALIZING, EXITED_WITH_SUCCESS, EXITED_WITH_FAILURE, KILLING,
+  REINITIALIZING, REINITIALIZING_AWAITING_KILL,
+  EXITED_WITH_SUCCESS, EXITED_WITH_FAILURE, KILLING,
   CONTAINER_CLEANEDUP_AFTER_KILL, CONTAINER_RESOURCES_CLEANINGUP, DONE,
   PAUSING, PAUSED, RESUMING
 }

+ 86 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/UpdateContainerTokenEvent.java

@@ -0,0 +1,86 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.server.nodemanager.containermanager.container;
+
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
+
+/**
+ * Update Event consumed by the Container.
+ */
+public class UpdateContainerTokenEvent extends ContainerEvent {
+  private final ContainerTokenIdentifier updatedToken;
+  private final boolean isResourceChange;
+  private final boolean isExecTypeUpdate;
+  private final boolean isIncrease;
+
+  /**
+   * Create Update event.
+   *
+   * @param cID Container Id.
+   * @param updatedToken Updated Container Token.
+   * @param isResourceChange Is Resource change.
+   * @param isExecTypeUpdate Is ExecutionType Update.
+   * @param isIncrease Is container increase.
+   */
+  public UpdateContainerTokenEvent(ContainerId cID,
+      ContainerTokenIdentifier updatedToken, boolean isResourceChange,
+      boolean isExecTypeUpdate, boolean isIncrease) {
+    super(cID, ContainerEventType.UPDATE_CONTAINER_TOKEN);
+    this.updatedToken = updatedToken;
+    this.isResourceChange = isResourceChange;
+    this.isExecTypeUpdate = isExecTypeUpdate;
+    this.isIncrease = isIncrease;
+  }
+
+  /**
+   * Update Container Token.
+   *
+   * @return Container Token.
+   */
+  public ContainerTokenIdentifier getUpdatedToken() {
+    return updatedToken;
+  }
+
+  /**
+   * Is this update a ResourceChange.
+   *
+   * @return isResourceChange.
+   */
+  public boolean isResourceChange() {
+    return isResourceChange;
+  }
+
+  /**
+   * Is this update an ExecType Update.
+   *
+   * @return isExecTypeUpdate.
+   */
+  public boolean isExecTypeUpdate() {
+    return isExecTypeUpdate;
+  }
+
+  /**
+   * Is this a container Increase.
+   *
+   * @return isIncrease.
+   */
+  public boolean isIncrease() {
+    return isIncrease;
+  }
+}

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

@@ -33,6 +33,7 @@ import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus;
 import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
 
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerImpl;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor
     .ChangeMonitoringContainerResourceEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerState;
@@ -151,7 +152,9 @@ public class ContainerScheduler extends AbstractService implements
     case SCHEDULE_CONTAINER:
       scheduleContainer(event.getContainer());
       break;
+    // NOTE: Is sent only after container state has changed to PAUSED...
     case CONTAINER_PAUSED:
+    // NOTE: Is sent only after container state has changed to DONE...
     case CONTAINER_COMPLETED:
       onResourcesReclaimed(event.getContainer());
       break;
@@ -180,58 +183,38 @@ public class ContainerScheduler extends AbstractService implements
     if (updateEvent.isResourceChange()) {
       if (runningContainers.containsKey(containerId)) {
         this.utilizationTracker.subtractContainerResource(
-            updateEvent.getContainer());
-        updateEvent.getContainer().setContainerTokenIdentifier(
-            updateEvent.getUpdatedToken());
+            new ContainerImpl(getConfig(), null, null, null, null,
+                updateEvent.getOriginalToken(), context));
         this.utilizationTracker.addContainerResources(
             updateEvent.getContainer());
         getContainersMonitor().handle(
             new ChangeMonitoringContainerResourceEvent(containerId,
                 updateEvent.getUpdatedToken().getResource()));
-      } else {
-        // Is Queued or localizing..
-        updateEvent.getContainer().setContainerTokenIdentifier(
-            updateEvent.getUpdatedToken());
-      }
-      try {
-        // Persist change in the state store.
-        this.context.getNMStateStore().storeContainerResourceChanged(
-            containerId,
-            updateEvent.getUpdatedToken().getVersion(),
-            updateEvent.getUpdatedToken().getResource());
-      } catch (IOException e) {
-        LOG.warn("Could not store container [" + containerId + "] resource " +
-            "change..", e);
       }
     }
 
     if (updateEvent.isExecTypeUpdate()) {
-      updateEvent.getContainer().setContainerTokenIdentifier(
-          updateEvent.getUpdatedToken());
-      // If this is a running container.. just change the execution type
-      // and be done with it.
-      if (!runningContainers.containsKey(containerId)) {
-        // Promotion or not (Increase signifies either a promotion
-        // or container size increase)
-        if (updateEvent.isIncrease()) {
-          // Promotion of queued container..
-          if (queuedOpportunisticContainers.remove(containerId) != null) {
-            queuedGuaranteedContainers.put(containerId,
-                updateEvent.getContainer());
-          }
+      // Promotion or not (Increase signifies either a promotion
+      // or container size increase)
+      if (updateEvent.isIncrease()) {
+        // Promotion of queued container..
+        if (queuedOpportunisticContainers.remove(containerId) != null) {
+          queuedGuaranteedContainers.put(containerId,
+              updateEvent.getContainer());
           //Kill/pause opportunistic containers if any to make room for
           // promotion request
           reclaimOpportunisticContainerResources(updateEvent.getContainer());
-        } else {
-          // Demotion of queued container.. Should not happen too often
-          // since you should not find too many queued guaranteed
-          // containers
-          if (queuedGuaranteedContainers.remove(containerId) != null) {
-            queuedOpportunisticContainers.put(containerId,
-                updateEvent.getContainer());
-          }
+        }
+      } else {
+        // Demotion of queued container.. Should not happen too often
+        // since you should not find too many queued guaranteed
+        // containers
+        if (queuedGuaranteedContainers.remove(containerId) != null) {
+          queuedOpportunisticContainers.put(containerId,
+              updateEvent.getContainer());
         }
       }
+      startPendingContainers(maxOppQueueLength <= 0);
     }
   }
 
@@ -290,6 +273,16 @@ public class ContainerScheduler extends AbstractService implements
       queuedGuaranteedContainers.remove(container.getContainerId());
     }
 
+    // Requeue PAUSED containers
+    if (container.getContainerState() == ContainerState.PAUSED) {
+      if (container.getContainerTokenIdentifier().getExecutionType() ==
+          ExecutionType.GUARANTEED) {
+        queuedGuaranteedContainers.put(container.getContainerId(), container);
+      } else {
+        queuedOpportunisticContainers.put(
+            container.getContainerId(), container);
+      }
+    }
     // decrement only if it was a running container
     Container completedContainer = runningContainers.remove(container
         .getContainerId());
@@ -301,7 +294,8 @@ public class ContainerScheduler extends AbstractService implements
           ExecutionType.OPPORTUNISTIC) {
         this.metrics.completeOpportunisticContainer(container.getResource());
       }
-      startPendingContainers(false);
+      boolean forceStartGuaranteedContainers = (maxOppQueueLength <= 0);
+      startPendingContainers(forceStartGuaranteedContainers);
     }
   }
 
@@ -311,26 +305,9 @@ public class ContainerScheduler extends AbstractService implements
    *        container without looking at available resource
    */
   private void startPendingContainers(boolean forceStartGuaranteedContaieners) {
-    // Start pending guaranteed containers, if resources available.
+    // Start guaranteed containers that are paused, if resources available.
     boolean resourcesAvailable = startContainers(
-        queuedGuaranteedContainers.values(), forceStartGuaranteedContaieners);
-    // Resume opportunistic containers, if resource available.
-    if (resourcesAvailable) {
-      List<Container> pausedContainers = new ArrayList<Container>();
-      Map<ContainerId, Container> containers =
-          context.getContainers();
-      for (Map.Entry<ContainerId, Container>entry : containers.entrySet()) {
-        ContainerId contId = entry.getKey();
-        // Find containers that were not already started and are in paused state
-        if(false == runningContainers.containsKey(contId)) {
-          if(containers.get(contId).getContainerState()
-              == ContainerState.PAUSED) {
-            pausedContainers.add(containers.get(contId));
-          }
-        }
-      }
-      resourcesAvailable = startContainers(pausedContainers, false);
-    }
+          queuedGuaranteedContainers.values(), forceStartGuaranteedContaieners);
     // Start opportunistic containers, if resources available.
     if (resourcesAvailable) {
       startContainers(queuedOpportunisticContainers.values(), false);
@@ -590,16 +567,19 @@ public class ContainerScheduler extends AbstractService implements
         queuedOpportunisticContainers.values().iterator();
     while (containerIter.hasNext()) {
       Container container = containerIter.next();
-      if (numAllowed <= 0) {
-        container.sendKillEvent(
-            ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER,
-            "Container De-queued to meet NM queuing limits.");
-        containerIter.remove();
-        LOG.info(
-            "Opportunistic container {} will be killed to meet NM queuing" +
-                " limits.", container.getContainerId());
+      // Do not shed PAUSED containers
+      if (container.getContainerState() != ContainerState.PAUSED) {
+        if (numAllowed <= 0) {
+          container.sendKillEvent(
+              ContainerExitStatus.KILLED_BY_CONTAINER_SCHEDULER,
+              "Container De-queued to meet NM queuing limits.");
+          containerIter.remove();
+          LOG.info(
+              "Opportunistic container {} will be killed to meet NM queuing" +
+                  " limits.", container.getContainerId());
+        }
+        numAllowed--;
       }
-      numAllowed--;
     }
   }
 

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

@@ -21,33 +21,37 @@ package org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler;
 import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container
     .Container;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.UpdateContainerTokenEvent;
+
 /**
  * Update Event consumed by the {@link ContainerScheduler}.
  */
 public class UpdateContainerSchedulerEvent extends ContainerSchedulerEvent {
 
-  private ContainerTokenIdentifier updatedToken;
-  private boolean isResourceChange;
-  private boolean isExecTypeUpdate;
-  private boolean isIncrease;
+  private final UpdateContainerTokenEvent containerEvent;
+  private final ContainerTokenIdentifier originalToken;
 
   /**
    * Create instance of Event.
    *
-   * @param originalContainer Original Container.
-   * @param updatedToken Updated Container Token.
-   * @param isResourceChange is this a Resource Change.
-   * @param isExecTypeUpdate is this an ExecTypeUpdate.
-   * @param isIncrease is this a Container Increase.
+   * @param container Container.
+   * @param origToken The Original Container Token.
+   * @param event The Container Event.
+   */
+  public UpdateContainerSchedulerEvent(Container container,
+      ContainerTokenIdentifier origToken, UpdateContainerTokenEvent event) {
+    super(container, ContainerSchedulerEventType.UPDATE_CONTAINER);
+    this.containerEvent = event;
+    this.originalToken = origToken;
+  }
+
+  /**
+   * Original Token before update.
+   *
+   * @return Container Token.
    */
-  public UpdateContainerSchedulerEvent(Container originalContainer,
-      ContainerTokenIdentifier updatedToken, boolean isResourceChange,
-      boolean isExecTypeUpdate, boolean isIncrease) {
-    super(originalContainer, ContainerSchedulerEventType.UPDATE_CONTAINER);
-    this.updatedToken = updatedToken;
-    this.isResourceChange = isResourceChange;
-    this.isExecTypeUpdate = isExecTypeUpdate;
-    this.isIncrease = isIncrease;
+  public ContainerTokenIdentifier getOriginalToken() {
+    return this.originalToken;
   }
 
   /**
@@ -56,7 +60,7 @@ public class UpdateContainerSchedulerEvent extends ContainerSchedulerEvent {
    * @return Container Token.
    */
   public ContainerTokenIdentifier getUpdatedToken() {
-    return updatedToken;
+    return containerEvent.getUpdatedToken();
   }
 
   /**
@@ -64,7 +68,7 @@ public class UpdateContainerSchedulerEvent extends ContainerSchedulerEvent {
    * @return isResourceChange.
    */
   public boolean isResourceChange() {
-    return isResourceChange;
+    return containerEvent.isResourceChange();
   }
 
   /**
@@ -72,7 +76,7 @@ public class UpdateContainerSchedulerEvent extends ContainerSchedulerEvent {
    * @return isExecTypeUpdate.
    */
   public boolean isExecTypeUpdate() {
-    return isExecTypeUpdate;
+    return containerEvent.isExecTypeUpdate();
   }
 
   /**
@@ -80,6 +84,6 @@ public class UpdateContainerSchedulerEvent extends ContainerSchedulerEvent {
    * @return isIncrease.
    */
   public boolean isIncrease() {
-    return isIncrease;
+    return containerEvent.isIncrease();
   }
 }

+ 1 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestEventFlow.java

@@ -159,8 +159,7 @@ public class TestEventFlow {
     containerManager.startContainers(allRequests);
 
     BaseContainerManagerTest.waitForContainerState(containerManager, cID,
-        Arrays.asList(ContainerState.RUNNING, ContainerState.SCHEDULED,
-            ContainerState.NEW), 20);
+        Arrays.asList(ContainerState.RUNNING, ContainerState.SCHEDULED), 20);
 
     List<ContainerId> containerIds = new ArrayList<ContainerId>();
     containerIds.add(cID);

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

@@ -208,6 +208,8 @@ public abstract class BaseContainerManagerTest {
     containerManager.init(conf);
     nodeStatusUpdater.start();
     ((NMContext)context).setNodeStatusUpdater(nodeStatusUpdater);
+    ((NMContext)context).setContainerStateTransitionListener(
+        new NodeManager.DefaultContainerStateListener());
   }
 
   protected ContainerManagerImpl

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

@@ -90,12 +90,16 @@ import org.apache.hadoop.yarn.server.nodemanager.CMgrCompletedAppsEvent;
 import org.apache.hadoop.yarn.server.nodemanager.CMgrSignalContainersEvent;
 import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
 import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.Signal;
+import org.apache.hadoop.yarn.server.nodemanager.ContainerStateTransitionListener;
+import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor;
 import org.apache.hadoop.yarn.server.nodemanager.DeletionService;
 import org.apache.hadoop.yarn.server.nodemanager.NodeManager;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.TestAuxServices.ServiceA;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationState;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEvent;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEventType;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerImpl;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ContainerLocalizer;
@@ -119,6 +123,41 @@ public class TestContainerManager extends BaseContainerManagerTest {
     LOG = LoggerFactory.getLogger(TestContainerManager.class);
   }
 
+  private static class Listener implements ContainerStateTransitionListener {
+
+    private final Map<ContainerId,
+        List<org.apache.hadoop.yarn.server.nodemanager.containermanager.
+            container.ContainerState>> states = new HashMap<>();
+    private final Map<ContainerId, List<ContainerEventType>> events =
+        new HashMap<>();
+
+    @Override
+    public void init(Context context) {}
+
+    @Override
+    public void preTransition(ContainerImpl op,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState beforeState,
+        ContainerEvent eventToBeProcessed) {
+      if (!states.containsKey(op.getContainerId())) {
+        states.put(op.getContainerId(), new ArrayList<>());
+        states.get(op.getContainerId()).add(beforeState);
+        events.put(op.getContainerId(), new ArrayList<>());
+      }
+    }
+
+    @Override
+    public void postTransition(ContainerImpl op,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState beforeState,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState afterState,
+        ContainerEvent processedEvent) {
+      states.get(op.getContainerId()).add(afterState);
+      events.get(op.getContainerId()).add(processedEvent.getType());
+    }
+  }
+
   private boolean delayContainers = false;
 
   @Override
@@ -144,7 +183,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
   @Override
   protected ContainerManagerImpl
       createContainerManager(DeletionService delSrvc) {
-    return new ContainerManagerImpl(context, exec, delSrvc, nodeStatusUpdater,
+    return  new ContainerManagerImpl(context, exec, delSrvc, nodeStatusUpdater,
       metrics, dirsHandler) {
 
       @Override
@@ -496,6 +535,9 @@ public class TestContainerManager extends BaseContainerManagerTest {
   @Test
   public void testContainerUpgradeSuccessAutoCommit() throws IOException,
       InterruptedException, YarnException {
+    Listener listener = new Listener();
+    ((NodeManager.DefaultContainerStateListener)containerManager.context.
+        getContainerStateTransitionListener()).addListener(listener);
     testContainerReInitSuccess(true);
     // Should not be able to Commit (since already auto committed)
     try {
@@ -504,6 +546,41 @@ public class TestContainerManager extends BaseContainerManagerTest {
     } catch (Exception e) {
       Assert.assertTrue(e.getMessage().contains("Nothing to Commit"));
     }
+
+    List<org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+        ContainerState> containerStates =
+        listener.states.get(createContainerId(0));
+    Assert.assertEquals(Arrays.asList(
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.NEW,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.LOCALIZING,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.SCHEDULED,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.RUNNING,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.REINITIALIZING,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.REINITIALIZING_AWAITING_KILL,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.REINITIALIZING_AWAITING_KILL,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.SCHEDULED,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.RUNNING), containerStates);
+
+    List<ContainerEventType> containerEventTypes =
+        listener.events.get(createContainerId(0));
+    Assert.assertEquals(Arrays.asList(
+        ContainerEventType.INIT_CONTAINER,
+        ContainerEventType.RESOURCE_LOCALIZED,
+        ContainerEventType.CONTAINER_LAUNCHED,
+        ContainerEventType.REINITIALIZE_CONTAINER,
+        ContainerEventType.RESOURCE_LOCALIZED,
+        ContainerEventType.UPDATE_DIAGNOSTICS_MSG,
+        ContainerEventType.CONTAINER_KILLED_ON_REQUEST,
+        ContainerEventType.CONTAINER_LAUNCHED), containerEventTypes);
   }
 
   @Test
@@ -524,6 +601,9 @@ public class TestContainerManager extends BaseContainerManagerTest {
   @Test
   public void testContainerUpgradeSuccessExplicitRollback() throws IOException,
       InterruptedException, YarnException {
+    Listener listener = new Listener();
+    ((NodeManager.DefaultContainerStateListener)containerManager.context.
+        getContainerStateTransitionListener()).addListener(listener);
     String[] pids = testContainerReInitSuccess(false);
 
     // Test that the container can be Restarted after the successful upgrrade.
@@ -575,6 +655,67 @@ public class TestContainerManager extends BaseContainerManagerTest {
 
     Assert.assertNotEquals("The Rolled-back process should be a different pid",
         pids[0], rolledBackPid);
+
+    List<org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+        ContainerState> containerStates =
+        listener.states.get(createContainerId(0));
+    Assert.assertEquals(Arrays.asList(
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.NEW,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.LOCALIZING,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.SCHEDULED,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.RUNNING,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.REINITIALIZING,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.REINITIALIZING_AWAITING_KILL,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.REINITIALIZING_AWAITING_KILL,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.SCHEDULED,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.RUNNING,
+        // This is the successful restart
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.REINITIALIZING_AWAITING_KILL,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.REINITIALIZING_AWAITING_KILL,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.SCHEDULED,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.RUNNING,
+        // This is the rollback
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.REINITIALIZING_AWAITING_KILL,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.REINITIALIZING_AWAITING_KILL,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.SCHEDULED,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.RUNNING), containerStates);
+
+    List<ContainerEventType> containerEventTypes =
+        listener.events.get(createContainerId(0));
+    Assert.assertEquals(Arrays.asList(
+        ContainerEventType.INIT_CONTAINER,
+        ContainerEventType.RESOURCE_LOCALIZED,
+        ContainerEventType.CONTAINER_LAUNCHED,
+        ContainerEventType.REINITIALIZE_CONTAINER,
+        ContainerEventType.RESOURCE_LOCALIZED,
+        ContainerEventType.UPDATE_DIAGNOSTICS_MSG,
+        ContainerEventType.CONTAINER_KILLED_ON_REQUEST,
+        ContainerEventType.CONTAINER_LAUNCHED,
+        ContainerEventType.REINITIALIZE_CONTAINER,
+        ContainerEventType.UPDATE_DIAGNOSTICS_MSG,
+        ContainerEventType.CONTAINER_KILLED_ON_REQUEST,
+        ContainerEventType.CONTAINER_LAUNCHED,
+        ContainerEventType.ROLLBACK_REINIT,
+        ContainerEventType.UPDATE_DIAGNOSTICS_MSG,
+        ContainerEventType.CONTAINER_KILLED_ON_REQUEST,
+        ContainerEventType.CONTAINER_LAUNCHED), containerEventTypes);
   }
 
   @Test
@@ -584,6 +725,9 @@ public class TestContainerManager extends BaseContainerManagerTest {
       return;
     }
     containerManager.start();
+    Listener listener = new Listener();
+    ((NodeManager.DefaultContainerStateListener)containerManager.context.
+        getContainerStateTransitionListener()).addListener(listener);
     // ////// Construct the Container-id
     ContainerId cId = createContainerId(0);
     File oldStartFile = new File(tmpDir, "start_file_o.txt").getAbsoluteFile();
@@ -598,6 +742,32 @@ public class TestContainerManager extends BaseContainerManagerTest {
     // since upgrade was terminated..
     Assert.assertTrue("Process is NOT alive!",
         DefaultContainerExecutor.containerIsAlive(pid));
+
+    List<org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+        ContainerState> containerStates =
+        listener.states.get(createContainerId(0));
+    Assert.assertEquals(Arrays.asList(
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.NEW,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.LOCALIZING,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.SCHEDULED,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.RUNNING,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.REINITIALIZING,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.RUNNING), containerStates);
+
+    List<ContainerEventType> containerEventTypes =
+        listener.events.get(createContainerId(0));
+    Assert.assertEquals(Arrays.asList(
+        ContainerEventType.INIT_CONTAINER,
+        ContainerEventType.RESOURCE_LOCALIZED,
+        ContainerEventType.CONTAINER_LAUNCHED,
+        ContainerEventType.REINITIALIZE_CONTAINER,
+        ContainerEventType.RESOURCE_FAILED), containerEventTypes);
   }
 
   @Test
@@ -632,6 +802,9 @@ public class TestContainerManager extends BaseContainerManagerTest {
       return;
     }
     containerManager.start();
+    Listener listener = new Listener();
+    ((NodeManager.DefaultContainerStateListener)containerManager.context.
+        getContainerStateTransitionListener()).addListener(listener);
     // ////// Construct the Container-id
     ContainerId cId = createContainerId(0);
     File oldStartFile = new File(tmpDir, "start_file_o.txt").getAbsoluteFile();
@@ -666,6 +839,50 @@ public class TestContainerManager extends BaseContainerManagerTest {
 
     Assert.assertNotEquals("The Rolled-back process should be a different pid",
         pid, rolledBackPid);
+
+    List<org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+        ContainerState> containerStates =
+        listener.states.get(createContainerId(0));
+    Assert.assertEquals(Arrays.asList(
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.NEW,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.LOCALIZING,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.SCHEDULED,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.RUNNING,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.REINITIALIZING,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.REINITIALIZING_AWAITING_KILL,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.REINITIALIZING_AWAITING_KILL,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.SCHEDULED,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.RUNNING,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.RUNNING,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.SCHEDULED,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.RUNNING), containerStates);
+
+    List<ContainerEventType> containerEventTypes =
+        listener.events.get(createContainerId(0));
+    Assert.assertEquals(Arrays.asList(
+        ContainerEventType.INIT_CONTAINER,
+        ContainerEventType.RESOURCE_LOCALIZED,
+        ContainerEventType.CONTAINER_LAUNCHED,
+        ContainerEventType.REINITIALIZE_CONTAINER,
+        ContainerEventType.RESOURCE_LOCALIZED,
+        ContainerEventType.UPDATE_DIAGNOSTICS_MSG,
+        ContainerEventType.CONTAINER_KILLED_ON_REQUEST,
+        ContainerEventType.CONTAINER_LAUNCHED,
+        ContainerEventType.UPDATE_DIAGNOSTICS_MSG,
+        ContainerEventType.CONTAINER_EXITED_WITH_FAILURE,
+        ContainerEventType.CONTAINER_LAUNCHED), containerEventTypes);
   }
 
   /**
@@ -1582,16 +1799,12 @@ public class TestContainerManager extends BaseContainerManagerTest {
         containerManager.updateContainer(updateRequest);
     // Check response
     Assert.assertEquals(
-        0, updateResponse.getSuccessfullyUpdatedContainers().size());
-    Assert.assertEquals(2, updateResponse.getFailedRequests().size());
+        1, updateResponse.getSuccessfullyUpdatedContainers().size());
+    Assert.assertEquals(1, updateResponse.getFailedRequests().size());
     for (Map.Entry<ContainerId, SerializedException> entry : updateResponse
         .getFailedRequests().entrySet()) {
       Assert.assertNotNull("Failed message", entry.getValue().getMessage());
-      if (cId0.equals(entry.getKey())) {
-        Assert.assertTrue(entry.getValue().getMessage()
-          .contains("Resource can only be changed when a "
-              + "container is in RUNNING or SCHEDULED state"));
-      } else if (cId7.equals(entry.getKey())) {
+      if (cId7.equals(entry.getKey())) {
         Assert.assertTrue(entry.getValue().getMessage()
             .contains("Container " + cId7.toString()
                 + " is not handled by this NodeManager"));

+ 0 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/TestContainer.java

@@ -122,8 +122,6 @@ public class TestContainer {
     try {
       wc = new WrappedContainer(7, 314159265358979L, 4344, "yak");
       assertEquals(ContainerState.NEW, wc.c.getContainerState());
-      ContainerImpl container = (ContainerImpl)wc.c;
-      assertEquals(org.apache.hadoop.yarn.api.records.ContainerState.NEW, container.getCurrentState());
       wc.initContainer();
 
       // Verify request for public/private resources to localizer
@@ -133,7 +131,6 @@ public class TestContainer {
               LocalResourceVisibility.APPLICATION));
       verify(wc.localizerBus).handle(argThat(matchesReq));
       assertEquals(ContainerState.LOCALIZING, wc.c.getContainerState());
-      assertEquals(org.apache.hadoop.yarn.api.records.ContainerState.SCHEDULED, container.getCurrentState());
     }
     finally {
       if (wc != null) {

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

@@ -23,6 +23,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 
@@ -47,11 +48,17 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.security.NMTokenIdentifier;
 import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit;
 import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
+import org.apache.hadoop.yarn.server.nodemanager.ContainerStateTransitionListener;
+import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor;
 import org.apache.hadoop.yarn.server.nodemanager.DeletionService;
+import org.apache.hadoop.yarn.server.nodemanager.NodeManager;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.BaseContainerManagerTest;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEvent;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEventType;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerImpl;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerState;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitorImpl;
@@ -76,6 +83,40 @@ public class TestContainerSchedulerQueuing extends BaseContainerManagerTest {
     LOG = LoggerFactory.getLogger(TestContainerSchedulerQueuing.class);
   }
 
+  private static class Listener implements ContainerStateTransitionListener {
+
+    private final Map<ContainerId,
+        List<ContainerState>> states = new HashMap<>();
+    private final Map<ContainerId, List<ContainerEventType>> events =
+        new HashMap<>();
+
+    @Override
+    public void init(Context context) {}
+
+    @Override
+    public void preTransition(ContainerImpl op,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState beforeState,
+        ContainerEvent eventToBeProcessed) {
+      if (!states.containsKey(op.getContainerId())) {
+        states.put(op.getContainerId(), new ArrayList<>());
+        states.get(op.getContainerId()).add(beforeState);
+        events.put(op.getContainerId(), new ArrayList<>());
+      }
+    }
+
+    @Override
+    public void postTransition(ContainerImpl op,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState beforeState,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState afterState,
+        ContainerEvent processedEvent) {
+      states.get(op.getContainerId()).add(afterState);
+      events.get(op.getContainerId()).add(processedEvent.getType());
+    }
+  }
+
   private boolean delayContainers = true;
 
   @Override
@@ -542,6 +583,10 @@ public class TestContainerSchedulerQueuing extends BaseContainerManagerTest {
     containerManager.start();
     containerManager.getContainerScheduler().
         setUsePauseEventForPreemption(true);
+
+    Listener listener = new Listener();
+    ((NodeManager.DefaultContainerStateListener)containerManager.getContext().
+        getContainerStateTransitionListener()).addListener(listener);
     ContainerLaunchContext containerLaunchContext =
         recordFactory.newRecordInstance(ContainerLaunchContext.class);
 
@@ -606,6 +651,39 @@ public class TestContainerSchedulerQueuing extends BaseContainerManagerTest {
     // starts running
     BaseContainerManagerTest.waitForNMContainerState(containerManager,
         createContainerId(0), ContainerState.DONE, 40);
+
+    List<org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+        ContainerState> containerStates =
+        listener.states.get(createContainerId(0));
+    Assert.assertEquals(Arrays.asList(
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.NEW,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.SCHEDULED,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.RUNNING,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.PAUSING,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.PAUSED,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.RESUMING,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.RUNNING,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.EXITED_WITH_SUCCESS,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.DONE), containerStates);
+    List<ContainerEventType> containerEventTypes =
+        listener.events.get(createContainerId(0));
+    Assert.assertEquals(Arrays.asList(ContainerEventType.INIT_CONTAINER,
+        ContainerEventType.CONTAINER_LAUNCHED,
+        ContainerEventType.PAUSE_CONTAINER,
+        ContainerEventType.CONTAINER_PAUSED,
+        ContainerEventType.RESUME_CONTAINER,
+        ContainerEventType.CONTAINER_RESUMED,
+        ContainerEventType.CONTAINER_EXITED_WITH_SUCCESS,
+        ContainerEventType.CONTAINER_RESOURCES_CLEANEDUP), containerEventTypes);
   }
 
   /**
@@ -1068,6 +1146,9 @@ public class TestContainerSchedulerQueuing extends BaseContainerManagerTest {
   @Test
   public void testPromotionOfOpportunisticContainers() throws Exception {
     containerManager.start();
+    Listener listener = new Listener();
+    ((NodeManager.DefaultContainerStateListener)containerManager.getContext().
+        getContainerStateTransitionListener()).addListener(listener);
 
     ContainerLaunchContext containerLaunchContext =
         recordFactory.newRecordInstance(ContainerLaunchContext.class);
@@ -1150,6 +1231,7 @@ public class TestContainerSchedulerQueuing extends BaseContainerManagerTest {
     containerStatuses = containerManager
         .getContainerStatuses(statRequest).getContainerStatuses();
     Assert.assertEquals(1, containerStatuses.size());
+
     for (ContainerStatus status : containerStatuses) {
       if (org.apache.hadoop.yarn.api.records.ContainerState.RUNNING ==
           status.getState()) {
@@ -1160,6 +1242,25 @@ public class TestContainerSchedulerQueuing extends BaseContainerManagerTest {
 
     // Ensure no containers are queued.
     Assert.assertEquals(0, containerScheduler.getNumQueuedContainers());
+
+    List<org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+        ContainerState> containerStates =
+        listener.states.get(createContainerId(1));
+    Assert.assertEquals(Arrays.asList(
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.NEW,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.SCHEDULED,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.SCHEDULED,
+        org.apache.hadoop.yarn.server.nodemanager.containermanager.container.
+            ContainerState.RUNNING), containerStates);
+    List<ContainerEventType> containerEventTypes =
+        listener.events.get(createContainerId(1));
+    Assert.assertEquals(Arrays.asList(
+        ContainerEventType.INIT_CONTAINER,
+        ContainerEventType.UPDATE_CONTAINER_TOKEN,
+        ContainerEventType.CONTAINER_LAUNCHED), containerEventTypes);
   }
 
   @Test

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

@@ -1459,7 +1459,7 @@ public class RMAppImpl implements RMApp, Recoverable {
       app.rmContext.getSystemMetricsPublisher()
           .appFinished(app, finalState, app.finishTime);
       // set the memory free
-      app.submissionContext.getAMContainerSpec().setTokensConf(null);
+      app.clearUnusedFields();
     };
   }
 
@@ -2021,4 +2021,13 @@ public class RMAppImpl implements RMApp, Recoverable {
   public void setApplicationPriority(Priority applicationPriority) {
     this.applicationPriority = applicationPriority;
   }
+
+  /**
+     * Clear Unused fields to free memory.
+     * @param app
+     */
+  private void clearUnusedFields() {
+    this.submissionContext.setAMContainerSpec(null);
+    this.submissionContext.setLogAggregationContext(null);
+  }
 }

+ 130 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockMemoryRMStateStore.java

@@ -0,0 +1,130 @@
+/**
+ * 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.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.MemoryRMStateStore;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.ApplicationStateData;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Test helper for MemoryRMStateStore will make sure the event.
+ */
+public class MockMemoryRMStateStore extends MemoryRMStateStore {
+
+  private Map<ApplicationId, ApplicationSubmissionContext> appSubCtxtCopy =
+      new HashMap<ApplicationId, ApplicationSubmissionContext>();
+
+  @SuppressWarnings("rawtypes")
+  @Override
+  protected EventHandler getRMStateStoreEventHandler() {
+    return rmStateStoreEventHandler;
+  }
+
+  @Override
+  public synchronized RMState loadState() throws Exception {
+
+    RMState cloneState = super.loadState();
+
+    for(Entry<ApplicationId, ApplicationStateData> state :
+        cloneState.getApplicationState().entrySet()) {
+      ApplicationStateData oldStateData = state.getValue();
+      oldStateData.setApplicationSubmissionContext(
+          this.appSubCtxtCopy.get(state.getKey()));
+      cloneState.getApplicationState().put(state.getKey(), oldStateData);
+    }
+    return cloneState;
+  }
+
+  @Override
+  public synchronized void storeApplicationStateInternal(
+      ApplicationId appId, ApplicationStateData appState)
+      throws Exception {
+    // Clone Application Submission Context
+    this.cloneAppSubmissionContext(appState);
+    super.storeApplicationStateInternal(appId, appState);
+  }
+
+  @Override
+  public synchronized void updateApplicationStateInternal(
+      ApplicationId appId, ApplicationStateData appState)
+      throws Exception {
+    // Clone Application Submission Context
+    this.cloneAppSubmissionContext(appState);
+    super.updateApplicationStateInternal(appId, appState);
+  }
+
+  /**
+   * Clone Application Submission Context and Store in Map for
+   * later use.
+   *
+   * @param appState
+   */
+  private void cloneAppSubmissionContext(ApplicationStateData appState) {
+    ApplicationSubmissionContext oldAppSubCtxt =
+        appState.getApplicationSubmissionContext();
+    ApplicationSubmissionContext context =
+        ApplicationSubmissionContext.newInstance(
+            oldAppSubCtxt.getApplicationId(),
+            oldAppSubCtxt.getApplicationName(),
+            oldAppSubCtxt.getQueue(),
+            oldAppSubCtxt.getPriority(),
+            oldAppSubCtxt.getAMContainerSpec(),
+            oldAppSubCtxt.getUnmanagedAM(),
+            oldAppSubCtxt.getCancelTokensWhenComplete(),
+            oldAppSubCtxt.getMaxAppAttempts(),
+            oldAppSubCtxt.getResource()
+            );
+    context.setAttemptFailuresValidityInterval(
+        oldAppSubCtxt.getAttemptFailuresValidityInterval());
+    context.setKeepContainersAcrossApplicationAttempts(
+        oldAppSubCtxt.getKeepContainersAcrossApplicationAttempts());
+    context.setAMContainerResourceRequests(
+        oldAppSubCtxt.getAMContainerResourceRequests());
+    context.setLogAggregationContext(oldAppSubCtxt.getLogAggregationContext());
+    context.setApplicationType(oldAppSubCtxt.getApplicationType());
+    this.appSubCtxtCopy.put(oldAppSubCtxt.getApplicationId(), context);
+  }
+
+  /**
+   * Traverse each app state and replace cloned app sub context
+   * into the state.
+   *
+   * @param actualState
+   * @return actualState
+   */
+  @VisibleForTesting
+  public RMState reloadStateWithClonedAppSubCtxt(RMState actualState) {
+    for(Entry<ApplicationId, ApplicationStateData> state :
+        actualState.getApplicationState().entrySet()) {
+      ApplicationStateData oldStateData = state.getValue();
+      oldStateData.setApplicationSubmissionContext(
+          this.appSubCtxtCopy.get(state.getKey()));
+      actualState.getApplicationState().put(state.getKey(),
+          oldStateData);
+    }
+    return actualState;
+  }
+}

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java

@@ -164,7 +164,7 @@ public class MockRM extends ResourceManager {
     } else {
       Class storeClass = getRMContext().getStateStore().getClass();
       if (storeClass.equals(MemoryRMStateStore.class)) {
-        MockRMMemoryStateStore mockStateStore = new MockRMMemoryStateStore();
+        MockMemoryRMStateStore mockStateStore = new MockMemoryRMStateStore();
         mockStateStore.init(conf);
         setRMStateStore(mockStateStore);
       } else if (storeClass.equals(NullRMStateStore.class)) {

+ 0 - 32
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRMMemoryStateStore.java

@@ -1,32 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.yarn.server.resourcemanager;
-
-import org.apache.hadoop.yarn.event.EventHandler;
-import org.apache.hadoop.yarn.server.resourcemanager.recovery.MemoryRMStateStore;
-
-/**
- * Test helper for MemoryRMStateStore will make sure the event.
- */
-public class MockRMMemoryStateStore extends MemoryRMStateStore {
-  @SuppressWarnings("rawtypes")
-  @Override
-  protected EventHandler getRMStateStoreEventHandler() {
-    return rmStateStoreEventHandler;
-  }
-}

+ 3 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationCleanup.java

@@ -293,6 +293,8 @@ public class TestApplicationCleanup {
     // start RM
     MockRM rm1 = new MockRM(conf);
     rm1.start();
+    MockMemoryRMStateStore memStore =
+        (MockMemoryRMStateStore) rm1.getRMStateStore();
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
     nm1.registerNode();
@@ -304,7 +306,7 @@ public class TestApplicationCleanup {
     rm1.waitForState(app0.getApplicationId(), RMAppState.FAILED);
 
     // start new RM
-    MockRM rm2 = new MockRM(conf, rm1.getRMStateStore());
+    MockRM rm2 = new MockRM(conf, memStore);
     rm2.start();
     
     // nm1 register to rm2, and do a heartbeat

+ 3 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestContainerResourceUsage.java

@@ -143,6 +143,8 @@ public class TestContainerResourceUsage {
     conf.set(YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName());
     MockRM rm0 = new MockRM(conf);
     rm0.start();
+    MockMemoryRMStateStore memStore =
+        (MockMemoryRMStateStore) rm0.getRMStateStore();
     MockNM nm =
         new MockNM("127.0.0.1:1234", 65536, rm0.getResourceTrackerService());
     nm.registerNode();
@@ -229,7 +231,7 @@ public class TestContainerResourceUsage {
         vcoreSeconds, metricsBefore.getVcoreSeconds());
 
     // create new RM to represent RM restart. Load up the state store.
-    MockRM rm1 = new MockRM(conf, rm0.getRMStateStore());
+    MockRM rm1 = new MockRM(conf, memStore);
     rm1.start();
     RMApp app0After =
         rm1.getRMContext().getRMApps().get(app0.getApplicationId());

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

@@ -415,7 +415,7 @@ public class TestRMHA {
     configuration.setBoolean(YarnConfiguration.AUTO_FAILOVER_ENABLED, false);
     Configuration conf = new YarnConfiguration(configuration);
 
-    MemoryRMStateStore memStore = new MockRMMemoryStateStore() {
+    MemoryRMStateStore memStore = new MockMemoryRMStateStore() {
       int count = 0;
 
       @Override
@@ -465,7 +465,7 @@ public class TestRMHA {
     configuration.setBoolean(YarnConfiguration.AUTO_FAILOVER_ENABLED, false);
     Configuration conf = new YarnConfiguration(configuration);
 
-    MemoryRMStateStore memStore = new MockRMMemoryStateStore() {
+    MemoryRMStateStore memStore = new MockMemoryRMStateStore() {
       @Override
       public void updateApplicationState(ApplicationStateData appState) {
         notifyStoreOperationFailed(new StoreFencedException());

+ 30 - 17
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java

@@ -199,7 +199,8 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
 
     // PHASE 1: create RM and get state
     MockRM rm1 = createMockRM(conf);
-    MemoryRMStateStore memStore = (MemoryRMStateStore) rm1.getRMStateStore();
+    MockMemoryRMStateStore memStore =
+        (MockMemoryRMStateStore) rm1.getRMStateStore();
     Map<ApplicationId, ApplicationStateData> rmAppState =
         memStore.getState().getApplicationState();
 
@@ -679,7 +680,7 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
   @Test (timeout = 60000)
   public void testRMRestartWaitForPreviousSucceededAttempt() throws Exception {
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
-    MemoryRMStateStore memStore = new MockRMMemoryStateStore() {
+    MemoryRMStateStore memStore = new MockMemoryRMStateStore() {
       int count = 0;
 
       @Override
@@ -734,7 +735,8 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1);
     // create RM
     MockRM rm1 = createMockRM(conf);
-    MemoryRMStateStore memStore = (MemoryRMStateStore) rm1.getRMStateStore();
+    MockMemoryRMStateStore memStore =
+        (MockMemoryRMStateStore) rm1.getRMStateStore();
     Map<ApplicationId, ApplicationStateData> rmAppState =
         memStore.getState().getApplicationState();
     // start RM
@@ -780,7 +782,8 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
       YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS);
     // create RM
     MockRM rm1 = createMockRM(conf);
-    MemoryRMStateStore memStore = (MemoryRMStateStore) rm1.getRMStateStore();
+    MockMemoryRMStateStore memStore =
+        (MockMemoryRMStateStore) rm1.getRMStateStore();
     Map<ApplicationId, ApplicationStateData> rmAppState =
         memStore.getState().getApplicationState();
     // start RM
@@ -824,18 +827,18 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
 
   @Test (timeout = 60000)
   public void testRMRestartKilledAppWithNoAttempts() throws Exception {
-    MemoryRMStateStore memStore = new MockRMMemoryStateStore() {
+    MockMemoryRMStateStore memStore = new MockMemoryRMStateStore() {
       @Override
       public synchronized void storeApplicationAttemptStateInternal(
-          ApplicationAttemptId attemptId,
-          ApplicationAttemptStateData attemptStateData) throws Exception {
+          ApplicationAttemptId appAttemptId,
+          ApplicationAttemptStateData attemptState) throws Exception {
         // ignore attempt saving request.
       }
 
       @Override
       public synchronized void updateApplicationAttemptStateInternal(
-          ApplicationAttemptId attemptId,
-          ApplicationAttemptStateData attemptStateData) throws Exception {
+          ApplicationAttemptId appAttemptId,
+          ApplicationAttemptStateData attemptState) throws Exception {
         // ignore attempt saving request.
       }
     };
@@ -868,7 +871,8 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
       YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS);
     // PHASE 1: create RM and get state
     MockRM rm1 = createMockRM(conf);
-    MemoryRMStateStore memStore = (MemoryRMStateStore) rm1.getRMStateStore();
+    MockMemoryRMStateStore memStore =
+        (MockMemoryRMStateStore) rm1.getRMStateStore();
     Map<ApplicationId, ApplicationStateData> rmAppState =
         memStore.getState().getApplicationState();
 
@@ -926,6 +930,9 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
         new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
     nm1.registerNode();
 
+    MockMemoryRMStateStore memStore =
+        (MockMemoryRMStateStore) rm1.getRMStateStore();
+
     // a succeeded app.
     RMApp app0 = rm1.submitApp(200, "name", "user", null,
       false, "default", 1, null, "myType");
@@ -953,7 +960,7 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
     .appCreated(any(RMApp.class), anyLong());
     // restart rm
 
-    MockRM rm2 = new MockRM(conf, rm1.getRMStateStore()) {
+    MockRM rm2 = new MockRM(conf, memStore) {
       @Override
       protected RMAppManager createRMAppManager() {
         return spy(super.createRMAppManager());
@@ -1625,7 +1632,8 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
     // start RM
     MockRM rm1 = createMockRM(conf);
     rm1.start();
-    MemoryRMStateStore memStore = (MemoryRMStateStore) rm1.getRMStateStore();
+    MockMemoryRMStateStore memStore =
+        (MockMemoryRMStateStore) rm1.getRMStateStore();
     RMState rmState = memStore.getState();
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
@@ -1664,7 +1672,7 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
   // This is to test RM does not get hang on shutdown.
   @Test (timeout = 10000)
   public void testRMShutdown() throws Exception {
-    MemoryRMStateStore memStore = new MockRMMemoryStateStore() {
+    MemoryRMStateStore memStore = new MockMemoryRMStateStore() {
       @Override
       public synchronized void checkVersion()
           throws Exception {
@@ -1743,7 +1751,8 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
       }
     };
     rm1.start();
-    MemoryRMStateStore memStore = (MemoryRMStateStore) rm1.getRMStateStore();
+    MockMemoryRMStateStore memStore =
+        (MockMemoryRMStateStore) rm1.getRMStateStore();
     RMApp app1 = null;
     try {
        app1 = rm1.submitApp(200, "name", "user",
@@ -1767,7 +1776,7 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
 
   @Test (timeout = 20000)
   public void testAppRecoveredInOrderOnRMRestart() throws Exception {
-    MemoryRMStateStore memStore = new MockRMMemoryStateStore();
+    MemoryRMStateStore memStore = new MemoryRMStateStore();
     memStore.init(conf);
 
     for (int i = 10; i > 0; i--) {
@@ -2405,6 +2414,8 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
     MockRM rm1 = new MockRM(conf);
     rm1.start();
     CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
+    MockMemoryRMStateStore memStore =
+        (MockMemoryRMStateStore) rm1.getRMStateStore();
 
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
@@ -2441,7 +2452,7 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
     MockRM rm2 = null;
     // start RM2
     try {
-      rm2 = new MockRM(conf, rm1.getRMStateStore());
+      rm2 = new MockRM(conf, memStore);
       rm2.start();
       Assert.assertTrue("RM start successfully", true);
     } catch (Exception e) {
@@ -2542,6 +2553,8 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
       }
     };
     rm1.start();
+    MockMemoryRMStateStore memStore =
+        (MockMemoryRMStateStore) rm1.getRMStateStore();
 
     // add node label "x" and set node to label mapping
     Set<String> clusterNodeLabels = new HashSet<String>();
@@ -2568,7 +2581,7 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
     conf.setBoolean(YarnConfiguration.NODE_LABELS_ENABLED, false);
     MockRM rm2 = new MockRM(
         TestUtils.getConfigurationWithDefaultQueueLabels(conf),
-        rm1.getRMStateStore()) {
+        memStore) {
       @Override
       protected RMNodeLabelsManager createNodeLabelManager() {
         RMNodeLabelsManager mgr = new RMNodeLabelsManager();

+ 37 - 14
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java

@@ -572,6 +572,8 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     conf.set(YarnConfiguration.YARN_ADMIN_ACL, "");
     rm1 = new MockRM(conf);
     rm1.start();
+    MockMemoryRMStateStore memStore =
+        (MockMemoryRMStateStore) rm1.getRMStateStore();
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 8192, rm1.getResourceTrackerService());
     nm1.registerNode();
@@ -583,7 +585,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[]{QUEUE_DOESNT_EXIST});
     final String noQueue = CapacitySchedulerConfiguration.ROOT + "." + QUEUE_DOESNT_EXIST;
     csConf.setCapacity(noQueue, 100);
-    rm2 = new MockRM(csConf, rm1.getRMStateStore());
+    rm2 = new MockRM(csConf, memStore);
 
     rm2.start();
     UserGroupInformation user2 = UserGroupInformation.createRemoteUser("user2");
@@ -721,11 +723,15 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
 
   private void verifyAppRecoveryWithWrongQueueConfig(
       CapacitySchedulerConfiguration csConf, RMApp app, String diagnostics,
-      MemoryRMStateStore memStore, RMState state) throws Exception {
+      MockMemoryRMStateStore memStore, RMState state) throws Exception {
     // Restart RM with fail-fast as false. App should be killed.
     csConf.setBoolean(YarnConfiguration.RM_FAIL_FAST, false);
     rm2 = new MockRM(csConf, memStore);
     rm2.start();
+
+    MockMemoryRMStateStore memStore2 =
+        (MockMemoryRMStateStore) rm2.getRMStateStore();
+
     // Wait for app to be killed.
     rm2.waitForState(app.getApplicationId(), RMAppState.KILLED);
     ApplicationReport report = rm2.getApplicationReport(app.getApplicationId());
@@ -734,24 +740,27 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     assertEquals(report.getYarnApplicationState(), YarnApplicationState.KILLED);
     assertEquals(report.getDiagnostics(), diagnostics);
 
+    //Reload previous state with cloned app sub context object
+    RMState newState = memStore2.reloadStateWithClonedAppSubCtxt(state);
+
     // Remove updated app info(app being KILLED) from state store and reinstate
     // state store to previous state i.e. which indicates app is RUNNING.
     // This is to simulate app recovery with fail fast config as true.
     for(Map.Entry<ApplicationId, ApplicationStateData> entry :
-        state.getApplicationState().entrySet()) {
+        newState.getApplicationState().entrySet()) {
       ApplicationStateData appState = mock(ApplicationStateData.class);
       ApplicationSubmissionContext ctxt =
           mock(ApplicationSubmissionContext.class);
       when(appState.getApplicationSubmissionContext()).thenReturn(ctxt);
       when(ctxt.getApplicationId()).thenReturn(entry.getKey());
-      memStore.removeApplicationStateInternal(appState);
-      memStore.storeApplicationStateInternal(
+      memStore2.removeApplicationStateInternal(appState);
+      memStore2.storeApplicationStateInternal(
           entry.getKey(), entry.getValue());
     }
 
     // Now restart RM with fail-fast as true. QueueException should be thrown.
     csConf.setBoolean(YarnConfiguration.RM_FAIL_FAST, true);
-    MockRM rm = new MockRM(csConf, memStore);
+    MockRM rm = new MockRM(csConf, memStore2);
     try {
       rm.start();
       Assert.fail("QueueException must have been thrown");
@@ -781,6 +790,9 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     setupQueueConfiguration(csConf);
     rm1 = new MockRM(csConf);
     rm1.start();
+
+    MockMemoryRMStateStore memStore =
+        (MockMemoryRMStateStore) rm1.getRMStateStore();
     MockNM nm =
         new MockNM("127.1.1.1:4321", 8192, rm1.getResourceTrackerService());
     nm.registerNode();
@@ -801,7 +813,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     String diags = "Application killed on recovery as it was submitted to " +
         "queue QueueB which is no longer a leaf queue after restart.";
     verifyAppRecoveryWithWrongQueueConfig(csConf, app, diags,
-        (MemoryRMStateStore) rm1.getRMStateStore(), state);
+        memStore, state);
   }
 
   //Test behavior of an app if queue is removed during recovery. Test case does
@@ -826,6 +838,8 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     setupQueueConfiguration(csConf);
     rm1 = new MockRM(csConf);
     rm1.start();
+    MockMemoryRMStateStore memStore =
+        (MockMemoryRMStateStore) rm1.getRMStateStore();
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 8192, rm1.getResourceTrackerService());
     MockNM nm2 =
@@ -853,7 +867,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     rm1.clearQueueMetrics(app2);
 
     // Take a copy of state store so that it can be reset to this state.
-    RMState state = rm1.getRMStateStore().loadState();
+    RMState state = memStore.loadState();
 
     // Set new configuration with QueueB removed.
     csConf = new CapacitySchedulerConfiguration(conf);
@@ -862,7 +876,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     String diags = "Application killed on recovery as it was submitted to " +
         "queue QueueB which no longer exists after restart.";
     verifyAppRecoveryWithWrongQueueConfig(csConf, app2, diags,
-        (MemoryRMStateStore) rm1.getRMStateStore(), state);
+        memStore, state);
   }
 
   private void checkParentQueue(ParentQueue parentQueue, int numContainers,
@@ -931,6 +945,8 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
   public void testContainersNotRecoveredForCompletedApps() throws Exception {
     rm1 = new MockRM(conf);
     rm1.start();
+    MockMemoryRMStateStore memStore =
+        (MockMemoryRMStateStore) rm1.getRMStateStore();
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 8192, rm1.getResourceTrackerService());
     nm1.registerNode();
@@ -938,7 +954,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
     MockRM.finishAMAndVerifyAppState(app1, rm1, nm1, am1);
 
-    rm2 = new MockRM(conf, rm1.getRMStateStore());
+    rm2 = new MockRM(conf, memStore);
     rm2.start();
     nm1.setResourceTrackerService(rm2.getResourceTrackerService());
     NMContainerStatus runningContainer =
@@ -1212,6 +1228,9 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     // start RM
     rm1 = new MockRM(conf);
     rm1.start();
+
+    MockMemoryRMStateStore memStore =
+        (MockMemoryRMStateStore) rm1.getRMStateStore();
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
     nm1.registerNode();
@@ -1230,7 +1249,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
 
 
     // start new RM
-    rm2 = new MockRM(conf, rm1.getRMStateStore());
+    rm2 = new MockRM(conf, memStore);
     rm2.start();
 
     am0.setAMRMProtocol(rm2.getApplicationMasterService(), rm2.getRMContext());
@@ -1370,7 +1389,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
   // RM should start correctly.
   @Test (timeout = 20000)
   public void testAppStateSavedButAttemptStateNotSaved() throws Exception {
-    MemoryRMStateStore memStore = new MemoryRMStateStore() {
+    MockMemoryRMStateStore memStore = new MockMemoryRMStateStore() {
       @Override public synchronized void updateApplicationAttemptStateInternal(
           ApplicationAttemptId appAttemptId,
           ApplicationAttemptStateData attemptState) {
@@ -1414,6 +1433,8 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     // start RM
     rm1 = new MockRM(conf);
     rm1.start();
+    MockMemoryRMStateStore memStore =
+        (MockMemoryRMStateStore) rm1.getRMStateStore();
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
     nm1.registerNode();
@@ -1438,8 +1459,10 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     }
 
     // start new RM
-    rm2 = new MockRM(conf, rm1.getRMStateStore());
+    rm2 = new MockRM(conf, memStore);
     rm2.start();
+    MockMemoryRMStateStore memStore2 =
+        (MockMemoryRMStateStore) rm2.getRMStateStore();
     rm2.waitForState(app0.getApplicationId(), RMAppState.ACCEPTED);
     rm2.waitForState(am0.getApplicationAttemptId(), RMAppAttemptState.LAUNCHED);
 
@@ -1488,7 +1511,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
         recoveredApp.getFinalApplicationStatus());
 
     // Restart RM once more to check UAM is not re-run
-    MockRM rm3 = new MockRM(conf, rm1.getRMStateStore());
+    MockRM rm3 = new MockRM(conf, memStore2);
     rm3.start();
     recoveredApp = rm3.getRMContext().getRMApps().get(app0.getApplicationId());
     Assert.assertEquals(RMAppState.FINISHED, recoveredApp.getState());

+ 10 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java

@@ -41,6 +41,7 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus;
 import org.apache.hadoop.yarn.server.resourcemanager.MockAM;
 import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
+import org.apache.hadoop.yarn.server.resourcemanager.MockMemoryRMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.MemoryRMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.ApplicationStateData;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
@@ -702,8 +703,11 @@ public class TestAMRestart {
     // explicitly set max-am-retry count as 2.
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
     MockRM rm1 = new MockRM(conf);
-    MemoryRMStateStore memStore = (MemoryRMStateStore) rm1.getRMStateStore();
     rm1.start();
+
+    MockMemoryRMStateStore memStore =
+        (MockMemoryRMStateStore) rm1.getRMStateStore();
+
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 8000, rm1.getResourceTrackerService());
     nm1.registerNode();
@@ -735,7 +739,6 @@ public class TestAMRestart {
     RMAppImpl app1 = (RMAppImpl)rm1.submitApp(200, 10000, false);
     app1.setSystemClock(clock);
     MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
-
     // Fail attempt1 normally
     nm1.nodeHeartbeat(am1.getApplicationAttemptId(),
       1, ContainerState.COMPLETE);
@@ -771,8 +774,12 @@ public class TestAMRestart {
     @SuppressWarnings("resource")
     MockRM rm2 = new MockRM(conf, memStore);
     rm2.start();
+
+    MockMemoryRMStateStore memStore1 =
+        (MockMemoryRMStateStore) rm2.getRMStateStore();
     ApplicationStateData app1State =
-        memStore.getState().getApplicationState().get(app1.getApplicationId());
+        memStore1.getState().getApplicationState().
+        get(app1.getApplicationId());
     Assert.assertEquals(1, app1State.getFirstAttemptId());
 
     // re-register the NM

+ 185 - 14
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java

@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.rmapp;
 
+import static org.junit.Assert.assertEquals;
 import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anyLong;
 import static org.mockito.Matchers.eq;
@@ -28,17 +29,23 @@ import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 
+import java.io.File;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Map;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.UnsupportedFileSystemException;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.security.Credentials;
@@ -46,14 +53,22 @@ import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
 import org.apache.hadoop.yarn.MockApps;
+import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.hadoop.yarn.api.records.ContainerRetryContext;
+import org.apache.hadoop.yarn.api.records.ContainerRetryPolicy;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.api.records.LocalResourceType;
+import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
+import org.apache.hadoop.yarn.api.records.LogAggregationContext;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.api.records.URL;
 import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationSubmissionContextPBImpl;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.DrainDispatcher;
@@ -251,7 +266,113 @@ public class TestRMAppTransitions {
     rmDispatcher.start();
   }
 
-  protected RMApp createNewTestApp(ApplicationSubmissionContext submissionContext) {
+  private ByteBuffer getTokens() throws IOException {
+    Credentials ts = new Credentials();
+    DataOutputBuffer dob = new DataOutputBuffer();
+    ts.writeTokenStorageToStream(dob);
+    ByteBuffer securityTokens =
+        ByteBuffer.wrap(dob.getData(), 0, dob.getLength());
+    return securityTokens;
+  }
+
+  private ByteBuffer getTokensConf() throws IOException {
+
+    DataOutputBuffer dob = new DataOutputBuffer();
+    Configuration appConf = new Configuration(false);
+    appConf.clear();
+    appConf.set("dfs.nameservices", "mycluster1,mycluster2");
+    appConf.set("dfs.namenode.rpc-address.mycluster2.nn1",
+        "123.0.0.1");
+    appConf.set("dfs.namenode.rpc-address.mycluster3.nn2",
+        "123.0.0.2");
+    appConf.write(dob);
+    ByteBuffer tokenConf =
+        ByteBuffer.wrap(dob.getData(), 0, dob.getLength());
+    return tokenConf;
+  }
+
+  private Map<String, LocalResource> getLocalResources()
+      throws UnsupportedFileSystemException {
+    FileContext localFS = FileContext.getLocalFSFileContext();
+    File tmpDir = new File("target");
+    File scriptFile = new File(tmpDir, "scriptFile.sh");
+    URL resourceURL =
+        URL.fromPath(localFS
+            .makeQualified(new Path(scriptFile.getAbsolutePath())));
+    LocalResource localRes =
+        Records.newRecord(LocalResource.class);
+    localRes.setResource(resourceURL);
+    localRes.setSize(-1);
+    localRes.setVisibility(LocalResourceVisibility.APPLICATION);
+    localRes.setType(LocalResourceType.FILE);
+    localRes.setTimestamp(scriptFile.lastModified());
+    String destinationFile = "dest_file";
+    Map<String, LocalResource> localResources =
+        new HashMap<String, LocalResource>();
+    localResources.put(destinationFile, localRes);
+    return localResources;
+  }
+
+  private Map<String, String> getEnvironment() {
+    Map<String, String> userSetEnv = new HashMap<String, String>();
+    userSetEnv.put(Environment.CONTAINER_ID.name(), "user_set_container_id");
+    userSetEnv.put(Environment.NM_HOST.name(), "user_set_NM_HOST");
+    userSetEnv.put(Environment.NM_PORT.name(), "user_set_NM_PORT");
+    userSetEnv.put(Environment.NM_HTTP_PORT.name(), "user_set_NM_HTTP_PORT");
+    userSetEnv.put(Environment.LOCAL_DIRS.name(), "user_set_LOCAL_DIR");
+    userSetEnv.put(Environment.USER.key(), "user_set_" +
+        Environment.USER.key());
+    userSetEnv.put(Environment.LOGNAME.name(), "user_set_LOGNAME");
+    userSetEnv.put(Environment.PWD.name(), "user_set_PWD");
+    userSetEnv.put(Environment.HOME.name(), "user_set_HOME");
+    return userSetEnv;
+  }
+
+  private ContainerRetryContext getContainerRetryContext() {
+    ContainerRetryContext containerRetryContext = ContainerRetryContext
+        .newInstance(
+            ContainerRetryPolicy.RETRY_ON_SPECIFIC_ERROR_CODES,
+            new HashSet<>(Arrays.asList(Integer.valueOf(111))), 0, 0);
+    return containerRetryContext;
+  }
+
+  private Map<String, ByteBuffer> getServiceData() {
+    Map<String, ByteBuffer> serviceData = new HashMap<String, ByteBuffer>();
+    String serviceName = "non_exist_auxService";
+    serviceData.put(serviceName, ByteBuffer.wrap(serviceName.getBytes()));
+    return serviceData;
+  }
+
+  private ContainerLaunchContext prepareContainerLaunchContext()
+      throws IOException {
+
+    ContainerLaunchContext clc =
+        Records.newRecord(ContainerLaunchContext.class);
+    clc.setCommands(Arrays.asList("/bin/sleep 5"));
+    if (UserGroupInformation.isSecurityEnabled()) {
+      clc.setTokens(getTokens());
+      clc.setTokensConf(getTokensConf());
+    }
+    clc.setLocalResources(getLocalResources());
+    clc.setEnvironment(getEnvironment());
+    clc.setContainerRetryContext(getContainerRetryContext());
+    clc.setServiceData(getServiceData());
+    return clc;
+  }
+
+  private LogAggregationContext getLogAggregationContext() {
+    LogAggregationContext logAggregationContext =
+        LogAggregationContext.newInstance(
+          "includePattern", "excludePattern",
+          "rolledLogsIncludePattern",
+          "rolledLogsExcludePattern",
+          "policyClass",
+          "policyParameters");
+    return logAggregationContext;
+  }
+
+  protected RMApp createNewTestApp(ApplicationSubmissionContext
+      submissionContext) throws IOException {
     ApplicationId applicationId = MockApps.newAppID(appId++);
     String user = MockApps.newUserName();
     String name = MockApps.newAppName();
@@ -270,7 +391,9 @@ public class TestRMAppTransitions {
     // but applicationId is still set for safety
     submissionContext.setApplicationId(applicationId);
     submissionContext.setPriority(Priority.newInstance(0));
-    submissionContext.setAMContainerSpec(mock(ContainerLaunchContext.class));
+    submissionContext.setAMContainerSpec(prepareContainerLaunchContext());
+    submissionContext.setLogAggregationContext(getLogAggregationContext());
+
     RMApp application = new RMAppImpl(applicationId, rmContext, conf, name,
         user, queue, submissionContext, scheduler, masterService,
         System.currentTimeMillis(), "YARN", null,
@@ -405,6 +528,7 @@ public class TestRMAppTransitions {
     // verify sendATSCreateEvent() is get called during
     // AddApplicationToSchedulerTransition.
     verify(publisher).appCreated(eq(application), anyLong());
+    verifyRMAppFieldsForNonFinalTransitions(application);
     return application;
   }
 
@@ -422,6 +546,7 @@ public class TestRMAppTransitions {
     application.handle(event);
     assertStartTimeSet(application);
     assertAppState(RMAppState.SUBMITTED, application);
+    verifyRMAppFieldsForNonFinalTransitions(application);
     return application;
   }
 
@@ -530,6 +655,7 @@ public class TestRMAppTransitions {
     assertFailed(application,
         ".*Unmanaged application.*Failing the application.*");
     assertAppFinalStateSaved(application);
+    verifyRMAppFieldsForFinalTransitions(application);
   }
   
   @Test
@@ -539,6 +665,7 @@ public class TestRMAppTransitions {
     RMApp application = testCreateAppFinished(null, diagMsg);
     Assert.assertTrue("Finished application missing diagnostics",
         application.getDiagnostics().indexOf(diagMsg) != -1);
+    verifyRMAppFieldsForFinalTransitions(application);
   }
 
   @Test (timeout = 30000)
@@ -546,15 +673,7 @@ public class TestRMAppTransitions {
     LOG.info("--- START: testAppRecoverPath ---");
     ApplicationSubmissionContext sub =
         Records.newRecord(ApplicationSubmissionContext.class);
-    ContainerLaunchContext clc =
-        Records.newRecord(ContainerLaunchContext.class);
-    Credentials credentials = new Credentials();
-    DataOutputBuffer dob = new DataOutputBuffer();
-    credentials.writeTokenStorageToStream(dob);
-    ByteBuffer securityTokens =
-        ByteBuffer.wrap(dob.getData(), 0, dob.getLength());
-    clc.setTokens(securityTokens);
-    sub.setAMContainerSpec(clc);
+    sub.setAMContainerSpec(prepareContainerLaunchContext());
     testCreateAppSubmittedRecovery(sub);
   }
 
@@ -577,6 +696,7 @@ public class TestRMAppTransitions {
     assertAppFinalStateNotSaved(application);
     verifyApplicationFinished(RMAppState.KILLED);
     verifyAppRemovedSchedulerEvent(RMAppState.KILLED);
+    verifyRMAppFieldsForFinalTransitions(application);
   }
 
   @Test
@@ -594,6 +714,7 @@ public class TestRMAppTransitions {
     assertFailed(application, rejectedText);
     assertAppFinalStateSaved(application);
     verifyApplicationFinished(RMAppState.FAILED);
+    verifyRMAppFieldsForFinalTransitions(application);
   }
 
   @Test (timeout = 30000)
@@ -611,6 +732,7 @@ public class TestRMAppTransitions {
     assertFailed(application, rejectedText);
     assertAppFinalStateSaved(application);
     verifyApplicationFinished(RMAppState.FAILED);
+    verifyRMAppFieldsForFinalTransitions(application);
     rmContext.getStateStore().removeApplication(application);
   }
 
@@ -633,6 +755,7 @@ public class TestRMAppTransitions {
     assertKilled(application);
     verifyApplicationFinished(RMAppState.KILLED);
     verifyAppRemovedSchedulerEvent(RMAppState.KILLED);
+    verifyRMAppFieldsForFinalTransitions(application);
   }
 
   @Test (timeout = 30000)
@@ -650,6 +773,7 @@ public class TestRMAppTransitions {
     assertFailed(application, rejectedText);
     assertAppFinalStateSaved(application);
     verifyApplicationFinished(RMAppState.FAILED);
+    verifyRMAppFieldsForFinalTransitions(application);
   }
 
   @Test (timeout = 30000)
@@ -684,6 +808,7 @@ public class TestRMAppTransitions {
     assertFailed(application, rejectedText);
     assertAppFinalStateSaved(application);
     verifyApplicationFinished(RMAppState.FAILED);
+    verifyRMAppFieldsForFinalTransitions(application);
   }
 
   @Test
@@ -706,6 +831,7 @@ public class TestRMAppTransitions {
     assertAppFinalStateSaved(application);
     verifyApplicationFinished(RMAppState.KILLED);
     verifyAppRemovedSchedulerEvent(RMAppState.KILLED);
+    verifyRMAppFieldsForFinalTransitions(application);
   }
 
   @Test
@@ -769,8 +895,9 @@ public class TestRMAppTransitions {
     assertAppFinalStateSaved(application);
     verifyApplicationFinished(RMAppState.KILLED);
     verifyAppRemovedSchedulerEvent(RMAppState.KILLED);
+    verifyRMAppFieldsForFinalTransitions(application);
   }
-  
+
   @Test
   public void testAppAcceptedAttemptKilled() throws IOException,
       InterruptedException {
@@ -816,6 +943,7 @@ public class TestRMAppTransitions {
     assertKilled(application);
     verifyApplicationFinished(RMAppState.KILLED);
     verifyAppRemovedSchedulerEvent(RMAppState.KILLED);
+    verifyRMAppFieldsForFinalTransitions(application);
   }
 
   @Test
@@ -873,6 +1001,7 @@ public class TestRMAppTransitions {
     assertFailed(application, ".*Failing the application.*");
     assertAppFinalStateSaved(application);
     verifyApplicationFinished(RMAppState.FAILED);
+    verifyRMAppFieldsForFinalTransitions(application);
   }
 
   @Test
@@ -914,6 +1043,7 @@ public class TestRMAppTransitions {
     assertFinalAppStatus(FinalApplicationStatus.FAILED, application);
     Assert.assertTrue("Finished app missing diagnostics", application
       .getDiagnostics().indexOf(diagMsg) != -1);
+    verifyRMAppFieldsForFinalTransitions(application);
   }
 
   @Test
@@ -933,6 +1063,7 @@ public class TestRMAppTransitions {
     Assert.assertEquals("application diagnostics is not correct",
         "", diag.toString());
     verifyApplicationFinished(RMAppState.FINISHED);
+    verifyRMAppFieldsForFinalTransitions(application);
   }
 
   @Test (timeout = 30000)
@@ -962,6 +1093,7 @@ public class TestRMAppTransitions {
 
     assertTimesAtFinish(application);
     assertAppState(RMAppState.FAILED, application);
+    verifyRMAppFieldsForFinalTransitions(application);
   }
 
   @Test (timeout = 30000)
@@ -1016,6 +1148,7 @@ public class TestRMAppTransitions {
 
     assertTimesAtFinish(application);
     assertAppState(RMAppState.KILLED, application);
+    verifyRMAppFieldsForFinalTransitions(application);
   }
   
   @Test(timeout = 30000)
@@ -1061,11 +1194,12 @@ public class TestRMAppTransitions {
     RMAppState finalState = appState.getState();
     Assert.assertEquals("Application is not in finalState.", finalState,
         application.getState());
+    verifyRMAppFieldsForFinalTransitions(application);
   }
   
   public void createRMStateForApplications(
       Map<ApplicationId, ApplicationStateData> applicationState,
-      RMAppState rmAppState) {
+      RMAppState rmAppState) throws IOException {
     RMApp app = createNewTestApp(null);
     ApplicationStateData appState =
         ApplicationStateData.newInstance(app.getSubmitTime(), app.getStartTime(),
@@ -1075,7 +1209,7 @@ public class TestRMAppTransitions {
   }
   
   @Test
-  public void testGetAppReport() {
+  public void testGetAppReport() throws IOException {
     RMApp app = createNewTestApp(null);
     assertAppState(RMAppState.NEW, app);
     ApplicationReport report = app.createAndGetApplicationReport(null, true);
@@ -1109,4 +1243,41 @@ public class TestRMAppTransitions {
       Assert.assertEquals(finalState, appRemovedEvent.getFinalState());
     }
   }
+
+  private void verifyRMAppFieldsForNonFinalTransitions(RMApp application)
+      throws IOException {
+    assertEquals(Arrays.asList("/bin/sleep 5"),
+        application.getApplicationSubmissionContext().
+        getAMContainerSpec().getCommands());
+    assertEquals(getLocalResources(),
+        application.getApplicationSubmissionContext().
+        getAMContainerSpec().getLocalResources());
+    if(UserGroupInformation.isSecurityEnabled()) {
+      assertEquals(getTokens(),
+          application.getApplicationSubmissionContext().
+          getAMContainerSpec().getTokens());
+      assertEquals(getTokensConf(),
+          application.getApplicationSubmissionContext().
+          getAMContainerSpec().getTokensConf());
+    }
+    assertEquals(getEnvironment(),
+        application.getApplicationSubmissionContext().
+        getAMContainerSpec().getEnvironment());
+    assertEquals(getContainerRetryContext(),
+        application.getApplicationSubmissionContext().
+        getAMContainerSpec().getContainerRetryContext());
+    assertEquals(getServiceData(),
+        application.getApplicationSubmissionContext().
+        getAMContainerSpec().getServiceData());
+    assertEquals(getLogAggregationContext(),
+        application.getApplicationSubmissionContext().
+        getLogAggregationContext());
+  }
+
+  private void verifyRMAppFieldsForFinalTransitions(RMApp application) {
+    assertEquals(null, application.getApplicationSubmissionContext().
+        getAMContainerSpec());
+    assertEquals(null, application.getApplicationSubmissionContext().
+        getLogAggregationContext());
+  }
 }

+ 31 - 25
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

@@ -34,11 +34,9 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.Enumeration;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.PriorityQueue;
 import java.util.Set;
 import java.util.concurrent.BrokenBarrierException;
 import java.util.concurrent.CyclicBarrier;
@@ -52,7 +50,6 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.apache.hadoop.util.Time;
 import org.apache.hadoop.yarn.LocalConfigurationProvider;
 import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
@@ -157,12 +154,8 @@ import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.DominantResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
-import org.apache.log4j.Level;
-import org.apache.log4j.LogManager;
-import org.apache.log4j.Logger;
 import org.junit.After;
 import org.junit.Assert;
-import org.junit.Assume;
 import org.junit.Before;
 import org.junit.Test;
 import org.mockito.Mockito;
@@ -1055,13 +1048,16 @@ public class TestCapacityScheduler {
   @Test
   public void testCapacitySchedulerInfo() throws Exception {
     QueueInfo queueInfo = resourceManager.getResourceScheduler().getQueueInfo("a", true, true);
-    Assert.assertEquals(queueInfo.getQueueName(), "a");
-    Assert.assertEquals(queueInfo.getChildQueues().size(), 2);
+    Assert.assertEquals("Queue Name should be a", "a",
+        queueInfo.getQueueName());
+    Assert.assertEquals("Child Queues size should be 2", 2,
+        queueInfo.getChildQueues().size());
 
     List<QueueUserACLInfo> userACLInfo = resourceManager.getResourceScheduler().getQueueUserAclInfo();
     Assert.assertNotNull(userACLInfo);
     for (QueueUserACLInfo queueUserACLInfo : userACLInfo) {
-      Assert.assertEquals(getQueueCount(userACLInfo, queueUserACLInfo.getQueueName()), 1);
+      Assert.assertEquals(1, getQueueCount(userACLInfo,
+          queueUserACLInfo.getQueueName()));
     }
 
   }
@@ -3054,8 +3050,8 @@ public class TestCapacityScheduler {
     CSQueue rootQueue = cs.getRootQueue();
     CSQueue queueA = findQueue(rootQueue, A);
     CSQueue queueA1 = findQueue(queueA, A1);
-    assertEquals("queue max allocation", ((LeafQueue) queueA1)
-        .getMaximumAllocation().getMemorySize(), 4096);
+    assertEquals("queue max allocation", 4096, ((LeafQueue) queueA1)
+        .getMaximumAllocation().getMemorySize());
 
     setMaxAllocMb(conf, A1, 6144);
     setMaxAllocVcores(conf, A1, 3);
@@ -3868,12 +3864,16 @@ public class TestCapacityScheduler {
     cs.start();
 
     QueueInfo queueInfoA = cs.getQueueInfo("a", true, false);
-    Assert.assertEquals(queueInfoA.getQueueName(), "a");
-    Assert.assertEquals(queueInfoA.getDefaultNodeLabelExpression(), "x");
+    Assert.assertEquals("Queue Name should be a", "a",
+        queueInfoA.getQueueName());
+    Assert.assertEquals("Default Node Label Expression should be x", "x",
+        queueInfoA.getDefaultNodeLabelExpression());
 
     QueueInfo queueInfoB = cs.getQueueInfo("b", true, false);
-    Assert.assertEquals(queueInfoB.getQueueName(), "b");
-    Assert.assertEquals(queueInfoB.getDefaultNodeLabelExpression(), "y");
+    Assert.assertEquals("Queue Name should be b", "b",
+        queueInfoB.getQueueName());
+    Assert.assertEquals("Default Node Label Expression should be y", "y",
+        queueInfoB.getDefaultNodeLabelExpression());
   }
 
   @Test(timeout = 60000)
@@ -4147,20 +4147,26 @@ public class TestCapacityScheduler {
     Resource usedResource =
         resourceManager.getResourceScheduler()
             .getSchedulerNode(nm_0.getNodeId()).getAllocatedResource();
-    Assert.assertEquals(usedResource.getMemorySize(), 1 * GB);
-    Assert.assertEquals(usedResource.getVirtualCores(), 1);
+    Assert.assertEquals("Used Resource Memory Size should be 1GB", 1 * GB,
+        usedResource.getMemorySize());
+    Assert.assertEquals("Used Resource Virtual Cores should be 1", 1,
+        usedResource.getVirtualCores());
     // Check total resource of scheduler node is also changed to 1 GB 1 core
     Resource totalResource =
         resourceManager.getResourceScheduler()
             .getSchedulerNode(nm_0.getNodeId()).getTotalResource();
-    Assert.assertEquals(totalResource.getMemorySize(), 1 * GB);
-    Assert.assertEquals(totalResource.getVirtualCores(), 1);
+    Assert.assertEquals("Total Resource Memory Size should be 1GB", 1 * GB,
+        totalResource.getMemorySize());
+    Assert.assertEquals("Total Resource Virtual Cores should be 1", 1,
+        totalResource.getVirtualCores());
     // Check the available resource is 0/0
     Resource availableResource =
         resourceManager.getResourceScheduler()
             .getSchedulerNode(nm_0.getNodeId()).getUnallocatedResource();
-    Assert.assertEquals(availableResource.getMemorySize(), 0);
-    Assert.assertEquals(availableResource.getVirtualCores(), 0);
+    Assert.assertEquals("Available Resource Memory Size should be 0", 0,
+        availableResource.getMemorySize());
+    Assert.assertEquals("Available Resource Memory Size should be 0", 0,
+        availableResource.getVirtualCores());
   }
 
   @Test
@@ -4709,7 +4715,7 @@ public class TestCapacityScheduler {
 
     String targetQueue = "b1";
     CSQueue b1 = cs.getQueue(targetQueue);
-    Assert.assertEquals(b1.getState(), QueueState.RUNNING);
+    Assert.assertEquals(QueueState.RUNNING, b1.getState());
 
     // test if we can convert a leaf queue which is in RUNNING state
     conf = new CapacitySchedulerConfiguration();
@@ -4727,7 +4733,7 @@ public class TestCapacityScheduler {
     setupQueueConfiguration(conf);
     conf.set("yarn.scheduler.capacity.root.b.b1.state", "STOPPED");
     cs.reinitialize(conf, mockContext);
-    Assert.assertEquals(b1.getState(), QueueState.STOPPED);
+    Assert.assertEquals(QueueState.STOPPED, b1.getState());
 
     // test if we can convert a leaf queue which is in STOPPED state
     conf = new CapacitySchedulerConfiguration();
@@ -4740,7 +4746,7 @@ public class TestCapacityScheduler {
     }
     b1 = cs.getQueue(targetQueue);
     Assert.assertTrue(b1 instanceof ParentQueue);
-    Assert.assertEquals(b1.getState(), QueueState.RUNNING);
+    Assert.assertEquals(QueueState.RUNNING, b1.getState());
     Assert.assertTrue(!b1.getChildQueues().isEmpty());
   }
 

+ 3 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestRMDelegationTokens.java

@@ -38,7 +38,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenResponse;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
-import org.apache.hadoop.yarn.server.resourcemanager.MockRMMemoryStateStore;
+import org.apache.hadoop.yarn.server.resourcemanager.MockMemoryRMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.RMSecretManagerService;
 import org.apache.hadoop.yarn.server.resourcemanager.TestRMRestart.TestSecurityMockRM;
@@ -78,7 +78,7 @@ public class TestRMDelegationTokens {
     UserGroupInformation.getLoginUser()
         .setAuthenticationMethod(AuthenticationMethod.KERBEROS);
 
-    MemoryRMStateStore memStore = new MockRMMemoryStateStore();
+    MemoryRMStateStore memStore = new MockMemoryRMStateStore();
     memStore.init(conf);
     RMState rmState = memStore.getState();
 
@@ -132,7 +132,7 @@ public class TestRMDelegationTokens {
   // Test all expired keys are removed from state-store.
   @Test(timeout = 15000)
   public void testRemoveExpiredMasterKeyInRMStateStore() throws Exception {
-    MemoryRMStateStore memStore = new MockRMMemoryStateStore();
+    MemoryRMStateStore memStore = new MockMemoryRMStateStore();
     memStore.init(testConf);
     RMState rmState = memStore.getState();