Преглед изворни кода

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 година
родитељ
комит
7c58cfea8f
99 измењених фајлова са 4190 додато и 694 уклоњено
  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 com.google.common.base.Charsets;
 import org.apache.commons.collections.map.UnmodifiableMap;
 import org.apache.commons.collections.map.UnmodifiableMap;
+import org.apache.commons.io.FilenameUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
@@ -2811,6 +2812,7 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
       String confName = null;
       String confName = null;
       String confValue = null;
       String confValue = null;
       String confInclude = null;
       String confInclude = null;
+      String confTag = null;
       boolean confFinal = false;
       boolean confFinal = false;
       boolean fallbackAllowed = false;
       boolean fallbackAllowed = false;
       boolean fallbackEntered = false;
       boolean fallbackEntered = false;
@@ -2825,6 +2827,7 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
             confName = null;
             confName = null;
             confValue = null;
             confValue = null;
             confFinal = false;
             confFinal = false;
+            confTag = null;
             confSource.clear();
             confSource.clear();
 
 
             // First test for short format configuration
             // First test for short format configuration
@@ -2843,9 +2846,8 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
                 confSource.add(StringInterner.weakIntern(
                 confSource.add(StringInterner.weakIntern(
                     reader.getAttributeValue(i)));
                     reader.getAttributeValue(i)));
               } else if ("tag".equals(propertyAttr)) {
               } 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;
             break;
@@ -2937,9 +2939,7 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
             break;
             break;
           case "tag":
           case "tag":
             if (token.length() > 0) {
             if (token.length() > 0) {
-              //Read tags and put them in propertyTagsMap
-              readTagFromConfig(token.toString(), confName,
-                  confValue, confSource);
+              confTag = StringInterner.weakIntern(token.toString());
             }
             }
             break;
             break;
           case "include":
           case "include":
@@ -2956,6 +2956,11 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
               break;
               break;
             }
             }
             confSource.add(name);
             confSource.add(name);
+            //Read tags and put them in propertyTagsMap
+            if (confTag != null) {
+              readTagFromConfig(confTag, confName, confValue, confSource);
+            }
+
             DeprecatedKeyInfo keyInfo =
             DeprecatedKeyInfo keyInfo =
                 deprecations.getDeprecatedKeyMap().get(confName);
                 deprecations.getDeprecatedKeyMap().get(confName);
             if (keyInfo != null) {
             if (keyInfo != null) {
@@ -3001,21 +3006,24 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
         if (confSource.size() > 0) {
         if (confSource.size() > 0) {
           for (String source : confSource) {
           for (String source : confSource) {
             PropertyTag tag1 = this.getPropertyTag(tagStr,
             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 {
         } 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))
             propertyTagsMap.get(CorePropertyTag.valueOf(tagStr))
                 .setProperty(confName, confValue);
                 .setProperty(confName, confValue);
           } else {
           } else {
@@ -3025,11 +3033,11 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
                 props);
                 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 "
         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 org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.CryptoExtension;
 
 
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Strings;
 import com.google.common.base.Strings;
@@ -133,6 +134,9 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
 
 
   private final ValueQueue<EncryptedKeyVersion> encKeyVersionQueue;
   private final ValueQueue<EncryptedKeyVersion> encKeyVersionQueue;
 
 
+  private static final ObjectWriter WRITER =
+      new ObjectMapper().writerWithDefaultPrettyPrinter();
+
   private class EncryptedQueueRefiller implements
   private class EncryptedQueueRefiller implements
     ValueQueue.QueueRefiller<EncryptedKeyVersion> {
     ValueQueue.QueueRefiller<EncryptedKeyVersion> {
 
 
@@ -226,8 +230,7 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
   private static void writeJson(Object obj, OutputStream os)
   private static void writeJson(Object obj, OutputStream os)
       throws IOException {
       throws IOException {
     Writer writer = new OutputStreamWriter(os, StandardCharsets.UTF_8);
     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 javax.management.ObjectName;
 
 
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Preconditions;
 import com.google.common.util.concurrent.AtomicDoubleArray;
 import com.google.common.util.concurrent.AtomicDoubleArray;
 import org.apache.commons.lang.exception.ExceptionUtils;
 import org.apache.commons.lang.exception.ExceptionUtils;
@@ -128,6 +129,8 @@ public class DecayRpcScheduler implements RpcScheduler,
   public static final Logger LOG =
   public static final Logger LOG =
       LoggerFactory.getLogger(DecayRpcScheduler.class);
       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
   // 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
   // identity from all previous decay windows: idx 0 for decayed call count and
   // idx 1 for the raw call count
   // idx 1 for the raw call count
@@ -909,8 +912,7 @@ public class DecayRpcScheduler implements RpcScheduler,
       return "{}";
       return "{}";
     } else {
     } else {
       try {
       try {
-        ObjectMapper om = new ObjectMapper();
-        return om.writeValueAsString(decisions);
+        return WRITER.writeValueAsString(decisions);
       } catch (Exception e) {
       } catch (Exception e) {
         return "Error: " + e.getMessage();
         return "Error: " + e.getMessage();
       }
       }
@@ -919,8 +921,7 @@ public class DecayRpcScheduler implements RpcScheduler,
 
 
   public String getCallVolumeSummary() {
   public String getCallVolumeSummary() {
     try {
     try {
-      ObjectMapper om = new ObjectMapper();
-      return om.writeValueAsString(getDecayedCallCounts());
+      return WRITER.writeValueAsString(getDecayedCallCounts());
     } catch (Exception e) {
     } catch (Exception e) {
       return "Error: " + e.getMessage();
       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 {
       throws Throwable {
       long startTime = 0;
       long startTime = 0;
       if (LOG.isDebugEnabled()) {
       if (LOG.isDebugEnabled()) {
-        startTime = Time.now();
+        startTime = Time.monotonicNow();
       }
       }
 
 
       // if Tracing is on then start a new span for this rpc.
       // 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 (traceScope != null) traceScope.close();
       }
       }
       if (LOG.isDebugEnabled()) {
       if (LOG.isDebugEnabled()) {
-        long callTime = Time.now() - startTime;
+        long callTime = Time.monotonicNow() - startTime;
         LOG.debug("Call: " + method.getName() + " " + callTime);
         LOG.debug("Call: " + method.getName() + " " + callTime);
       }
       }
       return value.get();
       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.JsonNode;
 import com.fasterxml.jackson.databind.MappingJsonFactory;
 import com.fasterxml.jackson.databind.MappingJsonFactory;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectReader;
 import com.fasterxml.jackson.databind.node.ContainerNode;
 import com.fasterxml.jackson.databind.node.ContainerNode;
 import org.apache.log4j.Layout;
 import org.apache.log4j.Layout;
 import org.apache.log4j.helpers.ISO8601DateFormat;
 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.
    * configuration it must be done in a static intializer block.
    */
    */
   private static final JsonFactory factory = new MappingJsonFactory();
   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 DATE = "date";
   public static final String EXCEPTION_CLASS = "exceptionclass";
   public static final String EXCEPTION_CLASS = "exceptionclass";
   public static final String LEVEL = "level";
   public static final String LEVEL = "level";
@@ -252,8 +254,7 @@ public class Log4Json extends Layout {
    * @throws IOException on any parsing problems
    * @throws IOException on any parsing problems
    */
    */
   public static ContainerNode parse(String json) throws IOException {
   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)) {
     if (!(jsonNode instanceof ContainerNode)) {
       throw new IOException("Wrong JSON data: " + json);
       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;
 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.commons.lang.exception.ExceptionUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.codehaus.jackson.map.ObjectMapper;
 import org.codehaus.jackson.map.ObjectMapper;
+import org.codehaus.jackson.map.ObjectWriter;
 import org.slf4j.Logger;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
 
 
+import java.io.IOException;
+import java.util.LinkedHashMap;
+import java.util.Map;
+
 /**
 /**
  * Build a JSON dump of the metrics.
  * Build a JSON dump of the metrics.
  *
  *
@@ -44,6 +45,9 @@ public class MetricsJsonBuilder extends MetricsRecordBuilder {
   private final MetricsCollector parent;
   private final MetricsCollector parent;
   private Map<String, Object> innerMetrics = new LinkedHashMap<>();
   private Map<String, Object> innerMetrics = new LinkedHashMap<>();
 
 
+  private static final ObjectWriter WRITER =
+      new ObjectMapper().writer();
+
   /**
   /**
    * Build an instance.
    * Build an instance.
    * @param parent parent collector. Unused in this instance; only used for
    * @param parent parent collector. Unused in this instance; only used for
@@ -116,7 +120,7 @@ public class MetricsJsonBuilder extends MetricsRecordBuilder {
   @Override
   @Override
   public String toString() {
   public String toString() {
     try {
     try {
-      return new ObjectMapper().writeValueAsString(innerMetrics);
+      return WRITER.writeValueAsString(innerMetrics);
     } catch (IOException e) {
     } catch (IOException e) {
       LOG.warn("Failed to dump to Json.", e);
       LOG.warn("Failed to dump to Json.", e);
       return ExceptionUtils.getStackTrace(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,
   private void snapshotMetrics(MetricsSourceAdapter sa,
                                MetricsBufferBuilder bufferBuilder) {
                                MetricsBufferBuilder bufferBuilder) {
-    long startTime = Time.now();
+    long startTime = Time.monotonicNow();
     bufferBuilder.add(sa.name(), sa.getMetrics(collector, true));
     bufferBuilder.add(sa.name(), sa.getMetrics(collector, true));
     collector.clear();
     collector.clear();
-    snapshotStat.add(Time.now() - startTime);
+    snapshotStat.add(Time.monotonicNow() - startTime);
     LOG.debug("Snapshotted source "+ sa.name());
     LOG.debug("Snapshotted source "+ sa.name());
   }
   }
 
 
@@ -431,7 +431,7 @@ public class MetricsSystemImpl extends MetricsSystem implements MetricsSource {
   synchronized void publishMetrics(MetricsBuffer buffer, boolean immediate) {
   synchronized void publishMetrics(MetricsBuffer buffer, boolean immediate) {
     int dropped = 0;
     int dropped = 0;
     for (MetricsSinkAdapter sa : sinks.values()) {
     for (MetricsSinkAdapter sa : sinks.values()) {
-      long startTime = Time.now();
+      long startTime = Time.monotonicNow();
       boolean result;
       boolean result;
       if (immediate) {
       if (immediate) {
         result = sa.putMetricsImmediate(buffer); 
         result = sa.putMetricsImmediate(buffer); 
@@ -439,7 +439,7 @@ public class MetricsSystemImpl extends MetricsSystem implements MetricsSource {
         result = sa.putMetrics(buffer, logicalTime);
         result = sa.putMetrics(buffer, logicalTime);
       }
       }
       dropped += result ? 0 : 1;
       dropped += result ? 0 : 1;
-      publishStat.add(Time.now() - startTime);
+      publishStat.add(Time.monotonicNow() - startTime);
     }
     }
     droppedPubAll.incr(dropped);
     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;
 package org.apache.hadoop.security.token.delegation.web;
 
 
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectReader;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.security.SecurityUtil;
 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 CONTENT_TYPE = "Content-Type";
   private static final String APPLICATION_JSON_MIME = "application/json";
   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_GET = "GET";
   private static final String HTTP_PUT = "PUT";
   private static final String HTTP_PUT = "PUT";
 
 
@@ -316,8 +320,7 @@ public abstract class DelegationTokenAuthenticator implements Authenticator {
         if (contentType != null &&
         if (contentType != null &&
             contentType.contains(APPLICATION_JSON_MIME)) {
             contentType.contains(APPLICATION_JSON_MIME)) {
           try {
           try {
-            ObjectMapper mapper = new ObjectMapper();
-            ret = mapper.readValue(conn.getInputStream(), Map.class);
+            ret = READER.readValue(conn.getInputStream());
           } catch (Exception ex) {
           } catch (Exception ex) {
             throw new AuthenticationException(String.format(
             throw new AuthenticationException(String.format(
                 "'%s' did not handle the '%s' delegation token operation: %s",
                 "'%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;
 package org.apache.hadoop.util;
 
 
 import com.fasterxml.jackson.databind.ObjectMapper;
 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.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 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 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.
    * 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());
     json.put(ERROR_CLASSNAME_JSON, ex.getClass().getName());
     Map<String, Object> jsonResponse = new LinkedHashMap<String, Object>();
     Map<String, Object> jsonResponse = new LinkedHashMap<String, Object>();
     jsonResponse.put(ERROR_JSON, json);
     jsonResponse.put(ERROR_JSON, json);
-    ObjectMapper jsonMapper = new ObjectMapper();
     Writer writer = response.getWriter();
     Writer writer = response.getWriter();
-    jsonMapper.writerWithDefaultPrettyPrinter().writeValue(writer, jsonResponse);
+    WRITER.writeValue(writer, jsonResponse);
     writer.flush();
     writer.flush();
   }
   }
 
 
@@ -144,8 +150,7 @@ public class HttpExceptionUtils {
       InputStream es = null;
       InputStream es = null;
       try {
       try {
         es = conn.getErrorStream();
         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);
         json = (Map) json.get(ERROR_JSON);
         String exClass = (String) json.get(ERROR_CLASSNAME_JSON);
         String exClass = (String) json.get(ERROR_CLASSNAME_JSON);
         String exMsg = (String) json.get(ERROR_MESSAGE_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;
     boolean dumpStack = false;
     if (log.isInfoEnabled()) {
     if (log.isInfoEnabled()) {
       synchronized (ReflectionUtils.class) {
       synchronized (ReflectionUtils.class) {
-        long now = Time.now();
+        long now = Time.monotonicNow();
         if (now - previousLogTime >= minInterval * 1000) {
         if (now - previousLogTime >= minInterval * 1000) {
           previousLogTime = now;
           previousLogTime = now;
           dumpStack = true;
           dumpStack = true;
@@ -241,7 +241,7 @@ public class ReflectionUtils {
     boolean dumpStack = false;
     boolean dumpStack = false;
     if (log.isInfoEnabled()) {
     if (log.isInfoEnabled()) {
       synchronized (ReflectionUtils.class) {
       synchronized (ReflectionUtils.class) {
-        long now = Time.now();
+        long now = Time.monotonicNow();
         if (now - previousLogTime >= minInterval * 1000) {
         if (now - previousLogTime >= minInterval * 1000) {
           previousLogTime = now;
           previousLogTime = now;
           dumpStack = true;
           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
 The minimum required versions of the native components on which Hadoop depends
 at compile time and/or runtime SHALL be considered
 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
 ### 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) |
 | `Syncs3600s90thPercentileLatencyMicros` | The 90th percentile of sync latency in microseconds (1 hour granularity) |
 | `Syncs3600s95thPercentileLatencyMicros` | The 95th 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) |
 | `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 |
 | `BatchesWritten` | Total number of batches written since startup |
 | `TxnsWritten` | Total number of transactions written since startup |
 | `TxnsWritten` | Total number of transactions written since startup |
 | `BytesWritten` | Total number of bytes 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;
   private Configuration conf;
   final static String CONFIG = new File("./test-config-TestConfiguration.xml").getAbsolutePath();
   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 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_ENUM = new File("./test-config-enum-TestConfiguration.xml").getAbsolutePath();
   final static String CONFIG_FOR_URI = "file://"
   final static String CONFIG_FOR_URI = "file://"
       + new File("./test-config-uri-TestConfiguration.xml").getAbsolutePath();
       + 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(new URI(CONFIG_FOR_URI)).delete();
     new File(CONFIG_MULTI_BYTE).delete();
     new File(CONFIG_MULTI_BYTE).delete();
     new File(CONFIG_MULTI_BYTE_SAVED).delete();
     new File(CONFIG_MULTI_BYTE_SAVED).delete();
+    new File(CONFIG_CORE).delete();
   }
   }
 
 
   private void startConfig() throws IOException{
   private void startConfig() throws IOException{
@@ -2248,14 +2251,14 @@ public class TestConfiguration {
   @Test
   @Test
   public void testGetAllPropertiesByTags() throws Exception {
   public void testGetAllPropertiesByTags() throws Exception {
 
 
-    out = new BufferedWriter(new FileWriter(CONFIG));
+    out = new BufferedWriter(new FileWriter(CONFIG_CORE));
     startConfig();
     startConfig();
     appendPropertyByTag("dfs.cblock.trace.io", "false", "DEBUG");
     appendPropertyByTag("dfs.cblock.trace.io", "false", "DEBUG");
     appendPropertyByTag("dfs.replication", "1", "PERFORMANCE,REQUIRED");
     appendPropertyByTag("dfs.replication", "1", "PERFORMANCE,REQUIRED");
     appendPropertyByTag("dfs.namenode.logging.level", "INFO", "CLIENT,DEBUG");
     appendPropertyByTag("dfs.namenode.logging.level", "INFO", "CLIENT,DEBUG");
     endConfig();
     endConfig();
 
 
-    Path fileResource = new Path(CONFIG);
+    Path fileResource = new Path(CONFIG_CORE);
     conf.addResource(fileResource);
     conf.addResource(fileResource);
     conf.getProps();
     conf.getProps();
 
 
@@ -2266,6 +2269,10 @@ public class TestConfiguration {
     tagList.add(CorePropertyTag.CLIENT);
     tagList.add(CorePropertyTag.CLIENT);
 
 
     Properties properties = conf.getAllPropertiesByTags(tagList);
     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(3, properties.size());
     assertEq(true, properties.containsKey("dfs.namenode.logging.level"));
     assertEq(true, properties.containsKey("dfs.namenode.logging.level"));
     assertEq(true, properties.containsKey("dfs.replication"));
     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 */
      * the entire file */
     private void read() throws IOException {
     private void read() throws IOException {
       String fileName = files.get(r.nextInt(files.size()));
       String fileName = files.get(r.nextInt(files.size()));
-      long startTime = Time.now();
+      long startTimestamp = Time.monotonicNow();
       InputStream in = fc.open(new Path(fileName));
       InputStream in = fc.open(new Path(fileName));
-      executionTime[OPEN] += (Time.now()-startTime);
+      executionTime[OPEN] += (Time.monotonicNow() - startTimestamp);
       totalNumOfOps[OPEN]++;
       totalNumOfOps[OPEN]++;
       while (in.read(buffer) != -1) {}
       while (in.read(buffer) != -1) {}
       in.close();
       in.close();
@@ -299,9 +299,9 @@ public class LoadGenerator extends Configured implements Tool {
       double fileSize = 0;
       double fileSize = 0;
       while ((fileSize = r.nextGaussian()+2)<=0) {}
       while ((fileSize = r.nextGaussian()+2)<=0) {}
       genFile(file, (long)(fileSize*BLOCK_SIZE));
       genFile(file, (long)(fileSize*BLOCK_SIZE));
-      long startTime = Time.now();
+      long startTimestamp = Time.monotonicNow();
       fc.delete(file, true);
       fc.delete(file, true);
-      executionTime[DELETE] += (Time.now()-startTime);
+      executionTime[DELETE] += (Time.monotonicNow() - startTimestamp);
       totalNumOfOps[DELETE]++;
       totalNumOfOps[DELETE]++;
     }
     }
     
     
@@ -310,9 +310,9 @@ public class LoadGenerator extends Configured implements Tool {
      */
      */
     private void list() throws IOException {
     private void list() throws IOException {
       String dirName = dirs.get(r.nextInt(dirs.size()));
       String dirName = dirs.get(r.nextInt(dirs.size()));
-      long startTime = Time.now();
+      long startTimestamp = Time.monotonicNow();
       fc.listStatus(new Path(dirName));
       fc.listStatus(new Path(dirName));
-      executionTime[LIST] += (Time.now()-startTime);
+      executionTime[LIST] += (Time.monotonicNow() - startTimestamp);
       totalNumOfOps[LIST]++;
       totalNumOfOps[LIST]++;
     }
     }
 
 
@@ -320,14 +320,14 @@ public class LoadGenerator extends Configured implements Tool {
      * The file is filled with 'a'.
      * The file is filled with 'a'.
      */
      */
     private void genFile(Path file, long fileSize) throws IOException {
     private void genFile(Path file, long fileSize) throws IOException {
-      long startTime = Time.now();
+      long startTimestamp = Time.monotonicNow();
       FSDataOutputStream out = null;
       FSDataOutputStream out = null;
       try {
       try {
         out = fc.create(file,
         out = fc.create(file,
             EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE),
             EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE),
             CreateOpts.createParent(), CreateOpts.bufferSize(4096),
             CreateOpts.createParent(), CreateOpts.bufferSize(4096),
             CreateOpts.repFac((short) 3));
             CreateOpts.repFac((short) 3));
-        executionTime[CREATE] += (Time.now() - startTime);
+        executionTime[CREATE] += (Time.monotonicNow() - startTimestamp);
         numOfOps[CREATE]++;
         numOfOps[CREATE]++;
 
 
         long i = fileSize;
         long i = fileSize;
@@ -337,8 +337,8 @@ public class LoadGenerator extends Configured implements Tool {
           i -= s;
           i -= s;
         }
         }
 
 
-        startTime = Time.now();
-        executionTime[WRITE_CLOSE] += (Time.now() - startTime);
+        startTimestamp = Time.monotonicNow();
+        executionTime[WRITE_CLOSE] += (Time.monotonicNow() - startTimestamp);
         numOfOps[WRITE_CLOSE]++;
         numOfOps[WRITE_CLOSE]++;
       } finally {
       } finally {
         IOUtils.cleanupWithLogger(LOG, out);
         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.
    * Priority of the mountd shutdown hook.
    */
    */
@@ -117,8 +128,7 @@ abstract public class MountdBase {
   private class Unregister implements Runnable {
   private class Unregister implements Runnable {
     @Override
     @Override
     public synchronized void run() {
     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();
     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.
    * Priority of the nfsd shutdown hook.
    */
    */
@@ -86,8 +93,7 @@ public abstract class Nfs3Base {
   private class NfsShutdownHook implements Runnable {
   private class NfsShutdownHook implements Runnable {
     @Override
     @Override
     public synchronized void run() {
     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.base.Preconditions;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Maps;
 import com.google.common.primitives.SignedBytes;
 import com.google.common.primitives.SignedBytes;
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.crypto.key.KeyProvider;
 import org.apache.hadoop.crypto.key.KeyProvider;
 import org.apache.hadoop.fs.BlockLocation;
 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_HA_NAMENODES_KEY_PREFIX;
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_NAMESERVICES;
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_NAMESERVICES;
 
 
+@InterfaceAudience.Private
 public class DFSUtilClient {
 public class DFSUtilClient {
   public static final byte[] EMPTY_BYTES = {};
   public static final byte[] EMPTY_BYTES = {};
   private static final Logger LOG = LoggerFactory.getLogger(
   private static final Logger LOG = LoggerFactory.getLogger(
@@ -406,7 +408,7 @@ public class DFSUtilClient {
    * @param keys list of keys in the order of preference
    * @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
    * @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) {
       Configuration conf, String... keys) {
     String value = null;
     String value = null;
     for (String key : keys) {
     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.ContentSummary.Builder;
 import org.apache.hadoop.fs.FileChecksum;
 import org.apache.hadoop.fs.FileChecksum;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.MD5MD5CRC32CastagnoliFileChecksum;
 import org.apache.hadoop.fs.MD5MD5CRC32CastagnoliFileChecksum;
 import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
 import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
 import org.apache.hadoop.fs.MD5MD5CRC32GzipFileChecksum;
 import org.apache.hadoop.fs.MD5MD5CRC32GzipFileChecksum;
@@ -65,6 +66,8 @@ import java.util.Map;
 
 
 class JsonUtilClient {
 class JsonUtilClient {
   static final DatanodeInfo[] EMPTY_DATANODE_INFO_ARRAY = {};
   static final DatanodeInfo[] EMPTY_DATANODE_INFO_ARRAY = {};
+  static final String UNSUPPPORTED_EXCEPTION_STR =
+      UnsupportedOperationException.class.getName();
 
 
   /** Convert a Json map to a RemoteException. */
   /** Convert a Json map to a RemoteException. */
   static RemoteException toRemoteException(final Map<?, ?> json) {
   static RemoteException toRemoteException(final Map<?, ?> json) {
@@ -72,6 +75,9 @@ class JsonUtilClient {
         RemoteException.class.getSimpleName());
         RemoteException.class.getSimpleName());
     final String message = (String)m.get("message");
     final String message = (String)m.get("message");
     final String javaClassName = (String)m.get("javaClassName");
     final String javaClassName = (String)m.get("javaClassName");
+    if (UNSUPPPORTED_EXCEPTION_STR.equals(javaClassName)) {
+      throw new UnsupportedOperationException(message);
+    }
     return new RemoteException(javaClassName, 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.FSInputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.GlobalStorageStatistics;
 import org.apache.hadoop.fs.GlobalStorageStatistics;
 import org.apache.hadoop.fs.GlobalStorageStatistics.StorageStatisticsProvider;
 import org.apache.hadoop.fs.GlobalStorageStatistics.StorageStatisticsProvider;
 import org.apache.hadoop.fs.StorageStatistics;
 import org.apache.hadoop.fs.StorageStatistics;
@@ -1766,6 +1767,22 @@ public class WebHdfsFileSystem extends FileSystem
     new FsPathRunner(op, src).run();
     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
   @VisibleForTesting
   InetSocketAddress[] getResolvedNNAddr() {
   InetSocketAddress[] getResolvedNNAddr() {
     return nnAddrs;
     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),
     NULL(false, HttpURLConnection.HTTP_NOT_IMPLEMENTED),
 
 
     CHECKACCESS(false, HttpURLConnection.HTTP_OK),
     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 boolean redirect;
     final int expectedHttpResponseCode;
     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);
     start(register);
   }
   }
   
   
-  static void startService(String[] args,
+  static Nfs3 startService(String[] args,
       DatagramSocket registrationSocket) throws IOException {
       DatagramSocket registrationSocket) throws IOException {
     StringUtils.startupShutdownMessage(Nfs3.class, args, LOG);
     StringUtils.startupShutdownMessage(Nfs3.class, args, LOG);
     NfsConfiguration conf = new NfsConfiguration();
     NfsConfiguration conf = new NfsConfiguration();
@@ -67,8 +67,14 @@ public class Nfs3 extends Nfs3Base {
     final Nfs3 nfsServer = new Nfs3(conf, registrationSocket,
     final Nfs3 nfsServer = new Nfs3(conf, registrationSocket,
         allowInsecurePorts);
         allowInsecurePorts);
     nfsServer.startServiceInternal(true);
     nfsServer.startServiceInternal(true);
+    return nfsServer;
   }
   }
-  
+
+  public void stop() {
+    super.stop();
+    mountd.stop();
+  }
+
   public static void main(String[] args) throws IOException {
   public static void main(String[] args) throws IOException {
     startService(args, null);
     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
    * HDFS directory size is always zero. Try to return something meaningful
    * here. Assume each child take 32bytes.
    * 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) {
   public static long getDirSize(int childNum) {
     return (childNum + 2) * 32;
     return (childNum + 2) * 32;
@@ -122,6 +124,9 @@ public class Nfs3Utils {
 
 
   /**
   /**
    * Send a write response to the netty network socket channel
    * 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) {
   public static void writeChannel(Channel channel, XDR out, int xid) {
     if (channel == null) {
     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);
   static final Log LOG = LogFactory.getLog(PrivilegedNfsGatewayStarter.class);
   private String[] args = null;
   private String[] args = null;
   private DatagramSocket registrationSocket = null;
   private DatagramSocket registrationSocket = null;
+  private Nfs3 nfs3Server = null;
 
 
   @Override
   @Override
   public void init(DaemonContext context) throws Exception {
   public void init(DaemonContext context) throws Exception {
@@ -68,12 +69,14 @@ public class PrivilegedNfsGatewayStarter implements Daemon {
 
 
   @Override
   @Override
   public void start() throws Exception {
   public void start() throws Exception {
-    Nfs3.startService(args, registrationSocket);
+    nfs3Server = Nfs3.startService(args, registrationSocket);
   }
   }
 
 
   @Override
   @Override
   public void stop() throws Exception {
   public void stop() throws Exception {
-    // Nothing to do.
+    if (nfs3Server != null) {
+      nfs3Server.stop();
+    }
   }
   }
 
 
   @Override
   @Override

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

@@ -144,22 +144,19 @@ fi
 #---------------------------------------------------------
 #---------------------------------------------------------
 # quorumjournal nodes (if any)
 # 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
 # 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)
 # 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
 # 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_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_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_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_NAMESERVICE_ID;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SERVER_HTTPS_KEYPASSWORD_KEY;
 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;
 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.InetSocketAddress;
 import java.net.URI;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URISyntaxException;
+import java.net.UnknownHostException;
+
 import java.security.SecureRandom;
 import java.security.SecureRandom;
 import java.util.Arrays;
 import java.util.Arrays;
 import java.util.Collection;
 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.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 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.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.http.HttpConfig;
 import org.apache.hadoop.http.HttpConfig;
 import org.apache.hadoop.http.HttpServer2;
 import org.apache.hadoop.http.HttpServer2;
@@ -453,6 +457,85 @@ public class DFSUtil {
     return principals;
     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 
    * Returns list of InetSocketAddress corresponding to  backup node rpc 
    * addresses from the configuration.
    * 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.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 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;
 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
  * 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 {
 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.annotation.JsonProperty;
 import com.fasterxml.jackson.core.JsonProcessingException;
 import com.fasterxml.jackson.core.JsonProcessingException;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Lists;
@@ -67,6 +68,11 @@ public class SlowDiskTracker {
    */
    */
   private final Timer timer;
   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
    * Number of disks to include in JSON report per operation. We will return
    * disks with the highest latency.
    * disks with the highest latency.
@@ -254,12 +260,11 @@ public class SlowDiskTracker {
    *         serialization failed.
    *         serialization failed.
    */
    */
   public String getSlowDiskReportAsJsonString() {
   public String getSlowDiskReportAsJsonString() {
-    ObjectMapper objectMapper = new ObjectMapper();
     try {
     try {
       if (slowDisksReport.isEmpty()) {
       if (slowDisksReport.isEmpty()) {
         return null;
         return null;
       }
       }
-      return objectMapper.writeValueAsString(slowDisksReport);
+      return WRITER.writeValueAsString(slowDisksReport);
     } catch (JsonProcessingException e) {
     } catch (JsonProcessingException e) {
       // Failed to serialize. Don't log the exception call stack.
       // Failed to serialize. Don't log the exception call stack.
       LOG.debug("Failed to serialize statistics" + e);
       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.annotation.JsonProperty;
 import com.fasterxml.jackson.core.JsonProcessingException;
 import com.fasterxml.jackson.core.JsonProcessingException;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.primitives.Ints;
 import com.google.common.primitives.Ints;
@@ -69,6 +70,10 @@ public class SlowPeerTracker {
    */
    */
   private final Timer timer;
   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
    * Number of nodes to include in JSON report. We will return nodes with
    * the highest number of votes from peers.
    * the highest number of votes from peers.
@@ -188,9 +193,8 @@ public class SlowPeerTracker {
   public String getJson() {
   public String getJson() {
     Collection<ReportForJson> validReports = getJsonReports(
     Collection<ReportForJson> validReports = getJsonReports(
         MAX_NODES_TO_REPORT);
         MAX_NODES_TO_REPORT);
-    ObjectMapper objectMapper = new ObjectMapper();
     try {
     try {
-      return objectMapper.writeValueAsString(validReports);
+      return WRITER.writeValueAsString(validReports);
     } catch (JsonProcessingException e) {
     } catch (JsonProcessingException e) {
       // Failed to serialize. Don't log the exception call stack.
       // Failed to serialize. Don't log the exception call stack.
       LOG.debug("Failed to serialize statistics" + e);
       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.
    * Shutdown disk balancer.
    */
    */
-  private  void shutdownDiskBalancer() {
+  private void shutdownDiskBalancer() {
     if (this.diskBalancer != null) {
     if (this.diskBalancer != null) {
       this.diskBalancer.shutdown();
       this.diskBalancer.shutdown();
       this.diskBalancer = null;
       this.diskBalancer = null;
@@ -2137,7 +2137,11 @@ public class DataNode extends ReconfigurableBase
       ipcServer.stop();
       ipcServer.stop();
     }
     }
 
 
-    if (blockPoolManager != null) {
+    if (ecWorker != null) {
+      ecWorker.shutDown();
+    }
+
+    if(blockPoolManager != null) {
       try {
       try {
         this.blockPoolManager.shutDownAll(bposArray);
         this.blockPoolManager.shutDownAll(bposArray);
       } catch (InterruptedException ie) {
       } 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.io.nativeio.SharedFileDescriptorFactory;
 import org.apache.hadoop.net.unix.DomainSocket;
 import org.apache.hadoop.net.unix.DomainSocket;
 import org.apache.hadoop.net.unix.DomainSocketWatcher;
 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.Joiner;
 import com.google.common.base.Preconditions;
 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.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
 import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
+import java.util.concurrent.ThreadPoolExecutor.CallerRunsPolicy;
 import java.util.concurrent.locks.LockSupport;
 import java.util.concurrent.locks.LockSupport;
 import java.util.logging.Level;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 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 org.slf4j.Logger;
 
 
 import java.util.Collection;
 import java.util.Collection;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ExecutorCompletionService;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.SynchronousQueue;
 import java.util.concurrent.SynchronousQueue;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.ThreadPoolExecutor;
@@ -149,7 +151,12 @@ public final class ErasureCodingWorker {
     return conf;
     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.nio.ByteBuffer;
 import java.util.BitSet;
 import java.util.BitSet;
 import java.util.concurrent.CompletionService;
 import java.util.concurrent.CompletionService;
-import java.util.concurrent.ExecutorCompletionService;
-import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicLong;
 
 
 /**
 /**
@@ -110,7 +108,7 @@ abstract class StripedReconstructor {
   // position in striped internal block
   // position in striped internal block
   private long positionInBlock;
   private long positionInBlock;
   private StripedReader stripedReader;
   private StripedReader stripedReader;
-  private ThreadPoolExecutor stripedReadPool;
+  private ErasureCodingWorker erasureCodingWorker;
   private final CachingStrategy cachingStrategy;
   private final CachingStrategy cachingStrategy;
   private long maxTargetLength = 0L;
   private long maxTargetLength = 0L;
   private final BitSet liveBitSet;
   private final BitSet liveBitSet;
@@ -122,7 +120,7 @@ abstract class StripedReconstructor {
 
 
   StripedReconstructor(ErasureCodingWorker worker,
   StripedReconstructor(ErasureCodingWorker worker,
       StripedReconstructionInfo stripedReconInfo) {
       StripedReconstructionInfo stripedReconInfo) {
-    this.stripedReadPool = worker.getStripedReadPool();
+    this.erasureCodingWorker = worker;
     this.datanode = worker.getDatanode();
     this.datanode = worker.getDatanode();
     this.conf = worker.getConf();
     this.conf = worker.getConf();
     this.ecPolicy = stripedReconInfo.getEcPolicy();
     this.ecPolicy = stripedReconInfo.getEcPolicy();
@@ -225,7 +223,7 @@ abstract class StripedReconstructor {
   }
   }
 
 
   CompletionService<Void> createReadService() {
   CompletionService<Void> createReadService() {
-    return new ExecutorCompletionService<>(stripedReadPool);
+    return erasureCodingWorker.createReadService();
   }
   }
 
 
   ExtendedBlock getBlockGroup() {
   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>>();
     deletingBlock = new HashMap<String, Set<Long>>();
 
 
     for (int idx = 0; idx < storage.getNumStorageDirs(); idx++) {
     for (int idx = 0; idx < storage.getNumStorageDirs(); idx++) {
-      addVolume(dataLocations, storage.getStorageDir(idx));
+      addVolume(storage.getStorageDir(idx));
     }
     }
     setupAsyncLazyPersistThreads();
     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();
     final StorageLocation storageLocation = sd.getStorageLocation();
 
 
     // If IOException raises from FsVolumeImpl() or getVolumeMap(), there is
     // 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;
 package org.apache.hadoop.hdfs.server.diskbalancer.planner;
 
 
 import com.fasterxml.jackson.annotation.JsonTypeInfo;
 import com.fasterxml.jackson.annotation.JsonTypeInfo;
-import com.fasterxml.jackson.databind.JavaType;
 import com.fasterxml.jackson.databind.ObjectMapper;
 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 com.google.common.base.Preconditions;
 
 
 import java.io.IOException;
 import java.io.IOException;
@@ -38,6 +39,10 @@ public class NodePlan {
   private int port;
   private int port;
   private long timeStamp;
   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.
    * returns timestamp when this plan was created.
    *
    *
@@ -153,8 +158,7 @@ public class NodePlan {
    * @throws IOException
    * @throws IOException
    */
    */
   public static NodePlan parseJson(String json) 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
    * @throws IOException
    */
    */
   public String toJson() 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_PERMISSIONS_SUPERUSERGROUP_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_KEY;
 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 static org.apache.hadoop.hdfs.server.namenode.FSDirStatAndListingOp.*;
 
 
 import org.apache.hadoop.hdfs.protocol.ReplicatedBlockStats;
 import org.apache.hadoop.hdfs.protocol.ReplicatedBlockStats;
@@ -351,7 +352,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         && !auditLoggers.isEmpty();
         && !auditLoggers.isEmpty();
   }
   }
 
 
-  private void logAuditEvent(boolean succeeded, String cmd, String src)
+  void logAuditEvent(boolean succeeded, String cmd, String src)
       throws IOException {
       throws IOException {
     logAuditEvent(succeeded, cmd, src, null, null);
     logAuditEvent(succeeded, cmd, src, null, null);
   }
   }
@@ -1095,7 +1096,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       if (!success) {
       if (!success) {
         fsImage.close();
         fsImage.close();
       }
       }
-      writeUnlock("loadFSImage");
+      writeUnlock("loadFSImage", true);
     }
     }
     imageLoadComplete();
     imageLoadComplete();
   }
   }
@@ -1586,6 +1587,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   public void writeUnlock(String opName) {
   public void writeUnlock(String opName) {
     this.fsLock.writeUnlock(opName);
     this.fsLock.writeUnlock(opName);
   }
   }
+
+  public void writeUnlock(String opName, boolean suppressWriteLockReport) {
+    this.fsLock.writeUnlock(opName, suppressWriteLockReport);
+  }
+
   @Override
   @Override
   public boolean hasWriteLock() {
   public boolean hasWriteLock() {
     return this.fsLock.isWriteLockedByCurrentThread();
     return this.fsLock.isWriteLockedByCurrentThread();
@@ -1710,7 +1716,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * Dump all metadata into specified file
    * Dump all metadata into specified file
    */
    */
   void metaSave(String filename) throws IOException {
   void metaSave(String filename) throws IOException {
-    checkSuperuserPrivilege();
+    String operationName = "metaSave";
+    checkSuperuserPrivilege(operationName);
     checkOperation(OperationCategory.UNCHECKED);
     checkOperation(OperationCategory.UNCHECKED);
     writeLock();
     writeLock();
     try {
     try {
@@ -1722,8 +1729,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       out.flush();
       out.flush();
       out.close();
       out.close();
     } finally {
     } finally {
-      writeUnlock("metaSave");
+      writeUnlock(operationName);
     }
     }
+    logAuditEvent(true, operationName, null);
   }
   }
 
 
   private void metaSave(PrintWriter out) {
   private void metaSave(PrintWriter out) {
@@ -1774,7 +1782,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     return sw.toString();
     return sw.toString();
   }
   }
 
 
-  FsServerDefaults getServerDefaults() throws StandbyException {
+  @VisibleForTesting
+  public FsServerDefaults getServerDefaults() throws StandbyException {
     checkOperation(OperationCategory.READ);
     checkOperation(OperationCategory.READ);
     return serverDefaults;
     return serverDefaults;
   }
   }
@@ -3118,7 +3127,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       requireEffectiveLayoutVersionForFeature(Feature.QUOTA_BY_STORAGE_TYPE);
       requireEffectiveLayoutVersionForFeature(Feature.QUOTA_BY_STORAGE_TYPE);
     }
     }
     checkOperation(OperationCategory.WRITE);
     checkOperation(OperationCategory.WRITE);
-    final String operationName = "setQuota";
+    final String operationName = getQuotaCommand(nsQuota, ssQuota);
     writeLock();
     writeLock();
     boolean success = false;
     boolean success = false;
     try {
     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);
     checkOperation(OperationCategory.UNCHECKED);
     readLock();
     readLock();
     try {
     try {
       checkOperation(OperationCategory.UNCHECKED);
       checkOperation(OperationCategory.UNCHECKED);
       final DatanodeManager dm = getBlockManager().getDatanodeManager();      
       final DatanodeManager dm = getBlockManager().getDatanodeManager();      
       final List<DatanodeDescriptor> results = dm.getDatanodeListForReport(type);
       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++) {
       for (int i=0; i<arr.length; i++) {
         arr[i] = new DatanodeInfoBuilder().setFrom(results.get(i))
         arr[i] = new DatanodeInfoBuilder().setFrom(results.get(i))
             .build();
             .build();
       }
       }
-      return arr;
     } finally {
     } finally {
-      readUnlock("datanodeReport");
+      readUnlock(operationName);
     }
     }
+    logAuditEvent(true, operationName, null);
+    return arr;
   }
   }
 
 
   DatanodeStorageReport[] getDatanodeStorageReport(final DatanodeReportType type
   DatanodeStorageReport[] getDatanodeStorageReport(final DatanodeReportType type
-      ) throws AccessControlException, StandbyException {
-    checkSuperuserPrivilege();
+      ) throws IOException {
+    String operationName = "getDatanodeStorageReport";
+    DatanodeStorageReport[] reports;
+    checkSuperuserPrivilege(operationName);
     checkOperation(OperationCategory.UNCHECKED);
     checkOperation(OperationCategory.UNCHECKED);
     readLock();
     readLock();
     try {
     try {
@@ -4233,17 +4246,18 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       final DatanodeManager dm = getBlockManager().getDatanodeManager();      
       final DatanodeManager dm = getBlockManager().getDatanodeManager();      
       final List<DatanodeDescriptor> datanodes = dm.getDatanodeListForReport(type);
       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++) {
       for (int i = 0; i < reports.length; i++) {
         final DatanodeDescriptor d = datanodes.get(i);
         final DatanodeDescriptor d = datanodes.get(i);
         reports[i] = new DatanodeStorageReport(
         reports[i] = new DatanodeStorageReport(
             new DatanodeInfoBuilder().setFrom(d).build(),
             new DatanodeInfoBuilder().setFrom(d).build(),
             d.getStorageReports());
             d.getStorageReports());
       }
       }
-      return reports;
     } finally {
     } finally {
       readUnlock("getDatanodeStorageReport");
       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)
   boolean saveNamespace(final long timeWindow, final long txGap)
       throws IOException {
       throws IOException {
+    String operationName = "saveNamespace";
     checkOperation(OperationCategory.UNCHECKED);
     checkOperation(OperationCategory.UNCHECKED);
-    checkSuperuserPrivilege();
+    checkSuperuserPrivilege(operationName);
 
 
     boolean saved = false;
     boolean saved = false;
     cpLock();  // Block if a checkpointing is in progress on standby.
     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);
       saved = getFSImage().saveNamespace(timeWindow, txGap, this);
     } finally {
     } finally {
-      readUnlock("saveNamespace");
+      readUnlock(operationName);
       cpUnlock();
       cpUnlock();
     }
     }
     if (saved) {
     if (saved) {
       LOG.info("New namespace image has been created");
       LOG.info("New namespace image has been created");
     }
     }
+    logAuditEvent(true, operationName, null);
     return saved;
     return saved;
   }
   }
   
   
@@ -4283,9 +4299,10 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * 
    * 
    * @throws AccessControlException if superuser privilege is violated.
    * @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);
     checkOperation(OperationCategory.UNCHECKED);
     cpLock();  // Block if a checkpointing is in progress on standby.
     cpLock();  // Block if a checkpointing is in progress on standby.
     writeLock();
     writeLock();
@@ -4293,17 +4310,18 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       checkOperation(OperationCategory.UNCHECKED);
       checkOperation(OperationCategory.UNCHECKED);
       
       
       // if it is disabled - enable it and vice versa.
       // 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 {
     } finally {
-      writeUnlock("restoreFailedStorage");
+      writeUnlock(operationName);
       cpUnlock();
       cpUnlock();
     }
     }
+    logAuditEvent(true, operationName, null);
+    return val;
   }
   }
 
 
   Date getStartTime() {
   Date getStartTime() {
@@ -4311,7 +4329,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
   }
     
     
   void finalizeUpgrade() throws IOException {
   void finalizeUpgrade() throws IOException {
-    checkSuperuserPrivilege();
+    String operationName = "finalizeUpgrade";
+    checkSuperuserPrivilege(operationName);
     checkOperation(OperationCategory.UNCHECKED);
     checkOperation(OperationCategory.UNCHECKED);
     cpLock();  // Block if a checkpointing is in progress on standby.
     cpLock();  // Block if a checkpointing is in progress on standby.
     writeLock();
     writeLock();
@@ -4319,26 +4338,33 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       checkOperation(OperationCategory.UNCHECKED);
       checkOperation(OperationCategory.UNCHECKED);
       getFSImage().finalizeUpgrade(this.isHaEnabled() && inActiveState());
       getFSImage().finalizeUpgrade(this.isHaEnabled() && inActiveState());
     } finally {
     } finally {
-      writeUnlock("finalizeUpgrade");
+      writeUnlock(operationName);
       cpUnlock();
       cpUnlock();
     }
     }
+    logAuditEvent(true, operationName, null);
   }
   }
 
 
   void refreshNodes() throws IOException {
   void refreshNodes() throws IOException {
+    String operationName = "refreshNodes";
     checkOperation(OperationCategory.UNCHECKED);
     checkOperation(OperationCategory.UNCHECKED);
-    checkSuperuserPrivilege();
+    checkSuperuserPrivilege(operationName);
     getBlockManager().getDatanodeManager().refreshNodes(new HdfsConfiguration());
     getBlockManager().getDatanodeManager().refreshNodes(new HdfsConfiguration());
+    logAuditEvent(true, operationName, null);
   }
   }
 
 
   void setBalancerBandwidth(long bandwidth) throws IOException {
   void setBalancerBandwidth(long bandwidth) throws IOException {
+    String operationName = "setBalancerBandwidth";
     checkOperation(OperationCategory.UNCHECKED);
     checkOperation(OperationCategory.UNCHECKED);
-    checkSuperuserPrivilege();
+    checkSuperuserPrivilege(operationName);
     getBlockManager().getDatanodeManager().setBalancerBandwidth(bandwidth);
     getBlockManager().getDatanodeManager().setBalancerBandwidth(bandwidth);
+    logAuditEvent(true, operationName, null);
   }
   }
 
 
   boolean setSafeMode(SafeModeAction action) throws IOException {
   boolean setSafeMode(SafeModeAction action) throws IOException {
+    String operationName = action.toString().toLowerCase();
+    boolean error = false;
     if (action != SafeModeAction.SAFEMODE_GET) {
     if (action != SafeModeAction.SAFEMODE_GET) {
-      checkSuperuserPrivilege();
+      checkSuperuserPrivilege(operationName);
       switch(action) {
       switch(action) {
       case SAFEMODE_LEAVE: // leave safe mode
       case SAFEMODE_LEAVE: // leave safe mode
         leaveSafeMode(false);
         leaveSafeMode(false);
@@ -4351,8 +4377,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         break;
         break;
       default:
       default:
         LOG.error("Unexpected safe mode action");
         LOG.error("Unexpected safe mode action");
+        error = true;
       }
       }
     }
     }
+    if (!error) {
+      logAuditEvent(true, operationName, null);
+    }
     return isInSafeMode();
     return isInSafeMode();
   }
   }
 
 
@@ -4488,7 +4518,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
   }
 
 
   CheckpointSignature rollEditLog() throws IOException {
   CheckpointSignature rollEditLog() throws IOException {
-    checkSuperuserPrivilege();
+    String operationName = "rollEditLog";
+    CheckpointSignature result = null;
+    checkSuperuserPrivilege(operationName);
     checkOperation(OperationCategory.JOURNAL);
     checkOperation(OperationCategory.JOURNAL);
     writeLock();
     writeLock();
     try {
     try {
@@ -4497,10 +4529,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       if (Server.isRpcInvocation()) {
       if (Server.isRpcInvocation()) {
         LOG.info("Roll Edit Log from " + Server.getRemoteAddress());
         LOG.info("Roll Edit Log from " + Server.getRemoteAddress());
       }
       }
-      return getFSImage().rollEditLog(getEffectiveLayoutVersion());
+      result = getFSImage().rollEditLog(getEffectiveLayoutVersion());
     } finally {
     } finally {
-      writeUnlock("rollEditLog");
+      writeUnlock(operationName);
     }
     }
+    logAuditEvent(true, operationName, null);
+    return result;
   }
   }
 
 
   NamenodeCommand startCheckpoint(NamenodeRegistration backupNode,
   NamenodeCommand startCheckpoint(NamenodeRegistration backupNode,
@@ -6189,11 +6223,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     checkOperation(OperationCategory.WRITE);
     checkOperation(OperationCategory.WRITE);
     final String operationName = "allowSnapshot";
     final String operationName = "allowSnapshot";
     boolean success = false;
     boolean success = false;
+    checkSuperuserPrivilege(operationName);
     writeLock();
     writeLock();
     try {
     try {
       checkOperation(OperationCategory.WRITE);
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot allow snapshot for " + path);
       checkNameNodeSafeMode("Cannot allow snapshot for " + path);
-      checkSuperuserPrivilege();
       FSDirSnapshotOp.allowSnapshot(dir, snapshotManager, path);
       FSDirSnapshotOp.allowSnapshot(dir, snapshotManager, path);
       success = true;
       success = true;
     } finally {
     } finally {
@@ -6207,12 +6241,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   void disallowSnapshot(String path) throws IOException {
   void disallowSnapshot(String path) throws IOException {
     checkOperation(OperationCategory.WRITE);
     checkOperation(OperationCategory.WRITE);
     final String operationName = "disallowSnapshot";
     final String operationName = "disallowSnapshot";
+    checkSuperuserPrivilege(operationName);
     boolean success = false;
     boolean success = false;
     writeLock();
     writeLock();
     try {
     try {
       checkOperation(OperationCategory.WRITE);
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot disallow snapshot for " + path);
       checkNameNodeSafeMode("Cannot disallow snapshot for " + path);
-      checkSuperuserPrivilege();
       FSDirSnapshotOp.disallowSnapshot(dir, snapshotManager, path);
       FSDirSnapshotOp.disallowSnapshot(dir, snapshotManager, path);
       success = true;
       success = true;
     } finally {
     } finally {
@@ -6405,7 +6439,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
   }
 
 
   RollingUpgradeInfo queryRollingUpgrade() throws IOException {
   RollingUpgradeInfo queryRollingUpgrade() throws IOException {
-    checkSuperuserPrivilege();
+    final String operationName = "queryRollingUpgrade";
+    checkSuperuserPrivilege(operationName);
     checkOperation(OperationCategory.READ);
     checkOperation(OperationCategory.READ);
     readLock();
     readLock();
     try {
     try {
@@ -6415,15 +6450,16 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       Preconditions.checkNotNull(rollingUpgradeInfo);
       Preconditions.checkNotNull(rollingUpgradeInfo);
       boolean hasRollbackImage = this.getFSImage().hasRollbackFSImage();
       boolean hasRollbackImage = this.getFSImage().hasRollbackFSImage();
       rollingUpgradeInfo.setCreatedRollbackImages(hasRollbackImage);
       rollingUpgradeInfo.setCreatedRollbackImages(hasRollbackImage);
-      return rollingUpgradeInfo;
     } finally {
     } finally {
-      readUnlock("queryRollingUpgrade");
+      readUnlock(operationName);
     }
     }
+    logAuditEvent(true, operationName, null, null, null);
+    return rollingUpgradeInfo;
   }
   }
 
 
   RollingUpgradeInfo startRollingUpgrade() throws IOException {
   RollingUpgradeInfo startRollingUpgrade() throws IOException {
     final String operationName = "startRollingUpgrade";
     final String operationName = "startRollingUpgrade";
-    checkSuperuserPrivilege();
+    checkSuperuserPrivilege(operationName);
     checkOperation(OperationCategory.WRITE);
     checkOperation(OperationCategory.WRITE);
     writeLock();
     writeLock();
     try {
     try {
@@ -6614,7 +6650,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
 
   RollingUpgradeInfo finalizeRollingUpgrade() throws IOException {
   RollingUpgradeInfo finalizeRollingUpgrade() throws IOException {
     final String operationName = "finalizeRollingUpgrade";
     final String operationName = "finalizeRollingUpgrade";
-    checkSuperuserPrivilege();
+    checkSuperuserPrivilege(operationName);
     checkOperation(OperationCategory.WRITE);
     checkOperation(OperationCategory.WRITE);
     writeLock();
     writeLock();
     try {
     try {
@@ -7739,5 +7775,38 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         .size();
         .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() {
   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) {
   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 currentTimeNanos = timer.monotonicNowNanos();
     final long writeLockIntervalNanos =
     final long writeLockIntervalNanos =
         currentTimeNanos - writeLockHeldTimeStampNanos;
         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) {
     if (this.serviceRpcServer != null) {
       this.serviceRpcServer.refreshServiceAcl(new Configuration(), new HDFSPolicyProvider());
       this.serviceRpcServer.refreshServiceAcl(new Configuration(), new HDFSPolicyProvider());
     }
     }
+    namesystem.logAuditEvent(true, "refreshServiceAcl", null);
   }
   }
 
 
   @Override // RefreshAuthorizationPolicyProtocol
   @Override // RefreshAuthorizationPolicyProtocol
@@ -1631,17 +1632,19 @@ public class NameNodeRpcServer implements NamenodeProtocols {
     LOG.info("Refreshing all user-to-groups mappings. Requested by user: " +
     LOG.info("Refreshing all user-to-groups mappings. Requested by user: " +
         getRemoteUser().getShortUserName());
         getRemoteUser().getShortUserName());
     Groups.getUserToGroupsMappingService().refresh();
     Groups.getUserToGroupsMappingService().refresh();
+    namesystem.logAuditEvent(true, "refreshUserToGroupsMappings", null);
   }
   }
 
 
   @Override // RefreshAuthorizationPolicyProtocol
   @Override // RefreshAuthorizationPolicyProtocol
-  public void refreshSuperUserGroupsConfiguration() {
+  public void refreshSuperUserGroupsConfiguration() throws IOException {
     LOG.info("Refreshing SuperUser proxy group mapping list ");
     LOG.info("Refreshing SuperUser proxy group mapping list ");
 
 
     ProxyUsers.refreshSuperUserGroupsConfiguration();
     ProxyUsers.refreshSuperUserGroupsConfiguration();
+    namesystem.logAuditEvent(true, "refreshSuperUserGroupsConfiguration", null);
   }
   }
 
 
   @Override // RefreshCallQueueProtocol
   @Override // RefreshCallQueueProtocol
-  public void refreshCallQueue() {
+  public void refreshCallQueue() throws IOException {
     LOG.info("Refreshing call queue.");
     LOG.info("Refreshing call queue.");
 
 
     Configuration conf = new Configuration();
     Configuration conf = new Configuration();
@@ -1649,6 +1652,7 @@ public class NameNodeRpcServer implements NamenodeProtocols {
     if (this.serviceRpcServer != null) {
     if (this.serviceRpcServer != null) {
       serviceRpcServer.refreshCallQueue(conf);
       serviceRpcServer.refreshCallQueue(conf);
     }
     }
+    namesystem.logAuditEvent(true, "refreshCallQueue", null);
   }
   }
 
 
   @Override // GenericRefreshProtocol
   @Override // GenericRefreshProtocol
@@ -2412,22 +2416,30 @@ public class NameNodeRpcServer implements NamenodeProtocols {
   @Override // ReconfigurationProtocol
   @Override // ReconfigurationProtocol
   public void startReconfiguration() throws IOException {
   public void startReconfiguration() throws IOException {
     checkNNStartup();
     checkNNStartup();
-    namesystem.checkSuperuserPrivilege();
+    String operationName = "startNamenodeReconfiguration";
+    namesystem.checkSuperuserPrivilege(operationName);
     nn.startReconfigurationTask();
     nn.startReconfigurationTask();
+    namesystem.logAuditEvent(true, operationName, null);
   }
   }
 
 
   @Override // ReconfigurationProtocol
   @Override // ReconfigurationProtocol
   public ReconfigurationTaskStatus getReconfigurationStatus()
   public ReconfigurationTaskStatus getReconfigurationStatus()
       throws IOException {
       throws IOException {
     checkNNStartup();
     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
   @Override // ReconfigurationProtocol
   public List<String> listReconfigurableProperties() throws IOException {
   public List<String> listReconfigurableProperties() throws IOException {
     checkNNStartup();
     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;
   final MutableQuantiles[] syncsQuantiles;
   @Metric("Journal transactions batched in sync")
   @Metric("Journal transactions batched in sync")
   MutableCounterLong transactionsBatchedInSync;
   MutableCounterLong transactionsBatchedInSync;
+  @Metric("Journal transactions batched in sync")
+  final MutableQuantiles[] numTransactionsBatchedInSync;
   @Metric("Number of blockReports from individual storages")
   @Metric("Number of blockReports from individual storages")
   MutableRate storageBlockReport;
   MutableRate storageBlockReport;
   final MutableQuantiles[] storageBlockReportQuantiles;
   final MutableQuantiles[] storageBlockReportQuantiles;
@@ -148,6 +150,7 @@ public class NameNodeMetrics {
     
     
     final int len = intervals.length;
     final int len = intervals.length;
     syncsQuantiles = new MutableQuantiles[len];
     syncsQuantiles = new MutableQuantiles[len];
+    numTransactionsBatchedInSync = new MutableQuantiles[len];
     storageBlockReportQuantiles = new MutableQuantiles[len];
     storageBlockReportQuantiles = new MutableQuantiles[len];
     cacheReportQuantiles = new MutableQuantiles[len];
     cacheReportQuantiles = new MutableQuantiles[len];
     generateEDEKTimeQuantiles = new MutableQuantiles[len];
     generateEDEKTimeQuantiles = new MutableQuantiles[len];
@@ -159,6 +162,10 @@ public class NameNodeMetrics {
       syncsQuantiles[i] = registry.newQuantiles(
       syncsQuantiles[i] = registry.newQuantiles(
           "syncs" + interval + "s",
           "syncs" + interval + "s",
           "Journal syncs", "ops", "latency", interval);
           "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(
       storageBlockReportQuantiles[i] = registry.newQuantiles(
           "storageBlockReport" + interval + "s",
           "storageBlockReport" + interval + "s",
           "Storage block report", "ops", "latency", interval);
           "Storage block report", "ops", "latency", interval);
@@ -304,6 +311,9 @@ public class NameNodeMetrics {
 
 
   public void incrTransactionsBatchedInSync(long count) {
   public void incrTransactionsBatchedInSync(long count) {
     transactionsBatchedInSync.incr(count);
     transactionsBatchedInSync.incr(count);
+    for (MutableQuantiles q : numTransactionsBatchedInSync) {
+      q.add(count);
+    }
   }
   }
 
 
   public void incSuccessfulReReplications() {
   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.ContentSummary;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.AclStatus;
@@ -115,6 +116,7 @@ public class NamenodeWebHdfsMethods {
   private Principal userPrincipal;
   private Principal userPrincipal;
   private String remoteAddr;
   private String remoteAddr;
 
 
+  private static volatile String serverDefaultsResponse = null;
   private @Context ServletContext context;
   private @Context ServletContext context;
   private @Context HttpServletResponse response;
   private @Context HttpServletResponse response;
 
 
@@ -1121,11 +1123,30 @@ public class NamenodeWebHdfsMethods {
       final String js = JsonUtil.toJsonString(storagePolicy);
       final String js = JsonUtil.toJsonString(storagePolicy);
       return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
       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:
     default:
       throw new UnsupportedOperationException(op + " is not supported");
       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,
   private static String getTrashRoot(String fullPath,
       Configuration conf) throws IOException {
       Configuration conf) throws IOException {
     FileSystem fs = FileSystem.get(conf != null ? conf : new Configuration());
     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.IOException;
 import java.io.PrintStream;
 import java.io.PrintStream;
 import java.net.InetSocketAddress;
 import java.net.InetSocketAddress;
+import java.net.URISyntaxException;
 import java.security.PrivilegedExceptionAction;
 import java.security.PrivilegedExceptionAction;
 import java.util.Arrays;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.HashMap;
 import java.util.List;
 import java.util.List;
 import java.util.Map;
 import java.util.Map;
 import java.util.Collection;
 import java.util.Collection;
+import java.util.Set;
 
 
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
@@ -71,6 +73,7 @@ public class GetConf extends Configured implements Tool {
     SECONDARY("-secondaryNameNodes", 
     SECONDARY("-secondaryNameNodes", 
         "gets list of secondary namenodes in the cluster."),
         "gets list of secondary namenodes in the cluster."),
     BACKUP("-backupNodes", "gets list of backup nodes 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",
     INCLUDE_FILE("-includeFile",
         "gets the include file path that defines the datanodes " +
         "gets the include file path that defines the datanodes " +
         "that can join the cluster."),
         "that can join the cluster."),
@@ -93,6 +96,8 @@ public class GetConf extends Configured implements Tool {
           new SecondaryNameNodesCommandHandler());
           new SecondaryNameNodesCommandHandler());
       map.put(StringUtils.toLowerCase(BACKUP.getName()),
       map.put(StringUtils.toLowerCase(BACKUP.getName()),
           new BackupNodesCommandHandler());
           new BackupNodesCommandHandler());
+      map.put(StringUtils.toLowerCase(JOURNALNODE.getName()),
+          new JournalNodeCommandHandler());
       map.put(StringUtils.toLowerCase(INCLUDE_FILE.getName()),
       map.put(StringUtils.toLowerCase(INCLUDE_FILE.getName()),
           new CommandHandler(DFSConfigKeys.DFS_HOSTS));
           new CommandHandler(DFSConfigKeys.DFS_HOSTS));
       map.put(StringUtils.toLowerCase(EXCLUDE_FILE.getName()),
       map.put(StringUtils.toLowerCase(EXCLUDE_FILE.getName()),
@@ -213,7 +218,19 @@ public class GetConf extends Configured implements Tool {
       return 0;
       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}
    * Handler for {@link Command#SECONDARY}
    */
    */
@@ -326,6 +343,18 @@ public class GetConf extends Configured implements Tool {
     printOut(buffer.toString());
     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() {
   private void printUsage() {
     printError(USAGE);
     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.ContentSummary;
 import org.apache.hadoop.fs.FileChecksum;
 import org.apache.hadoop.fs.FileChecksum;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
 import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.fs.XAttr;
@@ -470,4 +471,23 @@ public class JsonUtil {
   public static String toJsonString(BlockStoragePolicy storagePolicy) {
   public static String toJsonString(BlockStoragePolicy storagePolicy) {
     return toJsonString(BlockStoragePolicy.class, toJsonMap(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 -namenodes
        hdfs getconf -secondaryNameNodes
        hdfs getconf -secondaryNameNodes
        hdfs getconf -backupNodes
        hdfs getconf -backupNodes
+       hdfs getconf -journalNodes
        hdfs getconf -includeFile
        hdfs getconf -includeFile
        hdfs getconf -excludeFile
        hdfs getconf -excludeFile
        hdfs getconf -nnRpcAddresses
        hdfs getconf -nnRpcAddresses
@@ -126,6 +127,7 @@ Usage:
 | `-namenodes` | gets list of namenodes in the cluster. |
 | `-namenodes` | gets list of namenodes in the cluster. |
 | `-secondaryNameNodes` | gets list of secondary namenodes in the cluster. |
 | `-secondaryNameNodes` | gets list of secondary namenodes in the cluster. |
 | `-backupNodes` | gets list of backup nodes 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. |
 | `-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. |
 | `-excludeFile` | gets the exclude file path that defines the datanodes that need to decommissioned. |
 | `-nnRpcAddresses` | gets the namenode rpc addresses |
 | `-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());
     Assert.assertEquals(0L, cluster.getNamesystem().getPendingDeletionBlocks());
     // restart DataNodes
     // restart DataNodes
     for (int i = 0; i < REPLICATION; i++) {
     for (int i = 0; i < REPLICATION; i++) {
-      cluster.restartDataNode(dnprops[i], true);
+      cluster.restartDataNode(dnprops[i]);
     }
     }
     cluster.waitActive();
     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.core.type.TypeReference;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectReader;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableMap;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import static org.apache.hadoop.hdfs.DFSConfigKeys
 import static org.apache.hadoop.hdfs.DFSConfigKeys
@@ -76,6 +77,8 @@ public class TestSlowDiskTracker {
   private FakeTimer timer;
   private FakeTimer timer;
   private long reportValidityMs;
   private long reportValidityMs;
   private static final long OUTLIERS_REPORT_INTERVAL = 1000;
   private static final long OUTLIERS_REPORT_INTERVAL = 1000;
+  private static final ObjectReader READER = new ObjectMapper().readerFor(
+          new TypeReference<ArrayList<DiskLatency>>() {});
 
 
   static {
   static {
     conf = new HdfsConfiguration();
     conf = new HdfsConfiguration();
@@ -416,8 +419,7 @@ public class TestSlowDiskTracker {
 
 
   private ArrayList<DiskLatency> getAndDeserializeJson(
   private ArrayList<DiskLatency> getAndDeserializeJson(
       final String json) throws IOException {
       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,
   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.core.type.TypeReference;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectReader;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 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.apache.hadoop.util.FakeTimer;
 import org.junit.Before;
 import org.junit.Before;
 import org.junit.Rule;
 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.assertThat;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertTrue;
 
 
-
 /**
 /**
  * Tests for {@link SlowPeerTracker}.
  * Tests for {@link SlowPeerTracker}.
  */
  */
@@ -57,6 +58,8 @@ public class TestSlowPeerTracker {
   private SlowPeerTracker tracker;
   private SlowPeerTracker tracker;
   private FakeTimer timer;
   private FakeTimer timer;
   private long reportValidityMs;
   private long reportValidityMs;
+  private static final ObjectReader READER =
+      new ObjectMapper().readerFor(new TypeReference<Set<ReportForJson>>() {});
 
 
   @Before
   @Before
   public void setup() {
   public void setup() {
@@ -220,7 +223,6 @@ public class TestSlowPeerTracker {
       throws IOException {
       throws IOException {
     final String json = tracker.getJson();
     final String json = tracker.getJson();
     LOG.info("Got JSON: {}", json);
     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,
     conf.set(DFS_NAMENODE_AUDIT_LOGGERS_KEY,
         DummyAuditLogger.class.getName());
         DummyAuditLogger.class.getName());
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
-    
     GetOpParam.Op op = GetOpParam.Op.GETFILESTATUS;
     GetOpParam.Op op = GetOpParam.Op.GETFILESTATUS;
     try {
     try {
       cluster.waitClusterUp();
       cluster.waitClusterUp();
@@ -168,7 +167,8 @@ public class TestAuditLogger {
       conn.connect();
       conn.connect();
       assertEquals(200, conn.getResponseCode());
       assertEquals(200, conn.getResponseCode());
       conn.disconnect();
       conn.disconnect();
-      assertEquals(1, DummyAuditLogger.logCount);
+      assertEquals("getfileinfo", DummyAuditLogger.lastCommand);
+      DummyAuditLogger.resetLogCount();
       assertEquals("127.0.0.1", DummyAuditLogger.remoteAddr);
       assertEquals("127.0.0.1", DummyAuditLogger.remoteAddr);
       
       
       // non-trusted proxied request
       // non-trusted proxied request
@@ -178,7 +178,9 @@ public class TestAuditLogger {
       conn.connect();
       conn.connect();
       assertEquals(200, conn.getResponseCode());
       assertEquals(200, conn.getResponseCode());
       conn.disconnect();
       conn.disconnect();
-      assertEquals(2, DummyAuditLogger.logCount);
+      assertEquals("getfileinfo", DummyAuditLogger.lastCommand);
+      assertTrue(DummyAuditLogger.logCount == 1);
+      DummyAuditLogger.resetLogCount();
       assertEquals("127.0.0.1", DummyAuditLogger.remoteAddr);
       assertEquals("127.0.0.1", DummyAuditLogger.remoteAddr);
       
       
       // trusted proxied request
       // trusted proxied request
@@ -190,7 +192,8 @@ public class TestAuditLogger {
       conn.connect();
       conn.connect();
       assertEquals(200, conn.getResponseCode());
       assertEquals(200, conn.getResponseCode());
       conn.disconnect();
       conn.disconnect();
-      assertEquals(3, DummyAuditLogger.logCount);
+      assertEquals("getfileinfo", DummyAuditLogger.lastCommand);
+      assertTrue(DummyAuditLogger.logCount == 1);
       assertEquals("1.1.1.1", DummyAuditLogger.remoteAddr);
       assertEquals("1.1.1.1", DummyAuditLogger.remoteAddr);
     } finally {
     } finally {
       cluster.shutdown();
       cluster.shutdown();
@@ -547,6 +550,7 @@ public class TestAuditLogger {
     static int unsuccessfulCount;
     static int unsuccessfulCount;
     static short foundPermission;
     static short foundPermission;
     static String remoteAddr;
     static String remoteAddr;
+    private static String lastCommand;
     
     
     public void initialize(Configuration conf) {
     public void initialize(Configuration conf) {
       initialized = true;
       initialized = true;
@@ -565,11 +569,16 @@ public class TestAuditLogger {
       if (!succeeded) {
       if (!succeeded) {
         unsuccessfulCount++;
         unsuccessfulCount++;
       }
       }
+      lastCommand = cmd;
       if (stat != null) {
       if (stat != null) {
         foundPermission = stat.getPermission().toShort();
         foundPermission = stat.getPermission().toShort();
       }
       }
     }
     }
 
 
+    public static String getLastCommand() {
+      return lastCommand;
+    }
+
   }
   }
 
 
   public static class BrokenAuditLogger implements AuditLogger {
   public static class BrokenAuditLogger implements AuditLogger {
@@ -581,7 +590,9 @@ public class TestAuditLogger {
     public void logAuditEvent(boolean succeeded, String userName,
     public void logAuditEvent(boolean succeeded, String userName,
         InetAddress addr, String cmd, String src, String dst,
         InetAddress addr, String cmd, String src, String dst,
         FileStatus stat) {
         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.conf.Configuration;
 import org.apache.hadoop.fs.BatchedRemoteIterator;
 import org.apache.hadoop.fs.BatchedRemoteIterator;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 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.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
 import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 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.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.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
 import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
 import java.io.IOException;
 import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
 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 org.junit.Test;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY;
 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.assertTrue;
 import static org.junit.Assert.fail;
 import static org.junit.Assert.fail;
-import org.mockito.Mock;
 import org.mockito.Mockito;
 import org.mockito.Mockito;
 import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.when;
 
 
 public class TestAuditLoggerWithCommands {
 public class TestAuditLoggerWithCommands {
 
 
@@ -65,13 +72,15 @@ public class TestAuditLoggerWithCommands {
   static UserGroupInformation user2;
   static UserGroupInformation user2;
   private static NamenodeProtocols proto;
   private static NamenodeProtocols proto;
 
 
-  @BeforeClass
-  public static void initialize() throws Exception {
+  @Before
+  public void initialize() throws Exception {
     // start a cluster
     // start a cluster
     conf = new HdfsConfiguration();
     conf = new HdfsConfiguration();
     conf.setBoolean(DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY, true);
     conf.setBoolean(DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY, true);
     conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY,true);
     conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY,true);
     conf.setBoolean(DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY, true);
     conf.setBoolean(DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY, true);
+    conf.setBoolean(
+        CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION, true);
     cluster =
     cluster =
         new MiniDFSCluster.Builder(conf).numDataNodes(NUM_DATA_NODES).build();
         new MiniDFSCluster.Builder(conf).numDataNodes(NUM_DATA_NODES).build();
     cluster.waitActive();
     cluster.waitActive();
@@ -88,8 +97,9 @@ public class TestAuditLoggerWithCommands {
     fs = cluster.getFileSystem();
     fs = cluster.getFileSystem();
   }
   }
 
 
-  @AfterClass
-  public static void tearDown() throws Exception {
+  @After
+  public void tearDown() throws Exception {
+    Server.getCurCall().set(null);
     fs.close();
     fs.close();
     fs2.close();
     fs2.close();
     fileSys.close();
     fileSys.close();
@@ -126,22 +136,29 @@ public class TestAuditLoggerWithCommands {
     Path path = new Path("/testdir/testdir1");
     Path path = new Path("/testdir/testdir1");
     fs.mkdirs(path);
     fs.mkdirs(path);
     fileSys = DFSTestUtil.getFileSystemAs(user1, conf);
     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();
     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 {
     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
   @Test
@@ -180,7 +197,7 @@ public class TestAuditLoggerWithCommands {
         ".*allowed=false.*ugi=theDoctor.*cmd=renameSnapshot.*";
         ".*allowed=false.*ugi=theDoctor.*cmd=renameSnapshot.*";
     fs.mkdirs(srcDir);
     fs.mkdirs(srcDir);
     fileSys = DFSTestUtil.getFileSystemAs(user1, conf);
     fileSys = DFSTestUtil.getFileSystemAs(user1, conf);
-    cluster.getNamesystem().allowSnapshot(srcDir.toString());
+    ((DistributedFileSystem)fs).allowSnapshot(srcDir);
     try {
     try {
       fileSys.createSnapshot(srcDir);
       fileSys.createSnapshot(srcDir);
       fail("The operation should have failed with AccessControlException");
       fail("The operation should have failed with AccessControlException");
@@ -215,7 +232,7 @@ public class TestAuditLoggerWithCommands {
     Path s1;
     Path s1;
     fs.mkdirs(srcDir);
     fs.mkdirs(srcDir);
     fileSys = DFSTestUtil.getFileSystemAs(user1, conf);
     fileSys = DFSTestUtil.getFileSystemAs(user1, conf);
-    cluster.getNamesystem().allowSnapshot(srcDir.toString());
+    ((DistributedFileSystem)fs).allowSnapshot(srcDir);
     try {
     try {
       s1 = fs.createSnapshot(srcDir);
       s1 = fs.createSnapshot(srcDir);
       fileSys.deleteSnapshot(srcDir, s1.getName());
       fileSys.deleteSnapshot(srcDir, s1.getName());
@@ -236,13 +253,66 @@ public class TestAuditLoggerWithCommands {
         length+1 == auditlog.getOutput().split("\n").length);
         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
   @Test
   public void testAddCacheDirective() throws Exception {
   public void testAddCacheDirective() throws Exception {
     removeExistingCachePools(null);
     removeExistingCachePools(null);
     proto.addCachePool(new CachePoolInfo("pool1").
     proto.addCachePool(new CachePoolInfo("pool1").
         setMode(new FsPermission((short) 0)));
         setMode(new FsPermission((short) 0)));
     CacheDirectiveInfo alpha = new CacheDirectiveInfo.Builder().
     CacheDirectiveInfo alpha = new CacheDirectiveInfo.Builder().
-        setPath(new Path("/alpha")).
+        setPath(new Path(System.getProperty("user.dir"), "/alpha")).
         setPool("pool1").
         setPool("pool1").
         build();
         build();
     fileSys = DFSTestUtil.getFileSystemAs(user1, conf);
     fileSys = DFSTestUtil.getFileSystemAs(user1, conf);
@@ -618,6 +688,579 @@ public class TestAuditLoggerWithCommands {
     return verifyAuditLogs(".*allowed=" + allowed + pattern);
     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) {
   private int verifyAuditLogs(String pattern) {
     int length = auditlog.getOutput().split("\n").length;
     int length = auditlog.getOutput().split("\n").length;
     String lastAudit = auditlog.getOutput().split("\n")[length - 1];
     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.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.TimeoutException;
 import java.util.regex.Pattern;
 import java.util.regex.Pattern;
+import org.slf4j.LoggerFactory;
 
 
 import static org.junit.Assert.*;
 import static org.junit.Assert.*;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_FSLOCK_FAIR_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_FSLOCK_FAIR_KEY;
@@ -347,7 +348,7 @@ public class TestFSNamesystemLock {
 
 
     fsLock.writeLock();
     fsLock.writeLock();
     timer.advance(1);
     timer.advance(1);
-    fsLock.writeUnlock("baz");
+    fsLock.writeUnlock("baz", false);
 
 
     MetricsRecordBuilder rb = MetricsAsserts.mockMetricsRecordBuilder();
     MetricsRecordBuilder rb = MetricsAsserts.mockMetricsRecordBuilder();
     rates.snapshot(rb, true);
     rates.snapshot(rb, true);
@@ -360,4 +361,48 @@ public class TestFSNamesystemLock {
     assertCounter("FSNWriteLockBazNanosNumOps", 1L, rb);
     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());
       assertEquals("Unexpected num windows", 3, windows.size());
       for (Map<String, List<Map<String, Object>>> window : windows) {
       for (Map<String, List<Map<String, Object>>> window : windows) {
         final List<Map<String, Object>> ops = window.get("ops");
         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) {
         for (Map<String, Object> op: ops) {
+          if (op.get("opType").equals("datanodeReport")) {
+            continue;
+          }
           final long count = Long.parseLong(op.get("totalCount").toString());
           final long count = Long.parseLong(op.get("totalCount").toString());
           final String opType = op.get("opType").toString();
           final String opType = op.get("opType").toString();
           final int expected;
           final int expected;
           if (opType.equals(TopConf.ALL_CMDS)) {
           if (opType.equals(TopConf.ALL_CMDS)) {
-            expected = 2*NUM_OPS;
+            expected = 2 * NUM_OPS + 2;
+          } else if (opType.equals("datanodeReport")) {
+            expected = 2;
           } else {
           } else {
             expected = NUM_OPS;
             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_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_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_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_NAMESERVICES;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX;
+
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNotNull;
@@ -33,10 +36,14 @@ import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.IOException;
 import java.io.PrintStream;
 import java.io.PrintStream;
 import java.net.InetSocketAddress;
 import java.net.InetSocketAddress;
+import java.net.URISyntaxException;
+import java.net.UnknownHostException;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Arrays;
 import java.util.List;
 import java.util.List;
 import java.util.Map;
 import java.util.Map;
+import java.util.HashSet;
+import java.util.Set;
 import java.util.StringTokenizer;
 import java.util.StringTokenizer;
 
 
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
@@ -58,7 +65,7 @@ import com.google.common.base.Joiner;
  */
  */
 public class TestGetConf {
 public class TestGetConf {
   enum TestType {
   enum TestType {
-    NAMENODE, BACKUP, SECONDARY, NNRPCADDRESSES
+    NAMENODE, BACKUP, SECONDARY, NNRPCADDRESSES, JOURNALNODE
   }
   }
   FileSystem localFileSys; 
   FileSystem localFileSys; 
   /** Setup federation nameServiceIds in the configuration */
   /** Setup federation nameServiceIds in the configuration */
@@ -96,9 +103,10 @@ public class TestGetConf {
    * Add namenodes to the static resolution list to avoid going
    * Add namenodes to the static resolution list to avoid going
    * through DNS which can be really slow in some configurations.
    * 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++) {
     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:
     case NNRPCADDRESSES:
       args[0] = Command.NNRPCADDRESSES.getName();
       args[0] = Command.NNRPCADDRESSES.getName();
       break;
       break;
+    case JOURNALNODE:
+      args[0] = Command.JOURNALNODE.getName();
     }
     }
     return runTool(conf, args, success);
     return runTool(conf, args, success);
   }
   }
@@ -321,7 +331,7 @@ public class TestGetConf {
     String[] nnAddresses = setupAddress(conf,
     String[] nnAddresses = setupAddress(conf,
         DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY, nsCount, 1000);
         DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY, nsCount, 1000);
     setupAddress(conf, DFS_NAMENODE_RPC_ADDRESS_KEY, nsCount, 1500);
     setupAddress(conf, DFS_NAMENODE_RPC_ADDRESS_KEY, nsCount, 1500);
-    setupStaticHostResolution(nsCount);
+    setupStaticHostResolution(nsCount, "nn");
     String[] backupAddresses = setupAddress(conf,
     String[] backupAddresses = setupAddress(conf,
         DFS_NAMENODE_BACKUP_ADDRESS_KEY, nsCount, 2000);
         DFS_NAMENODE_BACKUP_ADDRESS_KEY, nsCount, 2000);
     String[] secondaryAddresses = setupAddress(conf,
     String[] secondaryAddresses = setupAddress(conf,
@@ -348,7 +358,160 @@ public class TestGetConf {
     verifyAddresses(conf, TestType.SECONDARY, false, secondaryAddresses);
     verifyAddresses(conf, TestType.SECONDARY, false, secondaryAddresses);
     verifyAddresses(conf, TestType.NNRPCADDRESSES, true, nnAddresses);
     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)
   @Test(timeout=10000)
   public void testGetSpecificKey() throws Exception {
   public void testGetSpecificKey() throws Exception {
     HdfsConfiguration conf = new HdfsConfiguration();
     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_SERVICE_RPC_ADDRESS_KEY, nsCount, 1000);
     setupAddress(conf, DFS_NAMENODE_RPC_ADDRESS_KEY, nsCount, 1500);
     setupAddress(conf, DFS_NAMENODE_RPC_ADDRESS_KEY, nsCount, 1500);
     conf.set(DFS_INTERNAL_NAMESERVICES_KEY, "ns1");
     conf.set(DFS_INTERNAL_NAMESERVICES_KEY, "ns1");
-    setupStaticHostResolution(nsCount);
+    setupStaticHostResolution(nsCount, "nn");
 
 
     String[] includedNN = new String[] {"nn1:1001"};
     String[] includedNN = new String[] {"nn1:1001"};
     verifyAddresses(conf, TestType.NAMENODE, false, includedNN);
     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;
 import com.google.common.collect.Lists;
 
 
 public class TestJsonUtil {
 public class TestJsonUtil {
+
+  private static final ObjectReader READER =
+      new ObjectMapper().readerFor(Map.class);
+
   static FileStatus toFileStatus(HdfsFileStatus f, String parent) {
   static FileStatus toFileStatus(HdfsFileStatus f, String parent) {
     return new FileStatus(f.getLen(), f.isDirectory(), f.getReplication(),
     return new FileStatus(f.getLen(), f.isDirectory(), f.getReplication(),
         f.getBlockSize(), f.getModificationTime(), f.getAccessTime(),
         f.getBlockSize(), f.getModificationTime(), f.getAccessTime(),
@@ -76,9 +80,8 @@ public class TestJsonUtil {
     System.out.println("fstatus = " + fstatus);
     System.out.println("fstatus = " + fstatus);
     final String json = JsonUtil.toJsonString(status, true);
     final String json = JsonUtil.toJsonString(status, true);
     System.out.println("json    = " + json.replace(",", ",\n  "));
     System.out.println("json    = " + json.replace(",", ",\n  "));
-    ObjectReader reader = new ObjectMapper().readerFor(Map.class);
     final HdfsFileStatus s2 =
     final HdfsFileStatus s2 =
-        JsonUtilClient.toFileStatus((Map<?, ?>) reader.readValue(json), true);
+        JsonUtilClient.toFileStatus((Map<?, ?>) READER.readValue(json), true);
     final FileStatus fs2 = toFileStatus(s2, parent);
     final FileStatus fs2 = toFileStatus(s2, parent);
     System.out.println("s2      = " + s2);
     System.out.println("s2      = " + s2);
     System.out.println("fs2     = " + fs2);
     System.out.println("fs2     = " + fs2);
@@ -164,8 +167,7 @@ public class TestJsonUtil {
   public void testToAclStatus() throws IOException {
   public void testToAclStatus() throws IOException {
     String jsonString =
     String jsonString =
         "{\"AclStatus\":{\"entries\":[\"user::rwx\",\"user:user1:rw-\",\"group::rw-\",\"other::r-x\"],\"group\":\"supergroup\",\"owner\":\"testuser\",\"stickyBit\":false}}";
         "{\"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 =
     List<AclEntry> aclSpec =
         Lists.newArrayList(aclEntry(ACCESS, USER, ALL),
         Lists.newArrayList(aclEntry(ACCESS, USER, ALL),
@@ -224,8 +226,7 @@ public class TestJsonUtil {
     String jsonString = 
     String jsonString = 
         "{\"XAttrs\":[{\"name\":\"user.a1\",\"value\":\"0x313233\"}," +
         "{\"XAttrs\":[{\"name\":\"user.a1\",\"value\":\"0x313233\"}," +
         "{\"name\":\"user.a2\",\"value\":\"0x313131\"}]}";
         "{\"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).
     XAttr xAttr1 = (new XAttr.Builder()).setNameSpace(XAttr.NameSpace.USER).
         setName("a1").setValue(XAttrCodec.decodeValue("0x313233")).build();
         setName("a1").setValue(XAttrCodec.decodeValue("0x313233")).build();
     XAttr xAttr2 = (new XAttr.Builder()).setNameSpace(XAttr.NameSpace.USER).
     XAttr xAttr2 = (new XAttr.Builder()).setNameSpace(XAttr.NameSpace.USER).
@@ -250,8 +251,7 @@ public class TestJsonUtil {
     String jsonString = 
     String jsonString = 
         "{\"XAttrs\":[{\"name\":\"user.a1\",\"value\":\"0x313233\"}," +
         "{\"XAttrs\":[{\"name\":\"user.a1\",\"value\":\"0x313233\"}," +
         "{\"name\":\"user.a2\",\"value\":\"0x313131\"}]}";
         "{\"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
     // Get xattr: user.a2
     byte[] value = JsonUtilClient.getXAttr(json, "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;
 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.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotEquals;
 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.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.StorageType;
 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.SystemErasureCodingPolicies;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 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.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.snapshot.SnapshotTestHelper;
 import org.apache.hadoop.hdfs.server.namenode.web.resources.NamenodeWebHdfsMethods;
 import org.apache.hadoop.hdfs.server.namenode.web.resources.NamenodeWebHdfsMethods;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 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.token.SecretManager.InvalidToken;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.util.DataChecksum;
 import org.apache.log4j.Level;
 import org.apache.log4j.Level;
 import org.codehaus.jettison.json.JSONException;
 import org.codehaus.jettison.json.JSONException;
 import org.codehaus.jettison.json.JSONObject;
 import org.codehaus.jettison.json.JSONObject;
 import org.junit.Assert;
 import org.junit.Assert;
 import org.junit.Test;
 import org.junit.Test;
+import org.mockito.Mockito;
 import org.mockito.internal.util.reflection.Whitebox;
 import org.mockito.internal.util.reflection.Whitebox;
 
 
 import static org.mockito.Matchers.any;
 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;
   boolean DEFAULT_FINISH_JOB_WHEN_REDUCERS_DONE = true;
 
 
   String MR_AM_STAGING_DIR_ERASURECODING_ENABLED =
   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;
   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>
     <make-maven-plugin.version>1.0-beta-1</make-maven-plugin.version>
     <native-maven-plugin.version>1.0-alpha-8</native-maven-plugin.version>
     <native-maven-plugin.version>1.0-alpha-8</native-maven-plugin.version>
     <surefire.fork.timeout>900</surefire.fork.timeout>
     <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>
     <hsqldb.version>2.3.4</hsqldb.version>
     <!-- the version of Hadoop declared in the version resources; can be overridden
     <!-- the version of Hadoop declared in the version resources; can be overridden
     so that Hadoop 3.x can declare itself a 2.x artifact. -->
     so that Hadoop 3.x can declare itself a 2.x artifact. -->
@@ -1251,6 +1251,15 @@
           </exclusion>
           </exclusion>
         </exclusions>
         </exclusions>
       </dependency>
       </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>
       <dependency>
         <groupId>org.skyscreamer</groupId>
         <groupId>org.skyscreamer</groupId>
         <artifactId>jsonassert</artifactId>
         <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);
       metadataStore = S3Guard.getMetadataStore(this);
       allowAuthoritative = conf.getBoolean(METADATASTORE_AUTHORITATIVE,
       allowAuthoritative = conf.getBoolean(METADATASTORE_AUTHORITATIVE,
           DEFAULT_METADATASTORE_AUTHORITATIVE);
           DEFAULT_METADATASTORE_AUTHORITATIVE);
+      if (hasMetadataStore()) {
+        LOG.debug("Using metadata store {}, authoritative={}",
+            getMetadataStore(), allowAuthoritative);
+      }
     } catch (AmazonClientException e) {
     } catch (AmazonClientException e) {
       throw translateException("initializing ", new Path(name), 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
    * @return the metadata store of this FS instance
    */
    */
   @VisibleForTesting
   @VisibleForTesting
-  MetadataStore getMetadataStore() {
+  public MetadataStore getMetadataStore() {
     return metadataStore;
     return metadataStore;
   }
   }
 
 
@@ -2474,9 +2478,11 @@ public class S3AFileSystem extends FileSystem {
     sb.append(", statistics {")
     sb.append(", statistics {")
         .append(statistics)
         .append(statistics)
         .append("}");
         .append("}");
-    sb.append(", metrics {")
-        .append(instrumentation.dump("{", "=", "} ", true))
-        .append("}");
+    if (instrumentation != null) {
+      sb.append(", metrics {")
+          .append(instrumentation.dump("{", "=", "} ", true))
+          .append("}");
+    }
     sb.append('}');
     sb.append('}');
     return sb.toString();
     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.
         // propagate the value, building a new origin field.
         // to track overwrites, the generic key is overwritten even if
         // to track overwrites, the generic key is overwritten even if
         // already matches the new one.
         // already matches the new one.
+        String origin = "[" + StringUtils.join(
+            source.getPropertySources(key), ", ") +"]";
         final String generic = FS_S3A_PREFIX + stripped;
         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;
     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.HashMap;
 import java.util.List;
 import java.util.List;
 import java.util.Map;
 import java.util.Map;
+import java.util.TreeMap;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeUnit;
 
 
 import com.amazonaws.AmazonClientException;
 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.ProvisionedThroughputDescription;
 import com.amazonaws.services.dynamodbv2.model.ResourceInUseException;
 import com.amazonaws.services.dynamodbv2.model.ResourceInUseException;
 import com.amazonaws.services.dynamodbv2.model.ResourceNotFoundException;
 import com.amazonaws.services.dynamodbv2.model.ResourceNotFoundException;
+import com.amazonaws.services.dynamodbv2.model.TableDescription;
 import com.amazonaws.services.dynamodbv2.model.WriteRequest;
 import com.amazonaws.services.dynamodbv2.model.WriteRequest;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Preconditions;
@@ -184,6 +186,18 @@ public class DynamoDBMetadataStore implements MetadataStore {
    * DynamoDB. Value is {@value} msec. */
    * DynamoDB. Value is {@value} msec. */
   public static final long MIN_RETRY_SLEEP_MSEC = 100;
   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 =
   private static ValueMap deleteTrackingValueMap =
       new ValueMap().withBoolean(":false", false);
       new ValueMap().withBoolean(":false", false);
 
 
@@ -788,7 +802,9 @@ public class DynamoDBMetadataStore implements MetadataStore {
     try {
     try {
       try {
       try {
         LOG.debug("Binding to table {}", tableName);
         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) {
         switch (status) {
         case "CREATING":
         case "CREATING":
         case "UPDATING":
         case "UPDATING":
@@ -824,9 +840,10 @@ public class DynamoDBMetadataStore implements MetadataStore {
 
 
           createTable(capacity);
           createTable(capacity);
         } else {
         } 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());
     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.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Preconditions;
+
+import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
@@ -31,6 +33,7 @@ import org.slf4j.LoggerFactory;
 import java.io.IOException;
 import java.io.IOException;
 import java.net.URI;
 import java.net.URI;
 import java.util.Collection;
 import java.util.Collection;
+import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.LinkedList;
 import java.util.Map;
 import java.util.Map;
@@ -100,7 +103,7 @@ public class LocalMetadataStore implements MetadataStore {
   public String toString() {
   public String toString() {
     final StringBuilder sb = new StringBuilder(
     final StringBuilder sb = new StringBuilder(
         "LocalMetadataStore{");
         "LocalMetadataStore{");
-    sb.append(", uriHost='").append(uriHost).append('\'');
+    sb.append("uriHost='").append(uriHost).append('\'');
     sb.append('}');
     sb.append('}');
     return sb.toString();
     return sb.toString();
   }
   }
@@ -153,7 +156,9 @@ public class LocalMetadataStore implements MetadataStore {
         m.setIsEmptyDirectory(isEmptyDirectory(p));
         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;
       return m;
     }
     }
   }
   }
@@ -424,12 +429,22 @@ public class LocalMetadataStore implements MetadataStore {
     Preconditions.checkArgument(p.isAbsolute(), "Path must be absolute");
     Preconditions.checkArgument(p.isAbsolute(), "Path must be absolute");
     URI uri = p.toUri();
     URI uri = p.toUri();
     if (uriHost != null) {
     if (uriHost != null) {
-      Preconditions.checkArgument(!isEmpty(uri.getHost()));
+      Preconditions.checkArgument(StringUtils.isNotEmpty(uri.getHost()));
     }
     }
     return p;
     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.Closeable;
 import java.io.IOException;
 import java.io.IOException;
 import java.util.Collection;
 import java.util.Collection;
+import java.util.Map;
 
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -218,4 +219,20 @@ public interface MetadataStore extends Closeable {
    * @throws UnsupportedOperationException if not implemented
    * @throws UnsupportedOperationException if not implemented
    */
    */
   void prune(long modTime) throws IOException, UnsupportedOperationException;
   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.io.IOException;
 import java.util.Collection;
 import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
 
 
 /**
 /**
  * A no-op implementation of MetadataStore.  Clients that use this
  * A no-op implementation of MetadataStore.  Clients that use this
@@ -101,4 +103,17 @@ public class NullMetadataStore implements MetadataStore {
   public String toString() {
   public String toString() {
     return "NullMetadataStore";
     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;
 package org.apache.hadoop.fs.s3a.s3guard;
 
 
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.IOException;
 import java.net.URI;
 import java.net.URI;
 import java.util.ArrayList;
 import java.util.ArrayList;
@@ -95,6 +96,10 @@ public final class S3Guard {
           msClass.getSimpleName(), fs.getScheme());
           msClass.getSimpleName(), fs.getScheme());
       msInstance.initialize(fs);
       msInstance.initialize(fs);
       return msInstance;
       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) {
     } catch (RuntimeException | IOException e) {
       String message = "Failed to instantiate metadata store " +
       String message = "Failed to instantiate metadata store " +
           conf.get(S3_METADATA_STORE_IMPL)
           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) {
       Configuration conf) {
     if (conf == null) {
     if (conf == null) {
       return NullMetadataStore.class;
       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.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
 
 
+import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.FileStatus;
 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.LocatedFileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RemoteIterator;
 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.S3AFileStatus;
 import org.apache.hadoop.fs.s3a.S3AFileSystem;
 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.fs.shell.CommandFormat;
+import org.apache.hadoop.util.ExitUtil;
 import org.apache.hadoop.util.GenericOptionsParser;
 import org.apache.hadoop.util.GenericOptionsParser;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.hadoop.util.ToolRunner;
 
 
 import static org.apache.hadoop.fs.s3a.Constants.*;
 import static org.apache.hadoop.fs.s3a.Constants.*;
+import static org.apache.hadoop.service.launcher.LauncherExitCodes.*;
 
 
 /**
 /**
  * CLI to manage S3Guard Metadata Store.
  * 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" + Init.NAME + " - " + Init.PURPOSE + "\n" +
       "\t" + Destroy.NAME + " - " + Destroy.PURPOSE + "\n" +
       "\t" + Destroy.NAME + " - " + Destroy.PURPOSE + "\n" +
       "\t" + Import.NAME + " - " + Import.PURPOSE + "\n" +
       "\t" + Import.NAME + " - " + Import.PURPOSE + "\n" +
+      "\t" + BucketInfo.NAME + " - " + BucketInfo.PURPOSE + "\n" +
       "\t" + Diff.NAME + " - " + Diff.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
   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();
   abstract public String getUsage();
 
 
   // Exit codes
   // 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 S3AFileSystem filesystem;
   private MetadataStore store;
   private MetadataStore store;
   private final CommandFormat commandFormat;
   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.
    * Constructor a S3Guard tool with HDFS configuration.
    * @param conf Configuration.
    * @param conf Configuration.
+   * @param opts any boolean options to support
    */
    */
-  protected S3GuardTool(Configuration conf) {
+  protected S3GuardTool(Configuration conf, String...opts) {
     super(conf);
     super(conf);
 
 
-    commandFormat = new CommandFormat(0, Integer.MAX_VALUE);
+    commandFormat = new CommandFormat(0, Integer.MAX_VALUE, opts);
     // For metadata store URI
     // For metadata store URI
     commandFormat.addOptionWithValue(META_FLAG);
     commandFormat.addOptionWithValue(META_FLAG);
     // DDB region.
     // DDB region.
@@ -126,10 +135,10 @@ public abstract class S3GuardTool extends Configured implements Tool {
    * {@link Destroy}.
    * {@link Destroy}.
    *
    *
    * @param paths remaining parameters from CLI.
    * @param paths remaining parameters from CLI.
-   * @return false for invalid parameters.
    * @throws IOException on I/O errors.
    * @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();
     Configuration conf = getConf();
     String fromCli = getCommandFormat().getOptValue(REGION_FLAG);
     String fromCli = getCommandFormat().getOptValue(REGION_FLAG);
     String fromConf = conf.get(S3GUARD_DDB_REGION_KEY);
     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 != null) {
       if (fromCli.isEmpty()) {
       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) {
       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 "
             + " flag is not supported. If you need to specify a different "
             + "region than the S3 bucket, configure " + S3GUARD_DDB_REGION_KEY);
             + "region than the S3 bucket, configure " + S3GUARD_DDB_REGION_KEY);
-        return false;
       }
       }
       conf.set(S3GUARD_DDB_REGION_KEY, fromCli);
       conf.set(S3GUARD_DDB_REGION_KEY, fromCli);
-      return true;
+      return;
     }
     }
 
 
     if (fromConf != null) {
     if (fromConf != null) {
       if (fromConf.isEmpty()) {
       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);
             S3GUARD_DDB_REGION_KEY);
-        return false;
       }
       }
-      return true;
+      return;
     }
     }
 
 
     if (hasS3Path) {
     if (hasS3Path) {
       String s3Path = paths.get(0);
       String s3Path = paths.get(0);
       initS3AFileSystem(s3Path);
       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");
         "config, or S3 bucket");
-    return false;
   }
   }
 
 
   /**
   /**
@@ -189,7 +195,7 @@ public abstract class S3GuardTool extends Configured implements Tool {
     String metaURI = getCommandFormat().getOptValue(META_FLAG);
     String metaURI = getCommandFormat().getOptValue(META_FLAG);
     if (metaURI != null && !metaURI.isEmpty()) {
     if (metaURI != null && !metaURI.isEmpty()) {
       URI uri = URI.create(metaURI);
       URI uri = URI.create(metaURI);
-      LOG.info("create metadata store: {}", uri + " scheme: "
+      LOG.info("Create metadata store: {}", uri + " scheme: "
           + uri.getScheme());
           + uri.getScheme());
       switch (uri.getScheme().toLowerCase(Locale.ENGLISH)) {
       switch (uri.getScheme().toLowerCase(Locale.ENGLISH)) {
       case "local":
       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
    * @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 {
   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
     // Make sure that S3AFileSystem does not hold an actual MetadataStore
     // implementation.
     // 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)) {
     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;
     filesystem = (S3AFileSystem) fs;
   }
   }
 
 
   /**
   /**
    * Parse CLI arguments and returns the position arguments.
    * 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.
    * @param args command line arguments.
    * @return the position arguments from CLI.
    * @return the position arguments from CLI.
@@ -285,11 +304,32 @@ public abstract class S3GuardTool extends Configured implements Tool {
     return commandFormat;
     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.
    * Create the metadata store.
    */
    */
   static class Init extends S3GuardTool {
   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";
     public static final String PURPOSE = "initialize metadata repository";
     private static final String USAGE = NAME + " [OPTIONS] [s3a://BUCKET]\n" +
     private static final String USAGE = NAME + " [OPTIONS] [s3a://BUCKET]\n" +
         "\t" + PURPOSE + "\n\n" +
         "\t" + PURPOSE + "\n\n" +
@@ -325,7 +365,7 @@ public abstract class S3GuardTool extends Configured implements Tool {
     }
     }
 
 
     @Override
     @Override
-    public int run(String[] args) throws IOException {
+    public int run(String[] args, PrintStream out) throws Exception {
       List<String> paths = parseArgs(args);
       List<String> paths = parseArgs(args);
 
 
       String readCap = getCommandFormat().getOptValue(READ_FLAG);
       String readCap = getCommandFormat().getOptValue(READ_FLAG);
@@ -340,20 +380,92 @@ public abstract class S3GuardTool extends Configured implements Tool {
       }
       }
 
 
       // Validate parameters.
       // 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;
       return SUCCESS;
     }
     }
   }
   }
 
 
+
   /**
   /**
    * Destroy a metadata store.
    * Destroy a metadata store.
    */
    */
   static class Destroy extends S3GuardTool {
   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 "
     public static final String PURPOSE = "destroy Metadata Store data "
         + DATA_IN_S3_IS_PRESERVED;
         + DATA_IN_S3_IS_PRESERVED;
     private static final String USAGE = NAME + " [OPTIONS] [s3a://BUCKET]\n" +
     private static final String USAGE = NAME + " [OPTIONS] [s3a://BUCKET]\n" +
@@ -383,19 +495,21 @@ public abstract class S3GuardTool extends Configured implements Tool {
       return USAGE;
       return USAGE;
     }
     }
 
 
-    public int run(String[] args) throws IOException {
+    public int run(String[] args, PrintStream out) throws Exception {
       List<String> paths = parseArgs(args);
       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 {
       try {
         initMetadataStore(false);
         initMetadataStore(false);
       } catch (FileNotFoundException e) {
       } catch (FileNotFoundException e) {
         // indication that the table was not found
         // 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.debug("Failed to bind to store to be destroyed", e);
-        LOG.info("Metadata Store does not exist.");
         return SUCCESS;
         return SUCCESS;
       }
       }
 
 
@@ -403,7 +517,7 @@ public abstract class S3GuardTool extends Configured implements Tool {
           "Metadata Store is not initialized");
           "Metadata Store is not initialized");
 
 
       getStore().destroy();
       getStore().destroy();
-      LOG.info("Metadata store is deleted.");
+      println(out, "Metadata store is deleted.");
       return SUCCESS;
       return SUCCESS;
     }
     }
   }
   }
@@ -412,7 +526,7 @@ public abstract class S3GuardTool extends Configured implements Tool {
    * Import s3 metadata to the metadata store.
    * Import s3 metadata to the metadata store.
    */
    */
   static class Import extends S3GuardTool {
   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 " +
     public static final String PURPOSE = "import metadata from existing S3 " +
         "data";
         "data";
     private static final String USAGE = NAME + " [OPTIONS] [s3a://BUCKET]\n" +
     private static final String USAGE = NAME + " [OPTIONS] [s3a://BUCKET]\n" +
@@ -498,21 +612,16 @@ public abstract class S3GuardTool extends Configured implements Tool {
     }
     }
 
 
     @Override
     @Override
-    public int run(String[] args) throws IOException {
+    public int run(String[] args, PrintStream out) throws Exception {
       List<String> paths = parseArgs(args);
       List<String> paths = parseArgs(args);
       if (paths.isEmpty()) {
       if (paths.isEmpty()) {
-        System.err.println(getUsage());
-        return INVALID_ARGUMENT;
+        errorln(getUsage());
+        throw invalidArgs("no arguments");
       }
       }
       String s3Path = paths.get(0);
       String s3Path = paths.get(0);
       initS3AFileSystem(s3Path);
       initS3AFileSystem(s3Path);
 
 
-      URI uri;
-      try {
-        uri = new URI(s3Path);
-      } catch (URISyntaxException e) {
-        throw new IOException(e);
-      }
+      URI uri = toUri(s3Path);
       String filePath = uri.getPath();
       String filePath = uri.getPath();
       if (filePath.isEmpty()) {
       if (filePath.isEmpty()) {
         // If they specify a naked S3 URI (e.g. s3a://bucket), we'll consider
         // 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);
       Path path = new Path(filePath);
       FileStatus status = getFilesystem().getFileStatus(path);
       FileStatus status = getFilesystem().getFileStatus(path);
 
 
-      initMetadataStore(false);
+      try {
+        initMetadataStore(false);
+      } catch (FileNotFoundException e) {
+        throw storeNotFound(e);
+      }
 
 
       long items = 1;
       long items = 1;
       if (status.isFile()) {
       if (status.isFile()) {
@@ -532,17 +645,18 @@ public abstract class S3GuardTool extends Configured implements Tool {
         items = importDir(status);
         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;
       return SUCCESS;
     }
     }
+
   }
   }
 
 
   /**
   /**
    * Show diffs between the s3 and metadata store.
    * Show diffs between the s3 and metadata store.
    */
    */
   static class Diff extends S3GuardTool {
   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 " +
     public static final String PURPOSE = "report on delta between S3 and " +
         "repository";
         "repository";
     private static final String USAGE = NAME + " [OPTIONS] s3a://BUCKET\n" +
     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 (differ(msStatus, s3Status)) {
         if (s3Status != null) {
         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) {
         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 {
       try {
         s3Status = getFilesystem().getFileStatus(qualified);
         s3Status = getFilesystem().getFileStatus(qualified);
       } catch (FileNotFoundException e) {
       } catch (FileNotFoundException e) {
+        /* ignored */
       }
       }
       PathMetadata meta = getStore().get(qualified);
       PathMetadata meta = getStore().get(qualified);
       FileStatus msStatus = (meta != null && !meta.isDeleted()) ?
       FileStatus msStatus = (meta != null && !meta.isDeleted()) ?
@@ -717,18 +832,13 @@ public abstract class S3GuardTool extends Configured implements Tool {
       List<String> paths = parseArgs(args);
       List<String> paths = parseArgs(args);
       if (paths.isEmpty()) {
       if (paths.isEmpty()) {
         out.println(USAGE);
         out.println(USAGE);
-        return INVALID_ARGUMENT;
+        throw invalidArgs("no arguments");
       }
       }
       String s3Path = paths.get(0);
       String s3Path = paths.get(0);
       initS3AFileSystem(s3Path);
       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;
       Path root;
       if (uri.getPath().isEmpty()) {
       if (uri.getPath().isEmpty()) {
         root = new Path("/");
         root = new Path("/");
@@ -741,17 +851,13 @@ public abstract class S3GuardTool extends Configured implements Tool {
       return SUCCESS;
       return SUCCESS;
     }
     }
 
 
-    @Override
-    public int run(String[] args) throws IOException {
-      return run(args, System.out);
-    }
   }
   }
 
 
   /**
   /**
    * Prune metadata that has not been modified recently.
    * Prune metadata that has not been modified recently.
    */
    */
   static class Prune extends S3GuardTool {
   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 " +
     public static final String PURPOSE = "truncate older metadata from " +
         "repository "
         "repository "
         + DATA_IN_S3_IS_PRESERVED;;
         + DATA_IN_S3_IS_PRESERVED;;
@@ -803,18 +909,19 @@ public abstract class S3GuardTool extends Configured implements Tool {
       return unit.toMillis(parsed);
       return unit.toMillis(parsed);
     }
     }
 
 
-    @VisibleForTesting
     public int run(String[] args, PrintStream out) throws
     public int run(String[] args, PrintStream out) throws
         InterruptedException, IOException {
         InterruptedException, IOException {
       List<String> paths = parseArgs(args);
       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);
       initMetadataStore(false);
 
 
       Configuration conf = getConf();
       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;
       long cliDelta = 0;
       cliDelta += getDeltaComponent(TimeUnit.DAYS, "days");
       cliDelta += getDeltaComponent(TimeUnit.DAYS, "days");
@@ -823,8 +930,7 @@ public abstract class S3GuardTool extends Configured implements Tool {
       cliDelta += getDeltaComponent(TimeUnit.SECONDS, "seconds");
       cliDelta += getDeltaComponent(TimeUnit.SECONDS, "seconds");
 
 
       if (confDelta <= 0 && cliDelta <= 0) {
       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
       // 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;
       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
     @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;
   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() {
   private static void printHelp() {
     if (command == null) {
     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.");
           "administrative commands.");
     } else {
     } 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();
-    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.
    * Execute the command with the given arguments.
    *
    *
-   * @param args command specific arguments.
    * @param conf Hadoop configuration.
    * @param conf Hadoop configuration.
+   * @param args command specific arguments.
    * @return exit code.
    * @return exit code.
    * @throws Exception on I/O errors.
    * @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 {
       Exception {
     /* ToolRunner.run does this too, but we must do it before looking at
     /* ToolRunner.run does this too, but we must do it before looking at
     subCommand or instantiating the cmd object below */
     subCommand or instantiating the cmd object below */
@@ -878,9 +1184,10 @@ public abstract class S3GuardTool extends Configured implements Tool {
         .getRemainingArgs();
         .getRemainingArgs();
     if (otherArgs.length == 0) {
     if (otherArgs.length == 0) {
       printHelp();
       printHelp();
-      return INVALID_ARGUMENT;
+      throw new ExitUtil.ExitException(E_USAGE, "No arguments provided");
     }
     }
     final String subCommand = otherArgs[0];
     final String subCommand = otherArgs[0];
+    LOG.debug("Executing command {}", subCommand);
     switch (subCommand) {
     switch (subCommand) {
     case Init.NAME:
     case Init.NAME:
       command = new Init(conf);
       command = new Init(conf);
@@ -891,15 +1198,22 @@ public abstract class S3GuardTool extends Configured implements Tool {
     case Import.NAME:
     case Import.NAME:
       command = new Import(conf);
       command = new Import(conf);
       break;
       break;
+    case BucketInfo.NAME:
+      command = new BucketInfo(conf);
+      break;
     case Diff.NAME:
     case Diff.NAME:
       command = new Diff(conf);
       command = new Diff(conf);
       break;
       break;
     case Prune.NAME:
     case Prune.NAME:
       command = new Prune(conf);
       command = new Prune(conf);
       break;
       break;
+    case SetCapacity.NAME:
+      command = new SetCapacity(conf);
+      break;
     default:
     default:
       printHelp();
       printHelp();
-      return INVALID_ARGUMENT;
+      throw new ExitUtil.ExitException(E_USAGE,
+          "Unknown command " + subCommand);
     }
     }
     return ToolRunner.run(conf, command, otherArgs);
     return ToolRunner.run(conf, command, otherArgs);
   }
   }
@@ -910,15 +1224,22 @@ public abstract class S3GuardTool extends Configured implements Tool {
    */
    */
   public static void main(String[] args) {
   public static void main(String[] args) {
     try {
     try {
-      int ret = run(args, new Configuration());
-      System.exit(ret);
+      int ret = run(new Configuration(), args);
+      exit(ret, "");
     } catch (CommandFormat.UnknownOptionException e) {
     } catch (CommandFormat.UnknownOptionException e) {
-      System.err.println(e.getMessage());
+      errorln(e.getMessage());
       printHelp();
       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) {
     } catch (Throwable e) {
       e.printStackTrace(System.err);
       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
 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`
 ### Import a bucket: `s3guard import`
@@ -421,6 +421,98 @@ Example
 hadoop s3guard diff s3a://ireland-1
 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`
 ### 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`
 ### Clean up a table, `s3guard prune`
 
 
 Delete all file entries in the MetadataStore table whose object "modification
 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 )
     [-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. This does not delete the entries in the bucket itself.
 1. The modification time is effectively the creation time of the objects
 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".
 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
 ## Debugging and Error Handling
 
 
@@ -607,6 +755,12 @@ or the configuration is preventing S3Guard from finding the table.
 region as the bucket being used.
 region as the bucket being used.
 1. Create the table if necessary.
 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
 ## 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");
     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
   @Test
   public void testBucketConfigurationSkipsUnmodifiable() throws Throwable {
   public void testBucketConfigurationSkipsUnmodifiable() throws Throwable {
     Configuration config = new Configuration(false);
     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
     // files to list are delaying visibility
     if (!recursive) {
     if (!recursive) {
       // in this case only the top level files are listed
       // in this case only the top level files are listed
+      verifyFileIsListed(listedFiles, baseTestDir, fileNames);
       assertEquals("Unexpected number of files returned by listFiles() call",
       assertEquals("Unexpected number of files returned by listFiles() call",
           normalFileNum + delayedFileNum, listedFiles.size());
           normalFileNum + delayedFileNum, listedFiles.size());
-      verifyFileIsListed(listedFiles, baseTestDir, fileNames);
     } else {
     } else {
-      assertEquals("Unexpected number of files returned by listFiles() call",
-          filesAndEmptyDirectories,
-          listedFiles.size());
       for (Path dir : testDirs) {
       for (Path dir : testDirs) {
         verifyFileIsListed(listedFiles, dir, fileNames);
         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.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
 import org.apache.hadoop.fs.s3a.s3guard.DirListingMetadata;
 import org.apache.hadoop.fs.s3a.s3guard.DirListingMetadata;
 import org.junit.Assume;
 import org.junit.Assume;
 import org.junit.Test;
 import org.junit.Test;
 
 
-import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.IOException;
 import java.net.URI;
 import java.net.URI;
 import java.util.Arrays;
 import java.util.Arrays;
 
 
+import static org.apache.hadoop.fs.s3a.Constants.*;
+
 /**
 /**
  * Test cases that validate S3Guard's behavior for writing things like
  * Test cases that validate S3Guard's behavior for writing things like
  * directory listings back to the MetadataStore.
  * directory listings back to the MetadataStore.
@@ -66,7 +68,7 @@ public class ITestS3GuardWriteBack extends AbstractS3ATestBase {
     noS3Guard.mkdirs(new Path(directory, "OnS3"));
     noS3Guard.mkdirs(new Path(directory, "OnS3"));
     // Create a directory on both S3 and metadata store
     // Create a directory on both S3 and metadata store
     Path p = new Path(directory, "OnS3AndMS");
     Path p = new Path(directory, "OnS3AndMS");
-    assertPathDoesntExist(noWriteBack, p);
+    ContractTestUtils.assertPathDoesNotExist(noWriteBack, "path", p);
     noWriteBack.mkdirs(p);
     noWriteBack.mkdirs(p);
 
 
     FileStatus[] fsResults;
     FileStatus[] fsResults;
@@ -87,7 +89,7 @@ public class ITestS3GuardWriteBack extends AbstractS3ATestBase {
 
 
     // FS should return both (and will write it back)
     // FS should return both (and will write it back)
     fsResults = yesWriteBack.listStatus(directory);
     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),
             + " both /OnS3 and /OnS3AndMS: " + Arrays.toString(fsResults),
         2, fsResults.length);
         2, fsResults.length);
 
 
@@ -104,7 +106,12 @@ public class ITestS3GuardWriteBack extends AbstractS3ATestBase {
         new Path(directory, "OnS3"));
         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,
   private S3AFileSystem createTestFS(URI fsURI, boolean disableS3Guard,
       boolean authoritativeMeta) throws IOException {
       boolean authoritativeMeta) throws IOException {
     Configuration conf;
     Configuration conf;
@@ -112,12 +119,22 @@ public class ITestS3GuardWriteBack extends AbstractS3ATestBase {
     // Create a FileSystem that is S3-backed only
     // Create a FileSystem that is S3-backed only
     conf = createConfiguration();
     conf = createConfiguration();
     S3ATestUtils.disableFilesystemCaching(conf);
     S3ATestUtils.disableFilesystemCaching(conf);
+    String host = fsURI.getHost();
     if (disableS3Guard) {
     if (disableS3Guard) {
       conf.set(Constants.S3_METADATA_STORE_IMPL,
       conf.set(Constants.S3_METADATA_STORE_IMPL,
           Constants.S3GUARD_METASTORE_NULL);
           Constants.S3GUARD_METASTORE_NULL);
+      S3AUtils.setBucketOption(conf, host,
+          S3_METADATA_STORE_IMPL,
+          S3GUARD_METASTORE_NULL);
     } else {
     } else {
       S3ATestUtils.maybeEnableS3Guard(conf);
       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);
     FileSystem fs = FileSystem.get(fsURI, conf);
     return asS3AFS(fs);
     return asS3AFS(fs);
@@ -128,14 +145,4 @@ public class ITestS3GuardWriteBack extends AbstractS3ATestBase {
     return (S3AFileSystem)fs;
     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());
     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.
    * Assert that a configuration option matches the expected value.
    * @param conf configuration
    * @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;
 package org.apache.hadoop.fs.s3a.s3guard;
 
 
+import java.io.ByteArrayOutputStream;
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.IOException;
+import java.io.PrintStream;
+import java.util.Collection;
+import java.util.concurrent.Callable;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeUnit;
 
 
+import org.junit.Assume;
 import org.junit.Test;
 import org.junit.Test;
 
 
 import org.apache.hadoop.conf.Configuration;
 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.S3AFileStatus;
 import org.apache.hadoop.fs.s3a.S3ATestUtils;
 import org.apache.hadoop.fs.s3a.S3ATestUtils;
 import org.apache.hadoop.io.IOUtils;
 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.fs.s3a.s3guard.S3GuardTool.SUCCESS;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 
 
 /**
 /**
  * Common functionality for S3GuardTool test cases.
  * 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 {
 public abstract class AbstractS3GuardToolTestBase extends AbstractS3ATestBase {
 
 
   protected static final String OWNER = "hdfs";
   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;
   private MetadataStore ms;
 
 
@@ -57,6 +69,51 @@ public abstract class AbstractS3GuardToolTestBase extends AbstractS3ATestBase {
     assertEquals(message, SUCCESS, tool.run(args));
     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() {
   protected MetadataStore getMetadataStore() {
     return ms;
     return ms;
   }
   }
@@ -134,16 +191,23 @@ public abstract class AbstractS3GuardToolTestBase extends AbstractS3ATestBase {
       Thread.sleep(TimeUnit.SECONDS.toMillis(2));
       Thread.sleep(TimeUnit.SECONDS.toMillis(2));
       createFile(new Path(parent, "fresh"), true, true);
       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 {
     } finally {
       getFileSystem().delete(parent, true);
       getFileSystem().delete(parent, true);
       ms.prune(Long.MAX_VALUE);
       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
   @Test
   public void testPruneCommandCLI() throws Exception {
   public void testPruneCommandCLI() throws Exception {
     String testPath = path("testPruneCommandCLI").toString();
     String testPath = path("testPruneCommandCLI").toString();
@@ -158,4 +222,70 @@ public abstract class AbstractS3GuardToolTestBase extends AbstractS3ATestBase {
     String testPath = path("testPruneCommandConf").toString();
     String testPath = path("testPruneCommandConf").toString();
     testPruneCommand(getConfiguration(), "prune", testPath);
     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;
 package org.apache.hadoop.fs.s3a.s3guard;
 
 
 import java.io.IOException;
 import java.io.IOException;
+import java.util.Map;
+import java.util.Objects;
 import java.util.Random;
 import java.util.Random;
 import java.util.concurrent.Callable;
 import java.util.concurrent.Callable;
+import java.util.concurrent.atomic.AtomicInteger;
 
 
 import com.amazonaws.services.dynamodbv2.document.DynamoDB;
 import com.amazonaws.services.dynamodbv2.document.DynamoDB;
 import com.amazonaws.services.dynamodbv2.document.Table;
 import com.amazonaws.services.dynamodbv2.document.Table;
 import com.amazonaws.services.dynamodbv2.model.ResourceNotFoundException;
 import com.amazonaws.services.dynamodbv2.model.ResourceNotFoundException;
 import org.junit.Test;
 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.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.Destroy;
 import org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.Init;
 import org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.Init;
 import org.apache.hadoop.test.LambdaTestUtils;
 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.
  * 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
   @Test
-  public void testDynamoDBInitDestroyCycle() throws Exception {
+  public void testDynamoDBInitDestroyCycle() throws Throwable {
     String testTableName = "testDynamoDBInitDestroy" + new Random().nextInt();
     String testTableName = "testDynamoDBInitDestroy" + new Random().nextInt();
     String testS3Url = path(testTableName).toString();
     String testS3Url = path(testTableName).toString();
     S3AFileSystem fs = getFileSystem();
     S3AFileSystem fs = getFileSystem();
@@ -99,11 +176,80 @@ public class ITestS3GuardToolDynamoDB extends AbstractS3GuardToolTestBase {
       assertTrue(String.format("%s does not exist", testTableName),
       assertTrue(String.format("%s does not exist", testTableName),
           exist(db, 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 MetadataStore
       Destroy destroyCmd = new Destroy(fs.getConf());
       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);
           "destroy", "-meta", "dynamodb://" + testTableName, testS3Url);
       // Verify it does not exist
       // Verify it does not exist
       assertFalse(String.format("%s still exists", testTableName),
       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.BufferedReader;
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.ByteArrayOutputStream;
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.IOException;
 import java.io.InputStreamReader;
 import java.io.InputStreamReader;
-import java.io.PrintStream;
 import java.util.HashSet;
 import java.util.HashSet;
 import java.util.Set;
 import java.util.Set;
+import java.util.concurrent.Callable;
 
 
 import org.junit.Test;
 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.S3AFileSystem;
 import org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.Diff;
 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.
  * Test S3Guard related CLI commands against a LocalMetadataStore.
  */
  */
 public class ITestS3GuardToolLocal extends AbstractS3GuardToolTestBase {
 public class ITestS3GuardToolLocal extends AbstractS3GuardToolTestBase {
 
 
+  private static final String LOCAL_METADATA = "local://metadata";
+
   @Override
   @Override
   protected MetadataStore newMetadataStore() {
   protected MetadataStore newMetadataStore() {
     return new LocalMetadataStore();
     return new LocalMetadataStore();
@@ -65,10 +69,7 @@ public class ITestS3GuardToolLocal extends AbstractS3GuardToolTestBase {
 
 
     S3GuardTool.Import cmd = new S3GuardTool.Import(fs.getConf());
     S3GuardTool.Import cmd = new S3GuardTool.Import(fs.getConf());
     cmd.setStore(ms);
     cmd.setStore(ms);
-
-    expectSuccess("Import command did not exit successfully - see output",
-        cmd,
-        "import", parent.toString());
+    exec(cmd, "import", parent.toString());
 
 
     DirListingMetadata children =
     DirListingMetadata children =
         ms.listChildren(dir);
         ms.listChildren(dir);
@@ -80,7 +81,7 @@ public class ITestS3GuardToolLocal extends AbstractS3GuardToolTestBase {
   }
   }
 
 
   @Test
   @Test
-  public void testDiffCommand() throws IOException {
+  public void testDiffCommand() throws Exception {
     S3AFileSystem fs = getFileSystem();
     S3AFileSystem fs = getFileSystem();
     MetadataStore ms = getMetadataStore();
     MetadataStore ms = getMetadataStore();
     Set<Path> filesOnS3 = new HashSet<>(); // files on S3.
     Set<Path> filesOnS3 = new HashSet<>(); // files on S3.
@@ -108,13 +109,10 @@ public class ITestS3GuardToolLocal extends AbstractS3GuardToolTestBase {
     }
     }
 
 
     ByteArrayOutputStream buf = new ByteArrayOutputStream();
     ByteArrayOutputStream buf = new ByteArrayOutputStream();
-    PrintStream out = new PrintStream(buf);
     Diff cmd = new Diff(fs.getConf());
     Diff cmd = new Diff(fs.getConf());
     cmd.setStore(ms);
     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> actualOnS3 = new HashSet<>();
     Set<Path> actualOnMS = 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,
     assertEquals("Mismatched metadata store outputs: " + actualOut,
         filesOnMS, actualOnMS);
         filesOnMS, actualOnMS);
     assertEquals("Mismatched s3 outputs: " + actualOut, filesOnS3, actualOnS3);
     assertEquals("Mismatched s3 outputs: " + actualOut, filesOnS3, actualOnS3);
     assertFalse("Diff contained duplicates", duplicates);
     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();
     handlerThread.interrupt();
     super.serviceStop();
     super.serviceStop();
   }
   }
-  
-  public void setHeartbeatInterval(int interval) {
-    heartbeatIntervalMs.set(interval);
-  }
-  
+
   public List<? extends Collection<T>> getMatchingRequests(
   public List<? extends Collection<T>> getMatchingRequests(
                                                    Priority priority, 
                                                    Priority priority, 
                                                    String resourceName, 
                                                    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.common.annotations.VisibleForTesting;
 import com.google.protobuf.ByteString;
 import com.google.protobuf.ByteString;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.UpdateContainerTokenEvent;
 import org.slf4j.Logger;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 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.ContainerScheduler;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler.ContainerSchedulerEventType;
 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.metrics.NodeManagerMetrics;
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService;
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService;
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService.RecoveredApplicationsState;
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService.RecoveredApplicationsState;
@@ -1251,29 +1251,6 @@ public class ContainerManagerImpl extends CompositeService implements
           + " [" + containerTokenIdentifier.getVersion() + "]");
           + " [" + 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.
     // Check validity of the target resource.
     Resource currentResource = container.getResource();
     Resource currentResource = container.getResource();
     ExecutionType currentExecType =
     ExecutionType currentExecType =
@@ -1313,11 +1290,11 @@ public class ContainerManagerImpl extends CompositeService implements
     this.readLock.lock();
     this.readLock.lock();
     try {
     try {
       if (!serviceStopped) {
       if (!serviceStopped) {
-        // Dispatch message to ContainerScheduler to actually
+        // Dispatch message to Container to actually
         // make the change.
         // 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 {
       } else {
         throw new YarnException(
         throw new YarnException(
             "Unable to change container resource as the NodeManager is "
             "Unable to change container resource as the NodeManager is "
@@ -1816,10 +1793,14 @@ public class ContainerManagerImpl extends CompositeService implements
     if (container == null) {
     if (container == null) {
       throw new YarnException("Specified " + containerId + " does not exist!");
       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
       throw new YarnException("Cannot perform " + op + " on [" + containerId
           + "]. Current state is [" + container.getContainerState() + ", " +
           + "]. Current state is [" + container.getContainerState() + ", " +
-          "isReInitializing=" + container.isReInitializing() + "].");
+          "isReInitializing=" + container.isReInitializing() + "]. Container"
+          + " Execution Type is [" + container.getContainerTokenIdentifier()
+          .getExecutionType() + "].");
     }
     }
     return container;
     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,
   ROLLBACK_REINIT,
   PAUSE_CONTAINER,
   PAUSE_CONTAINER,
   RESUME_CONTAINER,
   RESUME_CONTAINER,
+  UPDATE_CONTAINER_TOKEN,
 
 
   // DownloadManager
   // DownloadManager
   CONTAINER_INITED,
   CONTAINER_INITED,
@@ -42,5 +43,8 @@ public enum ContainerEventType {
   CONTAINER_EXITED_WITH_FAILURE,
   CONTAINER_EXITED_WITH_FAILURE,
   CONTAINER_KILLED_ON_REQUEST,
   CONTAINER_KILLED_ON_REQUEST,
   CONTAINER_PAUSED,
   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.Lock;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler.UpdateContainerSchedulerEvent;
 import org.slf4j.Logger;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
 
 
@@ -308,8 +310,8 @@ public class ContainerImpl implements Container {
         UPDATE_DIAGNOSTICS_TRANSITION)
         UPDATE_DIAGNOSTICS_TRANSITION)
     .addTransition(ContainerState.NEW, ContainerState.DONE,
     .addTransition(ContainerState.NEW, ContainerState.DONE,
         ContainerEventType.KILL_CONTAINER, new KillOnNewTransition())
         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
     // From LOCALIZING State
     .addTransition(ContainerState.LOCALIZING,
     .addTransition(ContainerState.LOCALIZING,
@@ -325,8 +327,9 @@ public class ContainerImpl implements Container {
     .addTransition(ContainerState.LOCALIZING, ContainerState.KILLING,
     .addTransition(ContainerState.LOCALIZING, ContainerState.KILLING,
         ContainerEventType.KILL_CONTAINER,
         ContainerEventType.KILL_CONTAINER,
         new KillBeforeRunningTransition())
         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
     // From LOCALIZATION_FAILED State
     .addTransition(ContainerState.LOCALIZATION_FAILED,
     .addTransition(ContainerState.LOCALIZATION_FAILED,
@@ -351,6 +354,9 @@ public class ContainerImpl implements Container {
     .addTransition(ContainerState.LOCALIZATION_FAILED,
     .addTransition(ContainerState.LOCALIZATION_FAILED,
         ContainerState.LOCALIZATION_FAILED,
         ContainerState.LOCALIZATION_FAILED,
         ContainerEventType.RESOURCE_FAILED)
         ContainerEventType.RESOURCE_FAILED)
+    .addTransition(ContainerState.LOCALIZATION_FAILED,
+        ContainerState.LOCALIZATION_FAILED,
+        ContainerEventType.UPDATE_CONTAINER_TOKEN, new UpdateTransition())
 
 
     // From SCHEDULED State
     // From SCHEDULED State
     .addTransition(ContainerState.SCHEDULED, ContainerState.RUNNING,
     .addTransition(ContainerState.SCHEDULED, ContainerState.RUNNING,
@@ -364,6 +370,9 @@ public class ContainerImpl implements Container {
     .addTransition(ContainerState.SCHEDULED, ContainerState.KILLING,
     .addTransition(ContainerState.SCHEDULED, ContainerState.KILLING,
         ContainerEventType.KILL_CONTAINER,
         ContainerEventType.KILL_CONTAINER,
         new KillBeforeRunningTransition())
         new KillBeforeRunningTransition())
+    .addTransition(ContainerState.SCHEDULED, ContainerState.SCHEDULED,
+        ContainerEventType.UPDATE_CONTAINER_TOKEN,
+        new NotifyContainerSchedulerOfUpdateTransition())
 
 
     // From RUNNING State
     // From RUNNING State
     .addTransition(ContainerState.RUNNING,
     .addTransition(ContainerState.RUNNING,
@@ -376,10 +385,16 @@ public class ContainerImpl implements Container {
             ContainerState.EXITED_WITH_FAILURE),
             ContainerState.EXITED_WITH_FAILURE),
         ContainerEventType.CONTAINER_EXITED_WITH_FAILURE,
         ContainerEventType.CONTAINER_EXITED_WITH_FAILURE,
         new RetryFailureTransition())
         new RetryFailureTransition())
-    .addTransition(ContainerState.RUNNING, ContainerState.REINITIALIZING,
+    .addTransition(ContainerState.RUNNING,
+        EnumSet.of(ContainerState.RUNNING,
+            ContainerState.REINITIALIZING,
+            ContainerState.REINITIALIZING_AWAITING_KILL),
         ContainerEventType.REINITIALIZE_CONTAINER,
         ContainerEventType.REINITIALIZE_CONTAINER,
         new ReInitializeContainerTransition())
         new ReInitializeContainerTransition())
-    .addTransition(ContainerState.RUNNING, ContainerState.REINITIALIZING,
+    .addTransition(ContainerState.RUNNING,
+        EnumSet.of(ContainerState.RUNNING,
+            ContainerState.REINITIALIZING,
+            ContainerState.REINITIALIZING_AWAITING_KILL),
         ContainerEventType.ROLLBACK_REINIT,
         ContainerEventType.ROLLBACK_REINIT,
         new RollbackContainerTransition())
         new RollbackContainerTransition())
     .addTransition(ContainerState.RUNNING, ContainerState.RUNNING,
     .addTransition(ContainerState.RUNNING, ContainerState.RUNNING,
@@ -398,9 +413,16 @@ public class ContainerImpl implements Container {
         ContainerEventType.CONTAINER_KILLED_ON_REQUEST,
         ContainerEventType.CONTAINER_KILLED_ON_REQUEST,
         new KilledExternallyTransition())
         new KilledExternallyTransition())
     .addTransition(ContainerState.RUNNING, ContainerState.PAUSING,
     .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
     // From PAUSING State
+    .addTransition(ContainerState.PAUSING, ContainerState.PAUSING,
+        ContainerEventType.RESOURCE_LOCALIZED,
+        new ResourceLocalizedWhileRunningTransition())
     .addTransition(ContainerState.PAUSING, ContainerState.KILLING,
     .addTransition(ContainerState.PAUSING, ContainerState.KILLING,
         ContainerEventType.KILL_CONTAINER, new KillTransition())
         ContainerEventType.KILL_CONTAINER, new KillTransition())
     .addTransition(ContainerState.PAUSING, ContainerState.PAUSING,
     .addTransition(ContainerState.PAUSING, ContainerState.PAUSING,
@@ -420,6 +442,12 @@ public class ContainerImpl implements Container {
     .addTransition(ContainerState.PAUSING, ContainerState.EXITED_WITH_FAILURE,
     .addTransition(ContainerState.PAUSING, ContainerState.EXITED_WITH_FAILURE,
         ContainerEventType.CONTAINER_KILLED_ON_REQUEST,
         ContainerEventType.CONTAINER_KILLED_ON_REQUEST,
         new KilledExternallyTransition())
         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
     // From PAUSED State
     .addTransition(ContainerState.PAUSED, ContainerState.KILLING,
     .addTransition(ContainerState.PAUSED, ContainerState.KILLING,
@@ -429,6 +457,10 @@ public class ContainerImpl implements Container {
         UPDATE_DIAGNOSTICS_TRANSITION)
         UPDATE_DIAGNOSTICS_TRANSITION)
     .addTransition(ContainerState.PAUSED, ContainerState.PAUSED,
     .addTransition(ContainerState.PAUSED, ContainerState.PAUSED,
         ContainerEventType.PAUSE_CONTAINER)
         ContainerEventType.PAUSE_CONTAINER)
+    // This can happen during re-initialization.
+    .addTransition(ContainerState.PAUSED, ContainerState.PAUSED,
+        ContainerEventType.RESOURCE_LOCALIZED,
+        new ResourceLocalizedWhileRunningTransition())
     .addTransition(ContainerState.PAUSED, ContainerState.RESUMING,
     .addTransition(ContainerState.PAUSED, ContainerState.RESUMING,
         ContainerEventType.RESUME_CONTAINER, new ResumeContainerTransition())
         ContainerEventType.RESUME_CONTAINER, new ResumeContainerTransition())
     // In case something goes wrong then container will exit from the
     // In case something goes wrong then container will exit from the
@@ -444,6 +476,9 @@ public class ContainerImpl implements Container {
         ContainerState.EXITED_WITH_SUCCESS,
         ContainerState.EXITED_WITH_SUCCESS,
         ContainerEventType.CONTAINER_EXITED_WITH_SUCCESS,
         ContainerEventType.CONTAINER_EXITED_WITH_SUCCESS,
         new ExitedWithSuccessTransition(true))
         new ExitedWithSuccessTransition(true))
+    .addTransition(ContainerState.PAUSED, ContainerState.PAUSED,
+        ContainerEventType.UPDATE_CONTAINER_TOKEN,
+        new NotifyContainerSchedulerOfUpdateTransition())
 
 
     // From RESUMING State
     // From RESUMING State
     .addTransition(ContainerState.RESUMING, ContainerState.KILLING,
     .addTransition(ContainerState.RESUMING, ContainerState.KILLING,
@@ -453,6 +488,10 @@ public class ContainerImpl implements Container {
     .addTransition(ContainerState.RESUMING, ContainerState.RESUMING,
     .addTransition(ContainerState.RESUMING, ContainerState.RESUMING,
         ContainerEventType.UPDATE_DIAGNOSTICS_MSG,
         ContainerEventType.UPDATE_DIAGNOSTICS_MSG,
         UPDATE_DIAGNOSTICS_TRANSITION)
         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
     // In case something goes wrong then container will exit from the
     // RESUMING state
     // RESUMING state
     .addTransition(ContainerState.RESUMING,
     .addTransition(ContainerState.RESUMING,
@@ -467,6 +506,10 @@ public class ContainerImpl implements Container {
         ContainerState.EXITED_WITH_SUCCESS,
         ContainerState.EXITED_WITH_SUCCESS,
         ContainerEventType.CONTAINER_EXITED_WITH_SUCCESS,
         ContainerEventType.CONTAINER_EXITED_WITH_SUCCESS,
         new ExitedWithSuccessTransition(true))
         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
     // From REINITIALIZING State
     .addTransition(ContainerState.REINITIALIZING,
     .addTransition(ContainerState.REINITIALIZING,
@@ -478,7 +521,8 @@ public class ContainerImpl implements Container {
         ContainerEventType.CONTAINER_EXITED_WITH_FAILURE,
         ContainerEventType.CONTAINER_EXITED_WITH_FAILURE,
         new ExitedWithFailureTransition(true))
         new ExitedWithFailureTransition(true))
     .addTransition(ContainerState.REINITIALIZING,
     .addTransition(ContainerState.REINITIALIZING,
-        ContainerState.REINITIALIZING,
+        EnumSet.of(ContainerState.REINITIALIZING,
+            ContainerState.REINITIALIZING_AWAITING_KILL),
         ContainerEventType.RESOURCE_LOCALIZED,
         ContainerEventType.RESOURCE_LOCALIZED,
         new ResourceLocalizedWhileReInitTransition())
         new ResourceLocalizedWhileReInitTransition())
     .addTransition(ContainerState.REINITIALIZING, ContainerState.RUNNING,
     .addTransition(ContainerState.REINITIALIZING, ContainerState.RUNNING,
@@ -490,12 +534,39 @@ public class ContainerImpl implements Container {
         UPDATE_DIAGNOSTICS_TRANSITION)
         UPDATE_DIAGNOSTICS_TRANSITION)
     .addTransition(ContainerState.REINITIALIZING, ContainerState.KILLING,
     .addTransition(ContainerState.REINITIALIZING, ContainerState.KILLING,
         ContainerEventType.KILL_CONTAINER, new KillTransition())
         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,
     .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,
         ContainerState.SCHEDULED,
         ContainerEventType.CONTAINER_KILLED_ON_REQUEST,
         ContainerEventType.CONTAINER_KILLED_ON_REQUEST,
         new KilledForReInitializationTransition())
         new KilledForReInitializationTransition())
+    .addTransition(ContainerState.REINITIALIZING_AWAITING_KILL,
+        ContainerState.REINITIALIZING_AWAITING_KILL,
+        ContainerEventType.UPDATE_CONTAINER_TOKEN,
+        new NotifyContainerSchedulerOfUpdateTransition())
 
 
     // From RELAUNCHING State
     // From RELAUNCHING State
     .addTransition(ContainerState.RELAUNCHING, ContainerState.RUNNING,
     .addTransition(ContainerState.RELAUNCHING, ContainerState.RUNNING,
@@ -511,6 +582,10 @@ public class ContainerImpl implements Container {
         ContainerEventType.KILL_CONTAINER, new KillTransition())
         ContainerEventType.KILL_CONTAINER, new KillTransition())
     .addTransition(ContainerState.RELAUNCHING, ContainerState.KILLING,
     .addTransition(ContainerState.RELAUNCHING, ContainerState.KILLING,
         ContainerEventType.PAUSE_CONTAINER, new KillOnPauseTransition())
         ContainerEventType.PAUSE_CONTAINER, new KillOnPauseTransition())
+    .addTransition(ContainerState.RELAUNCHING, ContainerState.RELAUNCHING,
+        ContainerEventType.UPDATE_CONTAINER_TOKEN,
+        new NotifyContainerSchedulerOfUpdateTransition())
+
 
 
     // From CONTAINER_EXITED_WITH_SUCCESS State
     // From CONTAINER_EXITED_WITH_SUCCESS State
     .addTransition(ContainerState.EXITED_WITH_SUCCESS, ContainerState.DONE,
     .addTransition(ContainerState.EXITED_WITH_SUCCESS, ContainerState.DONE,
@@ -524,6 +599,10 @@ public class ContainerImpl implements Container {
         ContainerState.EXITED_WITH_SUCCESS,
         ContainerState.EXITED_WITH_SUCCESS,
         EnumSet.of(ContainerEventType.KILL_CONTAINER,
         EnumSet.of(ContainerEventType.KILL_CONTAINER,
             ContainerEventType.PAUSE_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
     // From EXITED_WITH_FAILURE State
     .addTransition(ContainerState.EXITED_WITH_FAILURE, ContainerState.DONE,
     .addTransition(ContainerState.EXITED_WITH_FAILURE, ContainerState.DONE,
@@ -537,6 +616,10 @@ public class ContainerImpl implements Container {
                    ContainerState.EXITED_WITH_FAILURE,
                    ContainerState.EXITED_WITH_FAILURE,
         EnumSet.of(ContainerEventType.KILL_CONTAINER,
         EnumSet.of(ContainerEventType.KILL_CONTAINER,
             ContainerEventType.PAUSE_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.
     // From KILLING State.
     .addTransition(ContainerState.KILLING,
     .addTransition(ContainerState.KILLING,
@@ -572,6 +655,9 @@ public class ContainerImpl implements Container {
         ContainerState.KILLING,
         ContainerState.KILLING,
         EnumSet.of(ContainerEventType.CONTAINER_LAUNCHED,
         EnumSet.of(ContainerEventType.CONTAINER_LAUNCHED,
             ContainerEventType.PAUSE_CONTAINER))
             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.
     // From CONTAINER_CLEANEDUP_AFTER_KILL State.
     .addTransition(ContainerState.CONTAINER_CLEANEDUP_AFTER_KILL,
     .addTransition(ContainerState.CONTAINER_CLEANEDUP_AFTER_KILL,
@@ -589,6 +675,10 @@ public class ContainerImpl implements Container {
             ContainerEventType.CONTAINER_EXITED_WITH_SUCCESS,
             ContainerEventType.CONTAINER_EXITED_WITH_SUCCESS,
             ContainerEventType.CONTAINER_EXITED_WITH_FAILURE,
             ContainerEventType.CONTAINER_EXITED_WITH_FAILURE,
             ContainerEventType.PAUSE_CONTAINER))
             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
     // From DONE
     .addTransition(ContainerState.DONE, ContainerState.DONE,
     .addTransition(ContainerState.DONE, ContainerState.DONE,
@@ -606,6 +696,9 @@ public class ContainerImpl implements Container {
         EnumSet.of(ContainerEventType.RESOURCE_FAILED,
         EnumSet.of(ContainerEventType.RESOURCE_FAILED,
             ContainerEventType.CONTAINER_EXITED_WITH_SUCCESS,
             ContainerEventType.CONTAINER_EXITED_WITH_SUCCESS,
             ContainerEventType.CONTAINER_EXITED_WITH_FAILURE))
             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
     // create the topology tables
     .installTopology();
     .installTopology();
@@ -616,7 +709,6 @@ public class ContainerImpl implements Container {
   public org.apache.hadoop.yarn.api.records.ContainerState getCurrentState() {
   public org.apache.hadoop.yarn.api.records.ContainerState getCurrentState() {
     switch (stateMachine.getCurrentState()) {
     switch (stateMachine.getCurrentState()) {
     case NEW:
     case NEW:
-      return org.apache.hadoop.yarn.api.records.ContainerState.NEW;
     case LOCALIZING:
     case LOCALIZING:
     case LOCALIZATION_FAILED:
     case LOCALIZATION_FAILED:
     case SCHEDULED:
     case SCHEDULED:
@@ -626,6 +718,7 @@ public class ContainerImpl implements Container {
     case RUNNING:
     case RUNNING:
     case RELAUNCHING:
     case RELAUNCHING:
     case REINITIALIZING:
     case REINITIALIZING:
+    case REINITIALIZING_AWAITING_KILL:
     case EXITED_WITH_SUCCESS:
     case EXITED_WITH_SUCCESS:
     case EXITED_WITH_FAILURE:
     case EXITED_WITH_FAILURE:
     case KILLING:
     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
    * State transition when a NEW container receives the INIT_CONTAINER
    * message.
    * message.
@@ -1074,12 +1206,15 @@ public class ContainerImpl implements Container {
   /**
   /**
    * Transition to start the Re-Initialization process.
    * Transition to start the Re-Initialization process.
    */
    */
-  static class ReInitializeContainerTransition extends ContainerTransition {
+  static class ReInitializeContainerTransition implements
+      MultipleArcTransition<ContainerImpl, ContainerEvent, ContainerState> {
 
 
     @SuppressWarnings("unchecked")
     @SuppressWarnings("unchecked")
     @Override
     @Override
-    public void transition(ContainerImpl container, ContainerEvent event) {
+    public ContainerState transition(
+        ContainerImpl container, ContainerEvent event) {
       container.reInitContext = createReInitContext(container, event);
       container.reInitContext = createReInitContext(container, event);
+      boolean resourcesPresent = false;
       try {
       try {
         // 'reInitContext.newResourceSet' can be
         // 'reInitContext.newResourceSet' can be
         // a) current container resourceSet (In case of Restart)
         // a) current container resourceSet (In case of Restart)
@@ -1101,6 +1236,7 @@ public class ContainerImpl implements Container {
           container.dispatcher.getEventHandler().handle(
           container.dispatcher.getEventHandler().handle(
               new ContainersLauncherEvent(container,
               new ContainersLauncherEvent(container,
                   ContainersLauncherEventType.CLEANUP_CONTAINER_FOR_REINIT));
                   ContainersLauncherEventType.CLEANUP_CONTAINER_FOR_REINIT));
+          resourcesPresent = true;
         }
         }
         container.metrics.reInitingContainer();
         container.metrics.reInitingContainer();
         NMAuditLogger.logSuccess(container.user,
         NMAuditLogger.logSuccess(container.user,
@@ -1112,7 +1248,11 @@ public class ContainerImpl implements Container {
             " re-initialization failure..", e);
             " re-initialization failure..", e);
         container.addDiagnostics("Error re-initializing due to" +
         container.addDiagnostics("Error re-initializing due to" +
             "[" + e.getMessage() + "]");
             "[" + e.getMessage() + "]");
+        return ContainerState.RUNNING;
       }
       }
+      return resourcesPresent ?
+          ContainerState.REINITIALIZING_AWAITING_KILL :
+          ContainerState.REINITIALIZING;
     }
     }
 
 
     protected ReInitializationContext createReInitContext(
     protected ReInitializationContext createReInitContext(
@@ -1164,11 +1304,14 @@ public class ContainerImpl implements Container {
    * If all dependencies are met, then restart Container with new bits.
    * If all dependencies are met, then restart Container with new bits.
    */
    */
   static class ResourceLocalizedWhileReInitTransition
   static class ResourceLocalizedWhileReInitTransition
-      extends ContainerTransition {
+      implements MultipleArcTransition
+      <ContainerImpl, ContainerEvent, ContainerState> {
+
 
 
     @SuppressWarnings("unchecked")
     @SuppressWarnings("unchecked")
     @Override
     @Override
-    public void transition(ContainerImpl container, ContainerEvent event) {
+    public ContainerState transition(
+        ContainerImpl container, ContainerEvent event) {
       ContainerResourceLocalizedEvent rsrcEvent =
       ContainerResourceLocalizedEvent rsrcEvent =
           (ContainerResourceLocalizedEvent) event;
           (ContainerResourceLocalizedEvent) event;
       container.reInitContext.newResourceSet.resourceLocalized(
       container.reInitContext.newResourceSet.resourceLocalized(
@@ -1180,7 +1323,9 @@ public class ContainerImpl implements Container {
         container.dispatcher.getEventHandler().handle(
         container.dispatcher.getEventHandler().handle(
             new ContainersLauncherEvent(container,
             new ContainersLauncherEvent(container,
                 ContainersLauncherEventType.CLEANUP_CONTAINER_FOR_REINIT));
                 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 {
 public enum ContainerState {
   NEW, LOCALIZING, LOCALIZATION_FAILED, SCHEDULED, RUNNING, RELAUNCHING,
   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,
   CONTAINER_CLEANEDUP_AFTER_KILL, CONTAINER_RESOURCES_CLEANINGUP, DONE,
   PAUSING, PAUSED, RESUMING
   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.Context;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
 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
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor
     .ChangeMonitoringContainerResourceEvent;
     .ChangeMonitoringContainerResourceEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerState;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerState;
@@ -151,7 +152,9 @@ public class ContainerScheduler extends AbstractService implements
     case SCHEDULE_CONTAINER:
     case SCHEDULE_CONTAINER:
       scheduleContainer(event.getContainer());
       scheduleContainer(event.getContainer());
       break;
       break;
+    // NOTE: Is sent only after container state has changed to PAUSED...
     case CONTAINER_PAUSED:
     case CONTAINER_PAUSED:
+    // NOTE: Is sent only after container state has changed to DONE...
     case CONTAINER_COMPLETED:
     case CONTAINER_COMPLETED:
       onResourcesReclaimed(event.getContainer());
       onResourcesReclaimed(event.getContainer());
       break;
       break;
@@ -180,58 +183,38 @@ public class ContainerScheduler extends AbstractService implements
     if (updateEvent.isResourceChange()) {
     if (updateEvent.isResourceChange()) {
       if (runningContainers.containsKey(containerId)) {
       if (runningContainers.containsKey(containerId)) {
         this.utilizationTracker.subtractContainerResource(
         this.utilizationTracker.subtractContainerResource(
-            updateEvent.getContainer());
-        updateEvent.getContainer().setContainerTokenIdentifier(
-            updateEvent.getUpdatedToken());
+            new ContainerImpl(getConfig(), null, null, null, null,
+                updateEvent.getOriginalToken(), context));
         this.utilizationTracker.addContainerResources(
         this.utilizationTracker.addContainerResources(
             updateEvent.getContainer());
             updateEvent.getContainer());
         getContainersMonitor().handle(
         getContainersMonitor().handle(
             new ChangeMonitoringContainerResourceEvent(containerId,
             new ChangeMonitoringContainerResourceEvent(containerId,
                 updateEvent.getUpdatedToken().getResource()));
                 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()) {
     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
           //Kill/pause opportunistic containers if any to make room for
           // promotion request
           // promotion request
           reclaimOpportunisticContainerResources(updateEvent.getContainer());
           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());
       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
     // decrement only if it was a running container
     Container completedContainer = runningContainers.remove(container
     Container completedContainer = runningContainers.remove(container
         .getContainerId());
         .getContainerId());
@@ -301,7 +294,8 @@ public class ContainerScheduler extends AbstractService implements
           ExecutionType.OPPORTUNISTIC) {
           ExecutionType.OPPORTUNISTIC) {
         this.metrics.completeOpportunisticContainer(container.getResource());
         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
    *        container without looking at available resource
    */
    */
   private void startPendingContainers(boolean forceStartGuaranteedContaieners) {
   private void startPendingContainers(boolean forceStartGuaranteedContaieners) {
-    // Start pending guaranteed containers, if resources available.
+    // Start guaranteed containers that are paused, if resources available.
     boolean resourcesAvailable = startContainers(
     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.
     // Start opportunistic containers, if resources available.
     if (resourcesAvailable) {
     if (resourcesAvailable) {
       startContainers(queuedOpportunisticContainers.values(), false);
       startContainers(queuedOpportunisticContainers.values(), false);
@@ -590,16 +567,19 @@ public class ContainerScheduler extends AbstractService implements
         queuedOpportunisticContainers.values().iterator();
         queuedOpportunisticContainers.values().iterator();
     while (containerIter.hasNext()) {
     while (containerIter.hasNext()) {
       Container container = containerIter.next();
       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.security.ContainerTokenIdentifier;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container
     .Container;
     .Container;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.UpdateContainerTokenEvent;
+
 /**
 /**
  * Update Event consumed by the {@link ContainerScheduler}.
  * Update Event consumed by the {@link ContainerScheduler}.
  */
  */
 public class UpdateContainerSchedulerEvent extends ContainerSchedulerEvent {
 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.
    * 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.
    * @return Container Token.
    */
    */
   public ContainerTokenIdentifier getUpdatedToken() {
   public ContainerTokenIdentifier getUpdatedToken() {
-    return updatedToken;
+    return containerEvent.getUpdatedToken();
   }
   }
 
 
   /**
   /**
@@ -64,7 +68,7 @@ public class UpdateContainerSchedulerEvent extends ContainerSchedulerEvent {
    * @return isResourceChange.
    * @return isResourceChange.
    */
    */
   public boolean isResourceChange() {
   public boolean isResourceChange() {
-    return isResourceChange;
+    return containerEvent.isResourceChange();
   }
   }
 
 
   /**
   /**
@@ -72,7 +76,7 @@ public class UpdateContainerSchedulerEvent extends ContainerSchedulerEvent {
    * @return isExecTypeUpdate.
    * @return isExecTypeUpdate.
    */
    */
   public boolean isExecTypeUpdate() {
   public boolean isExecTypeUpdate() {
-    return isExecTypeUpdate;
+    return containerEvent.isExecTypeUpdate();
   }
   }
 
 
   /**
   /**
@@ -80,6 +84,6 @@ public class UpdateContainerSchedulerEvent extends ContainerSchedulerEvent {
    * @return isIncrease.
    * @return isIncrease.
    */
    */
   public boolean 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);
     containerManager.startContainers(allRequests);
 
 
     BaseContainerManagerTest.waitForContainerState(containerManager, cID,
     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>();
     List<ContainerId> containerIds = new ArrayList<ContainerId>();
     containerIds.add(cID);
     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);
     containerManager.init(conf);
     nodeStatusUpdater.start();
     nodeStatusUpdater.start();
     ((NMContext)context).setNodeStatusUpdater(nodeStatusUpdater);
     ((NMContext)context).setNodeStatusUpdater(nodeStatusUpdater);
+    ((NMContext)context).setContainerStateTransitionListener(
+        new NodeManager.DefaultContainerStateListener());
   }
   }
 
 
   protected ContainerManagerImpl
   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.CMgrSignalContainersEvent;
 import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
 import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
 import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.Signal;
 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.DefaultContainerExecutor;
 import org.apache.hadoop.yarn.server.nodemanager.DeletionService;
 import org.apache.hadoop.yarn.server.nodemanager.DeletionService;
 import org.apache.hadoop.yarn.server.nodemanager.NodeManager;
 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.TestAuxServices.ServiceA;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationState;
 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.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.ContainerImpl;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ContainerLocalizer;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ContainerLocalizer;
@@ -119,6 +123,41 @@ public class TestContainerManager extends BaseContainerManagerTest {
     LOG = LoggerFactory.getLogger(TestContainerManager.class);
     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;
   private boolean delayContainers = false;
 
 
   @Override
   @Override
@@ -144,7 +183,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
   @Override
   @Override
   protected ContainerManagerImpl
   protected ContainerManagerImpl
       createContainerManager(DeletionService delSrvc) {
       createContainerManager(DeletionService delSrvc) {
-    return new ContainerManagerImpl(context, exec, delSrvc, nodeStatusUpdater,
+    return  new ContainerManagerImpl(context, exec, delSrvc, nodeStatusUpdater,
       metrics, dirsHandler) {
       metrics, dirsHandler) {
 
 
       @Override
       @Override
@@ -496,6 +535,9 @@ public class TestContainerManager extends BaseContainerManagerTest {
   @Test
   @Test
   public void testContainerUpgradeSuccessAutoCommit() throws IOException,
   public void testContainerUpgradeSuccessAutoCommit() throws IOException,
       InterruptedException, YarnException {
       InterruptedException, YarnException {
+    Listener listener = new Listener();
+    ((NodeManager.DefaultContainerStateListener)containerManager.context.
+        getContainerStateTransitionListener()).addListener(listener);
     testContainerReInitSuccess(true);
     testContainerReInitSuccess(true);
     // Should not be able to Commit (since already auto committed)
     // Should not be able to Commit (since already auto committed)
     try {
     try {
@@ -504,6 +546,41 @@ public class TestContainerManager extends BaseContainerManagerTest {
     } catch (Exception e) {
     } catch (Exception e) {
       Assert.assertTrue(e.getMessage().contains("Nothing to Commit"));
       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
   @Test
@@ -524,6 +601,9 @@ public class TestContainerManager extends BaseContainerManagerTest {
   @Test
   @Test
   public void testContainerUpgradeSuccessExplicitRollback() throws IOException,
   public void testContainerUpgradeSuccessExplicitRollback() throws IOException,
       InterruptedException, YarnException {
       InterruptedException, YarnException {
+    Listener listener = new Listener();
+    ((NodeManager.DefaultContainerStateListener)containerManager.context.
+        getContainerStateTransitionListener()).addListener(listener);
     String[] pids = testContainerReInitSuccess(false);
     String[] pids = testContainerReInitSuccess(false);
 
 
     // Test that the container can be Restarted after the successful upgrrade.
     // 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",
     Assert.assertNotEquals("The Rolled-back process should be a different pid",
         pids[0], rolledBackPid);
         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
   @Test
@@ -584,6 +725,9 @@ public class TestContainerManager extends BaseContainerManagerTest {
       return;
       return;
     }
     }
     containerManager.start();
     containerManager.start();
+    Listener listener = new Listener();
+    ((NodeManager.DefaultContainerStateListener)containerManager.context.
+        getContainerStateTransitionListener()).addListener(listener);
     // ////// Construct the Container-id
     // ////// Construct the Container-id
     ContainerId cId = createContainerId(0);
     ContainerId cId = createContainerId(0);
     File oldStartFile = new File(tmpDir, "start_file_o.txt").getAbsoluteFile();
     File oldStartFile = new File(tmpDir, "start_file_o.txt").getAbsoluteFile();
@@ -598,6 +742,32 @@ public class TestContainerManager extends BaseContainerManagerTest {
     // since upgrade was terminated..
     // since upgrade was terminated..
     Assert.assertTrue("Process is NOT alive!",
     Assert.assertTrue("Process is NOT alive!",
         DefaultContainerExecutor.containerIsAlive(pid));
         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
   @Test
@@ -632,6 +802,9 @@ public class TestContainerManager extends BaseContainerManagerTest {
       return;
       return;
     }
     }
     containerManager.start();
     containerManager.start();
+    Listener listener = new Listener();
+    ((NodeManager.DefaultContainerStateListener)containerManager.context.
+        getContainerStateTransitionListener()).addListener(listener);
     // ////// Construct the Container-id
     // ////// Construct the Container-id
     ContainerId cId = createContainerId(0);
     ContainerId cId = createContainerId(0);
     File oldStartFile = new File(tmpDir, "start_file_o.txt").getAbsoluteFile();
     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",
     Assert.assertNotEquals("The Rolled-back process should be a different pid",
         pid, rolledBackPid);
         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);
         containerManager.updateContainer(updateRequest);
     // Check response
     // Check response
     Assert.assertEquals(
     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
     for (Map.Entry<ContainerId, SerializedException> entry : updateResponse
         .getFailedRequests().entrySet()) {
         .getFailedRequests().entrySet()) {
       Assert.assertNotNull("Failed message", entry.getValue().getMessage());
       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()
         Assert.assertTrue(entry.getValue().getMessage()
             .contains("Container " + cId7.toString()
             .contains("Container " + cId7.toString()
                 + " is not handled by this NodeManager"));
                 + " 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 {
     try {
       wc = new WrappedContainer(7, 314159265358979L, 4344, "yak");
       wc = new WrappedContainer(7, 314159265358979L, 4344, "yak");
       assertEquals(ContainerState.NEW, wc.c.getContainerState());
       assertEquals(ContainerState.NEW, wc.c.getContainerState());
-      ContainerImpl container = (ContainerImpl)wc.c;
-      assertEquals(org.apache.hadoop.yarn.api.records.ContainerState.NEW, container.getCurrentState());
       wc.initContainer();
       wc.initContainer();
 
 
       // Verify request for public/private resources to localizer
       // Verify request for public/private resources to localizer
@@ -133,7 +131,6 @@ public class TestContainer {
               LocalResourceVisibility.APPLICATION));
               LocalResourceVisibility.APPLICATION));
       verify(wc.localizerBus).handle(argThat(matchesReq));
       verify(wc.localizerBus).handle(argThat(matchesReq));
       assertEquals(ContainerState.LOCALIZING, wc.c.getContainerState());
       assertEquals(ContainerState.LOCALIZING, wc.c.getContainerState());
-      assertEquals(org.apache.hadoop.yarn.api.records.ContainerState.SCHEDULED, container.getCurrentState());
     }
     }
     finally {
     finally {
       if (wc != null) {
       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.Arrays;
 import java.util.HashMap;
 import java.util.HashMap;
 import java.util.List;
 import java.util.List;
+import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 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.security.NMTokenIdentifier;
 import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit;
 import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit;
 import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
 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.DefaultContainerExecutor;
 import org.apache.hadoop.yarn.server.nodemanager.DeletionService;
 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.BaseContainerManagerTest;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl;
 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.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.container.ContainerState;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitor;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitorImpl;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitorImpl;
@@ -76,6 +83,40 @@ public class TestContainerSchedulerQueuing extends BaseContainerManagerTest {
     LOG = LoggerFactory.getLogger(TestContainerSchedulerQueuing.class);
     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;
   private boolean delayContainers = true;
 
 
   @Override
   @Override
@@ -542,6 +583,10 @@ public class TestContainerSchedulerQueuing extends BaseContainerManagerTest {
     containerManager.start();
     containerManager.start();
     containerManager.getContainerScheduler().
     containerManager.getContainerScheduler().
         setUsePauseEventForPreemption(true);
         setUsePauseEventForPreemption(true);
+
+    Listener listener = new Listener();
+    ((NodeManager.DefaultContainerStateListener)containerManager.getContext().
+        getContainerStateTransitionListener()).addListener(listener);
     ContainerLaunchContext containerLaunchContext =
     ContainerLaunchContext containerLaunchContext =
         recordFactory.newRecordInstance(ContainerLaunchContext.class);
         recordFactory.newRecordInstance(ContainerLaunchContext.class);
 
 
@@ -606,6 +651,39 @@ public class TestContainerSchedulerQueuing extends BaseContainerManagerTest {
     // starts running
     // starts running
     BaseContainerManagerTest.waitForNMContainerState(containerManager,
     BaseContainerManagerTest.waitForNMContainerState(containerManager,
         createContainerId(0), ContainerState.DONE, 40);
         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
   @Test
   public void testPromotionOfOpportunisticContainers() throws Exception {
   public void testPromotionOfOpportunisticContainers() throws Exception {
     containerManager.start();
     containerManager.start();
+    Listener listener = new Listener();
+    ((NodeManager.DefaultContainerStateListener)containerManager.getContext().
+        getContainerStateTransitionListener()).addListener(listener);
 
 
     ContainerLaunchContext containerLaunchContext =
     ContainerLaunchContext containerLaunchContext =
         recordFactory.newRecordInstance(ContainerLaunchContext.class);
         recordFactory.newRecordInstance(ContainerLaunchContext.class);
@@ -1150,6 +1231,7 @@ public class TestContainerSchedulerQueuing extends BaseContainerManagerTest {
     containerStatuses = containerManager
     containerStatuses = containerManager
         .getContainerStatuses(statRequest).getContainerStatuses();
         .getContainerStatuses(statRequest).getContainerStatuses();
     Assert.assertEquals(1, containerStatuses.size());
     Assert.assertEquals(1, containerStatuses.size());
+
     for (ContainerStatus status : containerStatuses) {
     for (ContainerStatus status : containerStatuses) {
       if (org.apache.hadoop.yarn.api.records.ContainerState.RUNNING ==
       if (org.apache.hadoop.yarn.api.records.ContainerState.RUNNING ==
           status.getState()) {
           status.getState()) {
@@ -1160,6 +1242,25 @@ public class TestContainerSchedulerQueuing extends BaseContainerManagerTest {
 
 
     // Ensure no containers are queued.
     // Ensure no containers are queued.
     Assert.assertEquals(0, containerScheduler.getNumQueuedContainers());
     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
   @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()
       app.rmContext.getSystemMetricsPublisher()
           .appFinished(app, finalState, app.finishTime);
           .appFinished(app, finalState, app.finishTime);
       // set the memory free
       // 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) {
   public void setApplicationPriority(Priority applicationPriority) {
     this.applicationPriority = 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 {
     } else {
       Class storeClass = getRMContext().getStateStore().getClass();
       Class storeClass = getRMContext().getStateStore().getClass();
       if (storeClass.equals(MemoryRMStateStore.class)) {
       if (storeClass.equals(MemoryRMStateStore.class)) {
-        MockRMMemoryStateStore mockStateStore = new MockRMMemoryStateStore();
+        MockMemoryRMStateStore mockStateStore = new MockMemoryRMStateStore();
         mockStateStore.init(conf);
         mockStateStore.init(conf);
         setRMStateStore(mockStateStore);
         setRMStateStore(mockStateStore);
       } else if (storeClass.equals(NullRMStateStore.class)) {
       } 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
     // start RM
     MockRM rm1 = new MockRM(conf);
     MockRM rm1 = new MockRM(conf);
     rm1.start();
     rm1.start();
+    MockMemoryRMStateStore memStore =
+        (MockMemoryRMStateStore) rm1.getRMStateStore();
     MockNM nm1 =
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
         new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
     nm1.registerNode();
     nm1.registerNode();
@@ -304,7 +306,7 @@ public class TestApplicationCleanup {
     rm1.waitForState(app0.getApplicationId(), RMAppState.FAILED);
     rm1.waitForState(app0.getApplicationId(), RMAppState.FAILED);
 
 
     // start new RM
     // start new RM
-    MockRM rm2 = new MockRM(conf, rm1.getRMStateStore());
+    MockRM rm2 = new MockRM(conf, memStore);
     rm2.start();
     rm2.start();
     
     
     // nm1 register to rm2, and do a heartbeat
     // 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());
     conf.set(YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName());
     MockRM rm0 = new MockRM(conf);
     MockRM rm0 = new MockRM(conf);
     rm0.start();
     rm0.start();
+    MockMemoryRMStateStore memStore =
+        (MockMemoryRMStateStore) rm0.getRMStateStore();
     MockNM nm =
     MockNM nm =
         new MockNM("127.0.0.1:1234", 65536, rm0.getResourceTrackerService());
         new MockNM("127.0.0.1:1234", 65536, rm0.getResourceTrackerService());
     nm.registerNode();
     nm.registerNode();
@@ -229,7 +231,7 @@ public class TestContainerResourceUsage {
         vcoreSeconds, metricsBefore.getVcoreSeconds());
         vcoreSeconds, metricsBefore.getVcoreSeconds());
 
 
     // create new RM to represent RM restart. Load up the state store.
     // 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();
     rm1.start();
     RMApp app0After =
     RMApp app0After =
         rm1.getRMContext().getRMApps().get(app0.getApplicationId());
         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.setBoolean(YarnConfiguration.AUTO_FAILOVER_ENABLED, false);
     Configuration conf = new YarnConfiguration(configuration);
     Configuration conf = new YarnConfiguration(configuration);
 
 
-    MemoryRMStateStore memStore = new MockRMMemoryStateStore() {
+    MemoryRMStateStore memStore = new MockMemoryRMStateStore() {
       int count = 0;
       int count = 0;
 
 
       @Override
       @Override
@@ -465,7 +465,7 @@ public class TestRMHA {
     configuration.setBoolean(YarnConfiguration.AUTO_FAILOVER_ENABLED, false);
     configuration.setBoolean(YarnConfiguration.AUTO_FAILOVER_ENABLED, false);
     Configuration conf = new YarnConfiguration(configuration);
     Configuration conf = new YarnConfiguration(configuration);
 
 
-    MemoryRMStateStore memStore = new MockRMMemoryStateStore() {
+    MemoryRMStateStore memStore = new MockMemoryRMStateStore() {
       @Override
       @Override
       public void updateApplicationState(ApplicationStateData appState) {
       public void updateApplicationState(ApplicationStateData appState) {
         notifyStoreOperationFailed(new StoreFencedException());
         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
     // PHASE 1: create RM and get state
     MockRM rm1 = createMockRM(conf);
     MockRM rm1 = createMockRM(conf);
-    MemoryRMStateStore memStore = (MemoryRMStateStore) rm1.getRMStateStore();
+    MockMemoryRMStateStore memStore =
+        (MockMemoryRMStateStore) rm1.getRMStateStore();
     Map<ApplicationId, ApplicationStateData> rmAppState =
     Map<ApplicationId, ApplicationStateData> rmAppState =
         memStore.getState().getApplicationState();
         memStore.getState().getApplicationState();
 
 
@@ -679,7 +680,7 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
   @Test (timeout = 60000)
   @Test (timeout = 60000)
   public void testRMRestartWaitForPreviousSucceededAttempt() throws Exception {
   public void testRMRestartWaitForPreviousSucceededAttempt() throws Exception {
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
-    MemoryRMStateStore memStore = new MockRMMemoryStateStore() {
+    MemoryRMStateStore memStore = new MockMemoryRMStateStore() {
       int count = 0;
       int count = 0;
 
 
       @Override
       @Override
@@ -734,7 +735,8 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1);
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1);
     // create RM
     // create RM
     MockRM rm1 = createMockRM(conf);
     MockRM rm1 = createMockRM(conf);
-    MemoryRMStateStore memStore = (MemoryRMStateStore) rm1.getRMStateStore();
+    MockMemoryRMStateStore memStore =
+        (MockMemoryRMStateStore) rm1.getRMStateStore();
     Map<ApplicationId, ApplicationStateData> rmAppState =
     Map<ApplicationId, ApplicationStateData> rmAppState =
         memStore.getState().getApplicationState();
         memStore.getState().getApplicationState();
     // start RM
     // start RM
@@ -780,7 +782,8 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
       YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS);
       YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS);
     // create RM
     // create RM
     MockRM rm1 = createMockRM(conf);
     MockRM rm1 = createMockRM(conf);
-    MemoryRMStateStore memStore = (MemoryRMStateStore) rm1.getRMStateStore();
+    MockMemoryRMStateStore memStore =
+        (MockMemoryRMStateStore) rm1.getRMStateStore();
     Map<ApplicationId, ApplicationStateData> rmAppState =
     Map<ApplicationId, ApplicationStateData> rmAppState =
         memStore.getState().getApplicationState();
         memStore.getState().getApplicationState();
     // start RM
     // start RM
@@ -824,18 +827,18 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
 
 
   @Test (timeout = 60000)
   @Test (timeout = 60000)
   public void testRMRestartKilledAppWithNoAttempts() throws Exception {
   public void testRMRestartKilledAppWithNoAttempts() throws Exception {
-    MemoryRMStateStore memStore = new MockRMMemoryStateStore() {
+    MockMemoryRMStateStore memStore = new MockMemoryRMStateStore() {
       @Override
       @Override
       public synchronized void storeApplicationAttemptStateInternal(
       public synchronized void storeApplicationAttemptStateInternal(
-          ApplicationAttemptId attemptId,
-          ApplicationAttemptStateData attemptStateData) throws Exception {
+          ApplicationAttemptId appAttemptId,
+          ApplicationAttemptStateData attemptState) throws Exception {
         // ignore attempt saving request.
         // ignore attempt saving request.
       }
       }
 
 
       @Override
       @Override
       public synchronized void updateApplicationAttemptStateInternal(
       public synchronized void updateApplicationAttemptStateInternal(
-          ApplicationAttemptId attemptId,
-          ApplicationAttemptStateData attemptStateData) throws Exception {
+          ApplicationAttemptId appAttemptId,
+          ApplicationAttemptStateData attemptState) throws Exception {
         // ignore attempt saving request.
         // ignore attempt saving request.
       }
       }
     };
     };
@@ -868,7 +871,8 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
       YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS);
       YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS);
     // PHASE 1: create RM and get state
     // PHASE 1: create RM and get state
     MockRM rm1 = createMockRM(conf);
     MockRM rm1 = createMockRM(conf);
-    MemoryRMStateStore memStore = (MemoryRMStateStore) rm1.getRMStateStore();
+    MockMemoryRMStateStore memStore =
+        (MockMemoryRMStateStore) rm1.getRMStateStore();
     Map<ApplicationId, ApplicationStateData> rmAppState =
     Map<ApplicationId, ApplicationStateData> rmAppState =
         memStore.getState().getApplicationState();
         memStore.getState().getApplicationState();
 
 
@@ -926,6 +930,9 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
         new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
         new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
     nm1.registerNode();
     nm1.registerNode();
 
 
+    MockMemoryRMStateStore memStore =
+        (MockMemoryRMStateStore) rm1.getRMStateStore();
+
     // a succeeded app.
     // a succeeded app.
     RMApp app0 = rm1.submitApp(200, "name", "user", null,
     RMApp app0 = rm1.submitApp(200, "name", "user", null,
       false, "default", 1, null, "myType");
       false, "default", 1, null, "myType");
@@ -953,7 +960,7 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
     .appCreated(any(RMApp.class), anyLong());
     .appCreated(any(RMApp.class), anyLong());
     // restart rm
     // restart rm
 
 
-    MockRM rm2 = new MockRM(conf, rm1.getRMStateStore()) {
+    MockRM rm2 = new MockRM(conf, memStore) {
       @Override
       @Override
       protected RMAppManager createRMAppManager() {
       protected RMAppManager createRMAppManager() {
         return spy(super.createRMAppManager());
         return spy(super.createRMAppManager());
@@ -1625,7 +1632,8 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
     // start RM
     // start RM
     MockRM rm1 = createMockRM(conf);
     MockRM rm1 = createMockRM(conf);
     rm1.start();
     rm1.start();
-    MemoryRMStateStore memStore = (MemoryRMStateStore) rm1.getRMStateStore();
+    MockMemoryRMStateStore memStore =
+        (MockMemoryRMStateStore) rm1.getRMStateStore();
     RMState rmState = memStore.getState();
     RMState rmState = memStore.getState();
     MockNM nm1 =
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
         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.
   // This is to test RM does not get hang on shutdown.
   @Test (timeout = 10000)
   @Test (timeout = 10000)
   public void testRMShutdown() throws Exception {
   public void testRMShutdown() throws Exception {
-    MemoryRMStateStore memStore = new MockRMMemoryStateStore() {
+    MemoryRMStateStore memStore = new MockMemoryRMStateStore() {
       @Override
       @Override
       public synchronized void checkVersion()
       public synchronized void checkVersion()
           throws Exception {
           throws Exception {
@@ -1743,7 +1751,8 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
       }
       }
     };
     };
     rm1.start();
     rm1.start();
-    MemoryRMStateStore memStore = (MemoryRMStateStore) rm1.getRMStateStore();
+    MockMemoryRMStateStore memStore =
+        (MockMemoryRMStateStore) rm1.getRMStateStore();
     RMApp app1 = null;
     RMApp app1 = null;
     try {
     try {
        app1 = rm1.submitApp(200, "name", "user",
        app1 = rm1.submitApp(200, "name", "user",
@@ -1767,7 +1776,7 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
 
 
   @Test (timeout = 20000)
   @Test (timeout = 20000)
   public void testAppRecoveredInOrderOnRMRestart() throws Exception {
   public void testAppRecoveredInOrderOnRMRestart() throws Exception {
-    MemoryRMStateStore memStore = new MockRMMemoryStateStore();
+    MemoryRMStateStore memStore = new MemoryRMStateStore();
     memStore.init(conf);
     memStore.init(conf);
 
 
     for (int i = 10; i > 0; i--) {
     for (int i = 10; i > 0; i--) {
@@ -2405,6 +2414,8 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
     MockRM rm1 = new MockRM(conf);
     MockRM rm1 = new MockRM(conf);
     rm1.start();
     rm1.start();
     CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
     CapacityScheduler cs = (CapacityScheduler) rm1.getResourceScheduler();
+    MockMemoryRMStateStore memStore =
+        (MockMemoryRMStateStore) rm1.getRMStateStore();
 
 
     MockNM nm1 =
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
         new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
@@ -2441,7 +2452,7 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
     MockRM rm2 = null;
     MockRM rm2 = null;
     // start RM2
     // start RM2
     try {
     try {
-      rm2 = new MockRM(conf, rm1.getRMStateStore());
+      rm2 = new MockRM(conf, memStore);
       rm2.start();
       rm2.start();
       Assert.assertTrue("RM start successfully", true);
       Assert.assertTrue("RM start successfully", true);
     } catch (Exception e) {
     } catch (Exception e) {
@@ -2542,6 +2553,8 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
       }
       }
     };
     };
     rm1.start();
     rm1.start();
+    MockMemoryRMStateStore memStore =
+        (MockMemoryRMStateStore) rm1.getRMStateStore();
 
 
     // add node label "x" and set node to label mapping
     // add node label "x" and set node to label mapping
     Set<String> clusterNodeLabels = new HashSet<String>();
     Set<String> clusterNodeLabels = new HashSet<String>();
@@ -2568,7 +2581,7 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
     conf.setBoolean(YarnConfiguration.NODE_LABELS_ENABLED, false);
     conf.setBoolean(YarnConfiguration.NODE_LABELS_ENABLED, false);
     MockRM rm2 = new MockRM(
     MockRM rm2 = new MockRM(
         TestUtils.getConfigurationWithDefaultQueueLabels(conf),
         TestUtils.getConfigurationWithDefaultQueueLabels(conf),
-        rm1.getRMStateStore()) {
+        memStore) {
       @Override
       @Override
       protected RMNodeLabelsManager createNodeLabelManager() {
       protected RMNodeLabelsManager createNodeLabelManager() {
         RMNodeLabelsManager mgr = new RMNodeLabelsManager();
         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, "");
     conf.set(YarnConfiguration.YARN_ADMIN_ACL, "");
     rm1 = new MockRM(conf);
     rm1 = new MockRM(conf);
     rm1.start();
     rm1.start();
+    MockMemoryRMStateStore memStore =
+        (MockMemoryRMStateStore) rm1.getRMStateStore();
     MockNM nm1 =
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 8192, rm1.getResourceTrackerService());
         new MockNM("127.0.0.1:1234", 8192, rm1.getResourceTrackerService());
     nm1.registerNode();
     nm1.registerNode();
@@ -583,7 +585,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[]{QUEUE_DOESNT_EXIST});
     csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[]{QUEUE_DOESNT_EXIST});
     final String noQueue = CapacitySchedulerConfiguration.ROOT + "." + QUEUE_DOESNT_EXIST;
     final String noQueue = CapacitySchedulerConfiguration.ROOT + "." + QUEUE_DOESNT_EXIST;
     csConf.setCapacity(noQueue, 100);
     csConf.setCapacity(noQueue, 100);
-    rm2 = new MockRM(csConf, rm1.getRMStateStore());
+    rm2 = new MockRM(csConf, memStore);
 
 
     rm2.start();
     rm2.start();
     UserGroupInformation user2 = UserGroupInformation.createRemoteUser("user2");
     UserGroupInformation user2 = UserGroupInformation.createRemoteUser("user2");
@@ -721,11 +723,15 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
 
 
   private void verifyAppRecoveryWithWrongQueueConfig(
   private void verifyAppRecoveryWithWrongQueueConfig(
       CapacitySchedulerConfiguration csConf, RMApp app, String diagnostics,
       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.
     // Restart RM with fail-fast as false. App should be killed.
     csConf.setBoolean(YarnConfiguration.RM_FAIL_FAST, false);
     csConf.setBoolean(YarnConfiguration.RM_FAIL_FAST, false);
     rm2 = new MockRM(csConf, memStore);
     rm2 = new MockRM(csConf, memStore);
     rm2.start();
     rm2.start();
+
+    MockMemoryRMStateStore memStore2 =
+        (MockMemoryRMStateStore) rm2.getRMStateStore();
+
     // Wait for app to be killed.
     // Wait for app to be killed.
     rm2.waitForState(app.getApplicationId(), RMAppState.KILLED);
     rm2.waitForState(app.getApplicationId(), RMAppState.KILLED);
     ApplicationReport report = rm2.getApplicationReport(app.getApplicationId());
     ApplicationReport report = rm2.getApplicationReport(app.getApplicationId());
@@ -734,24 +740,27 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     assertEquals(report.getYarnApplicationState(), YarnApplicationState.KILLED);
     assertEquals(report.getYarnApplicationState(), YarnApplicationState.KILLED);
     assertEquals(report.getDiagnostics(), diagnostics);
     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
     // Remove updated app info(app being KILLED) from state store and reinstate
     // state store to previous state i.e. which indicates app is RUNNING.
     // state store to previous state i.e. which indicates app is RUNNING.
     // This is to simulate app recovery with fail fast config as true.
     // This is to simulate app recovery with fail fast config as true.
     for(Map.Entry<ApplicationId, ApplicationStateData> entry :
     for(Map.Entry<ApplicationId, ApplicationStateData> entry :
-        state.getApplicationState().entrySet()) {
+        newState.getApplicationState().entrySet()) {
       ApplicationStateData appState = mock(ApplicationStateData.class);
       ApplicationStateData appState = mock(ApplicationStateData.class);
       ApplicationSubmissionContext ctxt =
       ApplicationSubmissionContext ctxt =
           mock(ApplicationSubmissionContext.class);
           mock(ApplicationSubmissionContext.class);
       when(appState.getApplicationSubmissionContext()).thenReturn(ctxt);
       when(appState.getApplicationSubmissionContext()).thenReturn(ctxt);
       when(ctxt.getApplicationId()).thenReturn(entry.getKey());
       when(ctxt.getApplicationId()).thenReturn(entry.getKey());
-      memStore.removeApplicationStateInternal(appState);
-      memStore.storeApplicationStateInternal(
+      memStore2.removeApplicationStateInternal(appState);
+      memStore2.storeApplicationStateInternal(
           entry.getKey(), entry.getValue());
           entry.getKey(), entry.getValue());
     }
     }
 
 
     // Now restart RM with fail-fast as true. QueueException should be thrown.
     // Now restart RM with fail-fast as true. QueueException should be thrown.
     csConf.setBoolean(YarnConfiguration.RM_FAIL_FAST, true);
     csConf.setBoolean(YarnConfiguration.RM_FAIL_FAST, true);
-    MockRM rm = new MockRM(csConf, memStore);
+    MockRM rm = new MockRM(csConf, memStore2);
     try {
     try {
       rm.start();
       rm.start();
       Assert.fail("QueueException must have been thrown");
       Assert.fail("QueueException must have been thrown");
@@ -781,6 +790,9 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     setupQueueConfiguration(csConf);
     setupQueueConfiguration(csConf);
     rm1 = new MockRM(csConf);
     rm1 = new MockRM(csConf);
     rm1.start();
     rm1.start();
+
+    MockMemoryRMStateStore memStore =
+        (MockMemoryRMStateStore) rm1.getRMStateStore();
     MockNM nm =
     MockNM nm =
         new MockNM("127.1.1.1:4321", 8192, rm1.getResourceTrackerService());
         new MockNM("127.1.1.1:4321", 8192, rm1.getResourceTrackerService());
     nm.registerNode();
     nm.registerNode();
@@ -801,7 +813,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     String diags = "Application killed on recovery as it was submitted to " +
     String diags = "Application killed on recovery as it was submitted to " +
         "queue QueueB which is no longer a leaf queue after restart.";
         "queue QueueB which is no longer a leaf queue after restart.";
     verifyAppRecoveryWithWrongQueueConfig(csConf, app, diags,
     verifyAppRecoveryWithWrongQueueConfig(csConf, app, diags,
-        (MemoryRMStateStore) rm1.getRMStateStore(), state);
+        memStore, state);
   }
   }
 
 
   //Test behavior of an app if queue is removed during recovery. Test case does
   //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);
     setupQueueConfiguration(csConf);
     rm1 = new MockRM(csConf);
     rm1 = new MockRM(csConf);
     rm1.start();
     rm1.start();
+    MockMemoryRMStateStore memStore =
+        (MockMemoryRMStateStore) rm1.getRMStateStore();
     MockNM nm1 =
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 8192, rm1.getResourceTrackerService());
         new MockNM("127.0.0.1:1234", 8192, rm1.getResourceTrackerService());
     MockNM nm2 =
     MockNM nm2 =
@@ -853,7 +867,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     rm1.clearQueueMetrics(app2);
     rm1.clearQueueMetrics(app2);
 
 
     // Take a copy of state store so that it can be reset to this state.
     // 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.
     // Set new configuration with QueueB removed.
     csConf = new CapacitySchedulerConfiguration(conf);
     csConf = new CapacitySchedulerConfiguration(conf);
@@ -862,7 +876,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     String diags = "Application killed on recovery as it was submitted to " +
     String diags = "Application killed on recovery as it was submitted to " +
         "queue QueueB which no longer exists after restart.";
         "queue QueueB which no longer exists after restart.";
     verifyAppRecoveryWithWrongQueueConfig(csConf, app2, diags,
     verifyAppRecoveryWithWrongQueueConfig(csConf, app2, diags,
-        (MemoryRMStateStore) rm1.getRMStateStore(), state);
+        memStore, state);
   }
   }
 
 
   private void checkParentQueue(ParentQueue parentQueue, int numContainers,
   private void checkParentQueue(ParentQueue parentQueue, int numContainers,
@@ -931,6 +945,8 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
   public void testContainersNotRecoveredForCompletedApps() throws Exception {
   public void testContainersNotRecoveredForCompletedApps() throws Exception {
     rm1 = new MockRM(conf);
     rm1 = new MockRM(conf);
     rm1.start();
     rm1.start();
+    MockMemoryRMStateStore memStore =
+        (MockMemoryRMStateStore) rm1.getRMStateStore();
     MockNM nm1 =
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 8192, rm1.getResourceTrackerService());
         new MockNM("127.0.0.1:1234", 8192, rm1.getResourceTrackerService());
     nm1.registerNode();
     nm1.registerNode();
@@ -938,7 +954,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
     MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
     MockRM.finishAMAndVerifyAppState(app1, rm1, nm1, am1);
     MockRM.finishAMAndVerifyAppState(app1, rm1, nm1, am1);
 
 
-    rm2 = new MockRM(conf, rm1.getRMStateStore());
+    rm2 = new MockRM(conf, memStore);
     rm2.start();
     rm2.start();
     nm1.setResourceTrackerService(rm2.getResourceTrackerService());
     nm1.setResourceTrackerService(rm2.getResourceTrackerService());
     NMContainerStatus runningContainer =
     NMContainerStatus runningContainer =
@@ -1212,6 +1228,9 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     // start RM
     // start RM
     rm1 = new MockRM(conf);
     rm1 = new MockRM(conf);
     rm1.start();
     rm1.start();
+
+    MockMemoryRMStateStore memStore =
+        (MockMemoryRMStateStore) rm1.getRMStateStore();
     MockNM nm1 =
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
         new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
     nm1.registerNode();
     nm1.registerNode();
@@ -1230,7 +1249,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
 
 
 
 
     // start new RM
     // start new RM
-    rm2 = new MockRM(conf, rm1.getRMStateStore());
+    rm2 = new MockRM(conf, memStore);
     rm2.start();
     rm2.start();
 
 
     am0.setAMRMProtocol(rm2.getApplicationMasterService(), rm2.getRMContext());
     am0.setAMRMProtocol(rm2.getApplicationMasterService(), rm2.getRMContext());
@@ -1370,7 +1389,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
   // RM should start correctly.
   // RM should start correctly.
   @Test (timeout = 20000)
   @Test (timeout = 20000)
   public void testAppStateSavedButAttemptStateNotSaved() throws Exception {
   public void testAppStateSavedButAttemptStateNotSaved() throws Exception {
-    MemoryRMStateStore memStore = new MemoryRMStateStore() {
+    MockMemoryRMStateStore memStore = new MockMemoryRMStateStore() {
       @Override public synchronized void updateApplicationAttemptStateInternal(
       @Override public synchronized void updateApplicationAttemptStateInternal(
           ApplicationAttemptId appAttemptId,
           ApplicationAttemptId appAttemptId,
           ApplicationAttemptStateData attemptState) {
           ApplicationAttemptStateData attemptState) {
@@ -1414,6 +1433,8 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     // start RM
     // start RM
     rm1 = new MockRM(conf);
     rm1 = new MockRM(conf);
     rm1.start();
     rm1.start();
+    MockMemoryRMStateStore memStore =
+        (MockMemoryRMStateStore) rm1.getRMStateStore();
     MockNM nm1 =
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
         new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
     nm1.registerNode();
     nm1.registerNode();
@@ -1438,8 +1459,10 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
     }
     }
 
 
     // start new RM
     // start new RM
-    rm2 = new MockRM(conf, rm1.getRMStateStore());
+    rm2 = new MockRM(conf, memStore);
     rm2.start();
     rm2.start();
+    MockMemoryRMStateStore memStore2 =
+        (MockMemoryRMStateStore) rm2.getRMStateStore();
     rm2.waitForState(app0.getApplicationId(), RMAppState.ACCEPTED);
     rm2.waitForState(app0.getApplicationId(), RMAppState.ACCEPTED);
     rm2.waitForState(am0.getApplicationAttemptId(), RMAppAttemptState.LAUNCHED);
     rm2.waitForState(am0.getApplicationAttemptId(), RMAppAttemptState.LAUNCHED);
 
 
@@ -1488,7 +1511,7 @@ public class TestWorkPreservingRMRestart extends ParameterizedSchedulerTestBase
         recoveredApp.getFinalApplicationStatus());
         recoveredApp.getFinalApplicationStatus());
 
 
     // Restart RM once more to check UAM is not re-run
     // 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();
     rm3.start();
     recoveredApp = rm3.getRMContext().getRMApps().get(app0.getApplicationId());
     recoveredApp = rm3.getRMContext().getRMApps().get(app0.getApplicationId());
     Assert.assertEquals(RMAppState.FINISHED, recoveredApp.getState());
     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.MockAM;
 import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
 import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
 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.MemoryRMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.ApplicationStateData;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.ApplicationStateData;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
@@ -702,8 +703,11 @@ public class TestAMRestart {
     // explicitly set max-am-retry count as 2.
     // explicitly set max-am-retry count as 2.
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
     MockRM rm1 = new MockRM(conf);
     MockRM rm1 = new MockRM(conf);
-    MemoryRMStateStore memStore = (MemoryRMStateStore) rm1.getRMStateStore();
     rm1.start();
     rm1.start();
+
+    MockMemoryRMStateStore memStore =
+        (MockMemoryRMStateStore) rm1.getRMStateStore();
+
     MockNM nm1 =
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 8000, rm1.getResourceTrackerService());
         new MockNM("127.0.0.1:1234", 8000, rm1.getResourceTrackerService());
     nm1.registerNode();
     nm1.registerNode();
@@ -735,7 +739,6 @@ public class TestAMRestart {
     RMAppImpl app1 = (RMAppImpl)rm1.submitApp(200, 10000, false);
     RMAppImpl app1 = (RMAppImpl)rm1.submitApp(200, 10000, false);
     app1.setSystemClock(clock);
     app1.setSystemClock(clock);
     MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
     MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1);
-
     // Fail attempt1 normally
     // Fail attempt1 normally
     nm1.nodeHeartbeat(am1.getApplicationAttemptId(),
     nm1.nodeHeartbeat(am1.getApplicationAttemptId(),
       1, ContainerState.COMPLETE);
       1, ContainerState.COMPLETE);
@@ -771,8 +774,12 @@ public class TestAMRestart {
     @SuppressWarnings("resource")
     @SuppressWarnings("resource")
     MockRM rm2 = new MockRM(conf, memStore);
     MockRM rm2 = new MockRM(conf, memStore);
     rm2.start();
     rm2.start();
+
+    MockMemoryRMStateStore memStore1 =
+        (MockMemoryRMStateStore) rm2.getRMStateStore();
     ApplicationStateData app1State =
     ApplicationStateData app1State =
-        memStore.getState().getApplicationState().get(app1.getApplicationId());
+        memStore1.getState().getApplicationState().
+        get(app1.getApplicationId());
     Assert.assertEquals(1, app1State.getFirstAttemptId());
     Assert.assertEquals(1, app1State.getFirstAttemptId());
 
 
     // re-register the NM
     // 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;
 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.any;
 import static org.mockito.Matchers.anyLong;
 import static org.mockito.Matchers.anyLong;
 import static org.mockito.Matchers.eq;
 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.times;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.verify;
 
 
+import java.io.File;
 import java.io.IOException;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Map;
 import java.util.Map;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.UnsupportedFileSystemException;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.security.Credentials;
 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;
 import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
 import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
 import org.apache.hadoop.yarn.MockApps;
 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.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 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.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.Priority;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 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.api.records.impl.pb.ApplicationSubmissionContextPBImpl;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.DrainDispatcher;
 import org.apache.hadoop.yarn.event.DrainDispatcher;
@@ -251,7 +266,113 @@ public class TestRMAppTransitions {
     rmDispatcher.start();
     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++);
     ApplicationId applicationId = MockApps.newAppID(appId++);
     String user = MockApps.newUserName();
     String user = MockApps.newUserName();
     String name = MockApps.newAppName();
     String name = MockApps.newAppName();
@@ -270,7 +391,9 @@ public class TestRMAppTransitions {
     // but applicationId is still set for safety
     // but applicationId is still set for safety
     submissionContext.setApplicationId(applicationId);
     submissionContext.setApplicationId(applicationId);
     submissionContext.setPriority(Priority.newInstance(0));
     submissionContext.setPriority(Priority.newInstance(0));
-    submissionContext.setAMContainerSpec(mock(ContainerLaunchContext.class));
+    submissionContext.setAMContainerSpec(prepareContainerLaunchContext());
+    submissionContext.setLogAggregationContext(getLogAggregationContext());
+
     RMApp application = new RMAppImpl(applicationId, rmContext, conf, name,
     RMApp application = new RMAppImpl(applicationId, rmContext, conf, name,
         user, queue, submissionContext, scheduler, masterService,
         user, queue, submissionContext, scheduler, masterService,
         System.currentTimeMillis(), "YARN", null,
         System.currentTimeMillis(), "YARN", null,
@@ -405,6 +528,7 @@ public class TestRMAppTransitions {
     // verify sendATSCreateEvent() is get called during
     // verify sendATSCreateEvent() is get called during
     // AddApplicationToSchedulerTransition.
     // AddApplicationToSchedulerTransition.
     verify(publisher).appCreated(eq(application), anyLong());
     verify(publisher).appCreated(eq(application), anyLong());
+    verifyRMAppFieldsForNonFinalTransitions(application);
     return application;
     return application;
   }
   }
 
 
@@ -422,6 +546,7 @@ public class TestRMAppTransitions {
     application.handle(event);
     application.handle(event);
     assertStartTimeSet(application);
     assertStartTimeSet(application);
     assertAppState(RMAppState.SUBMITTED, application);
     assertAppState(RMAppState.SUBMITTED, application);
+    verifyRMAppFieldsForNonFinalTransitions(application);
     return application;
     return application;
   }
   }
 
 
@@ -530,6 +655,7 @@ public class TestRMAppTransitions {
     assertFailed(application,
     assertFailed(application,
         ".*Unmanaged application.*Failing the application.*");
         ".*Unmanaged application.*Failing the application.*");
     assertAppFinalStateSaved(application);
     assertAppFinalStateSaved(application);
+    verifyRMAppFieldsForFinalTransitions(application);
   }
   }
   
   
   @Test
   @Test
@@ -539,6 +665,7 @@ public class TestRMAppTransitions {
     RMApp application = testCreateAppFinished(null, diagMsg);
     RMApp application = testCreateAppFinished(null, diagMsg);
     Assert.assertTrue("Finished application missing diagnostics",
     Assert.assertTrue("Finished application missing diagnostics",
         application.getDiagnostics().indexOf(diagMsg) != -1);
         application.getDiagnostics().indexOf(diagMsg) != -1);
+    verifyRMAppFieldsForFinalTransitions(application);
   }
   }
 
 
   @Test (timeout = 30000)
   @Test (timeout = 30000)
@@ -546,15 +673,7 @@ public class TestRMAppTransitions {
     LOG.info("--- START: testAppRecoverPath ---");
     LOG.info("--- START: testAppRecoverPath ---");
     ApplicationSubmissionContext sub =
     ApplicationSubmissionContext sub =
         Records.newRecord(ApplicationSubmissionContext.class);
         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);
     testCreateAppSubmittedRecovery(sub);
   }
   }
 
 
@@ -577,6 +696,7 @@ public class TestRMAppTransitions {
     assertAppFinalStateNotSaved(application);
     assertAppFinalStateNotSaved(application);
     verifyApplicationFinished(RMAppState.KILLED);
     verifyApplicationFinished(RMAppState.KILLED);
     verifyAppRemovedSchedulerEvent(RMAppState.KILLED);
     verifyAppRemovedSchedulerEvent(RMAppState.KILLED);
+    verifyRMAppFieldsForFinalTransitions(application);
   }
   }
 
 
   @Test
   @Test
@@ -594,6 +714,7 @@ public class TestRMAppTransitions {
     assertFailed(application, rejectedText);
     assertFailed(application, rejectedText);
     assertAppFinalStateSaved(application);
     assertAppFinalStateSaved(application);
     verifyApplicationFinished(RMAppState.FAILED);
     verifyApplicationFinished(RMAppState.FAILED);
+    verifyRMAppFieldsForFinalTransitions(application);
   }
   }
 
 
   @Test (timeout = 30000)
   @Test (timeout = 30000)
@@ -611,6 +732,7 @@ public class TestRMAppTransitions {
     assertFailed(application, rejectedText);
     assertFailed(application, rejectedText);
     assertAppFinalStateSaved(application);
     assertAppFinalStateSaved(application);
     verifyApplicationFinished(RMAppState.FAILED);
     verifyApplicationFinished(RMAppState.FAILED);
+    verifyRMAppFieldsForFinalTransitions(application);
     rmContext.getStateStore().removeApplication(application);
     rmContext.getStateStore().removeApplication(application);
   }
   }
 
 
@@ -633,6 +755,7 @@ public class TestRMAppTransitions {
     assertKilled(application);
     assertKilled(application);
     verifyApplicationFinished(RMAppState.KILLED);
     verifyApplicationFinished(RMAppState.KILLED);
     verifyAppRemovedSchedulerEvent(RMAppState.KILLED);
     verifyAppRemovedSchedulerEvent(RMAppState.KILLED);
+    verifyRMAppFieldsForFinalTransitions(application);
   }
   }
 
 
   @Test (timeout = 30000)
   @Test (timeout = 30000)
@@ -650,6 +773,7 @@ public class TestRMAppTransitions {
     assertFailed(application, rejectedText);
     assertFailed(application, rejectedText);
     assertAppFinalStateSaved(application);
     assertAppFinalStateSaved(application);
     verifyApplicationFinished(RMAppState.FAILED);
     verifyApplicationFinished(RMAppState.FAILED);
+    verifyRMAppFieldsForFinalTransitions(application);
   }
   }
 
 
   @Test (timeout = 30000)
   @Test (timeout = 30000)
@@ -684,6 +808,7 @@ public class TestRMAppTransitions {
     assertFailed(application, rejectedText);
     assertFailed(application, rejectedText);
     assertAppFinalStateSaved(application);
     assertAppFinalStateSaved(application);
     verifyApplicationFinished(RMAppState.FAILED);
     verifyApplicationFinished(RMAppState.FAILED);
+    verifyRMAppFieldsForFinalTransitions(application);
   }
   }
 
 
   @Test
   @Test
@@ -706,6 +831,7 @@ public class TestRMAppTransitions {
     assertAppFinalStateSaved(application);
     assertAppFinalStateSaved(application);
     verifyApplicationFinished(RMAppState.KILLED);
     verifyApplicationFinished(RMAppState.KILLED);
     verifyAppRemovedSchedulerEvent(RMAppState.KILLED);
     verifyAppRemovedSchedulerEvent(RMAppState.KILLED);
+    verifyRMAppFieldsForFinalTransitions(application);
   }
   }
 
 
   @Test
   @Test
@@ -769,8 +895,9 @@ public class TestRMAppTransitions {
     assertAppFinalStateSaved(application);
     assertAppFinalStateSaved(application);
     verifyApplicationFinished(RMAppState.KILLED);
     verifyApplicationFinished(RMAppState.KILLED);
     verifyAppRemovedSchedulerEvent(RMAppState.KILLED);
     verifyAppRemovedSchedulerEvent(RMAppState.KILLED);
+    verifyRMAppFieldsForFinalTransitions(application);
   }
   }
-  
+
   @Test
   @Test
   public void testAppAcceptedAttemptKilled() throws IOException,
   public void testAppAcceptedAttemptKilled() throws IOException,
       InterruptedException {
       InterruptedException {
@@ -816,6 +943,7 @@ public class TestRMAppTransitions {
     assertKilled(application);
     assertKilled(application);
     verifyApplicationFinished(RMAppState.KILLED);
     verifyApplicationFinished(RMAppState.KILLED);
     verifyAppRemovedSchedulerEvent(RMAppState.KILLED);
     verifyAppRemovedSchedulerEvent(RMAppState.KILLED);
+    verifyRMAppFieldsForFinalTransitions(application);
   }
   }
 
 
   @Test
   @Test
@@ -873,6 +1001,7 @@ public class TestRMAppTransitions {
     assertFailed(application, ".*Failing the application.*");
     assertFailed(application, ".*Failing the application.*");
     assertAppFinalStateSaved(application);
     assertAppFinalStateSaved(application);
     verifyApplicationFinished(RMAppState.FAILED);
     verifyApplicationFinished(RMAppState.FAILED);
+    verifyRMAppFieldsForFinalTransitions(application);
   }
   }
 
 
   @Test
   @Test
@@ -914,6 +1043,7 @@ public class TestRMAppTransitions {
     assertFinalAppStatus(FinalApplicationStatus.FAILED, application);
     assertFinalAppStatus(FinalApplicationStatus.FAILED, application);
     Assert.assertTrue("Finished app missing diagnostics", application
     Assert.assertTrue("Finished app missing diagnostics", application
       .getDiagnostics().indexOf(diagMsg) != -1);
       .getDiagnostics().indexOf(diagMsg) != -1);
+    verifyRMAppFieldsForFinalTransitions(application);
   }
   }
 
 
   @Test
   @Test
@@ -933,6 +1063,7 @@ public class TestRMAppTransitions {
     Assert.assertEquals("application diagnostics is not correct",
     Assert.assertEquals("application diagnostics is not correct",
         "", diag.toString());
         "", diag.toString());
     verifyApplicationFinished(RMAppState.FINISHED);
     verifyApplicationFinished(RMAppState.FINISHED);
+    verifyRMAppFieldsForFinalTransitions(application);
   }
   }
 
 
   @Test (timeout = 30000)
   @Test (timeout = 30000)
@@ -962,6 +1093,7 @@ public class TestRMAppTransitions {
 
 
     assertTimesAtFinish(application);
     assertTimesAtFinish(application);
     assertAppState(RMAppState.FAILED, application);
     assertAppState(RMAppState.FAILED, application);
+    verifyRMAppFieldsForFinalTransitions(application);
   }
   }
 
 
   @Test (timeout = 30000)
   @Test (timeout = 30000)
@@ -1016,6 +1148,7 @@ public class TestRMAppTransitions {
 
 
     assertTimesAtFinish(application);
     assertTimesAtFinish(application);
     assertAppState(RMAppState.KILLED, application);
     assertAppState(RMAppState.KILLED, application);
+    verifyRMAppFieldsForFinalTransitions(application);
   }
   }
   
   
   @Test(timeout = 30000)
   @Test(timeout = 30000)
@@ -1061,11 +1194,12 @@ public class TestRMAppTransitions {
     RMAppState finalState = appState.getState();
     RMAppState finalState = appState.getState();
     Assert.assertEquals("Application is not in finalState.", finalState,
     Assert.assertEquals("Application is not in finalState.", finalState,
         application.getState());
         application.getState());
+    verifyRMAppFieldsForFinalTransitions(application);
   }
   }
   
   
   public void createRMStateForApplications(
   public void createRMStateForApplications(
       Map<ApplicationId, ApplicationStateData> applicationState,
       Map<ApplicationId, ApplicationStateData> applicationState,
-      RMAppState rmAppState) {
+      RMAppState rmAppState) throws IOException {
     RMApp app = createNewTestApp(null);
     RMApp app = createNewTestApp(null);
     ApplicationStateData appState =
     ApplicationStateData appState =
         ApplicationStateData.newInstance(app.getSubmitTime(), app.getStartTime(),
         ApplicationStateData.newInstance(app.getSubmitTime(), app.getStartTime(),
@@ -1075,7 +1209,7 @@ public class TestRMAppTransitions {
   }
   }
   
   
   @Test
   @Test
-  public void testGetAppReport() {
+  public void testGetAppReport() throws IOException {
     RMApp app = createNewTestApp(null);
     RMApp app = createNewTestApp(null);
     assertAppState(RMAppState.NEW, app);
     assertAppState(RMAppState.NEW, app);
     ApplicationReport report = app.createAndGetApplicationReport(null, true);
     ApplicationReport report = app.createAndGetApplicationReport(null, true);
@@ -1109,4 +1243,41 @@ public class TestRMAppTransitions {
       Assert.assertEquals(finalState, appRemovedEvent.getFinalState());
       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.Arrays;
 import java.util.Collection;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Collections;
-import java.util.Enumeration;
 import java.util.HashMap;
 import java.util.HashMap;
 import java.util.List;
 import java.util.List;
 import java.util.Map;
 import java.util.Map;
-import java.util.PriorityQueue;
 import java.util.Set;
 import java.util.Set;
 import java.util.concurrent.BrokenBarrierException;
 import java.util.concurrent.BrokenBarrierException;
 import java.util.concurrent.CyclicBarrier;
 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.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.apache.hadoop.util.Time;
 import org.apache.hadoop.yarn.LocalConfigurationProvider;
 import org.apache.hadoop.yarn.LocalConfigurationProvider;
 import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
 import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
 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.DefaultResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.DominantResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.DominantResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
 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.After;
 import org.junit.Assert;
 import org.junit.Assert;
-import org.junit.Assume;
 import org.junit.Before;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
 import org.mockito.Mockito;
 import org.mockito.Mockito;
@@ -1055,13 +1048,16 @@ public class TestCapacityScheduler {
   @Test
   @Test
   public void testCapacitySchedulerInfo() throws Exception {
   public void testCapacitySchedulerInfo() throws Exception {
     QueueInfo queueInfo = resourceManager.getResourceScheduler().getQueueInfo("a", true, true);
     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();
     List<QueueUserACLInfo> userACLInfo = resourceManager.getResourceScheduler().getQueueUserAclInfo();
     Assert.assertNotNull(userACLInfo);
     Assert.assertNotNull(userACLInfo);
     for (QueueUserACLInfo queueUserACLInfo : 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 rootQueue = cs.getRootQueue();
     CSQueue queueA = findQueue(rootQueue, A);
     CSQueue queueA = findQueue(rootQueue, A);
     CSQueue queueA1 = findQueue(queueA, A1);
     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);
     setMaxAllocMb(conf, A1, 6144);
     setMaxAllocVcores(conf, A1, 3);
     setMaxAllocVcores(conf, A1, 3);
@@ -3868,12 +3864,16 @@ public class TestCapacityScheduler {
     cs.start();
     cs.start();
 
 
     QueueInfo queueInfoA = cs.getQueueInfo("a", true, false);
     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);
     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)
   @Test(timeout = 60000)
@@ -4147,20 +4147,26 @@ public class TestCapacityScheduler {
     Resource usedResource =
     Resource usedResource =
         resourceManager.getResourceScheduler()
         resourceManager.getResourceScheduler()
             .getSchedulerNode(nm_0.getNodeId()).getAllocatedResource();
             .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
     // Check total resource of scheduler node is also changed to 1 GB 1 core
     Resource totalResource =
     Resource totalResource =
         resourceManager.getResourceScheduler()
         resourceManager.getResourceScheduler()
             .getSchedulerNode(nm_0.getNodeId()).getTotalResource();
             .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
     // Check the available resource is 0/0
     Resource availableResource =
     Resource availableResource =
         resourceManager.getResourceScheduler()
         resourceManager.getResourceScheduler()
             .getSchedulerNode(nm_0.getNodeId()).getUnallocatedResource();
             .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
   @Test
@@ -4709,7 +4715,7 @@ public class TestCapacityScheduler {
 
 
     String targetQueue = "b1";
     String targetQueue = "b1";
     CSQueue b1 = cs.getQueue(targetQueue);
     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
     // test if we can convert a leaf queue which is in RUNNING state
     conf = new CapacitySchedulerConfiguration();
     conf = new CapacitySchedulerConfiguration();
@@ -4727,7 +4733,7 @@ public class TestCapacityScheduler {
     setupQueueConfiguration(conf);
     setupQueueConfiguration(conf);
     conf.set("yarn.scheduler.capacity.root.b.b1.state", "STOPPED");
     conf.set("yarn.scheduler.capacity.root.b.b1.state", "STOPPED");
     cs.reinitialize(conf, mockContext);
     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
     // test if we can convert a leaf queue which is in STOPPED state
     conf = new CapacitySchedulerConfiguration();
     conf = new CapacitySchedulerConfiguration();
@@ -4740,7 +4746,7 @@ public class TestCapacityScheduler {
     }
     }
     b1 = cs.getQueue(targetQueue);
     b1 = cs.getQueue(targetQueue);
     Assert.assertTrue(b1 instanceof ParentQueue);
     Assert.assertTrue(b1 instanceof ParentQueue);
-    Assert.assertEquals(b1.getState(), QueueState.RUNNING);
+    Assert.assertEquals(QueueState.RUNNING, b1.getState());
     Assert.assertTrue(!b1.getChildQueues().isEmpty());
     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.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
 import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
 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.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.RMSecretManagerService;
 import org.apache.hadoop.yarn.server.resourcemanager.RMSecretManagerService;
 import org.apache.hadoop.yarn.server.resourcemanager.TestRMRestart.TestSecurityMockRM;
 import org.apache.hadoop.yarn.server.resourcemanager.TestRMRestart.TestSecurityMockRM;
@@ -78,7 +78,7 @@ public class TestRMDelegationTokens {
     UserGroupInformation.getLoginUser()
     UserGroupInformation.getLoginUser()
         .setAuthenticationMethod(AuthenticationMethod.KERBEROS);
         .setAuthenticationMethod(AuthenticationMethod.KERBEROS);
 
 
-    MemoryRMStateStore memStore = new MockRMMemoryStateStore();
+    MemoryRMStateStore memStore = new MockMemoryRMStateStore();
     memStore.init(conf);
     memStore.init(conf);
     RMState rmState = memStore.getState();
     RMState rmState = memStore.getState();
 
 
@@ -132,7 +132,7 @@ public class TestRMDelegationTokens {
   // Test all expired keys are removed from state-store.
   // Test all expired keys are removed from state-store.
   @Test(timeout = 15000)
   @Test(timeout = 15000)
   public void testRemoveExpiredMasterKeyInRMStateStore() throws Exception {
   public void testRemoveExpiredMasterKeyInRMStateStore() throws Exception {
-    MemoryRMStateStore memStore = new MockRMMemoryStateStore();
+    MemoryRMStateStore memStore = new MockMemoryRMStateStore();
     memStore.init(testConf);
     memStore.init(testConf);
     RMState rmState = memStore.getState();
     RMState rmState = memStore.getState();