Forráskód Böngészése

Merge branch 'trunk' into HDFS-7240

Anu Engineer 7 éve
szülő
commit
35c2d261d2
56 módosított fájl, 1435 hozzáadás és 305 törlés
  1. 2 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/ReconfigurableBase.java
  2. 10 10
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileStatus.java
  3. 26 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/source/TestJvmMetrics.java
  4. 15 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
  5. 8 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
  6. 13 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/FileChecksumHelper.java
  7. 7 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java
  8. 16 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
  9. 34 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/OpenFilesIterator.java
  10. 15 3
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
  11. 24 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
  12. 47 16
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java
  13. 7 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
  14. 6 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
  15. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
  16. 20 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeAdminManager.java
  17. 18 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
  18. 6 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java
  19. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ReportCommand.java
  20. 9 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
  21. 46 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  22. 9 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
  23. 27 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
  24. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
  25. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html
  26. 2 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.js
  27. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
  28. 4 0
      hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md
  29. 17 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/AdminStatesBaseTest.java
  30. 177 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
  31. 52 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptedTransfer.java
  32. 3 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHdfsAdmin.java
  33. 11 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java
  34. 9 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java
  35. 27 21
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestLeaseManager.java
  36. 22 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestListOpenFiles.java
  37. 8 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDNFencingWithReplication.java
  38. 39 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java
  39. 23 18
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java
  40. 39 37
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ContainerLogAppender.java
  41. 51 52
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ProcfsBasedProcessTree.java
  42. 4 11
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestProcfsBasedProcessTree.java
  43. 1 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
  44. 4 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java
  45. 3 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/metrics/NodeManagerMetrics.java
  46. 72 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMLeveldbStateStoreService.java
  47. 11 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMStateStoreService.java
  48. 275 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/TestContainersLauncher.java
  49. 23 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/metrics/TestNodeManagerMetrics.java
  50. 35 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/TestNMLeveldbStateStoreService.java
  51. 10 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
  52. 59 45
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationFileLoaderService.java
  53. 74 18
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestAllocationFileLoaderService.java
  54. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerHA.md
  55. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md
  56. 6 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRestart.md

+ 2 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/ReconfigurableBase.java

@@ -214,7 +214,7 @@ public abstract class ReconfigurableBase
    * This method makes the change to this objects {@link Configuration}
    * This method makes the change to this objects {@link Configuration}
    * and calls reconfigurePropertyImpl to update internal data structures.
    * and calls reconfigurePropertyImpl to update internal data structures.
    * This method cannot be overridden, subclasses should instead override
    * This method cannot be overridden, subclasses should instead override
-   * reconfigureProperty.
+   * reconfigurePropertyImpl.
    */
    */
   @Override
   @Override
   public final void reconfigureProperty(String property, String newVal)
   public final void reconfigureProperty(String property, String newVal)
@@ -262,7 +262,7 @@ public abstract class ReconfigurableBase
    * all internal data structures derived from the configuration property
    * all internal data structures derived from the configuration property
    * that is being changed. If this object owns other Reconfigurable objects
    * that is being changed. If this object owns other Reconfigurable objects
    * reconfigureProperty should be called recursively to make sure that
    * reconfigureProperty should be called recursively to make sure that
-   * to make sure that the configuration of these objects is updated.
+   * the configuration of these objects are updated.
    *
    *
    * @param property Name of the property that is being reconfigured.
    * @param property Name of the property that is being reconfigured.
    * @param newVal Proposed new value of the property.
    * @param newVal Proposed new value of the property.

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

@@ -444,18 +444,18 @@ public class FileStatus implements Writable, Comparable<Object>,
     StringBuilder sb = new StringBuilder();
     StringBuilder sb = new StringBuilder();
     sb.append(getClass().getSimpleName()); 
     sb.append(getClass().getSimpleName()); 
     sb.append("{");
     sb.append("{");
-    sb.append("path=" + getPath());
-    sb.append("; isDirectory=" + isDirectory());
+    sb.append("path=" + path);
+    sb.append("; isDirectory=" + isdir);
     if(!isDirectory()){
     if(!isDirectory()){
-      sb.append("; length=" + getLen());
-      sb.append("; replication=" + getReplication());
-      sb.append("; blocksize=" + getBlockSize());
+      sb.append("; length=" + length);
+      sb.append("; replication=" + block_replication);
+      sb.append("; blocksize=" + blocksize);
     }
     }
-    sb.append("; modification_time=" + getModificationTime());
-    sb.append("; access_time=" + getAccessTime());
-    sb.append("; owner=" + getOwner());
-    sb.append("; group=" + getGroup());
-    sb.append("; permission=" + getPermission());
+    sb.append("; modification_time=" + modification_time);
+    sb.append("; access_time=" + access_time);
+    sb.append("; owner=" + owner);
+    sb.append("; group=" + group);
+    sb.append("; permission=" + permission);
     sb.append("; isSymlink=" + isSymlink());
     sb.append("; isSymlink=" + isSymlink());
     if(isSymlink()) {
     if(isSymlink()) {
       try {
       try {

+ 26 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/source/TestJvmMetrics.java

@@ -200,4 +200,30 @@ public class TestJvmMetrics {
     Assert.assertTrue(alerter.numAlerts > 0);
     Assert.assertTrue(alerter.numAlerts > 0);
     Assert.assertTrue(alerter.maxGcTimePercentage >= alertGcPerc);
     Assert.assertTrue(alerter.maxGcTimePercentage >= alertGcPerc);
   }
   }
+
+  @Test
+  public void testJvmMetricsSingletonWithSameProcessName() {
+    JvmMetrics jvmMetrics1 = org.apache.hadoop.metrics2.source.JvmMetrics
+        .initSingleton("test", null);
+    JvmMetrics jvmMetrics2 = org.apache.hadoop.metrics2.source.JvmMetrics
+        .initSingleton("test", null);
+    Assert.assertEquals("initSingleton should return the singleton instance",
+        jvmMetrics1, jvmMetrics2);
+  }
+
+  @Test
+  public void testJvmMetricsSingletonWithDifferentProcessNames() {
+    final String process1Name = "process1";
+    JvmMetrics jvmMetrics1 = org.apache.hadoop.metrics2.source.JvmMetrics
+        .initSingleton(process1Name, null);
+    final String process2Name = "process2";
+    JvmMetrics jvmMetrics2 = org.apache.hadoop.metrics2.source.JvmMetrics
+        .initSingleton(process2Name, null);
+    Assert.assertEquals("initSingleton should return the singleton instance",
+        jvmMetrics1, jvmMetrics2);
+    Assert.assertEquals("unexpected process name of the singleton instance",
+        process1Name, jvmMetrics1.processName);
+    Assert.assertEquals("unexpected process name of the singleton instance",
+        process1Name, jvmMetrics2.processName);
+  }
 }
 }

+ 15 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -133,6 +133,7 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
 import org.apache.hadoop.hdfs.protocol.OpenFilesIterator;
 import org.apache.hadoop.hdfs.protocol.OpenFilesIterator;
+import org.apache.hadoop.hdfs.protocol.OpenFilesIterator.OpenFilesType;
 import org.apache.hadoop.hdfs.protocol.QuotaByStorageTypeExceededException;
 import org.apache.hadoop.hdfs.protocol.QuotaByStorageTypeExceededException;
 import org.apache.hadoop.hdfs.protocol.ReencryptionStatusIterator;
 import org.apache.hadoop.hdfs.protocol.ReencryptionStatusIterator;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
@@ -3084,8 +3085,21 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    *
    *
    * @throws IOException
    * @throws IOException
    */
    */
+  @Deprecated
   public RemoteIterator<OpenFileEntry> listOpenFiles() throws IOException {
   public RemoteIterator<OpenFileEntry> listOpenFiles() throws IOException {
     checkOpen();
     checkOpen();
-    return new OpenFilesIterator(namenode, tracer);
+    return listOpenFiles(EnumSet.of(OpenFilesType.ALL_OPEN_FILES));
+  }
+
+  /**
+   * Get a remote iterator to the open files list by type, managed by NameNode.
+   *
+   * @param openFilesTypes
+   * @throws IOException
+   */
+  public RemoteIterator<OpenFileEntry> listOpenFiles(
+      EnumSet<OpenFilesType> openFilesTypes) throws IOException {
+    checkOpen();
+    return new OpenFilesIterator(namenode, tracer, openFilesTypes);
   }
   }
 }
 }

+ 8 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java

@@ -90,6 +90,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsPathHandle;
 import org.apache.hadoop.hdfs.protocol.HdfsPathHandle;
 import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
+import org.apache.hadoop.hdfs.protocol.OpenFilesIterator.OpenFilesType;
 import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus;
 import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
@@ -3079,10 +3080,17 @@ public class DistributedFileSystem extends FileSystem
    * <p/>
    * <p/>
    * This method can only be called by HDFS superusers.
    * This method can only be called by HDFS superusers.
    */
    */
+  @Deprecated
   public RemoteIterator<OpenFileEntry> listOpenFiles() throws IOException {
   public RemoteIterator<OpenFileEntry> listOpenFiles() throws IOException {
     return dfs.listOpenFiles();
     return dfs.listOpenFiles();
   }
   }
 
 
+  public RemoteIterator<OpenFileEntry> listOpenFiles(
+      EnumSet<OpenFilesType> openFilesTypes) throws IOException {
+    return dfs.listOpenFiles(openFilesTypes);
+  }
+
+
   /**
   /**
    * Create a {@link HdfsDataOutputStreamBuilder} to append a file on DFS.
    * Create a {@link HdfsDataOutputStreamBuilder} to append a file on DFS.
    *
    *

+ 13 - 2
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/FileChecksumHelper.java

@@ -33,6 +33,7 @@ import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
 import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
 import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
 import org.apache.hadoop.hdfs.protocol.datatransfer.Op;
 import org.apache.hadoop.hdfs.protocol.datatransfer.Op;
 import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
 import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
+import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto;
 import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
 import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
@@ -303,8 +304,7 @@ final class FileChecksumHelper {
      * Return true when sounds good to continue or retry, false when severe
      * Return true when sounds good to continue or retry, false when severe
      * condition or totally failed.
      * condition or totally failed.
      */
      */
-    private boolean checksumBlock(
-        LocatedBlock locatedBlock) throws IOException {
+    private boolean checksumBlock(LocatedBlock locatedBlock) {
       ExtendedBlock block = locatedBlock.getBlock();
       ExtendedBlock block = locatedBlock.getBlock();
       if (getRemaining() < block.getNumBytes()) {
       if (getRemaining() < block.getNumBytes()) {
         block.setNumBytes(getRemaining());
         block.setNumBytes(getRemaining());
@@ -334,6 +334,17 @@ final class FileChecksumHelper {
             blockIdx--; // repeat at blockIdx-th block
             blockIdx--; // repeat at blockIdx-th block
             setRefetchBlocks(true);
             setRefetchBlocks(true);
           }
           }
+        } catch (InvalidEncryptionKeyException iee) {
+          if (blockIdx > getLastRetriedIndex()) {
+            LOG.debug("Got invalid encryption key error in response to "
+                    + "OP_BLOCK_CHECKSUM for file {} for block {} from "
+                    + "datanode {}. Will retry " + "the block once.",
+                  getSrc(), block, datanodes[j]);
+            setLastRetriedIndex(blockIdx);
+            done = true; // actually it's not done; but we'll retry
+            blockIdx--; // repeat at i-th block
+            getClient().clearDataEncryptionKey();
+          }
         } catch (IOException ie) {
         } catch (IOException ie) {
           LOG.warn("src={}" + ", datanodes[{}]={}",
           LOG.warn("src={}" + ", datanodes[{}]={}",
               getSrc(), j, datanodes[j], ie);
               getSrc(), j, datanodes[j], ie);

+ 7 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java

@@ -46,6 +46,7 @@ import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicyInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.ReencryptAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.ReencryptAction;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
+import org.apache.hadoop.hdfs.protocol.OpenFilesIterator.OpenFilesType;
 import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus;
 import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.AccessControlException;
 
 
@@ -652,8 +653,14 @@ public class HdfsAdmin {
    * <p/>
    * <p/>
    * This method can only be called by HDFS superusers.
    * This method can only be called by HDFS superusers.
    */
    */
+  @Deprecated
   public RemoteIterator<OpenFileEntry> listOpenFiles() throws IOException {
   public RemoteIterator<OpenFileEntry> listOpenFiles() throws IOException {
     return dfs.listOpenFiles();
     return dfs.listOpenFiles();
   }
   }
 
 
+  public RemoteIterator<OpenFileEntry> listOpenFiles(
+      EnumSet<OpenFilesType> openFilesTypes) throws IOException {
+    return dfs.listOpenFiles(openFilesTypes);
+  }
+
 }
 }

+ 16 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java

@@ -43,6 +43,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.inotify.EventBatchList;
 import org.apache.hadoop.hdfs.inotify.EventBatchList;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.ReencryptAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.ReencryptAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
+import org.apache.hadoop.hdfs.protocol.OpenFilesIterator.OpenFilesType;
 import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
 import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
@@ -1713,5 +1714,20 @@ public interface ClientProtocol {
    * @throws IOException
    * @throws IOException
    */
    */
   @Idempotent
   @Idempotent
+  @Deprecated
   BatchedEntries<OpenFileEntry> listOpenFiles(long prevId) throws IOException;
   BatchedEntries<OpenFileEntry> listOpenFiles(long prevId) throws IOException;
+
+  /**
+   * List open files in the system in batches. INode id is the cursor and the
+   * open files returned in a batch will have their INode ids greater than
+   * the cursor INode id. Open files can only be requested by super user and
+   * the the list across batches are not atomic.
+   *
+   * @param prevId the cursor INode id.
+   * @param openFilesTypes types to filter the open files
+   * @throws IOException
+   */
+  @Idempotent
+  BatchedEntries<OpenFileEntry> listOpenFiles(long prevId,
+      EnumSet<OpenFilesType> openFilesTypes) throws IOException;
 }
 }

+ 34 - 2
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/OpenFilesIterator.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.hdfs.protocol;
 package org.apache.hadoop.hdfs.protocol;
 
 
 import java.io.IOException;
 import java.io.IOException;
+import java.util.EnumSet;
 
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -35,20 +36,51 @@ import org.apache.htrace.core.Tracer;
 @InterfaceStability.Evolving
 @InterfaceStability.Evolving
 public class OpenFilesIterator extends
 public class OpenFilesIterator extends
     BatchedRemoteIterator<Long, OpenFileEntry> {
     BatchedRemoteIterator<Long, OpenFileEntry> {
+
+  /**
+   * Open file types to filter the results.
+   */
+  public enum OpenFilesType {
+
+    ALL_OPEN_FILES((short) 0x01),
+    BLOCKING_DECOMMISSION((short) 0x02);
+
+    private final short mode;
+    OpenFilesType(short mode) {
+      this.mode = mode;
+    }
+
+    public short getMode() {
+      return mode;
+    }
+
+    public static OpenFilesType valueOf(short num) {
+      for (OpenFilesType type : OpenFilesType.values()) {
+        if (type.getMode() == num) {
+          return type;
+        }
+      }
+      return null;
+    }
+  }
+
   private final ClientProtocol namenode;
   private final ClientProtocol namenode;
   private final Tracer tracer;
   private final Tracer tracer;
+  private final EnumSet<OpenFilesType> types;
 
 
-  public OpenFilesIterator(ClientProtocol namenode, Tracer tracer) {
+  public OpenFilesIterator(ClientProtocol namenode, Tracer tracer,
+      EnumSet<OpenFilesType> types) {
     super(HdfsConstants.GRANDFATHER_INODE_ID);
     super(HdfsConstants.GRANDFATHER_INODE_ID);
     this.namenode = namenode;
     this.namenode = namenode;
     this.tracer = tracer;
     this.tracer = tracer;
+    this.types = types;
   }
   }
 
 
   @Override
   @Override
   public BatchedEntries<OpenFileEntry> makeRequest(Long prevId)
   public BatchedEntries<OpenFileEntry> makeRequest(Long prevId)
       throws IOException {
       throws IOException {
     try (TraceScope ignored = tracer.newScope("listOpenFiles")) {
     try (TraceScope ignored = tracer.newScope("listOpenFiles")) {
-      return namenode.listOpenFiles(prevId);
+      return namenode.listOpenFiles(prevId, types);
     }
     }
   }
   }
 
 

+ 15 - 3
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java

@@ -75,6 +75,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.protocol.OpenFilesIterator.OpenFilesType;
 import org.apache.hadoop.hdfs.protocol.ReplicatedBlockStats;
 import org.apache.hadoop.hdfs.protocol.ReplicatedBlockStats;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
 import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus;
 import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus;
@@ -1893,13 +1894,24 @@ public class ClientNamenodeProtocolTranslatorPB implements
     }
     }
   }
   }
 
 
+  @Deprecated
   @Override
   @Override
   public BatchedEntries<OpenFileEntry> listOpenFiles(long prevId)
   public BatchedEntries<OpenFileEntry> listOpenFiles(long prevId)
       throws IOException {
       throws IOException {
-    ListOpenFilesRequestProto req =
-        ListOpenFilesRequestProto.newBuilder().setId(prevId).build();
+    return listOpenFiles(prevId, EnumSet.of(OpenFilesType.ALL_OPEN_FILES));
+  }
+
+  @Override
+  public BatchedEntries<OpenFileEntry> listOpenFiles(long prevId,
+      EnumSet<OpenFilesType> openFilesTypes) throws IOException {
+    ListOpenFilesRequestProto.Builder req =
+        ListOpenFilesRequestProto.newBuilder().setId(prevId);
+    if (openFilesTypes != null) {
+      req.addAllTypes(PBHelperClient.convertOpenFileTypes(openFilesTypes));
+    }
     try {
     try {
-      ListOpenFilesResponseProto response = rpcProxy.listOpenFiles(null, req);
+      ListOpenFilesResponseProto response =
+          rpcProxy.listOpenFiles(null, req.build());
       List<OpenFileEntry> openFileEntries =
       List<OpenFileEntry> openFileEntries =
           Lists.newArrayListWithCapacity(response.getEntriesCount());
           Lists.newArrayListWithCapacity(response.getEntriesCount());
       for (OpenFilesBatchResponseProto p : response.getEntriesList()) {
       for (OpenFilesBatchResponseProto p : response.getEntriesList()) {

+ 24 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java

@@ -94,6 +94,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
+import org.apache.hadoop.hdfs.protocol.OpenFilesIterator.OpenFilesType;
 import org.apache.hadoop.hdfs.protocol.ReplicatedBlockStats;
 import org.apache.hadoop.hdfs.protocol.ReplicatedBlockStats;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
 import org.apache.hadoop.hdfs.protocol.ProvidedStorageLocation;
 import org.apache.hadoop.hdfs.protocol.ProvidedStorageLocation;
@@ -131,6 +132,7 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsE
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsReplicatedBlockStatsResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsReplicatedBlockStatsResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsStatsResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsStatsResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.OpenFilesBatchResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.OpenFilesBatchResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.OpenFilesTypeProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollingUpgradeActionProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollingUpgradeActionProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollingUpgradeInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollingUpgradeInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SafeModeActionProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SafeModeActionProto;
@@ -3258,5 +3260,27 @@ public class PBHelperClient {
         .build();
         .build();
   }
   }
 
 
+  public static EnumSet<OpenFilesType> convertOpenFileTypes(
+      List<OpenFilesTypeProto> openFilesTypeProtos) {
+    EnumSet<OpenFilesType> types = EnumSet.noneOf(OpenFilesType.class);
+    for (OpenFilesTypeProto af : openFilesTypeProtos) {
+      OpenFilesType type = OpenFilesType.valueOf((short)af.getNumber());
+      if (type != null) {
+        types.add(type);
+      }
+    }
+    return types;
+  }
 
 
+  public static List<OpenFilesTypeProto> convertOpenFileTypes(
+      EnumSet<OpenFilesType> types) {
+    List<OpenFilesTypeProto> typeProtos = new ArrayList<>();
+    for (OpenFilesType type : types) {
+      OpenFilesTypeProto typeProto = OpenFilesTypeProto.valueOf(type.getMode());
+      if (typeProto != null) {
+        typeProtos.add(typeProto);
+      }
+    }
+    return typeProtos;
+  }
 }
 }

+ 47 - 16
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java

@@ -396,7 +396,9 @@ public class StripedBlockUtil {
       long rangeStartInBlockGroup, long rangeEndInBlockGroup) {
       long rangeStartInBlockGroup, long rangeEndInBlockGroup) {
     Preconditions.checkArgument(
     Preconditions.checkArgument(
         rangeStartInBlockGroup <= rangeEndInBlockGroup &&
         rangeStartInBlockGroup <= rangeEndInBlockGroup &&
-            rangeEndInBlockGroup < blockGroup.getBlockSize());
+            rangeEndInBlockGroup < blockGroup.getBlockSize(),
+        "start=%s end=%s blockSize=%s", rangeStartInBlockGroup,
+        rangeEndInBlockGroup, blockGroup.getBlockSize());
     long len = rangeEndInBlockGroup - rangeStartInBlockGroup + 1;
     long len = rangeEndInBlockGroup - rangeStartInBlockGroup + 1;
     int firstCellIdxInBG = (int) (rangeStartInBlockGroup / cellSize);
     int firstCellIdxInBG = (int) (rangeStartInBlockGroup / cellSize);
     int lastCellIdxInBG = (int) (rangeEndInBlockGroup / cellSize);
     int lastCellIdxInBG = (int) (rangeEndInBlockGroup / cellSize);
@@ -578,28 +580,39 @@ public class StripedBlockUtil {
   public static class StripingCell {
   public static class StripingCell {
     final ErasureCodingPolicy ecPolicy;
     final ErasureCodingPolicy ecPolicy;
     /** Logical order in a block group, used when doing I/O to a block group. */
     /** Logical order in a block group, used when doing I/O to a block group. */
-    final int idxInBlkGroup;
-    final int idxInInternalBlk;
-    final int idxInStripe;
+    private final long idxInBlkGroup;
+    private final long idxInInternalBlk;
+    private final int idxInStripe;
     /**
     /**
      * When a logical byte range is mapped to a set of cells, it might
      * When a logical byte range is mapped to a set of cells, it might
      * partially overlap with the first and last cells. This field and the
      * partially overlap with the first and last cells. This field and the
      * {@link #size} variable represent the start offset and size of the
      * {@link #size} variable represent the start offset and size of the
      * overlap.
      * overlap.
      */
      */
-    final int offset;
-    final int size;
+    private final long offset;
+    private final int size;
 
 
-    StripingCell(ErasureCodingPolicy ecPolicy, int cellSize, int idxInBlkGroup,
-        int offset) {
+    StripingCell(ErasureCodingPolicy ecPolicy, int cellSize, long idxInBlkGroup,
+        long offset) {
       this.ecPolicy = ecPolicy;
       this.ecPolicy = ecPolicy;
       this.idxInBlkGroup = idxInBlkGroup;
       this.idxInBlkGroup = idxInBlkGroup;
       this.idxInInternalBlk = idxInBlkGroup / ecPolicy.getNumDataUnits();
       this.idxInInternalBlk = idxInBlkGroup / ecPolicy.getNumDataUnits();
-      this.idxInStripe = idxInBlkGroup -
-          this.idxInInternalBlk * ecPolicy.getNumDataUnits();
+      this.idxInStripe = (int)(idxInBlkGroup -
+          this.idxInInternalBlk * ecPolicy.getNumDataUnits());
       this.offset = offset;
       this.offset = offset;
       this.size = cellSize;
       this.size = cellSize;
     }
     }
+
+    int getIdxInStripe() {
+      return idxInStripe;
+    }
+
+    @Override
+    public String toString() {
+      return String.format("StripingCell(idxInBlkGroup=%d, " +
+          "idxInInternalBlk=%d, idxInStrip=%d, offset=%d, size=%d)",
+          idxInBlkGroup, idxInInternalBlk, idxInStripe, offset, size);
+    }
   }
   }
 
 
   /**
   /**
@@ -646,7 +659,9 @@ public class StripedBlockUtil {
     public int missingChunksNum = 0;
     public int missingChunksNum = 0;
 
 
     public AlignedStripe(long offsetInBlock, long length, int width) {
     public AlignedStripe(long offsetInBlock, long length, int width) {
-      Preconditions.checkArgument(offsetInBlock >= 0 && length >= 0);
+      Preconditions.checkArgument(offsetInBlock >= 0 && length >= 0,
+          "OffsetInBlock(%s) and length(%s) must be non-negative",
+          offsetInBlock, length);
       this.range = new VerticalRange(offsetInBlock, length);
       this.range = new VerticalRange(offsetInBlock, length);
       this.chunks = new StripingChunk[width];
       this.chunks = new StripingChunk[width];
     }
     }
@@ -665,9 +680,9 @@ public class StripedBlockUtil {
 
 
     @Override
     @Override
     public String toString() {
     public String toString() {
-      return "Offset=" + range.offsetInBlock + ", length=" + range.spanInBlock +
-          ", fetchedChunksNum=" + fetchedChunksNum +
-          ", missingChunksNum=" + missingChunksNum;
+      return "AlignedStripe(Offset=" + range.offsetInBlock + ", length=" +
+          range.spanInBlock + ", fetchedChunksNum=" + fetchedChunksNum +
+          ", missingChunksNum=" + missingChunksNum + ")";
     }
     }
   }
   }
 
 
@@ -698,7 +713,9 @@ public class StripedBlockUtil {
     public long spanInBlock;
     public long spanInBlock;
 
 
     public VerticalRange(long offsetInBlock, long length) {
     public VerticalRange(long offsetInBlock, long length) {
-      Preconditions.checkArgument(offsetInBlock >= 0 && length >= 0);
+      Preconditions.checkArgument(offsetInBlock >= 0 && length >= 0,
+          "OffsetInBlock(%s) and length(%s) must be non-negative",
+          offsetInBlock, length);
       this.offsetInBlock = offsetInBlock;
       this.offsetInBlock = offsetInBlock;
       this.spanInBlock = length;
       this.spanInBlock = length;
     }
     }
@@ -707,6 +724,12 @@ public class StripedBlockUtil {
     public boolean include(long pos) {
     public boolean include(long pos) {
       return pos >= offsetInBlock && pos < offsetInBlock + spanInBlock;
       return pos >= offsetInBlock && pos < offsetInBlock + spanInBlock;
     }
     }
+
+    @Override
+    public String toString() {
+      return String.format("VerticalRange(offsetInBlock=%d, spanInBlock=%d)",
+          this.offsetInBlock, this.spanInBlock);
+    }
   }
   }
 
 
   /**
   /**
@@ -880,7 +903,9 @@ public class StripedBlockUtil {
     final long length;
     final long length;
 
 
     public StripeRange(long offsetInBlock, long length) {
     public StripeRange(long offsetInBlock, long length) {
-      Preconditions.checkArgument(offsetInBlock >= 0 && length >= 0);
+      Preconditions.checkArgument(offsetInBlock >= 0 && length >= 0,
+          "Offset(%s) and length(%s) must be non-negative", offsetInBlock,
+          length);
       this.offsetInBlock = offsetInBlock;
       this.offsetInBlock = offsetInBlock;
       this.length = length;
       this.length = length;
     }
     }
@@ -892,6 +917,12 @@ public class StripedBlockUtil {
     public long getLength() {
     public long getLength() {
       return length;
       return length;
     }
     }
+
+    @Override
+    public String toString() {
+      return String.format("StripeRange(offsetInBlock=%d, length=%d)",
+          offsetInBlock, length);
+    }
   }
   }
 
 
   /**
   /**

+ 7 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto

@@ -796,8 +796,14 @@ message GetEditsFromTxidResponseProto {
   required EventsListProto eventsList = 1;
   required EventsListProto eventsList = 1;
 }
 }
 
 
+enum OpenFilesTypeProto {
+  ALL_OPEN_FILES = 1;
+  BLOCKING_DECOMMISSION = 2;
+}
+
 message ListOpenFilesRequestProto {
 message ListOpenFilesRequestProto {
   required int64 id = 1;
   required int64 id = 1;
+  repeated OpenFilesTypeProto types = 2;
 }
 }
 
 
 message OpenFilesBatchResponseProto {
 message OpenFilesBatchResponseProto {
@@ -810,6 +816,7 @@ message OpenFilesBatchResponseProto {
 message ListOpenFilesResponseProto {
 message ListOpenFilesResponseProto {
   repeated OpenFilesBatchResponseProto entries = 1;
   repeated OpenFilesBatchResponseProto entries = 1;
   required bool hasMore = 2;
   required bool hasMore = 2;
+  repeated OpenFilesTypeProto types = 3;
 }
 }
 
 
 service ClientNamenodeProtocol {
 service ClientNamenodeProtocol {

+ 6 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java

@@ -53,6 +53,7 @@ import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
+import org.apache.hadoop.hdfs.protocol.OpenFilesIterator.OpenFilesType;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing;
@@ -1852,13 +1853,17 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
   public ListOpenFilesResponseProto listOpenFiles(RpcController controller,
   public ListOpenFilesResponseProto listOpenFiles(RpcController controller,
       ListOpenFilesRequestProto req) throws ServiceException {
       ListOpenFilesRequestProto req) throws ServiceException {
     try {
     try {
-      BatchedEntries<OpenFileEntry> entries = server.listOpenFiles(req.getId());
+      EnumSet<OpenFilesType> openFilesTypes =
+          PBHelperClient.convertOpenFileTypes(req.getTypesList());
+      BatchedEntries<OpenFileEntry> entries = server.listOpenFiles(req.getId(),
+          openFilesTypes);
       ListOpenFilesResponseProto.Builder builder =
       ListOpenFilesResponseProto.Builder builder =
           ListOpenFilesResponseProto.newBuilder();
           ListOpenFilesResponseProto.newBuilder();
       builder.setHasMore(entries.hasMore());
       builder.setHasMore(entries.hasMore());
       for (int i = 0; i < entries.size(); i++) {
       for (int i = 0; i < entries.size(); i++) {
         builder.addEntries(PBHelperClient.convert(entries.get(i)));
         builder.addEntries(PBHelperClient.convert(entries.get(i)));
       }
       }
+      builder.addAllTypes(req.getTypesList());
       return builder.build();
       return builder.build();
     } catch (IOException e) {
     } catch (IOException e) {
       throw new ServiceException(e);
       throw new ServiceException(e);

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java

@@ -2294,7 +2294,7 @@ public class BlockManager implements BlockStatsMXBean {
    * If there were any reconstruction requests that timed out, reap them
    * If there were any reconstruction requests that timed out, reap them
    * and put them back into the neededReconstruction queue
    * and put them back into the neededReconstruction queue
    */
    */
-  private void processPendingReconstructions() {
+  void processPendingReconstructions() {
     BlockInfo[] timedOutItems = pendingReconstruction.getTimedOutBlocks();
     BlockInfo[] timedOutItems = pendingReconstruction.getTimedOutBlocks();
     if (timedOutItems != null) {
     if (timedOutItems != null) {
       namesystem.writeLock();
       namesystem.writeLock();

+ 20 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeAdminManager.java

@@ -36,10 +36,14 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.server.namenode.INode;
+import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.INodeId;
 import org.apache.hadoop.hdfs.server.namenode.INodeId;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.Namesystem;
 import org.apache.hadoop.hdfs.server.namenode.Namesystem;
 import org.apache.hadoop.hdfs.util.CyclicIteration;
 import org.apache.hadoop.hdfs.util.CyclicIteration;
+import org.apache.hadoop.hdfs.util.LightWeightHashSet;
+import org.apache.hadoop.hdfs.util.LightWeightLinkedSet;
 import org.apache.hadoop.util.ChunkedArrayList;
 import org.apache.hadoop.util.ChunkedArrayList;
 import org.slf4j.Logger;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
@@ -649,8 +653,10 @@ public class DatanodeAdminManager {
         boolean pruneReliableBlocks) {
         boolean pruneReliableBlocks) {
       boolean firstReplicationLog = true;
       boolean firstReplicationLog = true;
       // Low redundancy in UC Blocks only
       // Low redundancy in UC Blocks only
-      int lowRedundancyInOpenFiles = 0;
-      // All low redundancy blocks. Includes lowRedundancyInOpenFiles.
+      int lowRedundancyBlocksInOpenFiles = 0;
+      LightWeightHashSet<Long> lowRedundancyOpenFiles =
+          new LightWeightLinkedSet<>();
+      // All low redundancy blocks. Includes lowRedundancyOpenFiles.
       int lowRedundancyBlocks = 0;
       int lowRedundancyBlocks = 0;
       // All maintenance and decommission replicas.
       // All maintenance and decommission replicas.
       int outOfServiceOnlyReplicas = 0;
       int outOfServiceOnlyReplicas = 0;
@@ -737,15 +743,24 @@ public class DatanodeAdminManager {
         // Update various counts
         // Update various counts
         lowRedundancyBlocks++;
         lowRedundancyBlocks++;
         if (bc.isUnderConstruction()) {
         if (bc.isUnderConstruction()) {
-          lowRedundancyInOpenFiles++;
+          INode ucFile = namesystem.getFSDirectory().getInode(bc.getId());
+          if(!(ucFile instanceof  INodeFile) ||
+              !ucFile.asFile().isUnderConstruction()) {
+            LOG.warn("File " + ucFile.getLocalName() + " is not under " +
+                "construction. Skipping add to low redundancy open files!");
+          } else {
+            lowRedundancyBlocksInOpenFiles++;
+            lowRedundancyOpenFiles.add(ucFile.getId());
+          }
         }
         }
         if ((liveReplicas == 0) && (num.outOfServiceReplicas() > 0)) {
         if ((liveReplicas == 0) && (num.outOfServiceReplicas() > 0)) {
           outOfServiceOnlyReplicas++;
           outOfServiceOnlyReplicas++;
         }
         }
       }
       }
 
 
-      datanode.getLeavingServiceStatus().set(lowRedundancyInOpenFiles,
-          lowRedundancyBlocks, outOfServiceOnlyReplicas);
+      datanode.getLeavingServiceStatus().set(lowRedundancyBlocksInOpenFiles,
+          lowRedundancyOpenFiles, lowRedundancyBlocks,
+          outOfServiceOnlyReplicas);
     }
     }
   }
   }
 
 

+ 18 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java

@@ -50,6 +50,7 @@ import org.apache.hadoop.hdfs.server.protocol.VolumeFailureSummary;
 import org.apache.hadoop.hdfs.util.EnumCounters;
 import org.apache.hadoop.hdfs.util.EnumCounters;
 import org.apache.hadoop.hdfs.util.LightWeightHashSet;
 import org.apache.hadoop.hdfs.util.LightWeightHashSet;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.apache.hadoop.hdfs.util.LightWeightLinkedSet;
 import org.apache.hadoop.util.IntrusiveCollection;
 import org.apache.hadoop.util.IntrusiveCollection;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.Time;
 import org.slf4j.Logger;
 import org.slf4j.Logger;
@@ -831,17 +832,21 @@ public class DatanodeDescriptor extends DatanodeInfo {
   /** Leaving service status. */
   /** Leaving service status. */
   public class LeavingServiceStatus {
   public class LeavingServiceStatus {
     private int underReplicatedBlocks;
     private int underReplicatedBlocks;
+    private int underReplicatedBlocksInOpenFiles;
     private int outOfServiceOnlyReplicas;
     private int outOfServiceOnlyReplicas;
-    private int underReplicatedInOpenFiles;
+    private LightWeightHashSet<Long> underReplicatedOpenFiles =
+        new LightWeightLinkedSet<>();
     private long startTime;
     private long startTime;
     
     
-    synchronized void set(int underRepInOpenFiles, int underRepBlocks,
-        int outOfServiceOnlyRep) {
+    synchronized void set(int lowRedundancyBlocksInOpenFiles,
+        LightWeightHashSet<Long> underRepInOpenFiles,
+        int underRepBlocks, int outOfServiceOnlyRep) {
       if (!isDecommissionInProgress() && !isEnteringMaintenance()) {
       if (!isDecommissionInProgress() && !isEnteringMaintenance()) {
         return;
         return;
       }
       }
-      underReplicatedInOpenFiles = underRepInOpenFiles;
+      underReplicatedOpenFiles = underRepInOpenFiles;
       underReplicatedBlocks = underRepBlocks;
       underReplicatedBlocks = underRepBlocks;
+      underReplicatedBlocksInOpenFiles = lowRedundancyBlocksInOpenFiles;
       outOfServiceOnlyReplicas = outOfServiceOnlyRep;
       outOfServiceOnlyReplicas = outOfServiceOnlyRep;
     }
     }
 
 
@@ -864,7 +869,14 @@ public class DatanodeDescriptor extends DatanodeInfo {
       if (!isDecommissionInProgress() && !isEnteringMaintenance()) {
       if (!isDecommissionInProgress() && !isEnteringMaintenance()) {
         return 0;
         return 0;
       }
       }
-      return underReplicatedInOpenFiles;
+      return underReplicatedBlocksInOpenFiles;
+    }
+    /** @return the collection of under-replicated blocks in open files */
+    public synchronized LightWeightHashSet<Long> getOpenFiles() {
+      if (!isDecommissionInProgress() && !isEnteringMaintenance()) {
+        return new LightWeightLinkedSet<>();
+      }
+      return underReplicatedOpenFiles;
     }
     }
     /** Set start time */
     /** Set start time */
     public synchronized void setStartTime(long time) {
     public synchronized void setStartTime(long time) {
@@ -880,7 +892,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
       }
       }
       return startTime;
       return startTime;
     }
     }
-  }  // End of class DecommissioningStatus
+  }  // End of class LeavingServiceStatus
 
 
   /**
   /**
    * Set the flag to indicate if this datanode is disallowed from communicating
    * Set the flag to indicate if this datanode is disallowed from communicating

+ 6 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/Command.java

@@ -501,7 +501,8 @@ public abstract class Command extends Configured implements Closeable {
    * Parse top number of nodes to be processed.
    * Parse top number of nodes to be processed.
    * @return top number of nodes to be processed.
    * @return top number of nodes to be processed.
    */
    */
-  protected int parseTopNodes(final CommandLine cmd, final StrBuilder result) {
+  protected int parseTopNodes(final CommandLine cmd, final StrBuilder result)
+      throws IllegalArgumentException {
     String outputLine = "";
     String outputLine = "";
     int nodes = 0;
     int nodes = 0;
     final String topVal = cmd.getOptionValue(DiskBalancerCLI.TOP);
     final String topVal = cmd.getOptionValue(DiskBalancerCLI.TOP);
@@ -523,6 +524,10 @@ public abstract class Command extends Configured implements Closeable {
         result.appendln(outputLine);
         result.appendln(outputLine);
         nodes = getDefaultTop();
         nodes = getDefaultTop();
       }
       }
+      if (nodes <= 0) {
+        throw new IllegalArgumentException(
+            "Top limit input should be a positive numeric value");
+      }
     }
     }
 
 
     return Math.min(nodes, cluster.getNodes().size());
     return Math.min(nodes, cluster.getNodes().size());

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ReportCommand.java

@@ -100,7 +100,7 @@ public class ReportCommand extends Command {
   }
   }
 
 
   private void handleTopReport(final CommandLine cmd, final StrBuilder result,
   private void handleTopReport(final CommandLine cmd, final StrBuilder result,
-      final String nodeFormat) {
+      final String nodeFormat) throws IllegalArgumentException {
     Collections.sort(getCluster().getNodes(), Collections.reverseOrder());
     Collections.sort(getCluster().getNodes(), Collections.reverseOrder());
 
 
     /* extract value that identifies top X DataNode(s) */
     /* extract value that identifies top X DataNode(s) */

+ 9 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java

@@ -90,6 +90,7 @@ import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
+import org.apache.hadoop.hdfs.protocol.OpenFilesIterator.OpenFilesType;
 import org.apache.hadoop.hdfs.protocol.ReplicatedBlockStats;
 import org.apache.hadoop.hdfs.protocol.ReplicatedBlockStats;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
@@ -1935,9 +1936,16 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
     return null;
     return null;
   }
   }
 
 
+  @Deprecated
   @Override
   @Override
-  public BatchedEntries<OpenFileEntry> listOpenFiles(long arg0)
+  public BatchedEntries<OpenFileEntry> listOpenFiles(long prevId)
       throws IOException {
       throws IOException {
+    return listOpenFiles(prevId, EnumSet.of(OpenFilesType.ALL_OPEN_FILES));
+  }
+
+  @Override
+  public BatchedEntries<OpenFileEntry> listOpenFiles(long prevId,
+      EnumSet<OpenFilesType> openFilesTypes) throws IOException {
     checkOperation(OperationCategory.READ, false);
     checkOperation(OperationCategory.READ, false);
     return null;
     return null;
   }
   }

+ 46 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -93,6 +93,7 @@ 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.ErasureCodingPolicyInfo;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicyInfo;
+import org.apache.hadoop.hdfs.protocol.OpenFilesIterator.OpenFilesType;
 import org.apache.hadoop.hdfs.protocol.ReplicatedBlockStats;
 import org.apache.hadoop.hdfs.protocol.ReplicatedBlockStats;
 import org.apache.hadoop.hdfs.protocol.ECBlockGroupStats;
 import org.apache.hadoop.hdfs.protocol.ECBlockGroupStats;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
@@ -276,6 +277,7 @@ import org.apache.hadoop.hdfs.server.protocol.SlowPeerReports;
 import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
 import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.hdfs.server.protocol.VolumeFailureSummary;
 import org.apache.hadoop.hdfs.server.protocol.VolumeFailureSummary;
+import org.apache.hadoop.hdfs.util.LightWeightHashSet;
 import org.apache.hadoop.hdfs.web.JsonUtil;
 import org.apache.hadoop.hdfs.web.JsonUtil;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
@@ -1762,12 +1764,14 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * the open files returned in a batch will have their INode ids greater than
    * the open files returned in a batch will have their INode ids greater than
    * this cursor. Open files can only be requested by super user and the the
    * this cursor. Open files can only be requested by super user and the the
    * list across batches does not represent a consistent view of all open files.
    * list across batches does not represent a consistent view of all open files.
+   * TODO: HDFS-12969 - to report open files by type.
    *
    *
    * @param prevId the cursor INode id.
    * @param prevId the cursor INode id.
+   * @param openFilesTypes
    * @throws IOException
    * @throws IOException
    */
    */
-  BatchedListEntries<OpenFileEntry> listOpenFiles(long prevId)
-      throws IOException {
+  BatchedListEntries<OpenFileEntry> listOpenFiles(long prevId,
+      EnumSet<OpenFilesType> openFilesTypes) throws IOException {
     final String operationName = "listOpenFiles";
     final String operationName = "listOpenFiles";
     checkSuperuserPrivilege();
     checkSuperuserPrivilege();
     checkOperation(OperationCategory.READ);
     checkOperation(OperationCategory.READ);
@@ -1775,7 +1779,16 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     BatchedListEntries<OpenFileEntry> batchedListEntries;
     BatchedListEntries<OpenFileEntry> batchedListEntries;
     try {
     try {
       checkOperation(OperationCategory.READ);
       checkOperation(OperationCategory.READ);
-      batchedListEntries = leaseManager.getUnderConstructionFiles(prevId);
+      if(openFilesTypes.contains(OpenFilesType.ALL_OPEN_FILES)) {
+        batchedListEntries = leaseManager.getUnderConstructionFiles(prevId);
+      } else {
+        if(openFilesTypes.contains(OpenFilesType.BLOCKING_DECOMMISSION)) {
+          batchedListEntries = getFilesBlockingDecom(prevId);
+        } else {
+          throw new IllegalArgumentException("Unknown OpenFileType: "
+              + openFilesTypes);
+        }
+      }
     } catch (AccessControlException e) {
     } catch (AccessControlException e) {
       logAuditEvent(false, operationName, null);
       logAuditEvent(false, operationName, null);
       throw e;
       throw e;
@@ -1786,6 +1799,36 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     return batchedListEntries;
     return batchedListEntries;
   }
   }
 
 
+  public BatchedListEntries<OpenFileEntry> getFilesBlockingDecom(long prevId) {
+    assert hasReadLock();
+    final List<OpenFileEntry> openFileEntries = Lists.newArrayList();
+    LightWeightHashSet<Long> openFileIds = new LightWeightHashSet<>();
+    for (DatanodeDescriptor dataNode :
+        blockManager.getDatanodeManager().getDatanodes()) {
+      for (long ucFileId : dataNode.getLeavingServiceStatus().getOpenFiles()) {
+        INode ucFile = getFSDirectory().getInode(ucFileId);
+        if (ucFile == null || ucFileId <= prevId ||
+            openFileIds.contains(ucFileId)) {
+          // probably got deleted or
+          // part of previous batch or
+          // already part of the current batch
+          continue;
+        }
+        Preconditions.checkState(ucFile instanceof INodeFile);
+        openFileIds.add(ucFileId);
+        INodeFile inodeFile = ucFile.asFile();
+        openFileEntries.add(new OpenFileEntry(
+            inodeFile.getId(), inodeFile.getFullPathName(),
+            inodeFile.getFileUnderConstructionFeature().getClientName(),
+            inodeFile.getFileUnderConstructionFeature().getClientMachine()));
+        if (openFileIds.size() >= this.maxListOpenFilesResponses) {
+          return new BatchedListEntries<>(openFileEntries, true);
+        }
+      }
+    }
+    return new BatchedListEntries<>(openFileEntries, false);
+  }
+
   private String metaSaveAsString() {
   private String metaSaveAsString() {
     StringWriter sw = new StringWriter();
     StringWriter sw = new StringWriter();
     PrintWriter pw = new PrintWriter(sw);
     PrintWriter pw = new PrintWriter(sw);

+ 9 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java

@@ -115,6 +115,7 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
+import org.apache.hadoop.hdfs.protocol.OpenFilesIterator.OpenFilesType;
 import org.apache.hadoop.hdfs.protocol.QuotaByStorageTypeExceededException;
 import org.apache.hadoop.hdfs.protocol.QuotaByStorageTypeExceededException;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
 import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
@@ -1334,11 +1335,18 @@ public class NameNodeRpcServer implements NamenodeProtocols {
     namesystem.metaSave(filename);
     namesystem.metaSave(filename);
   }
   }
 
 
+  @Deprecated
   @Override // ClientProtocol
   @Override // ClientProtocol
   public BatchedEntries<OpenFileEntry> listOpenFiles(long prevId)
   public BatchedEntries<OpenFileEntry> listOpenFiles(long prevId)
       throws IOException {
       throws IOException {
+    return listOpenFiles(prevId, EnumSet.of(OpenFilesType.ALL_OPEN_FILES));
+  }
+
+  @Override // ClientProtocol
+  public BatchedEntries<OpenFileEntry> listOpenFiles(long prevId,
+      EnumSet<OpenFilesType> openFilesTypes) throws IOException {
     checkNNStartup();
     checkNNStartup();
-    return namesystem.listOpenFiles(prevId);
+    return namesystem.listOpenFiles(prevId, openFilesTypes);
   }
   }
 
 
   @Override // ClientProtocol
   @Override // ClientProtocol

+ 27 - 9
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java

@@ -29,6 +29,7 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Collections;
 import java.util.Date;
 import java.util.Date;
+import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.List;
@@ -66,6 +67,7 @@ import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.NameNodeProxies;
 import org.apache.hadoop.hdfs.NameNodeProxies;
 import org.apache.hadoop.hdfs.NameNodeProxiesClient.ProxyAndInfo;
 import org.apache.hadoop.hdfs.NameNodeProxiesClient.ProxyAndInfo;
+import org.apache.hadoop.hdfs.protocol.OpenFilesIterator.OpenFilesType;
 import org.apache.hadoop.hdfs.protocol.ReplicatedBlockStats;
 import org.apache.hadoop.hdfs.protocol.ReplicatedBlockStats;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
@@ -462,7 +464,7 @@ public class DFSAdmin extends FsShell {
     "\t[-getDatanodeInfo <datanode_host:ipc_port>]\n" +
     "\t[-getDatanodeInfo <datanode_host:ipc_port>]\n" +
     "\t[-metasave filename]\n" +
     "\t[-metasave filename]\n" +
     "\t[-triggerBlockReport [-incremental] <datanode_host:ipc_port>]\n" +
     "\t[-triggerBlockReport [-incremental] <datanode_host:ipc_port>]\n" +
-    "\t[-listOpenFiles]\n" +
+    "\t[-listOpenFiles [-blockingDecommission]]\n" +
     "\t[-help [cmd]]\n";
     "\t[-help [cmd]]\n";
 
 
   /**
   /**
@@ -913,8 +915,21 @@ public class DFSAdmin extends FsShell {
    * Usage: hdfs dfsadmin -listOpenFiles
    * Usage: hdfs dfsadmin -listOpenFiles
    *
    *
    * @throws IOException
    * @throws IOException
+   * @param argv
    */
    */
-  public int listOpenFiles() throws IOException {
+  public int listOpenFiles(String[] argv) throws IOException {
+    List<OpenFilesType> types = new ArrayList<>();
+    if (argv != null) {
+      List<String> args = new ArrayList<>(Arrays.asList(argv));
+      if (StringUtils.popOption("-blockingDecommission", args)) {
+        types.add(OpenFilesType.BLOCKING_DECOMMISSION);
+      }
+    }
+    if (types.isEmpty()) {
+      types.add(OpenFilesType.ALL_OPEN_FILES);
+    }
+    EnumSet<OpenFilesType> openFilesTypes = EnumSet.copyOf(types);
+
     DistributedFileSystem dfs = getDFS();
     DistributedFileSystem dfs = getDFS();
     Configuration dfsConf = dfs.getConf();
     Configuration dfsConf = dfs.getConf();
     URI dfsUri = dfs.getUri();
     URI dfsUri = dfs.getUri();
@@ -926,9 +941,9 @@ public class DFSAdmin extends FsShell {
           dfsConf, HAUtil.getAddressOfActive(getDFS()), ClientProtocol.class,
           dfsConf, HAUtil.getAddressOfActive(getDFS()), ClientProtocol.class,
           UserGroupInformation.getCurrentUser(), false);
           UserGroupInformation.getCurrentUser(), false);
       openFilesRemoteIterator = new OpenFilesIterator(proxy.getProxy(),
       openFilesRemoteIterator = new OpenFilesIterator(proxy.getProxy(),
-          FsTracer.get(dfsConf));
+          FsTracer.get(dfsConf), openFilesTypes);
     } else {
     } else {
-      openFilesRemoteIterator = dfs.listOpenFiles();
+      openFilesRemoteIterator = dfs.listOpenFiles(openFilesTypes);
     }
     }
     printOpenFiles(openFilesRemoteIterator);
     printOpenFiles(openFilesRemoteIterator);
     return 0;
     return 0;
@@ -1214,9 +1229,11 @@ public class DFSAdmin extends FsShell {
         + "\tIf 'incremental' is specified, it will be an incremental\n"
         + "\tIf 'incremental' is specified, it will be an incremental\n"
         + "\tblock report; otherwise, it will be a full block report.\n";
         + "\tblock report; otherwise, it will be a full block report.\n";
 
 
-    String listOpenFiles = "-listOpenFiles\n"
+    String listOpenFiles = "-listOpenFiles [-blockingDecommission]\n"
         + "\tList all open files currently managed by the NameNode along\n"
         + "\tList all open files currently managed by the NameNode along\n"
-        + "\twith client name and client machine accessing them.\n";
+        + "\twith client name and client machine accessing them.\n"
+        + "\tIf 'blockingDecommission' option is specified, it will list the\n"
+        + "\topen files only that are blocking the ongoing Decommission.";
 
 
     String help = "-help [cmd]: \tDisplays help for the given command or all commands if none\n" +
     String help = "-help [cmd]: \tDisplays help for the given command or all commands if none\n" +
       "\t\tis specified.\n";
       "\t\tis specified.\n";
@@ -1964,7 +1981,8 @@ public class DFSAdmin extends FsShell {
       System.err.println("Usage: hdfs dfsadmin"
       System.err.println("Usage: hdfs dfsadmin"
           + " [-triggerBlockReport [-incremental] <datanode_host:ipc_port>]");
           + " [-triggerBlockReport [-incremental] <datanode_host:ipc_port>]");
     } else if ("-listOpenFiles".equals(cmd)) {
     } else if ("-listOpenFiles".equals(cmd)) {
-      System.err.println("Usage: hdfs dfsadmin [-listOpenFiles]");
+      System.err.println("Usage: hdfs dfsadmin"
+          + " [-listOpenFiles [-blockingDecommission]]");
     } else {
     } else {
       System.err.println("Usage: hdfs dfsadmin");
       System.err.println("Usage: hdfs dfsadmin");
       System.err.println("Note: Administrative commands can only be run as the HDFS superuser.");
       System.err.println("Note: Administrative commands can only be run as the HDFS superuser.");
@@ -2119,7 +2137,7 @@ public class DFSAdmin extends FsShell {
         return exitCode;
         return exitCode;
       }
       }
     } else if ("-listOpenFiles".equals(cmd)) {
     } else if ("-listOpenFiles".equals(cmd)) {
-      if (argv.length != 1) {
+      if ((argv.length != 1) && (argv.length != 2)) {
         printUsage(cmd);
         printUsage(cmd);
         return exitCode;
         return exitCode;
       }
       }
@@ -2205,7 +2223,7 @@ public class DFSAdmin extends FsShell {
       } else if ("-triggerBlockReport".equals(cmd)) {
       } else if ("-triggerBlockReport".equals(cmd)) {
         exitCode = triggerBlockReport(argv);
         exitCode = triggerBlockReport(argv);
       } else if ("-listOpenFiles".equals(cmd)) {
       } else if ("-listOpenFiles".equals(cmd)) {
-        exitCode = listOpenFiles();
+        exitCode = listOpenFiles(argv);
       } else if ("-help".equals(cmd)) {
       } else if ("-help".equals(cmd)) {
         if (i < argv.length) {
         if (i < argv.length) {
           printHelp(argv[i]);
           printHelp(argv[i]);

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

@@ -475,8 +475,8 @@
   <description>
   <description>
     The NameNode periodically scans the namespace for LazyPersist files with
     The NameNode periodically scans the namespace for LazyPersist files with
     missing blocks and unlinks them from the namespace. This configuration key
     missing blocks and unlinks them from the namespace. This configuration key
-    controls the interval between successive scans. Set it to a negative value
-    to disable this behavior.
+    controls the interval between successive scans. If this value is set to 0,
+    the file scrubber is disabled.
   </description>
   </description>
 </property>
 </property>
 <property>
 <property>

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html

@@ -147,7 +147,7 @@
 
 
 <p>
 <p>
   {#fs}
   {#fs}
-  {FilesTotal|fmt_number} files and directories, {BlocksTotal|fmt_number} blocks = {ObjectsTotal|fmt_number} total filesystem object(s).
+  {FilesTotal|fmt_number} files and directories, {BlocksTotal|fmt_number} blocks ({#replicastat}{TotalReplicatedBlocks|fmt_number}{/replicastat} replicated blocks, {#ecstat}{TotalECBlockGroups|fmt_number}{/ecstat} erasure coded block groups) = {ObjectsTotal|fmt_number} total filesystem object(s).
   {#helper_fs_max_objects/}
   {#helper_fs_max_objects/}
   {/fs}
   {/fs}
 </p>
 </p>

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.js

@@ -37,6 +37,8 @@
       {"name": "nnstat",  "url": "/jmx?qry=Hadoop:service=NameNode,name=NameNodeStatus"},
       {"name": "nnstat",  "url": "/jmx?qry=Hadoop:service=NameNode,name=NameNodeStatus"},
       {"name": "fs",      "url": "/jmx?qry=Hadoop:service=NameNode,name=FSNamesystemState"},
       {"name": "fs",      "url": "/jmx?qry=Hadoop:service=NameNode,name=FSNamesystemState"},
       {"name": "fsn",     "url": "/jmx?qry=Hadoop:service=NameNode,name=FSNamesystem"},
       {"name": "fsn",     "url": "/jmx?qry=Hadoop:service=NameNode,name=FSNamesystem"},
+      {"name": "replicastat",      "url": "/jmx?qry=Hadoop:service=NameNode,name=ReplicatedBlocksState"},
+      {"name": "ecstat",      "url": "/jmx?qry=Hadoop:service=NameNode,name=ECBlockGroupsState"},
       {"name": "blockstats",      "url": "/jmx?qry=Hadoop:service=NameNode,name=BlockStats"},
       {"name": "blockstats",      "url": "/jmx?qry=Hadoop:service=NameNode,name=BlockStats"},
       {"name": "mem",     "url": "/jmx?qry=java.lang:type=Memory"}
       {"name": "mem",     "url": "/jmx?qry=java.lang:type=Memory"}
     ];
     ];

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

@@ -409,7 +409,7 @@ Usage:
 | `-getDatanodeInfo` \<datanode\_host:ipc\_port\> | Get the information about the given datanode. See [Rolling Upgrade document](./HdfsRollingUpgrade.html#dfsadmin_-getDatanodeInfo) for the detail. |
 | `-getDatanodeInfo` \<datanode\_host:ipc\_port\> | Get the information about the given datanode. See [Rolling Upgrade document](./HdfsRollingUpgrade.html#dfsadmin_-getDatanodeInfo) for the detail. |
 | `-metasave` filename | Save Namenode's primary data structures to *filename* in the directory specified by hadoop.log.dir property. *filename* is overwritten if it exists. *filename* will contain one line for each of the following<br/>1. Datanodes heart beating with Namenode<br/>2. Blocks waiting to be replicated<br/>3. Blocks currently being replicated<br/>4. Blocks waiting to be deleted |
 | `-metasave` filename | Save Namenode's primary data structures to *filename* in the directory specified by hadoop.log.dir property. *filename* is overwritten if it exists. *filename* will contain one line for each of the following<br/>1. Datanodes heart beating with Namenode<br/>2. Blocks waiting to be replicated<br/>3. Blocks currently being replicated<br/>4. Blocks waiting to be deleted |
 | `-triggerBlockReport` `[-incremental]` \<datanode\_host:ipc\_port\> | Trigger a block report for the given datanode. If 'incremental' is specified, it will be otherwise, it will be a full block report. |
 | `-triggerBlockReport` `[-incremental]` \<datanode\_host:ipc\_port\> | Trigger a block report for the given datanode. If 'incremental' is specified, it will be otherwise, it will be a full block report. |
-| `-listOpenFiles` | List all open files currently managed by the NameNode along with client name and client machine accessing them. |
+| `-listOpenFiles` `[-blockingDecommission]` | List all open files currently managed by the NameNode along with client name and client machine accessing them. |
 | `-help` [cmd] | Displays help for the given command or all commands if none is specified. |
 | `-help` [cmd] | Displays help for the given command or all commands if none is specified. |
 
 
 Runs a HDFS dfsadmin client.
 Runs a HDFS dfsadmin client.

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md

@@ -99,6 +99,10 @@ Deployment
 
 
   Encoding and decoding work consumes additional CPU on both HDFS clients and DataNodes.
   Encoding and decoding work consumes additional CPU on both HDFS clients and DataNodes.
 
 
+  Erasure coding requires a minimum of as many DataNodes in the cluster as
+  the configured EC stripe width. For EC policy RS (6,3), this means
+  a minimum of 9 DataNodes.
+
   Erasure coded files are also spread across racks for rack fault-tolerance.
   Erasure coded files are also spread across racks for rack fault-tolerance.
   This means that when reading and writing striped files, most operations are off-rack.
   This means that when reading and writing striped files, most operations are off-rack.
   Network bisection bandwidth is thus very important.
   Network bisection bandwidth is thus very important.

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

@@ -388,9 +388,19 @@ public class AdminStatesBaseTest {
   protected void startCluster(int numNameNodes, int numDatanodes,
   protected void startCluster(int numNameNodes, int numDatanodes,
       boolean setupHostsFile, long[] nodesCapacity,
       boolean setupHostsFile, long[] nodesCapacity,
       boolean checkDataNodeHostConfig) throws IOException {
       boolean checkDataNodeHostConfig) throws IOException {
+    startCluster(numNameNodes, numDatanodes, setupHostsFile, nodesCapacity,
+        checkDataNodeHostConfig, true);
+  }
+
+  protected void startCluster(int numNameNodes, int numDatanodes,
+      boolean setupHostsFile, long[] nodesCapacity,
+      boolean checkDataNodeHostConfig, boolean federation) throws IOException {
     MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf)
     MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf)
-        .nnTopology(MiniDFSNNTopology.simpleFederatedTopology(numNameNodes))
         .numDataNodes(numDatanodes);
         .numDataNodes(numDatanodes);
+    if (federation) {
+      builder.nnTopology(
+          MiniDFSNNTopology.simpleFederatedTopology(numNameNodes));
+    }
     if (setupHostsFile) {
     if (setupHostsFile) {
       builder.setupHostsFile(setupHostsFile);
       builder.setupHostsFile(setupHostsFile);
     }
     }
@@ -413,6 +423,12 @@ public class AdminStatesBaseTest {
     startCluster(numNameNodes, numDatanodes, false, null, false);
     startCluster(numNameNodes, numDatanodes, false, null, false);
   }
   }
 
 
+  protected void startSimpleCluster(int numNameNodes, int numDatanodes)
+      throws IOException {
+    startCluster(numNameNodes, numDatanodes, false, null, false, false);
+  }
+
+
   protected void startSimpleHACluster(int numDatanodes) throws IOException {
   protected void startSimpleHACluster(int numDatanodes) throws IOException {
     cluster = new MiniDFSCluster.Builder(conf)
     cluster = new MiniDFSCluster.Builder(conf)
         .nnTopology(MiniDFSNNTopology.simpleHATopology()).numDataNodes(
         .nnTopology(MiniDFSNNTopology.simpleHATopology()).numDataNodes(

+ 177 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java

@@ -22,16 +22,23 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertTrue;
 
 
+import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.IOException;
+import java.io.PrintStream;
 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.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.List;
 import java.util.Map;
 import java.util.Map;
+import java.util.Scanner;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 
 import com.google.common.base.Supplier;
 import com.google.common.base.Supplier;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Lists;
+import org.apache.commons.lang.text.StrBuilder;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -60,7 +67,9 @@ import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
 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.NameNodeAdapter;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStatistics;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStatistics;
+import org.apache.hadoop.hdfs.tools.DFSAdmin;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.util.ToolRunner;
 import org.apache.log4j.Level;
 import org.apache.log4j.Level;
 import org.junit.Assert;
 import org.junit.Assert;
 import org.junit.Ignore;
 import org.junit.Ignore;
@@ -651,6 +660,174 @@ public class TestDecommission extends AdminStatesBaseTest {
     fdos.close();
     fdos.close();
   }
   }
 
 
+  private static String scanIntoString(final ByteArrayOutputStream baos) {
+    final StrBuilder sb = new StrBuilder();
+    final Scanner scanner = new Scanner(baos.toString());
+    while (scanner.hasNextLine()) {
+      sb.appendln(scanner.nextLine());
+    }
+    scanner.close();
+    return sb.toString();
+  }
+
+  private boolean verifyOpenFilesListing(String message,
+      HashSet<Path> closedFileSet,
+      HashMap<Path, FSDataOutputStream> openFilesMap,
+      ByteArrayOutputStream out, int expOpenFilesListSize) {
+    final String outStr = scanIntoString(out);
+    LOG.info(message + " - stdout: \n" + outStr);
+    for (Path closedFilePath : closedFileSet) {
+      if(outStr.contains(closedFilePath.toString())) {
+        return false;
+      }
+    }
+    HashSet<Path> openFilesNotListed = new HashSet<>();
+    for (Path openFilePath : openFilesMap.keySet()) {
+      if(!outStr.contains(openFilePath.toString())) {
+        openFilesNotListed.add(openFilePath);
+      }
+    }
+    int actualOpenFilesListedSize =
+        openFilesMap.size() - openFilesNotListed.size();
+    if (actualOpenFilesListedSize >= expOpenFilesListSize) {
+      return true;
+    } else {
+      LOG.info("Open files that are not listed yet: " + openFilesNotListed);
+      return false;
+    }
+  }
+
+  private void verifyOpenFilesBlockingDecommission(HashSet<Path> closedFileSet,
+      HashMap<Path, FSDataOutputStream> openFilesMap, final int maxOpenFiles)
+      throws Exception {
+    final PrintStream oldStreamOut = System.out;
+    try {
+      final ByteArrayOutputStream toolOut = new ByteArrayOutputStream();
+      System.setOut(new PrintStream(toolOut));
+      final DFSAdmin dfsAdmin = new DFSAdmin(getConf());
+
+      GenericTestUtils.waitFor(new Supplier<Boolean>() {
+        @Override
+        public Boolean get() {
+          try {
+            toolOut.reset();
+            assertEquals(0, ToolRunner.run(dfsAdmin,
+                new String[]{"-listOpenFiles", "-blockingDecommission"}));
+            toolOut.flush();
+            return verifyOpenFilesListing(
+                "dfsadmin -listOpenFiles -blockingDecommission",
+                closedFileSet, openFilesMap, toolOut, maxOpenFiles);
+          } catch (Exception e) {
+            LOG.warn("Unexpected exception: " + e);
+          }
+          return false;
+        }
+      }, 1000, 60000);
+    } finally {
+      System.setOut(oldStreamOut);
+    }
+  }
+
+  @Test(timeout=180000)
+  public void testDecommissionWithOpenfileReporting()
+      throws Exception {
+    LOG.info("Starting test testDecommissionWithOpenfileReporting");
+
+    // Disable redundancy monitor check so that open files blocking
+    // decommission can be listed and verified.
+    getConf().setInt(
+        DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY,
+        1000);
+    getConf().setLong(
+        DFSConfigKeys.DFS_NAMENODE_LIST_OPENFILES_NUM_RESPONSES, 1);
+
+    //At most 1 node can be decommissioned
+    startSimpleCluster(1, 4);
+
+    FileSystem fileSys = getCluster().getFileSystem(0);
+    FSNamesystem ns = getCluster().getNamesystem(0);
+
+    final String[] closedFiles = new String[3];
+    final String[] openFiles = new String[3];
+    HashSet<Path> closedFileSet = new HashSet<>();
+    HashMap<Path, FSDataOutputStream> openFilesMap = new HashMap<>();
+    for (int i = 0; i < 3; i++) {
+      closedFiles[i] = "/testDecommissionWithOpenfileReporting.closed." + i;
+      openFiles[i] = "/testDecommissionWithOpenfileReporting.open." + i;
+      writeFile(fileSys, new Path(closedFiles[i]), (short)3, 10);
+      closedFileSet.add(new Path(closedFiles[i]));
+      writeFile(fileSys, new Path(openFiles[i]), (short)3, 10);
+      FSDataOutputStream fdos =  fileSys.append(new Path(openFiles[i]));
+      openFilesMap.put(new Path(openFiles[i]), fdos);
+    }
+
+    HashMap<DatanodeInfo, Integer> dnInfoMap = new HashMap<>();
+    for (int i = 0; i < 3; i++) {
+      LocatedBlocks lbs = NameNodeAdapter.getBlockLocations(
+          getCluster().getNameNode(0), openFiles[i], 0, blockSize * 10);
+      for (DatanodeInfo dn : lbs.getLastLocatedBlock().getLocations()) {
+        if (dnInfoMap.containsKey(dn)) {
+          dnInfoMap.put(dn, dnInfoMap.get(dn) + 1);
+        } else {
+          dnInfoMap.put(dn, 1);
+        }
+      }
+    }
+
+    DatanodeInfo dnToDecommission = null;
+    int maxDnOccurance = 0;
+    for (Map.Entry<DatanodeInfo, Integer> entry : dnInfoMap.entrySet()) {
+      if (entry.getValue() > maxDnOccurance) {
+        maxDnOccurance = entry.getValue();
+        dnToDecommission = entry.getKey();
+      }
+    }
+    LOG.info("XXX Dn to decommission: " + dnToDecommission + ", max: "
+        + maxDnOccurance);
+
+    //decommission one of the 3 nodes which have last block
+    DatanodeManager dm = ns.getBlockManager().getDatanodeManager();
+    ArrayList<String> nodes = new ArrayList<>();
+    dnToDecommission = dm.getDatanode(dnToDecommission.getDatanodeUuid());
+    nodes.add(dnToDecommission.getXferAddr());
+    initExcludeHosts(nodes);
+    refreshNodes(0);
+    waitNodeState(dnToDecommission, AdminStates.DECOMMISSION_INPROGRESS);
+
+    // list and verify all the open files that are blocking decommission
+    verifyOpenFilesBlockingDecommission(
+        closedFileSet, openFilesMap, maxDnOccurance);
+
+    final AtomicBoolean stopRedundancyMonitor = new AtomicBoolean(false);
+    Thread monitorThread = new Thread(new Runnable() {
+      @Override
+      public void run() {
+        while (!stopRedundancyMonitor.get()) {
+          try {
+            BlockManagerTestUtil.checkRedundancy(
+                getCluster().getNamesystem().getBlockManager());
+            BlockManagerTestUtil.updateState(
+                getCluster().getNamesystem().getBlockManager());
+            Thread.sleep(1000);
+          } catch (Exception e) {
+            LOG.warn("Encountered exception during redundancy monitor: " + e);
+          }
+        }
+      }
+    });
+    monitorThread.start();
+
+    waitNodeState(dnToDecommission, AdminStates.DECOMMISSIONED);
+    stopRedundancyMonitor.set(true);
+    monitorThread.join();
+
+    // Open file is no more blocking decommission as all its blocks
+    // are re-replicated.
+    openFilesMap.clear();
+    verifyOpenFilesBlockingDecommission(
+        closedFileSet, openFilesMap, 0);
+  }
+
   @Test(timeout = 360000)
   @Test(timeout = 360000)
   public void testDecommissionWithOpenFileAndBlockRecovery()
   public void testDecommissionWithOpenFileAndBlockRecovery()
       throws IOException, InterruptedException {
       throws IOException, InterruptedException {

+ 52 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptedTransfer.java

@@ -59,6 +59,7 @@ import org.junit.After;
 import org.junit.Before;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.Test;
+import org.junit.Assert;
 import org.junit.rules.Timeout;
 import org.junit.rules.Timeout;
 import org.junit.runner.RunWith;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized;
@@ -318,6 +319,57 @@ public class TestEncryptedTransfer {
     assertEquals(checksum, fs.getFileChecksum(TEST_PATH));
     assertEquals(checksum, fs.getFileChecksum(TEST_PATH));
   }
   }
 
 
+  @Test
+  public void testFileChecksumWithInvalidEncryptionKey()
+      throws IOException, InterruptedException, TimeoutException {
+    if (resolverClazz != null) {
+      // TestTrustedChannelResolver does not use encryption keys.
+      return;
+    }
+    setEncryptionConfigKeys();
+    cluster = new MiniDFSCluster.Builder(conf).build();
+
+    fs = getFileSystem(conf);
+    DFSClient client = DFSClientAdapter.getDFSClient((DistributedFileSystem)fs);
+    DFSClient spyClient = Mockito.spy(client);
+    DFSClientAdapter.setDFSClient((DistributedFileSystem) fs, spyClient);
+    writeTestDataToFile(fs);
+    FileChecksum checksum = fs.getFileChecksum(TEST_PATH);
+
+    BlockTokenSecretManager btsm = cluster.getNamesystem().getBlockManager()
+        .getBlockTokenSecretManager();
+    // Reduce key update interval and token life for testing.
+    btsm.setKeyUpdateIntervalForTesting(2 * 1000);
+    btsm.setTokenLifetime(2 * 1000);
+    btsm.clearAllKeysForTesting();
+
+    // Wait until the encryption key becomes invalid.
+    LOG.info("Wait until encryption keys become invalid...");
+
+    DataEncryptionKey encryptionKey = spyClient.getEncryptionKey();
+    List<DataNode> dataNodes = cluster.getDataNodes();
+    for (DataNode dn: dataNodes) {
+      GenericTestUtils.waitFor(
+          new Supplier<Boolean>() {
+            @Override
+            public Boolean get() {
+              return !dn.getBlockPoolTokenSecretManager().
+                  get(encryptionKey.blockPoolId)
+                  .hasKey(encryptionKey.keyId);
+            }
+          }, 100, 30*1000
+      );
+    }
+    LOG.info("The encryption key is invalid on all nodes now.");
+    fs.getFileChecksum(TEST_PATH);
+    // verify that InvalidEncryptionKeyException is handled properly
+    Assert.assertTrue(client.getEncryptionKey() == null);
+    Mockito.verify(spyClient, times(1)).clearDataEncryptionKey();
+    // Retry the operation after clearing the encryption key
+    FileChecksum verifyChecksum = fs.getFileChecksum(TEST_PATH);
+    Assert.assertEquals(checksum, verifyChecksum);
+  }
+
   @Test
   @Test
   public void testLongLivedClientPipelineRecovery()
   public void testLongLivedClientPipelineRecovery()
       throws IOException, InterruptedException, TimeoutException {
       throws IOException, InterruptedException, TimeoutException {

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

@@ -25,6 +25,7 @@ import java.io.File;
 import java.io.IOException;
 import java.io.IOException;
 import java.net.URI;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URISyntaxException;
+import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.HashSet;
 import java.util.Set;
 import java.util.Set;
@@ -41,6 +42,7 @@ import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.hdfs.client.HdfsAdmin;
 import org.apache.hadoop.hdfs.client.HdfsAdmin;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
+import org.apache.hadoop.hdfs.protocol.OpenFilesIterator.OpenFilesType;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.junit.After;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Assert;
@@ -254,7 +256,7 @@ public class TestHdfsAdmin {
     HdfsAdmin hdfsAdmin = new HdfsAdmin(FileSystem.getDefaultUri(conf), conf);
     HdfsAdmin hdfsAdmin = new HdfsAdmin(FileSystem.getDefaultUri(conf), conf);
     HashSet<Path> openFiles = new HashSet<>(openFileMap.keySet());
     HashSet<Path> openFiles = new HashSet<>(openFileMap.keySet());
     RemoteIterator<OpenFileEntry> openFilesRemoteItr =
     RemoteIterator<OpenFileEntry> openFilesRemoteItr =
-        hdfsAdmin.listOpenFiles();
+        hdfsAdmin.listOpenFiles(EnumSet.of(OpenFilesType.ALL_OPEN_FILES));
     while (openFilesRemoteItr.hasNext()) {
     while (openFilesRemoteItr.hasNext()) {
       String filePath = openFilesRemoteItr.next().getFilePath();
       String filePath = openFilesRemoteItr.next().getFilePath();
       assertFalse(filePath + " should not be listed under open files!",
       assertFalse(filePath + " should not be listed under open files!",

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

@@ -168,7 +168,17 @@ public class BlockManagerTestUtil {
   public static int computeInvalidationWork(BlockManager bm) {
   public static int computeInvalidationWork(BlockManager bm) {
     return bm.computeInvalidateWork(Integer.MAX_VALUE);
     return bm.computeInvalidateWork(Integer.MAX_VALUE);
   }
   }
-  
+
+  /**
+   * Check the redundancy of blocks and trigger replication if needed.
+   * @param blockManager
+   */
+  public static void checkRedundancy(final BlockManager blockManager) {
+    blockManager.computeDatanodeWork();
+    blockManager.processPendingReconstructions();
+    blockManager.rescanPostponedMisreplicatedBlocks();
+  }
+
   /**
   /**
    * Compute all the replication and invalidation work for the
    * Compute all the replication and invalidation work for the
    * given BlockManager.
    * given BlockManager.

+ 9 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java

@@ -244,6 +244,15 @@ public class TestDiskBalancerCommand {
 
 
   }
   }
 
 
+  /* test basic report with negative top limit */
+  @Test(timeout = 60000)
+  public void testReportWithNegativeTopLimit()
+      throws Exception {
+    final String cmdLine = "hdfs diskbalancer -report -top -32";
+    thrown.expect(java.lang.IllegalArgumentException.class);
+    thrown.expectMessage("Top limit input should be a positive numeric value");
+    runCommand(cmdLine);
+  }
   /* test less than 64 DataNode(s) as total, e.g., -report -top 32 */
   /* test less than 64 DataNode(s) as total, e.g., -report -top 32 */
   @Test(timeout = 60000)
   @Test(timeout = 60000)
   public void testReportLessThanTotal() throws Exception {
   public void testReportLessThanTotal() throws Exception {

+ 27 - 21
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestLeaseManager.java

@@ -206,7 +206,7 @@ public class TestLeaseManager {
         HdfsConstants.GRANDFATHER_INODE_ID, DFSUtil.string2Bytes(""),
         HdfsConstants.GRANDFATHER_INODE_ID, DFSUtil.string2Bytes(""),
         perm, 0L);
         perm, 0L);
     when(fsDirectory.getRoot()).thenReturn(rootInodeDirectory);
     when(fsDirectory.getRoot()).thenReturn(rootInodeDirectory);
-    verifyINodeLeaseCounts(lm, rootInodeDirectory, 0, 0, 0);
+    verifyINodeLeaseCounts(fsNamesystem, lm, rootInodeDirectory, 0, 0, 0);
 
 
     for (Long iNodeId : iNodeIds) {
     for (Long iNodeId : iNodeIds) {
       INodeFile iNodeFile = stubInodeFile(iNodeId);
       INodeFile iNodeFile = stubInodeFile(iNodeId);
@@ -215,13 +215,13 @@ public class TestLeaseManager {
       when(fsDirectory.getInode(iNodeId)).thenReturn(iNodeFile);
       when(fsDirectory.getInode(iNodeId)).thenReturn(iNodeFile);
       lm.addLease("holder_" + iNodeId, iNodeId);
       lm.addLease("holder_" + iNodeId, iNodeId);
     }
     }
-    verifyINodeLeaseCounts(lm, rootInodeDirectory, iNodeIds.size(),
-        iNodeIds.size(), iNodeIds.size());
+    verifyINodeLeaseCounts(fsNamesystem, lm, rootInodeDirectory,
+        iNodeIds.size(), iNodeIds.size(), iNodeIds.size());
 
 
     for (Long iNodeId : iNodeIds) {
     for (Long iNodeId : iNodeIds) {
       lm.removeLease(iNodeId);
       lm.removeLease(iNodeId);
     }
     }
-    verifyINodeLeaseCounts(lm, rootInodeDirectory, 0, 0, 0);
+    verifyINodeLeaseCounts(fsNamesystem, lm, rootInodeDirectory, 0, 0, 0);
   }
   }
 
 
   /**
   /**
@@ -246,41 +246,44 @@ public class TestLeaseManager {
 
 
     // Case 1: No open files
     // Case 1: No open files
     int scale = 0;
     int scale = 0;
-    testInodeWithLeasesAtScaleImpl(lm, fsDirectory, rootInodeDirectory, scale);
+    testInodeWithLeasesAtScaleImpl(fsNamesystem, lm, fsDirectory,
+        rootInodeDirectory, scale);
 
 
     for (int workerCount = 1;
     for (int workerCount = 1;
          workerCount <= LeaseManager.INODE_FILTER_WORKER_COUNT_MAX / 2;
          workerCount <= LeaseManager.INODE_FILTER_WORKER_COUNT_MAX / 2;
          workerCount++) {
          workerCount++) {
       // Case 2: Open files count is half of worker task size
       // Case 2: Open files count is half of worker task size
       scale = workerCount * LeaseManager.INODE_FILTER_WORKER_TASK_MIN / 2;
       scale = workerCount * LeaseManager.INODE_FILTER_WORKER_TASK_MIN / 2;
-      testInodeWithLeasesAtScaleImpl(lm, fsDirectory,
+      testInodeWithLeasesAtScaleImpl(fsNamesystem, lm, fsDirectory,
           rootInodeDirectory, scale);
           rootInodeDirectory, scale);
 
 
       // Case 3: Open files count is 1 less of worker task size
       // Case 3: Open files count is 1 less of worker task size
       scale = workerCount * LeaseManager.INODE_FILTER_WORKER_TASK_MIN - 1;
       scale = workerCount * LeaseManager.INODE_FILTER_WORKER_TASK_MIN - 1;
-      testInodeWithLeasesAtScaleImpl(lm, fsDirectory,
+      testInodeWithLeasesAtScaleImpl(fsNamesystem, lm, fsDirectory,
           rootInodeDirectory, scale);
           rootInodeDirectory, scale);
 
 
       // Case 4: Open files count is equal to worker task size
       // Case 4: Open files count is equal to worker task size
       scale = workerCount * LeaseManager.INODE_FILTER_WORKER_TASK_MIN;
       scale = workerCount * LeaseManager.INODE_FILTER_WORKER_TASK_MIN;
-      testInodeWithLeasesAtScaleImpl(lm, fsDirectory,
+      testInodeWithLeasesAtScaleImpl(fsNamesystem, lm, fsDirectory,
           rootInodeDirectory, scale);
           rootInodeDirectory, scale);
 
 
       // Case 5: Open files count is 1 more than worker task size
       // Case 5: Open files count is 1 more than worker task size
       scale = workerCount * LeaseManager.INODE_FILTER_WORKER_TASK_MIN + 1;
       scale = workerCount * LeaseManager.INODE_FILTER_WORKER_TASK_MIN + 1;
-      testInodeWithLeasesAtScaleImpl(lm, fsDirectory,
+      testInodeWithLeasesAtScaleImpl(fsNamesystem, lm, fsDirectory,
           rootInodeDirectory, scale);
           rootInodeDirectory, scale);
     }
     }
 
 
     // Case 6: Open files count is way more than worker count
     // Case 6: Open files count is way more than worker count
     scale = 1279;
     scale = 1279;
-    testInodeWithLeasesAtScaleImpl(lm, fsDirectory, rootInodeDirectory, scale);
+    testInodeWithLeasesAtScaleImpl(fsNamesystem, lm, fsDirectory,
+        rootInodeDirectory, scale);
   }
   }
 
 
-  private void testInodeWithLeasesAtScaleImpl(final LeaseManager leaseManager,
-      final FSDirectory fsDirectory, INodeDirectory ancestorDirectory,
-      int scale) throws IOException {
-    verifyINodeLeaseCounts(leaseManager, ancestorDirectory, 0, 0, 0);
+  private void testInodeWithLeasesAtScaleImpl(FSNamesystem fsNamesystem,
+      final LeaseManager leaseManager, final FSDirectory fsDirectory,
+      INodeDirectory ancestorDirectory, int scale) throws IOException {
+    verifyINodeLeaseCounts(
+        fsNamesystem, leaseManager, ancestorDirectory, 0, 0, 0);
 
 
     Set<Long> iNodeIds = new HashSet<>();
     Set<Long> iNodeIds = new HashSet<>();
     for (int i = 0; i < scale; i++) {
     for (int i = 0; i < scale; i++) {
@@ -293,11 +296,12 @@ public class TestLeaseManager {
       when(fsDirectory.getInode(iNodeId)).thenReturn(iNodeFile);
       when(fsDirectory.getInode(iNodeId)).thenReturn(iNodeFile);
       leaseManager.addLease("holder_" + iNodeId, iNodeId);
       leaseManager.addLease("holder_" + iNodeId, iNodeId);
     }
     }
-    verifyINodeLeaseCounts(leaseManager, ancestorDirectory, iNodeIds.size(),
-        iNodeIds.size(), iNodeIds.size());
+    verifyINodeLeaseCounts(fsNamesystem, leaseManager,
+        ancestorDirectory, iNodeIds.size(), iNodeIds.size(), iNodeIds.size());
 
 
     leaseManager.removeAllLeases();
     leaseManager.removeAllLeases();
-    verifyINodeLeaseCounts(leaseManager, ancestorDirectory, 0, 0, 0);
+    verifyINodeLeaseCounts(fsNamesystem, leaseManager,
+        ancestorDirectory, 0, 0, 0);
   }
   }
 
 
   /**
   /**
@@ -389,10 +393,10 @@ public class TestLeaseManager {
 
 
   }
   }
 
 
-  private void verifyINodeLeaseCounts(final LeaseManager leaseManager,
-      INodeDirectory ancestorDirectory, int iNodeIdWithLeaseCount,
-      int iNodeWithLeaseCount, int iNodeUnderAncestorLeaseCount)
-      throws IOException {
+  private void verifyINodeLeaseCounts(FSNamesystem fsNamesystem,
+      LeaseManager leaseManager, INodeDirectory ancestorDirectory,
+      int iNodeIdWithLeaseCount, int iNodeWithLeaseCount,
+      int iNodeUnderAncestorLeaseCount) throws IOException {
     assertEquals(iNodeIdWithLeaseCount,
     assertEquals(iNodeIdWithLeaseCount,
         leaseManager.getINodeIdWithLeases().size());
         leaseManager.getINodeIdWithLeases().size());
     assertEquals(iNodeWithLeaseCount,
     assertEquals(iNodeWithLeaseCount,
@@ -401,6 +405,8 @@ public class TestLeaseManager {
         leaseManager.getINodeWithLeases(ancestorDirectory).size());
         leaseManager.getINodeWithLeases(ancestorDirectory).size());
     assertEquals(iNodeIdWithLeaseCount,
     assertEquals(iNodeIdWithLeaseCount,
         leaseManager.getUnderConstructionFiles(0).size());
         leaseManager.getUnderConstructionFiles(0).size());
+    assertEquals(0, (fsNamesystem.getFilesBlockingDecom(0) == null ?
+        0 : fsNamesystem.getFilesBlockingDecom(0).size()));
   }
   }
 
 
   private Map<String, INode> createINodeTree(INodeDirectory parentDir,
   private Map<String, INode> createINodeTree(INodeDirectory parentDir,

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

@@ -22,6 +22,7 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertTrue;
 
 
 import java.io.IOException;
 import java.io.IOException;
+import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.HashSet;
 import java.util.List;
 import java.util.List;
@@ -43,6 +44,7 @@ import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
+import org.apache.hadoop.hdfs.protocol.OpenFilesIterator.OpenFilesType;
 import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
 import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
@@ -95,9 +97,13 @@ public class TestListOpenFiles {
     verifyOpenFiles(openFiles);
     verifyOpenFiles(openFiles);
 
 
     BatchedEntries<OpenFileEntry> openFileEntryBatchedEntries =
     BatchedEntries<OpenFileEntry> openFileEntryBatchedEntries =
-        nnRpc.listOpenFiles(0);
+        nnRpc.listOpenFiles(0, EnumSet.of(OpenFilesType.ALL_OPEN_FILES));
     assertTrue("Open files list should be empty!",
     assertTrue("Open files list should be empty!",
         openFileEntryBatchedEntries.size() == 0);
         openFileEntryBatchedEntries.size() == 0);
+    BatchedEntries<OpenFileEntry> openFilesBlockingDecomEntries =
+        nnRpc.listOpenFiles(0, EnumSet.of(OpenFilesType.BLOCKING_DECOMMISSION));
+    assertTrue("Open files list blocking decommission should be empty!",
+        openFilesBlockingDecomEntries.size() == 0);
 
 
     openFiles.putAll(
     openFiles.putAll(
         DFSTestUtil.createOpenFiles(fs, "open-1", 1));
         DFSTestUtil.createOpenFiles(fs, "open-1", 1));
@@ -121,16 +127,16 @@ public class TestListOpenFiles {
     }
     }
   }
   }
 
 
-  private void verifyOpenFiles(Map<Path, FSDataOutputStream> openFiles)
-      throws IOException {
+  private void verifyOpenFiles(Map<Path, FSDataOutputStream> openFiles,
+      EnumSet<OpenFilesType> openFilesTypes) throws IOException {
     HashSet<Path> remainingFiles = new HashSet<>(openFiles.keySet());
     HashSet<Path> remainingFiles = new HashSet<>(openFiles.keySet());
     OpenFileEntry lastEntry = null;
     OpenFileEntry lastEntry = null;
     BatchedEntries<OpenFileEntry> batchedEntries;
     BatchedEntries<OpenFileEntry> batchedEntries;
     do {
     do {
       if (lastEntry == null) {
       if (lastEntry == null) {
-        batchedEntries = nnRpc.listOpenFiles(0);
+        batchedEntries = nnRpc.listOpenFiles(0, openFilesTypes);
       } else {
       } else {
-        batchedEntries = nnRpc.listOpenFiles(lastEntry.getId());
+        batchedEntries = nnRpc.listOpenFiles(lastEntry.getId(), openFilesTypes);
       }
       }
       assertTrue("Incorrect open files list size!",
       assertTrue("Incorrect open files list size!",
           batchedEntries.size() <= BATCH_SIZE);
           batchedEntries.size() <= BATCH_SIZE);
@@ -146,6 +152,13 @@ public class TestListOpenFiles {
         remainingFiles.size() == 0);
         remainingFiles.size() == 0);
   }
   }
 
 
+  private void verifyOpenFiles(Map<Path, FSDataOutputStream> openFiles)
+      throws IOException {
+    verifyOpenFiles(openFiles, EnumSet.of(OpenFilesType.ALL_OPEN_FILES));
+    verifyOpenFiles(new HashMap<>(),
+        EnumSet.of(OpenFilesType.BLOCKING_DECOMMISSION));
+  }
+
   private Set<Path> createFiles(FileSystem fileSystem, String fileNamePrefix,
   private Set<Path> createFiles(FileSystem fileSystem, String fileNamePrefix,
       int numFilesToCreate) throws IOException {
       int numFilesToCreate) throws IOException {
     HashSet<Path> files = new HashSet<>();
     HashSet<Path> files = new HashSet<>();
@@ -197,6 +210,8 @@ public class TestListOpenFiles {
             try {
             try {
               assertEquals(0, ToolRunner.run(dfsAdmin,
               assertEquals(0, ToolRunner.run(dfsAdmin,
                   new String[] {"-listOpenFiles"}));
                   new String[] {"-listOpenFiles"}));
+              assertEquals(0, ToolRunner.run(dfsAdmin,
+                  new String[] {"-listOpenFiles", "-blockingDecommission"}));
               // Sleep for some time to avoid
               // Sleep for some time to avoid
               // flooding logs with listing.
               // flooding logs with listing.
               Thread.sleep(listingIntervalMsec);
               Thread.sleep(listingIntervalMsec);
@@ -222,6 +237,8 @@ public class TestListOpenFiles {
 
 
       assertEquals(0, ToolRunner.run(dfsAdmin,
       assertEquals(0, ToolRunner.run(dfsAdmin,
           new String[] {"-listOpenFiles"}));
           new String[] {"-listOpenFiles"}));
+      assertEquals(0, ToolRunner.run(dfsAdmin,
+          new String[] {"-listOpenFiles", "-blockingDecommission"}));
       assertFalse("Client Error!", listOpenFilesError.get());
       assertFalse("Client Error!", listOpenFilesError.get());
 
 
       clientThread.join();
       clientThread.join();

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

@@ -60,11 +60,14 @@ public class TestDNFencingWithReplication {
   private static class ReplicationToggler extends RepeatingTestThread {
   private static class ReplicationToggler extends RepeatingTestThread {
     private final FileSystem fs;
     private final FileSystem fs;
     private final Path path;
     private final Path path;
+    private final MiniDFSCluster cluster;
 
 
-    public ReplicationToggler(TestContext ctx, FileSystem fs, Path p) {
+    ReplicationToggler(TestContext ctx, FileSystem fs, Path p,
+                       MiniDFSCluster cluster) {
       super(ctx);
       super(ctx);
       this.fs = fs;
       this.fs = fs;
       this.path = p;
       this.path = p;
+      this.cluster = cluster;
     }
     }
 
 
     @Override
     @Override
@@ -81,6 +84,7 @@ public class TestDNFencingWithReplication {
           @Override
           @Override
           public Boolean get() {
           public Boolean get() {
             try {
             try {
+              cluster.waitActive();
               BlockLocation[] blocks = fs.getFileBlockLocations(path, 0, 10);
               BlockLocation[] blocks = fs.getFileBlockLocations(path, 0, 10);
               Assert.assertEquals(1, blocks.length);
               Assert.assertEquals(1, blocks.length);
               return blocks[0].getHosts().length == replicas;
               return blocks[0].getHosts().length == replicas;
@@ -90,8 +94,8 @@ public class TestDNFencingWithReplication {
           }
           }
         }, 100, 60000);
         }, 100, 60000);
       } catch (TimeoutException te) {
       } catch (TimeoutException te) {
-        throw new IOException("Timed out waiting for " + replicas + " replicas " +
-            "on path " + path);
+        throw new IOException("Timed out waiting for " + replicas +
+                " replicas on path " + path);
       }
       }
     }
     }
     
     
@@ -122,7 +126,7 @@ public class TestDNFencingWithReplication {
       for (int i = 0; i < NUM_THREADS; i++) {
       for (int i = 0; i < NUM_THREADS; i++) {
         Path p = new Path("/test-" + i);
         Path p = new Path("/test-" + i);
         DFSTestUtil.createFile(fs, p, BLOCK_SIZE*10, (short)3, (long)i);
         DFSTestUtil.createFile(fs, p, BLOCK_SIZE*10, (short)3, (long)i);
-        togglers.addThread(new ReplicationToggler(togglers, fs, p));
+        togglers.addThread(new ReplicationToggler(togglers, fs, p, cluster));
       }
       }
       
       
       // Start a separate thread which will make sure that replication
       // Start a separate thread which will make sure that replication

+ 39 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java

@@ -40,6 +40,7 @@ import java.util.Random;
 
 
 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.assertTrue;
 
 
 /**
 /**
  * Need to cover the following combinations:
  * Need to cover the following combinations:
@@ -127,7 +128,7 @@ public class TestStripedBlockUtil {
     return (byte) (((i + 13) * 29) & BYTE_MASK);
     return (byte) (((i + 13) * 29) & BYTE_MASK);
   }
   }
 
 
-  private LocatedStripedBlock createDummyLocatedBlock(int bgSize) {
+  private LocatedStripedBlock createDummyLocatedBlock(long bgSize) {
     final long blockGroupID = -1048576;
     final long blockGroupID = -1048576;
     DatanodeInfo[] locs = new DatanodeInfo[groupSize];
     DatanodeInfo[] locs = new DatanodeInfo[groupSize];
     String[] storageIDs = new String[groupSize];
     String[] storageIDs = new String[groupSize];
@@ -160,7 +161,7 @@ public class TestStripedBlockUtil {
       Preconditions.checkState(done % cellSize == 0);
       Preconditions.checkState(done % cellSize == 0);
       StripingCell cell =
       StripingCell cell =
           new StripingCell(ecPolicy, cellSize, done / cellSize, 0);
           new StripingCell(ecPolicy, cellSize, done / cellSize, 0);
-      int idxInStripe = cell.idxInStripe;
+      int idxInStripe = cell.getIdxInStripe();
       int size = Math.min(cellSize, bgSize - done);
       int size = Math.min(cellSize, bgSize - done);
       for (int i = 0; i < size; i++) {
       for (int i = 0; i < size; i++) {
         bufs[idxInStripe][pos[idxInStripe] + i] = hashIntToByte(done + i);
         bufs[idxInStripe][pos[idxInStripe] + i] = hashIntToByte(done + i);
@@ -283,4 +284,40 @@ public class TestStripedBlockUtil {
       }
       }
     }
     }
   }
   }
+
+  /**
+   * Test dividing a byte range that located above the 2GB range, which is
+   * {@link Integer#MAX_VALUE}.
+   *
+   * HDFS-12860 occurs when {@link VerticalRange#offsetInBlock} is larger than
+   * {@link Integer#MAX_VALUE}
+   *
+   * Take RS-6-3-1024k EC policy as example:
+   *  <li>cellSize = 1MB</li>
+   *  <li>The first {@link VerticalRange#offsetInBlock} that is larger than
+   *  {@link Integer#MAX_VALUE} is Math.ceilInteger.MAX_VALUE / cellSize = 2048
+   *  </li>
+   *  <li>The first offset in block group that causes HDFS-12860 is:
+   *  2048 * cellSize * dataBlocks (6)</li>
+   */
+  @Test
+  public void testDivideOneStripeLargeBlockSize() {
+    ByteBuffer buffer = ByteBuffer.allocate(stripeSize);
+
+    // This offset will cause overflow before HDFS-12860.
+    long offsetInInternalBlk = Integer.MAX_VALUE / cellSize + 10;
+    long rangeStartInBlockGroup = offsetInInternalBlk * dataBlocks * cellSize;
+    long rangeEndInBlockGroup = rangeStartInBlockGroup +
+        dataBlocks / 2 * cellSize - 1;
+    // each block is 4GB, each block group has 4GB * (6 + 3) = 36GB.
+    long blockGroupSize = 4096L * cellSize * groupSize;
+    LocatedStripedBlock blockGroup = createDummyLocatedBlock(blockGroupSize);
+    AlignedStripe[] stripes = StripedBlockUtil.divideOneStripe(ecPolicy,
+        cellSize, blockGroup, rangeStartInBlockGroup, rangeEndInBlockGroup,
+        buffer);
+    long offset = offsetInInternalBlk * cellSize;
+    assertTrue(offset > Integer.MAX_VALUE);
+    assertEquals(offset, stripes[0].range.offsetInBlock);
+    assertEquals(1, stripes.length);
+  }
 }
 }

+ 23 - 18
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java

@@ -585,33 +585,38 @@ public class TaskAttemptListenerImpl extends CompositeService
   private void coalesceStatusUpdate(TaskAttemptId yarnAttemptID,
   private void coalesceStatusUpdate(TaskAttemptId yarnAttemptID,
       TaskAttemptStatus taskAttemptStatus,
       TaskAttemptStatus taskAttemptStatus,
       AtomicReference<TaskAttemptStatus> lastStatusRef) {
       AtomicReference<TaskAttemptStatus> lastStatusRef) {
-    boolean asyncUpdatedNeeded = false;
-    TaskAttemptStatus lastStatus = lastStatusRef.get();
-
-    if (lastStatus == null) {
-      lastStatusRef.set(taskAttemptStatus);
-      asyncUpdatedNeeded = true;
-    } else {
-      List<TaskAttemptId> oldFetchFailedMaps =
-          taskAttemptStatus.fetchFailedMaps;
-
-      // merge fetchFailedMaps from the previous update
-      if (lastStatus.fetchFailedMaps != null) {
+    List<TaskAttemptId> fetchFailedMaps = taskAttemptStatus.fetchFailedMaps;
+    TaskAttemptStatus lastStatus = null;
+    boolean done = false;
+    while (!done) {
+      lastStatus = lastStatusRef.get();
+      if (lastStatus != null && lastStatus.fetchFailedMaps != null) {
+        // merge fetchFailedMaps from the previous update
         if (taskAttemptStatus.fetchFailedMaps == null) {
         if (taskAttemptStatus.fetchFailedMaps == null) {
           taskAttemptStatus.fetchFailedMaps = lastStatus.fetchFailedMaps;
           taskAttemptStatus.fetchFailedMaps = lastStatus.fetchFailedMaps;
         } else {
         } else {
-          taskAttemptStatus.fetchFailedMaps.addAll(lastStatus.fetchFailedMaps);
+          taskAttemptStatus.fetchFailedMaps =
+              new ArrayList<>(lastStatus.fetchFailedMaps.size() +
+                  fetchFailedMaps.size());
+          taskAttemptStatus.fetchFailedMaps.addAll(
+              lastStatus.fetchFailedMaps);
+          taskAttemptStatus.fetchFailedMaps.addAll(
+              fetchFailedMaps);
         }
         }
       }
       }
 
 
-      if (!lastStatusRef.compareAndSet(lastStatus, taskAttemptStatus)) {
-        // update failed - async dispatcher has processed it in the meantime
-        taskAttemptStatus.fetchFailedMaps = oldFetchFailedMaps;
-        lastStatusRef.set(taskAttemptStatus);
-        asyncUpdatedNeeded = true;
+      // lastStatusRef may be changed by either the AsyncDispatcher when
+      // it processes the update, or by another IPC server handler
+      done = lastStatusRef.compareAndSet(lastStatus, taskAttemptStatus);
+      if (!done) {
+        LOG.info("TaskAttempt " + yarnAttemptID +
+            ": lastStatusRef changed by another thread, retrying...");
+        // let's revert taskAttemptStatus.fetchFailedMaps
+        taskAttemptStatus.fetchFailedMaps = fetchFailedMaps;
       }
       }
     }
     }
 
 
+    boolean asyncUpdatedNeeded = (lastStatus == null);
     if (asyncUpdatedNeeded) {
     if (asyncUpdatedNeeded) {
       context.getEventHandler().handle(
       context.getEventHandler().handle(
           new TaskAttemptStatusUpdateEvent(taskAttemptStatus.id,
           new TaskAttemptStatusUpdateEvent(taskAttemptStatus.id,

+ 39 - 37
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ContainerLogAppender.java

@@ -20,8 +20,8 @@ package org.apache.hadoop.yarn;
 
 
 import java.io.File;
 import java.io.File;
 import java.io.Flushable;
 import java.io.Flushable;
-import java.util.LinkedList;
-import java.util.Queue;
+import java.util.ArrayDeque;
+import java.util.Deque;
 
 
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
@@ -30,49 +30,43 @@ import org.apache.log4j.spi.LoggingEvent;
 
 
 /**
 /**
  * A simple log4j-appender for container's logs.
  * A simple log4j-appender for container's logs.
- * 
  */
  */
 @Public
 @Public
 @Unstable
 @Unstable
 public class ContainerLogAppender extends FileAppender
 public class ContainerLogAppender extends FileAppender
-  implements Flushable
-{
+    implements Flushable {
+
   private String containerLogDir;
   private String containerLogDir;
   private String containerLogFile;
   private String containerLogFile;
-  //so that log4j can configure it from the configuration(log4j.properties). 
   private int maxEvents;
   private int maxEvents;
-  private Queue<LoggingEvent> tail = null;
-  private boolean closing = false;
+  private Deque<LoggingEvent> eventBuffer;
+  private boolean closed = false;
 
 
   @Override
   @Override
-  public void activateOptions() {
-    synchronized (this) {
-      if (maxEvents > 0) {
-        tail = new LinkedList<LoggingEvent>();
-      }
-      setFile(new File(this.containerLogDir, containerLogFile).toString());
-      setAppend(true);
-      super.activateOptions();
+  public synchronized void activateOptions() {
+    if (maxEvents > 0) {
+      this.eventBuffer = new ArrayDeque<>();
     }
     }
+    setFile(new File(this.containerLogDir, containerLogFile).toString());
+    setAppend(true);
+    super.activateOptions();
   }
   }
-  
+
   @Override
   @Override
-  public void append(LoggingEvent event) {
-    synchronized (this) {
-      if (closing) { // When closing drop any new/transitive CLA appending
-        return;
-      }
-      if (tail == null) {
-        super.append(event);
-      } else {
-        if (tail.size() >= maxEvents) {
-          tail.remove();
-        }
-        tail.add(event);
+  public synchronized void append(LoggingEvent event) {
+    if (closed) {
+      return;
+    }
+    if (eventBuffer != null) {
+      if (eventBuffer.size() == maxEvents) {
+        eventBuffer.removeFirst();
       }
       }
+      eventBuffer.addLast(event);
+    } else {
+      super.append(event);
     }
     }
   }
   }
-  
+
   @Override
   @Override
   public void flush() {
   public void flush() {
     if (qw != null) {
     if (qw != null) {
@@ -82,13 +76,17 @@ public class ContainerLogAppender extends FileAppender
 
 
   @Override
   @Override
   public synchronized void close() {
   public synchronized void close() {
-    closing = true;
-    if (tail != null) {
-      for (LoggingEvent event : tail) {
-        super.append(event);
+    if (!closed) {
+      closed = true;
+      if (eventBuffer != null) {
+        for (LoggingEvent event : eventBuffer) {
+          super.append(event);
+        }
+        // let garbage collection do its work
+        eventBuffer = null;
       }
       }
+      super.close();
     }
     }
-    super.close();
   }
   }
 
 
   /**
   /**
@@ -111,13 +109,17 @@ public class ContainerLogAppender extends FileAppender
     this.containerLogFile = containerLogFile;
     this.containerLogFile = containerLogFile;
   }
   }
 
 
-  private static final int EVENT_SIZE = 100;
+  private static final long EVENT_SIZE = 100;
   
   
   public long getTotalLogFileSize() {
   public long getTotalLogFileSize() {
     return maxEvents * EVENT_SIZE;
     return maxEvents * EVENT_SIZE;
   }
   }
 
 
+  /**
+   *  Setter so that log4j can configure it from the
+   *  configuration(log4j.properties).
+   */
   public void setTotalLogFileSize(long logSize) {
   public void setTotalLogFileSize(long logSize) {
-    maxEvents = (int) logSize / EVENT_SIZE;
+    maxEvents = (int)(logSize / EVENT_SIZE);
   }
   }
 }
 }

+ 51 - 52
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ProcfsBasedProcessTree.java

@@ -20,21 +20,30 @@ package org.apache.hadoop.yarn.util;
 
 
 import java.io.BufferedReader;
 import java.io.BufferedReader;
 import java.io.File;
 import java.io.File;
+import java.io.FileFilter;
 import java.io.FileInputStream;
 import java.io.FileInputStream;
 import java.io.FileNotFoundException;
 import java.io.FileNotFoundException;
-import java.io.InputStreamReader;
 import java.io.IOException;
 import java.io.IOException;
+import java.io.InputStreamReader;
 import java.math.BigInteger;
 import java.math.BigInteger;
 import java.nio.charset.Charset;
 import java.nio.charset.Charset;
+import java.util.ArrayDeque;
 import java.util.ArrayList;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.List;
 import java.util.Map;
 import java.util.Map;
+import java.util.Queue;
 import java.util.regex.Matcher;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 import java.util.regex.Pattern;
 
 
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.io.IOUtils;
+import org.apache.commons.io.filefilter.AndFileFilter;
+import org.apache.commons.io.filefilter.DirectoryFileFilter;
+import org.apache.commons.io.filefilter.RegexFileFilter;
+import org.apache.commons.lang.ArrayUtils;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -85,8 +94,9 @@ public class ProcfsBasedProcessTree extends ResourceCalculatorProcessTree {
     }
     }
 
 
     public static MemInfo getMemInfoByName(String name) {
     public static MemInfo getMemInfoByName(String name) {
+      String searchName = StringUtils.trimToNull(name);
       for (MemInfo info : MemInfo.values()) {
       for (MemInfo info : MemInfo.values()) {
-        if (info.name.trim().equalsIgnoreCase(name.trim())) {
+        if (info.name.trim().equalsIgnoreCase(searchName)) {
           return info;
           return info;
         }
         }
       }
       }
@@ -170,7 +180,7 @@ public class ProcfsBasedProcessTree extends ResourceCalculatorProcessTree {
         return false;
         return false;
       }
       }
     } catch (SecurityException se) {
     } catch (SecurityException se) {
-      LOG.warn("Failed to get Operating System name. " + se);
+      LOG.warn("Failed to get Operating System name.", se);
       return false;
       return false;
     }
     }
     return true;
     return true;
@@ -214,12 +224,12 @@ public class ProcfsBasedProcessTree extends ResourceCalculatorProcessTree {
       // Add each process to its parent.
       // Add each process to its parent.
       for (Map.Entry<String, ProcessInfo> entry : allProcessInfo.entrySet()) {
       for (Map.Entry<String, ProcessInfo> entry : allProcessInfo.entrySet()) {
         String pID = entry.getKey();
         String pID = entry.getKey();
-        if (!pID.equals("1")) {
+        if (!"1".equals(pID)) {
           ProcessInfo pInfo = entry.getValue();
           ProcessInfo pInfo = entry.getValue();
           String ppid = pInfo.getPpid();
           String ppid = pInfo.getPpid();
           // If parent is init and process is not session leader,
           // If parent is init and process is not session leader,
           // attach to sessionID
           // attach to sessionID
-          if (ppid.equals("1")) {
+          if ("1".equals(ppid)) {
               String sid = pInfo.getSessionId().toString();
               String sid = pInfo.getSessionId().toString();
               if (!pID.equals(sid)) {
               if (!pID.equals(sid)) {
                  ppid = sid;
                  ppid = sid;
@@ -233,8 +243,8 @@ public class ProcfsBasedProcessTree extends ResourceCalculatorProcessTree {
       }
       }
 
 
       // now start constructing the process-tree
       // now start constructing the process-tree
-      LinkedList<ProcessInfo> pInfoQueue = new LinkedList<ProcessInfo>();
-      pInfoQueue.addAll(me.getChildren());
+      List<ProcessInfo> children = me.getChildren();
+      Queue<ProcessInfo> pInfoQueue = new ArrayDeque<ProcessInfo>(children);
       while (!pInfoQueue.isEmpty()) {
       while (!pInfoQueue.isEmpty()) {
         ProcessInfo pInfo = pInfoQueue.remove();
         ProcessInfo pInfo = pInfoQueue.remove();
         if (!processTree.containsKey(pInfo.getPid())) {
         if (!processTree.containsKey(pInfo.getPid())) {
@@ -254,12 +264,10 @@ public class ProcfsBasedProcessTree extends ResourceCalculatorProcessTree {
         }
         }
       }
       }
 
 
-      if (LOG.isDebugEnabled()) {
-        // Log.debug the ProcfsBasedProcessTree
-        LOG.debug(this.toString());
-      }
+      LOG.debug(this);
+
       if (smapsEnabled) {
       if (smapsEnabled) {
-        //Update smaps info
+        // Update smaps info
         processSMAPTree.clear();
         processSMAPTree.clear();
         for (ProcessInfo p : processTree.values()) {
         for (ProcessInfo p : processTree.values()) {
           if (p != null) {
           if (p != null) {
@@ -296,9 +304,7 @@ public class ProcfsBasedProcessTree extends ResourceCalculatorProcessTree {
       "\t|- %s %s %d %d %s %d %d %d %d %s%n";
       "\t|- %s %s %d %d %s %d %d %d %d %s%n";
 
 
   public List<String> getCurrentProcessIDs() {
   public List<String> getCurrentProcessIDs() {
-    List<String> currentPIDs = new ArrayList<String>();
-    currentPIDs.addAll(processTree.keySet());
-    return currentPIDs;
+    return Collections.unmodifiableList(new ArrayList<>(processTree.keySet()));
   }
   }
 
 
   /**
   /**
@@ -327,18 +333,17 @@ public class ProcfsBasedProcessTree extends ResourceCalculatorProcessTree {
 
 
   @Override
   @Override
   public long getVirtualMemorySize(int olderThanAge) {
   public long getVirtualMemorySize(int olderThanAge) {
-    long total = UNAVAILABLE;
+    long total = 0L;
+    boolean isAvailable = false;
     for (ProcessInfo p : processTree.values()) {
     for (ProcessInfo p : processTree.values()) {
       if (p != null) {
       if (p != null) {
-        if (total == UNAVAILABLE ) {
-          total = 0;
-        }
+        isAvailable = true;
         if (p.getAge() > olderThanAge) {
         if (p.getAge() > olderThanAge) {
           total += p.getVmem();
           total += p.getVmem();
         }
         }
       }
       }
     }
     }
-    return total;
+    return isAvailable ? total : UNAVAILABLE;
   }
   }
 
 
   @Override
   @Override
@@ -352,11 +357,11 @@ public class ProcfsBasedProcessTree extends ResourceCalculatorProcessTree {
     boolean isAvailable = false;
     boolean isAvailable = false;
     long totalPages = 0;
     long totalPages = 0;
     for (ProcessInfo p : processTree.values()) {
     for (ProcessInfo p : processTree.values()) {
-      if ((p != null) ) {
+      if (p != null) {
+        isAvailable = true;
         if (p.getAge() > olderThanAge) {
         if (p.getAge() > olderThanAge) {
           totalPages += p.getRssmemPage();
           totalPages += p.getRssmemPage();
         }
         }
-        isAvailable = true;
       }
       }
     }
     }
     return isAvailable ? totalPages * PAGE_SIZE : UNAVAILABLE; // convert # pages to byte
     return isAvailable ? totalPages * PAGE_SIZE : UNAVAILABLE; // convert # pages to byte
@@ -405,9 +410,7 @@ public class ProcfsBasedProcessTree extends ResourceCalculatorProcessTree {
               }
               }
             }
             }
           }
           }
-          if (LOG.isDebugEnabled()) {
-            LOG.debug(procMemInfo.toString());
-          }
+          LOG.debug(procMemInfo);
         }
         }
       }
       }
     }
     }
@@ -427,9 +430,9 @@ public class ProcfsBasedProcessTree extends ResourceCalculatorProcessTree {
     boolean isAvailable = false;
     boolean isAvailable = false;
     for (ProcessInfo p : processTree.values()) {
     for (ProcessInfo p : processTree.values()) {
       if (p != null) {
       if (p != null) {
-        incJiffies += p.getDtime();
         // data is available
         // data is available
         isAvailable = true;
         isAvailable = true;
+        incJiffies += p.getDtime();
       }
       }
     }
     }
     if (isAvailable) {
     if (isAvailable) {
@@ -481,21 +484,17 @@ public class ProcfsBasedProcessTree extends ResourceCalculatorProcessTree {
    * Get the list of all processes in the system.
    * Get the list of all processes in the system.
    */
    */
   private List<String> getProcessList() {
   private List<String> getProcessList() {
-    List<String> processList = new ArrayList<String>();
-    String[] processDirs = (new File(procfsDir)).list();
-    if (processDirs != null) {
-      for (String dir : processDirs) {
-        Matcher m = numberPattern.matcher(dir);
-        if (!m.matches()) {
-          continue;
-        }
-        try {
-          if ((new File(procfsDir, dir)).isDirectory()) {
-            processList.add(dir);
-          }
-        } catch (SecurityException s) {
-          // skip this process
-        }
+    List<String> processList = Collections.emptyList();
+    FileFilter procListFileFilter = new AndFileFilter(
+        DirectoryFileFilter.INSTANCE, new RegexFileFilter(numberPattern));
+
+    File dir = new File(procfsDir);
+    File[] processDirs = dir.listFiles(procListFileFilter);
+
+    if (ArrayUtils.isNotEmpty(processDirs)) {
+      processList = new ArrayList<String>(processDirs.length);
+      for (File processDir : processDirs) {
+        processList.add(processDir.getName());
       }
       }
     }
     }
     return processList;
     return processList;
@@ -547,7 +546,7 @@ public class ProcfsBasedProcessTree extends ResourceCalculatorProcessTree {
         ret = null;
         ret = null;
       }
       }
     } catch (IOException io) {
     } catch (IOException io) {
-      LOG.warn("Error reading the stream " + io);
+      LOG.warn("Error reading the stream", io);
       ret = null;
       ret = null;
     } finally {
     } finally {
       // Close the streams
       // Close the streams
@@ -556,10 +555,10 @@ public class ProcfsBasedProcessTree extends ResourceCalculatorProcessTree {
         try {
         try {
           in.close();
           in.close();
         } catch (IOException i) {
         } catch (IOException i) {
-          LOG.warn("Error closing the stream " + in);
+          LOG.warn("Error closing the stream", i);
         }
         }
       } catch (IOException i) {
       } catch (IOException i) {
-        LOG.warn("Error closing the stream " + fReader);
+        LOG.warn("Error closing the stream", i);
       }
       }
     }
     }
 
 
@@ -729,14 +728,14 @@ public class ProcfsBasedProcessTree extends ResourceCalculatorProcessTree {
           ret = "N/A";
           ret = "N/A";
         } else {
         } else {
           ret = ret.replace('\0', ' '); // Replace each null char with a space
           ret = ret.replace('\0', ' '); // Replace each null char with a space
-          if (ret.equals("")) {
+          if (ret.isEmpty()) {
             // The cmdline might be empty because the process is swapped out or
             // The cmdline might be empty because the process is swapped out or
             // is a zombie.
             // is a zombie.
             ret = "N/A";
             ret = "N/A";
           }
           }
         }
         }
       } catch (IOException io) {
       } catch (IOException io) {
-        LOG.warn("Error reading the stream " + io);
+        LOG.warn("Error reading the stream", io);
         ret = "N/A";
         ret = "N/A";
       } finally {
       } finally {
         // Close the streams
         // Close the streams
@@ -745,10 +744,10 @@ public class ProcfsBasedProcessTree extends ResourceCalculatorProcessTree {
           try {
           try {
             in.close();
             in.close();
           } catch (IOException i) {
           } catch (IOException i) {
-            LOG.warn("Error closing the stream " + in);
+            LOG.warn("Error closing the stream", i);
           }
           }
         } catch (IOException i) {
         } catch (IOException i) {
-          LOG.warn("Error closing the stream " + fReader);
+          LOG.warn("Error closing the stream", i);
         }
         }
       }
       }
 
 
@@ -805,11 +804,11 @@ public class ProcfsBasedProcessTree extends ResourceCalculatorProcessTree {
         }
         }
       }
       }
     } catch (FileNotFoundException f) {
     } catch (FileNotFoundException f) {
-      LOG.error(f.getMessage());
+      LOG.error(f);
     } catch (IOException e) {
     } catch (IOException e) {
-      LOG.error(e.getMessage());
+      LOG.error(e);
     } catch (Throwable t) {
     } catch (Throwable t) {
-      LOG.error(t.getMessage());
+      LOG.error(t);
     } finally {
     } finally {
       IOUtils.closeQuietly(in);
       IOUtils.closeQuietly(in);
     }
     }
@@ -839,7 +838,7 @@ public class ProcfsBasedProcessTree extends ResourceCalculatorProcessTree {
       StringBuilder sb = new StringBuilder();
       StringBuilder sb = new StringBuilder();
       for (ProcessSmapMemoryInfo info : memoryInfoList) {
       for (ProcessSmapMemoryInfo info : memoryInfoList) {
         sb.append("\n");
         sb.append("\n");
-        sb.append(info.toString());
+        sb.append(info);
       }
       }
       return sb.toString();
       return sb.toString();
     }
     }

+ 4 - 11
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestProcfsBasedProcessTree.java

@@ -118,7 +118,6 @@ public class TestProcfsBasedProcessTree {
   }
   }
 
 
   @Test(timeout = 30000)
   @Test(timeout = 30000)
-  @SuppressWarnings("deprecation")
   public void testProcessTree() throws Exception {
   public void testProcessTree() throws Exception {
     try {
     try {
       Assert.assertTrue(ProcfsBasedProcessTree.isAvailable());
       Assert.assertTrue(ProcfsBasedProcessTree.isAvailable());
@@ -163,7 +162,7 @@ public class TestProcfsBasedProcessTree {
     LOG.info("Root process pid: " + pid);
     LOG.info("Root process pid: " + pid);
     ProcfsBasedProcessTree p = createProcessTree(pid);
     ProcfsBasedProcessTree p = createProcessTree(pid);
     p.updateProcessTree(); // initialize
     p.updateProcessTree(); // initialize
-    LOG.info("ProcessTree: " + p.toString());
+    LOG.info("ProcessTree: " + p);
 
 
     File leaf = new File(lowestDescendant);
     File leaf = new File(lowestDescendant);
     // wait till lowest descendant process of Rougue Task starts execution
     // wait till lowest descendant process of Rougue Task starts execution
@@ -176,7 +175,7 @@ public class TestProcfsBasedProcessTree {
     }
     }
 
 
     p.updateProcessTree(); // reconstruct
     p.updateProcessTree(); // reconstruct
-    LOG.info("ProcessTree: " + p.toString());
+    LOG.info("ProcessTree: " + p);
 
 
     // Verify the orphaned pid is In process tree
     // Verify the orphaned pid is In process tree
     String lostpid = getPidFromPidFile(lostDescendant);
     String lostpid = getPidFromPidFile(lostDescendant);
@@ -395,7 +394,6 @@ public class TestProcfsBasedProcessTree {
    *           files.
    *           files.
    */
    */
   @Test(timeout = 30000)
   @Test(timeout = 30000)
-  @SuppressWarnings("deprecation")
   public void testCpuAndMemoryForProcessTree() throws IOException {
   public void testCpuAndMemoryForProcessTree() throws IOException {
 
 
     // test processes
     // test processes
@@ -908,13 +906,8 @@ public class TestProcfsBasedProcessTree {
       throws IOException {
       throws IOException {
     for (String pid : pids) {
     for (String pid : pids) {
       File pidDir = new File(procfsRootDir, pid);
       File pidDir = new File(procfsRootDir, pid);
-      pidDir.mkdir();
-      if (!pidDir.exists()) {
-        throw new IOException("couldn't make process directory under "
-            + "fake procfs");
-      } else {
-        LOG.info("created pid dir");
-      }
+      FileUtils.forceMkdir(pidDir);
+      LOG.info("created pid dir: " + pidDir);
     }
     }
   }
   }
 
 

+ 1 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java

@@ -451,6 +451,7 @@ public class NodeManager extends CompositeService
     // so that we make sure everything is up before registering with RM. 
     // so that we make sure everything is up before registering with RM. 
     addService(nodeStatusUpdater);
     addService(nodeStatusUpdater);
     ((NMContext) context).setNodeStatusUpdater(nodeStatusUpdater);
     ((NMContext) context).setNodeStatusUpdater(nodeStatusUpdater);
+    nmStore.setNodeStatusUpdater(nodeStatusUpdater);
 
 
     super.serviceInit(conf);
     super.serviceInit(conf);
     // TODO add local dirs to del
     // TODO add local dirs to del

+ 4 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java

@@ -59,6 +59,8 @@ public class ContainersMonitorImpl extends AbstractService implements
 
 
   private final static Logger LOG =
   private final static Logger LOG =
        LoggerFactory.getLogger(ContainersMonitorImpl.class);
        LoggerFactory.getLogger(ContainersMonitorImpl.class);
+  private final static Logger AUDITLOG =
+       LoggerFactory.getLogger(ContainersMonitorImpl.class.getName()+".audit");
 
 
   private long monitoringInterval;
   private long monitoringInterval;
   private MonitoringThread monitoringThread;
   private MonitoringThread monitoringThread;
@@ -595,8 +597,8 @@ public class ContainersMonitorImpl extends AbstractService implements
               * maxVCoresAllottedForContainers /nodeCpuPercentageForYARN);
               * maxVCoresAllottedForContainers /nodeCpuPercentageForYARN);
       long vmemLimit = ptInfo.getVmemLimit();
       long vmemLimit = ptInfo.getVmemLimit();
       long pmemLimit = ptInfo.getPmemLimit();
       long pmemLimit = ptInfo.getPmemLimit();
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(String.format(
+      if (AUDITLOG.isDebugEnabled()) {
+        AUDITLOG.debug(String.format(
                 "Memory usage of ProcessTree %s for container-id %s: ",
                 "Memory usage of ProcessTree %s for container-id %s: ",
                 pId, containerId.toString()) +
                 pId, containerId.toString()) +
                 formatUsageString(
                 formatUsageString(

+ 3 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/metrics/NodeManagerMetrics.java

@@ -99,7 +99,7 @@ public class NodeManagerMetrics {
   private long availableMB;
   private long availableMB;
   private long allocatedOpportunisticMB;
   private long allocatedOpportunisticMB;
 
 
-  public NodeManagerMetrics(JvmMetrics jvmMetrics) {
+  private NodeManagerMetrics(JvmMetrics jvmMetrics) {
     this.jvmMetrics = jvmMetrics;
     this.jvmMetrics = jvmMetrics;
   }
   }
 
 
@@ -107,8 +107,8 @@ public class NodeManagerMetrics {
     return create(DefaultMetricsSystem.instance());
     return create(DefaultMetricsSystem.instance());
   }
   }
 
 
-  static NodeManagerMetrics create(MetricsSystem ms) {
-    JvmMetrics jm = JvmMetrics.create("NodeManager", null, ms);
+  private static NodeManagerMetrics create(MetricsSystem ms) {
+    JvmMetrics jm = JvmMetrics.initSingleton("NodeManager", null);
     return ms.register(new NodeManagerMetrics(jm));
     return ms.register(new NodeManagerMetrics(jm));
   }
   }
 
 

+ 72 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMLeveldbStateStoreService.java

@@ -47,6 +47,7 @@ import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainerRequestProto
 import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
 import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
 import org.apache.hadoop.yarn.server.api.records.MasterKey;
 import org.apache.hadoop.yarn.server.api.records.MasterKey;
 import org.apache.hadoop.yarn.server.api.records.impl.pb.MasterKeyPBImpl;
 import org.apache.hadoop.yarn.server.api.records.impl.pb.MasterKeyPBImpl;
+import org.apache.hadoop.yarn.server.nodemanager.NodeStatusUpdater;
 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.ResourceMappings;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ResourceMappings;
 import org.apache.hadoop.yarn.server.records.Version;
 import org.apache.hadoop.yarn.server.records.Version;
@@ -155,6 +156,7 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
 
 
   private DB db;
   private DB db;
   private boolean isNewlyCreated;
   private boolean isNewlyCreated;
+  private boolean isHealthy;
   private Timer compactionTimer;
   private Timer compactionTimer;
 
 
   /**
   /**
@@ -169,6 +171,8 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
 
 
   @Override
   @Override
   protected void startStorage() throws IOException {
   protected void startStorage() throws IOException {
+    // Assume that we're healthy when we start
+    isHealthy = true;
   }
   }
 
 
   @Override
   @Override
@@ -187,6 +191,36 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
     return isNewlyCreated;
     return isNewlyCreated;
   }
   }
 
 
+  /**
+   * If the state store throws an error after recovery has been performed
+   * then we can not trust it any more to reflect the NM state. We need to
+   * mark the store and node unhealthy.
+   * Errors during the recovery will cause a service failure and thus a NM
+   * start failure. Do not need to mark the store unhealthy for those.
+   * @param dbErr Exception
+   */
+  private void markStoreUnHealthy(DBException dbErr) {
+    // Always log the error here, we might not see the error in the caller
+    LOG.error("Statestore exception: ", dbErr);
+    // We have already been marked unhealthy so no need to do it again.
+    if (!isHealthy) {
+      return;
+    }
+    // Mark unhealthy, an out of band heartbeat will be sent and the state
+    // will remain unhealthy (not recoverable).
+    // No need to close the store: does not make any difference at this point.
+    isHealthy = false;
+    // We could get here before the nodeStatusUpdater is set
+    NodeStatusUpdater nsu = getNodeStatusUpdater();
+    if (nsu != null) {
+      nsu.reportException(dbErr);
+    }
+  }
+
+  @VisibleForTesting
+  boolean isHealthy() {
+    return isHealthy;
+  }
 
 
   @Override
   @Override
   public List<RecoveredContainerState> loadContainersState()
   public List<RecoveredContainerState> loadContainersState()
@@ -354,6 +388,7 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
         batch.close();
         batch.close();
       }
       }
     } catch (DBException e) {
     } catch (DBException e) {
+      markStoreUnHealthy(e);
       throw new IOException(e);
       throw new IOException(e);
     }
     }
   }
   }
@@ -378,6 +413,7 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
     try {
     try {
       db.put(bytes(key), EMPTY_VALUE);
       db.put(bytes(key), EMPTY_VALUE);
     } catch (DBException e) {
     } catch (DBException e) {
+      markStoreUnHealthy(e);
       throw new IOException(e);
       throw new IOException(e);
     }
     }
   }
   }
@@ -393,6 +429,7 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
     try {
     try {
       db.delete(bytes(key));
       db.delete(bytes(key));
     } catch (DBException e) {
     } catch (DBException e) {
+      markStoreUnHealthy(e);
       throw new IOException(e);
       throw new IOException(e);
     }
     }
   }
   }
@@ -408,6 +445,7 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
     try {
     try {
       db.put(bytes(key), EMPTY_VALUE);
       db.put(bytes(key), EMPTY_VALUE);
     } catch (DBException e) {
     } catch (DBException e) {
+      markStoreUnHealthy(e);
       throw new IOException(e);
       throw new IOException(e);
     }
     }
   }
   }
@@ -424,6 +462,7 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
     try {
     try {
       db.delete(bytes(key));
       db.delete(bytes(key));
     } catch (DBException e) {
     } catch (DBException e) {
+      markStoreUnHealthy(e);
       throw new IOException(e);
       throw new IOException(e);
     }
     }
   }
   }
@@ -441,6 +480,7 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
     try {
     try {
       db.put(bytes(key), bytes(diagnostics.toString()));
       db.put(bytes(key), bytes(diagnostics.toString()));
     } catch (DBException e) {
     } catch (DBException e) {
+      markStoreUnHealthy(e);
       throw new IOException(e);
       throw new IOException(e);
     }
     }
   }
   }
@@ -459,6 +499,7 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
     try {
     try {
       db.put(bytes(key), EMPTY_VALUE);
       db.put(bytes(key), EMPTY_VALUE);
     } catch (DBException e) {
     } catch (DBException e) {
+      markStoreUnHealthy(e);
       throw new IOException(e);
       throw new IOException(e);
     }
     }
   }
   }
@@ -488,6 +529,7 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
         batch.close();
         batch.close();
       }
       }
     } catch (DBException e) {
     } catch (DBException e) {
+      markStoreUnHealthy(e);
       throw new IOException(e);
       throw new IOException(e);
     }
     }
   }
   }
@@ -504,6 +546,7 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
     try {
     try {
       db.put(bytes(key), EMPTY_VALUE);
       db.put(bytes(key), EMPTY_VALUE);
     } catch (DBException e) {
     } catch (DBException e) {
+      markStoreUnHealthy(e);
       throw new IOException(e);
       throw new IOException(e);
     }
     }
   }
   }
@@ -520,6 +563,7 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
     try {
     try {
       db.put(bytes(key), bytes(Integer.toString(exitCode)));
       db.put(bytes(key), bytes(Integer.toString(exitCode)));
     } catch (DBException e) {
     } catch (DBException e) {
+      markStoreUnHealthy(e);
       throw new IOException(e);
       throw new IOException(e);
     }
     }
   }
   }
@@ -532,6 +576,7 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
     try {
     try {
       db.put(bytes(key), bytes(Integer.toString(remainingRetryAttempts)));
       db.put(bytes(key), bytes(Integer.toString(remainingRetryAttempts)));
     } catch (DBException e) {
     } catch (DBException e) {
+      markStoreUnHealthy(e);
       throw new IOException(e);
       throw new IOException(e);
     }
     }
   }
   }
@@ -544,6 +589,7 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
     try {
     try {
       db.put(bytes(key), bytes(workDir));
       db.put(bytes(key), bytes(workDir));
     } catch (DBException e) {
     } catch (DBException e) {
+      markStoreUnHealthy(e);
       throw new IOException(e);
       throw new IOException(e);
     }
     }
   }
   }
@@ -556,6 +602,7 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
     try {
     try {
       db.put(bytes(key), bytes(logDir));
       db.put(bytes(key), bytes(logDir));
     } catch (DBException e) {
     } catch (DBException e) {
+      markStoreUnHealthy(e);
       throw new IOException(e);
       throw new IOException(e);
     }
     }
   }
   }
@@ -589,6 +636,7 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
         batch.close();
         batch.close();
       }
       }
     } catch (DBException e) {
     } catch (DBException e) {
+      markStoreUnHealthy(e);
       throw new IOException(e);
       throw new IOException(e);
     }
     }
   }
   }
@@ -638,6 +686,7 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
     try {
     try {
       db.put(bytes(key), p.toByteArray());
       db.put(bytes(key), p.toByteArray());
     } catch (DBException e) {
     } catch (DBException e) {
+      markStoreUnHealthy(e);
       throw new IOException(e);
       throw new IOException(e);
     }
     }
   }
   }
@@ -659,6 +708,7 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
         batch.close();
         batch.close();
       }
       }
     } catch (DBException e) {
     } catch (DBException e) {
+      markStoreUnHealthy(e);
       throw new IOException(e);
       throw new IOException(e);
     }
     }
   }
   }
@@ -815,6 +865,7 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
     try {
     try {
       db.put(bytes(key), proto.toByteArray());
       db.put(bytes(key), proto.toByteArray());
     } catch (DBException e) {
     } catch (DBException e) {
+      markStoreUnHealthy(e);
       throw new IOException(e);
       throw new IOException(e);
     }
     }
   }
   }
@@ -838,6 +889,7 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
         batch.close();
         batch.close();
       }
       }
     } catch (DBException e) {
     } catch (DBException e) {
+      markStoreUnHealthy(e);
       throw new IOException(e);
       throw new IOException(e);
     }
     }
   }
   }
@@ -861,6 +913,7 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
         batch.close();
         batch.close();
       }
       }
     } catch (DBException e) {
     } catch (DBException e) {
+      markStoreUnHealthy(e);
       throw new IOException(e);
       throw new IOException(e);
     }
     }
   }
   }
@@ -926,6 +979,7 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
     try {
     try {
       db.put(bytes(key), taskProto.toByteArray());
       db.put(bytes(key), taskProto.toByteArray());
     } catch (DBException e) {
     } catch (DBException e) {
+      markStoreUnHealthy(e);
       throw new IOException(e);
       throw new IOException(e);
     }
     }
   }
   }
@@ -936,6 +990,7 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
     try {
     try {
       db.delete(bytes(key));
       db.delete(bytes(key));
     } catch (DBException e) {
     } catch (DBException e) {
+      markStoreUnHealthy(e);
       throw new IOException(e);
       throw new IOException(e);
     }
     }
   }
   }
@@ -1009,6 +1064,7 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
     try {
     try {
       db.delete(bytes(key));
       db.delete(bytes(key));
     } catch (DBException e) {
     } catch (DBException e) {
+      markStoreUnHealthy(e);
       throw new IOException(e);
       throw new IOException(e);
     }
     }
   }
   }
@@ -1023,6 +1079,7 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
     try {
     try {
       db.put(bytes(dbKey), pb.getProto().toByteArray());
       db.put(bytes(dbKey), pb.getProto().toByteArray());
     } catch (DBException e) {
     } catch (DBException e) {
+      markStoreUnHealthy(e);
       throw new IOException(e);
       throw new IOException(e);
     }
     }
   }
   }
@@ -1096,6 +1153,7 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
     try {
     try {
       db.put(bytes(key), bytes(expTime.toString()));
       db.put(bytes(key), bytes(expTime.toString()));
     } catch (DBException e) {
     } catch (DBException e) {
+      markStoreUnHealthy(e);
       throw new IOException(e);
       throw new IOException(e);
     }
     }
   }
   }
@@ -1107,6 +1165,7 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
     try {
     try {
       db.delete(bytes(key));
       db.delete(bytes(key));
     } catch (DBException e) {
     } catch (DBException e) {
+      markStoreUnHealthy(e);
       throw new IOException(e);
       throw new IOException(e);
     }
     }
   }
   }
@@ -1157,6 +1216,7 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
     try {
     try {
       db.put(bytes(key), proto.toByteArray());
       db.put(bytes(key), proto.toByteArray());
     } catch (DBException e) {
     } catch (DBException e) {
+      markStoreUnHealthy(e);
       throw new IOException(e);
       throw new IOException(e);
     }
     }
   }
   }
@@ -1167,6 +1227,7 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
     try {
     try {
       db.delete(bytes(key));
       db.delete(bytes(key));
     } catch (DBException e) {
     } catch (DBException e) {
+      markStoreUnHealthy(e);
       throw new IOException(e);
       throw new IOException(e);
     }
     }
   }
   }
@@ -1198,6 +1259,7 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
         batch.close();
         batch.close();
       }
       }
     } catch (DBException e) {
     } catch (DBException e) {
+      markStoreUnHealthy(e);
       throw new IOException(e);
       throw new IOException(e);
     }
     }
 
 
@@ -1361,6 +1423,7 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
       try {
       try {
         db.delete(bytes(dbkey));
         db.delete(bytes(dbkey));
       } catch (DBException e) {
       } catch (DBException e) {
+        markStoreUnHealthy(e);
         throw new IOException(e);
         throw new IOException(e);
       }
       }
       return;
       return;
@@ -1375,6 +1438,7 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
     try {
     try {
       db.put(bytes(fullkey), data);
       db.put(bytes(fullkey), data);
     } catch (DBException e) {
     } catch (DBException e) {
+      markStoreUnHealthy(e);
       throw new IOException(e);
       throw new IOException(e);
     }
     }
   }
   }
@@ -1386,6 +1450,7 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
     try {
     try {
       db.delete(bytes(fullkey));
       db.delete(bytes(fullkey));
     } catch (DBException e) {
     } catch (DBException e) {
+      markStoreUnHealthy(e);
       throw new IOException(e);
       throw new IOException(e);
     }
     }
   }
   }
@@ -1409,6 +1474,7 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
         candidates.add(key);
         candidates.add(key);
       }
       }
     } catch (DBException e) {
     } catch (DBException e) {
+      markStoreUnHealthy(e);
       throw new IOException(e);
       throw new IOException(e);
     } finally {
     } finally {
       if (iter != null) {
       if (iter != null) {
@@ -1422,6 +1488,7 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
         db.delete(bytes(key));
         db.delete(bytes(key));
       }
       }
     } catch (DBException e) {
     } catch (DBException e) {
+      markStoreUnHealthy(e);
       throw new IOException(e);
       throw new IOException(e);
     }
     }
   }
   }
@@ -1555,6 +1622,11 @@ public class NMLeveldbStateStoreService extends NMStateStoreService {
     return db;
     return db;
   }
   }
 
 
+  @VisibleForTesting
+  void setDB(DB testDb) {
+    this.db = testDb;
+  }
+
   /**
   /**
    * 1) Versioning scheme: major.minor. For e.g. 1.0, 1.1, 1.2...1.25, 2.0 etc.
    * 1) Versioning scheme: major.minor. For e.g. 1.0, 1.1, 1.2...1.25, 2.0 etc.
    * 2) Any incompatible change of state-store is a major upgrade, and any
    * 2) Any incompatible change of state-store is a major upgrade, and any

+ 11 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/recovery/NMStateStoreService.java

@@ -44,6 +44,7 @@ import org.apache.hadoop.yarn.proto.YarnServerNodemanagerRecoveryProtos.Localize
 import org.apache.hadoop.yarn.proto.YarnServerNodemanagerRecoveryProtos.LogDeleterProto;
 import org.apache.hadoop.yarn.proto.YarnServerNodemanagerRecoveryProtos.LogDeleterProto;
 import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
 import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
 import org.apache.hadoop.yarn.server.api.records.MasterKey;
 import org.apache.hadoop.yarn.server.api.records.MasterKey;
+import org.apache.hadoop.yarn.server.nodemanager.NodeStatusUpdater;
 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.ResourceMappings;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ResourceMappings;
 
 
@@ -51,10 +52,20 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Reso
 @Unstable
 @Unstable
 public abstract class NMStateStoreService extends AbstractService {
 public abstract class NMStateStoreService extends AbstractService {
 
 
+  private NodeStatusUpdater nodeStatusUpdater = null;
+
   public NMStateStoreService(String name) {
   public NMStateStoreService(String name) {
     super(name);
     super(name);
   }
   }
 
 
+  protected NodeStatusUpdater getNodeStatusUpdater() {
+    return nodeStatusUpdater;
+  }
+
+  public void setNodeStatusUpdater(NodeStatusUpdater nodeStatusUpdater) {
+    this.nodeStatusUpdater = nodeStatusUpdater;
+  }
+
   public static class RecoveredApplicationsState {
   public static class RecoveredApplicationsState {
     List<ContainerManagerApplicationProto> applications;
     List<ContainerManagerApplicationProto> applications;
 
 

+ 275 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/TestContainersLauncher.java

@@ -0,0 +1,275 @@
+/* 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.launcher;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.SignalContainerCommand;
+import org.apache.hadoop.yarn.event.AsyncDispatcher;
+import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
+import org.apache.hadoop.yarn.server.nodemanager.LocalDirsHandlerService;
+import org.apache.hadoop.yarn.server.nodemanager.NodeManager;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationImpl;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerImpl;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.InjectMocks;
+import org.mockito.Mock;
+import org.mockito.Mockito;
+import org.mockito.MockitoAnnotations;
+import org.mockito.internal.util.reflection.Whitebox;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutorService;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Mockito.*;
+
+/**
+ * Tests to verify all the Container's Launcher Events in
+ * {@link ContainersLauncher} are handled as expected.
+ */
+public class TestContainersLauncher {
+
+  @Mock
+  private ApplicationImpl app1;
+
+  @Mock
+  private ContainerImpl container;
+
+  @Mock
+  private ApplicationId appId;
+
+  @Mock
+  private ApplicationAttemptId appAttemptId;
+
+  @Mock
+  private ContainerId containerId;
+
+  @Mock
+  private ContainersLauncherEvent event;
+
+  @Mock
+  private NodeManager.NMContext context;
+
+  @Mock
+  private AsyncDispatcher dispatcher;
+
+  @Mock
+  private ContainerExecutor exec;
+
+  @Mock
+  private LocalDirsHandlerService dirsHandler;
+
+  @Mock
+  private ContainerManagerImpl containerManager;
+
+  @Mock
+  private ExecutorService containerLauncher;
+
+  @Mock
+  private Configuration conf;
+
+  @Mock
+  private ContainerLaunch containerLaunch;
+
+  @InjectMocks
+  private ContainersLauncher tempContainersLauncher = new ContainersLauncher(
+      context, dispatcher, exec, dirsHandler, containerManager);
+
+  private ContainersLauncher spy;
+
+  @Before
+  public void setup() throws IllegalArgumentException, IllegalAccessException {
+    MockitoAnnotations.initMocks(this);
+    ConcurrentMap<ApplicationId, Application> applications =
+        new ConcurrentHashMap<>();
+    applications.put(appId, app1);
+    spy = spy(tempContainersLauncher);
+    conf = doReturn(conf).when(spy).getConfig();
+    when(event.getContainer()).thenReturn(container);
+    when(container.getContainerId()).thenReturn(containerId);
+    when(containerId.getApplicationAttemptId()).thenReturn(appAttemptId);
+    when(containerId.getApplicationAttemptId().getApplicationId())
+        .thenReturn(appId);
+    when(context.getApplications()).thenReturn(applications);
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testLaunchContainerEvent()
+      throws IllegalArgumentException, IllegalAccessException {
+    Map<ContainerId, ContainerLaunch> dummyMap =
+        (Map<ContainerId, ContainerLaunch>) Whitebox.getInternalState(spy,
+            "running");
+    when(event.getType())
+        .thenReturn(ContainersLauncherEventType.LAUNCH_CONTAINER);
+    assertEquals(0, dummyMap.size());
+    spy.handle(event);
+    assertEquals(1, dummyMap.size());
+    Mockito.verify(containerLauncher, Mockito.times(1))
+        .submit(Mockito.any(ContainerLaunch.class));
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testRelaunchContainerEvent()
+      throws IllegalArgumentException, IllegalAccessException {
+    Map<ContainerId, ContainerLaunch> dummyMap =
+        (Map<ContainerId, ContainerLaunch>) Whitebox.getInternalState(spy,
+            "running");
+    when(event.getType())
+        .thenReturn(ContainersLauncherEventType.RELAUNCH_CONTAINER);
+    assertEquals(0, dummyMap.size());
+    spy.handle(event);
+    assertEquals(1, dummyMap.size());
+    Mockito.verify(containerLauncher, Mockito.times(1))
+        .submit(Mockito.any(ContainerRelaunch.class));
+    for (ContainerId cid : dummyMap.keySet()) {
+      Object o = dummyMap.get(cid);
+      assertEquals(true, (o instanceof ContainerRelaunch));
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testRecoverContainerEvent()
+      throws IllegalArgumentException, IllegalAccessException {
+    Map<ContainerId, ContainerLaunch> dummyMap =
+        (Map<ContainerId, ContainerLaunch>) Whitebox.getInternalState(spy,
+            "running");
+    when(event.getType())
+        .thenReturn(ContainersLauncherEventType.RECOVER_CONTAINER);
+    assertEquals(0, dummyMap.size());
+    spy.handle(event);
+    assertEquals(1, dummyMap.size());
+    Mockito.verify(containerLauncher, Mockito.times(1))
+        .submit(Mockito.any(RecoveredContainerLaunch.class));
+    for (ContainerId cid : dummyMap.keySet()) {
+      Object o = dummyMap.get(cid);
+      assertEquals(true, (o instanceof RecoveredContainerLaunch));
+    }
+  }
+
+  @Test
+  public void testRecoverPausedContainerEvent()
+      throws IllegalArgumentException, IllegalAccessException {
+    when(event.getType())
+        .thenReturn(ContainersLauncherEventType.RECOVER_PAUSED_CONTAINER);
+    spy.handle(event);
+    Mockito.verify(containerLauncher, Mockito.times(1))
+        .submit(Mockito.any(RecoverPausedContainerLaunch.class));
+  }
+
+  @Test
+  public void testCleanupContainerEvent()
+      throws IllegalArgumentException, IllegalAccessException, IOException {
+    Map<ContainerId, ContainerLaunch> dummyMap = Collections
+        .synchronizedMap(new HashMap<ContainerId, ContainerLaunch>());
+    dummyMap.put(containerId, containerLaunch);
+    Whitebox.setInternalState(spy, "running", dummyMap);
+    when(event.getType())
+        .thenReturn(ContainersLauncherEventType.CLEANUP_CONTAINER);
+    doNothing().when(containerLaunch).cleanupContainer();
+    spy.handle(event);
+    assertEquals(0, dummyMap.size());
+    Mockito.verify(containerLaunch, Mockito.times(1)).cleanupContainer();
+  }
+
+  @Test
+  public void testCleanupContainerForReINITEvent()
+      throws IllegalArgumentException, IllegalAccessException, IOException {
+    Map<ContainerId, ContainerLaunch> dummyMap = Collections
+        .synchronizedMap(new HashMap<ContainerId, ContainerLaunch>());
+    dummyMap.put(containerId, containerLaunch);
+    Whitebox.setInternalState(spy, "running", dummyMap);
+    when(event.getType())
+        .thenReturn(ContainersLauncherEventType.CLEANUP_CONTAINER_FOR_REINIT);
+    doNothing().when(containerLaunch).cleanupContainer();
+    spy.handle(event);
+    assertEquals(0, dummyMap.size());
+    Mockito.verify(containerLaunch, Mockito.times(1)).cleanupContainer();
+  }
+
+  @Test
+  public void testSignalContainerEvent()
+      throws IllegalArgumentException, IllegalAccessException, IOException {
+    Map<ContainerId, ContainerLaunch> dummyMap = Collections
+        .synchronizedMap(new HashMap<ContainerId, ContainerLaunch>());
+    dummyMap.put(containerId, containerLaunch);
+
+    SignalContainersLauncherEvent dummyEvent =
+        mock(SignalContainersLauncherEvent.class);
+    when(dummyEvent.getContainer()).thenReturn(container);
+    when(container.getContainerId()).thenReturn(containerId);
+    when(containerId.getApplicationAttemptId()).thenReturn(appAttemptId);
+    when(containerId.getApplicationAttemptId().getApplicationId())
+        .thenReturn(appId);
+
+    Whitebox.setInternalState(spy, "running", dummyMap);
+    when(dummyEvent.getType())
+        .thenReturn(ContainersLauncherEventType.SIGNAL_CONTAINER);
+    when(dummyEvent.getCommand())
+        .thenReturn(SignalContainerCommand.GRACEFUL_SHUTDOWN);
+    doNothing().when(containerLaunch)
+        .signalContainer(SignalContainerCommand.GRACEFUL_SHUTDOWN);
+    spy.handle(dummyEvent);
+    assertEquals(1, dummyMap.size());
+    Mockito.verify(containerLaunch, Mockito.times(1))
+        .signalContainer(SignalContainerCommand.GRACEFUL_SHUTDOWN);
+  }
+
+  @Test
+  public void testPauseContainerEvent()
+      throws IllegalArgumentException, IllegalAccessException, IOException {
+    Map<ContainerId, ContainerLaunch> dummyMap = Collections
+        .synchronizedMap(new HashMap<ContainerId, ContainerLaunch>());
+    dummyMap.put(containerId, containerLaunch);
+    Whitebox.setInternalState(spy, "running", dummyMap);
+    when(event.getType())
+        .thenReturn(ContainersLauncherEventType.PAUSE_CONTAINER);
+    doNothing().when(containerLaunch).pauseContainer();
+    spy.handle(event);
+    assertEquals(1, dummyMap.size());
+    Mockito.verify(containerLaunch, Mockito.times(1)).pauseContainer();
+  }
+
+  @Test
+  public void testResumeContainerEvent()
+      throws IllegalArgumentException, IllegalAccessException, IOException {
+    Map<ContainerId, ContainerLaunch> dummyMap = Collections
+        .synchronizedMap(new HashMap<ContainerId, ContainerLaunch>());
+    dummyMap.put(containerId, containerLaunch);
+    Whitebox.setInternalState(spy, "running", dummyMap);
+    when(event.getType())
+        .thenReturn(ContainersLauncherEventType.RESUME_CONTAINER);
+    doNothing().when(containerLaunch).resumeContainer();
+    spy.handle(event);
+    assertEquals(1, dummyMap.size());
+    Mockito.verify(containerLaunch, Mockito.times(1)).resumeContainer();
+  }
+}

+ 23 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/metrics/TestNodeManagerMetrics.java

@@ -19,19 +19,40 @@ package org.apache.hadoop.yarn.server.nodemanager.metrics;
 
 
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+import org.apache.hadoop.metrics2.source.JvmMetrics;
 import static org.apache.hadoop.test.MetricsAsserts.*;
 import static org.apache.hadoop.test.MetricsAsserts.*;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.util.Records;
 import org.apache.hadoop.yarn.util.Records;
 
 
+import org.junit.After;
 import org.junit.Assert;
 import org.junit.Assert;
+import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
 
 
 public class TestNodeManagerMetrics {
 public class TestNodeManagerMetrics {
   static final int GiB = 1024; // MiB
   static final int GiB = 1024; // MiB
 
 
-  @Test public void testNames() {
+  private NodeManagerMetrics metrics;
+
+  @Before
+  public void setup() {
     DefaultMetricsSystem.initialize("NodeManager");
     DefaultMetricsSystem.initialize("NodeManager");
-    NodeManagerMetrics metrics = NodeManagerMetrics.create();
+    metrics = NodeManagerMetrics.create();
+  }
+
+  @After
+  public void tearDown() {
+    DefaultMetricsSystem.shutdown();
+  }
+
+  @Test
+  public void testReferenceOfSingletonJvmMetrics()  {
+    JvmMetrics jvmMetrics = JvmMetrics.initSingleton("NodeManagerModule", null);
+    Assert.assertEquals("NodeManagerMetrics should reference the singleton" +
+        " JvmMetrics instance", jvmMetrics, metrics.getJvmMetrics());
+  }
+
+  @Test public void testNames() {
     Resource total = Records.newRecord(Resource.class);
     Resource total = Records.newRecord(Resource.class);
     total.setMemorySize(8*GiB);
     total.setMemorySize(8*GiB);
     total.setVirtualCores(16);
     total.setVirtualCores(16);

+ 35 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/TestNMLeveldbStateStoreService.java

@@ -25,6 +25,7 @@ import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.isNull;
 import static org.mockito.Mockito.isNull;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.timeout;
 import static org.mockito.Mockito.timeout;
@@ -89,10 +90,12 @@ import org.apache.hadoop.yarn.server.security.BaseContainerTokenSecretManager;
 import org.apache.hadoop.yarn.server.security.BaseNMTokenSecretManager;
 import org.apache.hadoop.yarn.server.security.BaseNMTokenSecretManager;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.iq80.leveldb.DB;
 import org.iq80.leveldb.DB;
+import org.iq80.leveldb.DBException;
 import org.junit.After;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
+import org.mockito.Mockito;
 
 
 public class TestNMLeveldbStateStoreService {
 public class TestNMLeveldbStateStoreService {
   private static final File TMP_DIR = new File(
   private static final File TMP_DIR = new File(
@@ -1165,6 +1168,38 @@ public class TestNMLeveldbStateStoreService {
         resourceMappings.getAssignedResources("numa").equals(numaRes));
         resourceMappings.getAssignedResources("numa").equals(numaRes));
   }
   }
 
 
+  @Test
+  public void testStateStoreNodeHealth() throws IOException {
+    // keep the working DB clean, break a temp DB
+    DB keepDB = stateStore.getDB();
+    DB myMocked = mock(DB.class);
+    stateStore.setDB(myMocked);
+
+    ApplicationId appId = ApplicationId.newInstance(1234, 1);
+    ApplicationAttemptId appAttemptId =
+        ApplicationAttemptId.newInstance(appId, 1);
+    DBException toThrow = new DBException();
+    Mockito.doThrow(toThrow).when(myMocked).
+        put(any(byte[].class), any(byte[].class));
+    // write some data
+    try {
+      // chosen a simple method could be any of the "void" methods
+      ContainerId containerId = ContainerId.newContainerId(appAttemptId, 1);
+      stateStore.storeContainerKilled(containerId);
+    } catch (IOException ioErr) {
+      // Cause should be wrapped DBException
+      assertTrue(ioErr.getCause() instanceof DBException);
+      // check the store is marked unhealthy
+      assertFalse("Statestore should have been unhealthy",
+          stateStore.isHealthy());
+      return;
+    } finally {
+      // restore the working DB
+      stateStore.setDB(keepDB);
+    }
+    Assert.fail("Expected exception not thrown");
+  }
+
   private StartContainerRequest storeMockContainer(ContainerId containerId)
   private StartContainerRequest storeMockContainer(ContainerId containerId)
       throws IOException {
       throws IOException {
     // create a container request
     // create a container request

+ 10 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java

@@ -393,10 +393,16 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
     // Verify and get the update application priority and set back to
     // Verify and get the update application priority and set back to
     // submissionContext
     // submissionContext
     UserGroupInformation userUgi = UserGroupInformation.createRemoteUser(user);
     UserGroupInformation userUgi = UserGroupInformation.createRemoteUser(user);
-    Priority appPriority = scheduler.checkAndGetApplicationPriority(
-        submissionContext.getPriority(), userUgi, submissionContext.getQueue(),
-        applicationId);
-    submissionContext.setPriority(appPriority);
+
+    // Application priority needed to be validated only while submitting. During
+    // recovery, validated priority could be recovered from submission context.
+    if (!isRecovery) {
+      Priority appPriority = scheduler.checkAndGetApplicationPriority(
+          submissionContext.getPriority(), userUgi,
+          submissionContext.getQueue(),
+          applicationId);
+      submissionContext.setPriority(appPriority);
+    }
 
 
     // Since FairScheduler queue mapping is done inside scheduler,
     // Since FairScheduler queue mapping is done inside scheduler,
     // if FairScheduler is used and the queue doesn't exist, we should not
     // if FairScheduler is used and the queue doesn't exist, we should not

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

@@ -17,25 +17,15 @@
 */
 */
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
 
 
-import java.io.File;
-import java.io.IOException;
-import java.net.URL;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import javax.xml.parsers.DocumentBuilder;
-import javax.xml.parsers.DocumentBuilderFactory;
-import javax.xml.parsers.ParserConfigurationException;
-
+import com.google.common.annotations.VisibleForTesting;
 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.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.UnsupportedFileSystemException;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueACL;
@@ -45,8 +35,8 @@ import org.apache.hadoop.yarn.security.AccessType;
 import org.apache.hadoop.yarn.security.Permission;
 import org.apache.hadoop.yarn.security.Permission;
 import org.apache.hadoop.yarn.security.PrivilegedEntity;
 import org.apache.hadoop.yarn.security.PrivilegedEntity;
 import org.apache.hadoop.yarn.security.PrivilegedEntity.EntityType;
 import org.apache.hadoop.yarn.security.PrivilegedEntity.EntityType;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.FifoPolicy;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.FifoPolicy;
 import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.SystemClock;
 import org.apache.hadoop.yarn.util.SystemClock;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.apache.hadoop.yarn.util.resource.Resources;
@@ -57,7 +47,17 @@ import org.w3c.dom.NodeList;
 import org.w3c.dom.Text;
 import org.w3c.dom.Text;
 import org.xml.sax.SAXException;
 import org.xml.sax.SAXException;
 
 
-import com.google.common.annotations.VisibleForTesting;
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+import javax.xml.parsers.ParserConfigurationException;
+import java.io.IOException;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
 
 
 @Public
 @Public
 @Unstable
 @Unstable
@@ -77,6 +77,9 @@ public class AllocationFileLoaderService extends AbstractService {
 
 
   public static final long THREAD_JOIN_TIMEOUT_MS = 1000;
   public static final long THREAD_JOIN_TIMEOUT_MS = 1000;
 
 
+  //Permitted allocation file filesystems (case insensitive)
+  private static final String SUPPORTED_FS_REGEX =
+      "(?i)(hdfs)|(file)|(s3a)|(viewfs)";
   private static final String ROOT = "root";
   private static final String ROOT = "root";
   private static final AccessControlList EVERYBODY_ACL =
   private static final AccessControlList EVERYBODY_ACL =
       new AccessControlList("*");
       new AccessControlList("*");
@@ -85,12 +88,14 @@ public class AllocationFileLoaderService extends AbstractService {
 
 
   private final Clock clock;
   private final Clock clock;
 
 
-  private long lastSuccessfulReload; // Last time we successfully reloaded queues
-  private boolean lastReloadAttemptFailed = false;
-  
-  // Path to XML file containing allocations. 
-  private File allocFile;
-  
+  // Last time we successfully reloaded queues
+  private volatile long lastSuccessfulReload;
+  private volatile boolean lastReloadAttemptFailed = false;
+
+  // Path to XML file containing allocations.
+  private Path allocFile;
+  private FileSystem fs;
+
   private Listener reloadListener;
   private Listener reloadListener;
   
   
   @VisibleForTesting
   @VisibleForTesting
@@ -108,19 +113,19 @@ public class AllocationFileLoaderService extends AbstractService {
   public AllocationFileLoaderService(Clock clock) {
   public AllocationFileLoaderService(Clock clock) {
     super(AllocationFileLoaderService.class.getName());
     super(AllocationFileLoaderService.class.getName());
     this.clock = clock;
     this.clock = clock;
-    
   }
   }
   
   
   @Override
   @Override
   public void serviceInit(Configuration conf) throws Exception {
   public void serviceInit(Configuration conf) throws Exception {
     this.allocFile = getAllocationFile(conf);
     this.allocFile = getAllocationFile(conf);
-    if (allocFile != null) {
-      reloadThread = new Thread() {
-        @Override
-        public void run() {
-          while (running) {
+    if(this.allocFile != null) {
+      this.fs = allocFile.getFileSystem(conf);
+      reloadThread = new Thread(() -> {
+        while (running) {
+          try {
             long time = clock.getTime();
             long time = clock.getTime();
-            long lastModified = allocFile.lastModified();
+            long lastModified =
+                fs.getFileStatus(allocFile).getModificationTime();
             if (lastModified > lastSuccessfulReload &&
             if (lastModified > lastSuccessfulReload &&
                 time > lastModified + ALLOC_RELOAD_WAIT_MS) {
                 time > lastModified + ALLOC_RELOAD_WAIT_MS) {
               try {
               try {
@@ -136,19 +141,21 @@ public class AllocationFileLoaderService extends AbstractService {
               if (!lastReloadAttemptFailed) {
               if (!lastReloadAttemptFailed) {
                 LOG.warn("Failed to reload fair scheduler config file because" +
                 LOG.warn("Failed to reload fair scheduler config file because" +
                     " last modified returned 0. File exists: "
                     " last modified returned 0. File exists: "
-                    + allocFile.exists());
+                    + fs.exists(allocFile));
               }
               }
               lastReloadAttemptFailed = true;
               lastReloadAttemptFailed = true;
             }
             }
-            try {
-              Thread.sleep(reloadIntervalMs);
-            } catch (InterruptedException ex) {
-              LOG.info(
-                  "Interrupted while waiting to reload alloc configuration");
-            }
+          } catch (IOException e) {
+            LOG.info("Exception while loading allocation file: " + e);
+          }
+          try {
+            Thread.sleep(reloadIntervalMs);
+          } catch (InterruptedException ex) {
+            LOG.info(
+                "Interrupted while waiting to reload alloc configuration");
           }
           }
         }
         }
-      };
+      });
       reloadThread.setName("AllocationFileReloader");
       reloadThread.setName("AllocationFileReloader");
       reloadThread.setDaemon(true);
       reloadThread.setDaemon(true);
     }
     }
@@ -182,24 +189,31 @@ public class AllocationFileLoaderService extends AbstractService {
    * path is relative, it is searched for in the
    * path is relative, it is searched for in the
    * classpath, but loaded like a regular File.
    * classpath, but loaded like a regular File.
    */
    */
-  public File getAllocationFile(Configuration conf) {
+  public Path getAllocationFile(Configuration conf)
+      throws UnsupportedFileSystemException {
     String allocFilePath = conf.get(FairSchedulerConfiguration.ALLOCATION_FILE,
     String allocFilePath = conf.get(FairSchedulerConfiguration.ALLOCATION_FILE,
         FairSchedulerConfiguration.DEFAULT_ALLOCATION_FILE);
         FairSchedulerConfiguration.DEFAULT_ALLOCATION_FILE);
-    File allocFile = new File(allocFilePath);
-    if (!allocFile.isAbsolute()) {
+    Path allocPath = new Path(allocFilePath);
+    String allocPathScheme = allocPath.toUri().getScheme();
+    if(allocPathScheme != null && !allocPathScheme.matches(SUPPORTED_FS_REGEX)){
+      throw new UnsupportedFileSystemException("Allocation file "
+          + allocFilePath + " uses an unsupported filesystem");
+    } else if (!allocPath.isAbsolute()) {
       URL url = Thread.currentThread().getContextClassLoader()
       URL url = Thread.currentThread().getContextClassLoader()
           .getResource(allocFilePath);
           .getResource(allocFilePath);
       if (url == null) {
       if (url == null) {
         LOG.warn(allocFilePath + " not found on the classpath.");
         LOG.warn(allocFilePath + " not found on the classpath.");
-        allocFile = null;
+        allocPath = null;
       } else if (!url.getProtocol().equalsIgnoreCase("file")) {
       } else if (!url.getProtocol().equalsIgnoreCase("file")) {
         throw new RuntimeException("Allocation file " + url
         throw new RuntimeException("Allocation file " + url
             + " found on the classpath is not on the local filesystem.");
             + " found on the classpath is not on the local filesystem.");
       } else {
       } else {
-        allocFile = new File(url.getPath());
+        allocPath = new Path(url.getProtocol(), null, url.getPath());
       }
       }
+    } else if (allocPath.isAbsoluteAndSchemeAuthorityNull()){
+      allocPath = new Path("file", null, allocFilePath);
     }
     }
-    return allocFile;
+    return allocPath;
   }
   }
   
   
   public synchronized void setReloadListener(Listener reloadListener) {
   public synchronized void setReloadListener(Listener reloadListener) {
@@ -274,7 +288,7 @@ public class AllocationFileLoaderService extends AbstractService {
       DocumentBuilderFactory.newInstance();
       DocumentBuilderFactory.newInstance();
     docBuilderFactory.setIgnoringComments(true);
     docBuilderFactory.setIgnoringComments(true);
     DocumentBuilder builder = docBuilderFactory.newDocumentBuilder();
     DocumentBuilder builder = docBuilderFactory.newDocumentBuilder();
-    Document doc = builder.parse(allocFile);
+    Document doc = builder.parse(fs.open(allocFile));
     Element root = doc.getDocumentElement();
     Element root = doc.getDocumentElement();
     if (!"allocations".equals(root.getTagName()))
     if (!"allocations".equals(root.getTagName()))
       throw new AllocationConfigurationException("Bad fair scheduler config " +
       throw new AllocationConfigurationException("Bad fair scheduler config " +
@@ -437,7 +451,7 @@ public class AllocationFileLoaderService extends AbstractService {
           fairSharePreemptionTimeouts, fairSharePreemptionThresholds, queueAcls,
           fairSharePreemptionTimeouts, fairSharePreemptionThresholds, queueAcls,
           reservationAcls, newPlacementPolicy, configuredQueues,
           reservationAcls, newPlacementPolicy, configuredQueues,
           globalReservationQueueConfig, reservableQueues, nonPreemptableQueues);
           globalReservationQueueConfig, reservableQueues, nonPreemptableQueues);
-    
+
     lastSuccessfulReload = clock.getTime();
     lastSuccessfulReload = clock.getTime();
     lastReloadAttemptFailed = false;
     lastReloadAttemptFailed = false;
 
 

+ 74 - 18
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestAllocationFileLoaderService.java

@@ -17,17 +17,12 @@
 */
 */
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
 
 
-import static org.junit.Assert.*;
-
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.FileWriter;
-import java.io.OutputStreamWriter;
-import java.io.PrintWriter;
-import java.nio.charset.StandardCharsets;
-import java.util.List;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.UnsupportedFileSystemException;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSchedulerConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSchedulerConfiguration;
@@ -38,6 +33,23 @@ import org.apache.hadoop.yarn.util.ControlledClock;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.junit.Test;
 import org.junit.Test;
 
 
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.io.PrintWriter;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
 public class TestAllocationFileLoaderService {
 public class TestAllocationFileLoaderService {
   
   
   final static String TEST_DIR = new File(System.getProperty("test.build.data",
   final static String TEST_DIR = new File(System.getProperty("test.build.data",
@@ -45,16 +57,60 @@ public class TestAllocationFileLoaderService {
 
 
   final static String ALLOC_FILE = new File(TEST_DIR,
   final static String ALLOC_FILE = new File(TEST_DIR,
       "test-queues").getAbsolutePath();
       "test-queues").getAbsolutePath();
-  
+  private static final String TEST_FAIRSCHED_XML = "test-fair-scheduler.xml";
+
   @Test
   @Test
-  public void testGetAllocationFileFromClasspath() {
-    Configuration conf = new Configuration();
-    conf.set(FairSchedulerConfiguration.ALLOCATION_FILE,
-        "test-fair-scheduler.xml");
+  public void testGetAllocationFileFromFileSystem()
+      throws IOException, URISyntaxException {
+    Configuration conf = new YarnConfiguration();
+    File baseDir =
+        new File(TEST_DIR + Path.SEPARATOR + "getAllocHDFS").getAbsoluteFile();
+    FileUtil.fullyDelete(baseDir);
+    conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, baseDir.getAbsolutePath());
+    MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf);
+    MiniDFSCluster hdfsCluster = builder.build();
+    String fsAllocPath = "hdfs://localhost:" + hdfsCluster.getNameNodePort()
+        + Path.SEPARATOR + TEST_FAIRSCHED_XML;
+
+    URL fschedURL = Thread.currentThread().getContextClassLoader()
+        .getResource(TEST_FAIRSCHED_XML);
+    FileSystem fs = FileSystem.get(conf);
+    fs.copyFromLocalFile(new Path(fschedURL.toURI()), new Path(fsAllocPath));
+    conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, fsAllocPath);
+
     AllocationFileLoaderService allocLoader = new AllocationFileLoaderService();
     AllocationFileLoaderService allocLoader = new AllocationFileLoaderService();
-    File allocationFile = allocLoader.getAllocationFile(conf);
-    assertEquals("test-fair-scheduler.xml", allocationFile.getName());
-    assertTrue(allocationFile.exists());
+    Path allocationFile = allocLoader.getAllocationFile(conf);
+    assertEquals(fsAllocPath, allocationFile.toString());
+    assertTrue(fs.exists(allocationFile));
+
+    hdfsCluster.shutdown(true);
+  }
+
+  @Test (expected = UnsupportedFileSystemException.class)
+  public void testDenyGetAllocationFileFromUnsupportedFileSystem()
+      throws UnsupportedFileSystemException {
+    Configuration conf = new YarnConfiguration();
+    conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, "badfs:///badfile");
+    AllocationFileLoaderService allocLoader = new AllocationFileLoaderService();
+
+    allocLoader.getAllocationFile(conf);
+  }
+
+  @Test
+  public void testGetAllocationFileFromClasspath() {
+    try {
+      Configuration conf = new Configuration();
+      FileSystem fs = FileSystem.get(conf);
+      conf.set(FairSchedulerConfiguration.ALLOCATION_FILE,
+          TEST_FAIRSCHED_XML);
+      AllocationFileLoaderService allocLoader =
+          new AllocationFileLoaderService();
+      Path allocationFile = allocLoader.getAllocationFile(conf);
+      assertEquals(TEST_FAIRSCHED_XML, allocationFile.getName());
+      assertTrue(fs.exists(allocationFile));
+    } catch (IOException e) {
+      fail("Unable to access allocation file from classpath: " + e);
+    }
   }
   }
   
   
   @Test (timeout = 10000)
   @Test (timeout = 10000)

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerHA.md

@@ -56,7 +56,7 @@ Most of the failover functionality is tunable using various configuration proper
 
 
 | Configuration Properties | Description |
 | Configuration Properties | Description |
 |:---- |:---- |
 |:---- |:---- |
-| `yarn.resourcemanager.zk-address` | Address of the ZK-quorum. Used both for the state-store and embedded leader-election. |
+| `hadoop.zk.address` | Address of the ZK-quorum. Used both for the state-store and embedded leader-election. |
 | `yarn.resourcemanager.ha.enabled` | Enable RM HA. |
 | `yarn.resourcemanager.ha.enabled` | Enable RM HA. |
 | `yarn.resourcemanager.ha.rm-ids` | List of logical IDs for the RMs. e.g., "rm1,rm2". |
 | `yarn.resourcemanager.ha.rm-ids` | List of logical IDs for the RMs. e.g., "rm1,rm2". |
 | `yarn.resourcemanager.hostname.`*rm-id* | For each *rm-id*, specify the hostname the RM corresponds to. Alternately, one could set each of the RM's service addresses. |
 | `yarn.resourcemanager.hostname.`*rm-id* | For each *rm-id*, specify the hostname the RM corresponds to. Alternately, one could set each of the RM's service addresses. |

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRest.md

@@ -1316,13 +1316,13 @@ With the Applications API, you can obtain a collection of resources, each of whi
 
 
 ### Query Parameters Supported
 ### Query Parameters Supported
 
 
-Multiple parameters can be specified for GET operations. The started and finished times have a begin and end parameter to allow you to specify ranges. For example, one could request all applications that started between 1:00am and 2:00pm on 12/19/2011 with startedTimeBegin=1324256400&startedTimeEnd=1324303200. If the Begin parameter is not specified, it defaults to 0, and if the End parameter is not specified, it defaults to infinity.
+Multiple parameters can be specified for GET operations. The started and finished times have a begin and end parameter to allow you to specify ranges. For example, one could request all applications that started between 1:00am and 2:00pm on 12/19/2011 with startedTimeBegin=1324256400&startedTimeEnd=1324303200. If the Begin parameter is not specified, it defaults to 0, and if the End parameter is not specified, it defaults to infinity. All query parameters for this api will filter on all applications. However the `queue` query parameter will only implicitly filter on unfinished applications that are currently in the given queue.
 
 
       * state [deprecated] - state of the application
       * state [deprecated] - state of the application
       * states - applications matching the given application states, specified as a comma-separated list.
       * states - applications matching the given application states, specified as a comma-separated list.
       * finalStatus - the final status of the application - reported by the application itself
       * finalStatus - the final status of the application - reported by the application itself
       * user - user name
       * user - user name
-      * queue - queue name
+      * queue - unfinished applications that are currently in this queue
       * limit - total number of app objects to be returned
       * limit - total number of app objects to be returned
       * startedTimeBegin - applications with start time beginning with this time, specified in ms since epoch
       * startedTimeBegin - applications with start time beginning with this time, specified in ms since epoch
       * startedTimeEnd - applications with start time ending with this time, specified in ms since epoch
       * startedTimeEnd - applications with start time ending with this time, specified in ms since epoch

+ 6 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerRestart.md

@@ -93,22 +93,22 @@ This section describes the configurations involved to enable RM Restart feature.
 
 
 | Property | Description |
 | Property | Description |
 |:---- |:---- |
 |:---- |:---- |
-| `yarn.resourcemanager.zk-address` | Comma separated list of Host:Port pairs. Each corresponds to a ZooKeeper server (e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002") to be used by the RM for storing RM state. |
+| `hadoop.zk.address` | Comma separated list of Host:Port pairs. Each corresponds to a ZooKeeper server (e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002") to be used by the RM for storing RM state. |
 | `yarn.resourcemanager.zk-state-store.parent-path` | The full path of the root znode where RM state will be stored. Default value is /rmstore. |
 | `yarn.resourcemanager.zk-state-store.parent-path` | The full path of the root znode where RM state will be stored. Default value is /rmstore. |
 
 
 * Configure the retry policy state-store client uses to connect with the ZooKeeper server.
 * Configure the retry policy state-store client uses to connect with the ZooKeeper server.
 
 
 | Property | Description |
 | Property | Description |
 |:---- |:---- |
 |:---- |:---- |
-| `yarn.resourcemanager.zk-num-retries` | Number of times RM tries to connect to ZooKeeper server if the connection is lost. Default value is 500. |
-| `yarn.resourcemanager.zk-retry-interval-ms` | The interval in milliseconds between retries when connecting to a ZooKeeper server. Default value is 2 seconds. |
-| `yarn.resourcemanager.zk-timeout-ms` | ZooKeeper session timeout in milliseconds. This configuration is used by the ZooKeeper server to determine when the session expires. Session expiration happens when the server does not hear from the client (i.e. no heartbeat) within the session timeout period specified by this configuration. Default value is 10 seconds |
+| `hadoop.zk.num-retries` | Number of times RM tries to connect to ZooKeeper server if the connection is lost. Default value is 500. |
+| `hadoop.zk.retry-interval-ms` | The interval in milliseconds between retries when connecting to a ZooKeeper server. Default value is 2 seconds. |
+| `hadoop.zk.timeout-ms` | ZooKeeper session timeout in milliseconds. This configuration is used by the ZooKeeper server to determine when the session expires. Session expiration happens when the server does not hear from the client (i.e. no heartbeat) within the session timeout period specified by this configuration. Default value is 10 seconds |
 
 
 * Configure the ACLs to be used for setting permissions on ZooKeeper znodes.
 * Configure the ACLs to be used for setting permissions on ZooKeeper znodes.
 
 
 | Property | Description |
 | Property | Description |
 |:---- |:---- |
 |:---- |:---- |
-| `yarn.resourcemanager.zk-acl` | ACLs to be used for setting permissions on ZooKeeper znodes. Default value is `world:anyone:rwcda` |
+| `hadoop.zk.acl` | ACLs to be used for setting permissions on ZooKeeper znodes. Default value is `world:anyone:rwcda` |
 
 
 ### Configurations for LevelDB based state-store implementation
 ### Configurations for LevelDB based state-store implementation
 
 
@@ -157,7 +157,7 @@ Below is a minimum set of configurations for enabling RM work-preserving restart
        (e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002") to be used by the RM for storing RM state.
        (e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002") to be used by the RM for storing RM state.
        This must be supplied when using org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore
        This must be supplied when using org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore
        as the value for yarn.resourcemanager.store.class</description>
        as the value for yarn.resourcemanager.store.class</description>
-       <name>yarn.resourcemanager.zk-address</name>
+       <name>hadoop.zk.address</name>
        <value>127.0.0.1:2181</value>
        <value>127.0.0.1:2181</value>
      </property>
      </property>