You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by cu...@apache.org on 2018/02/21 01:14:47 UTC

[01/10] hadoop git commit: HDFS-11847. Enhance dfsadmin listOpenFiles command to list files blocking datanode decommissioning.

Repository: hadoop
Updated Branches:
  refs/heads/branch-3 [created] 5c6adb3dd


HDFS-11847. Enhance dfsadmin listOpenFiles command to list files blocking datanode decommissioning.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/ee447835
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/ee447835
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/ee447835

Branch: refs/heads/branch-3
Commit: ee44783515a55ab9fd368660c5cc2c2bc392132e
Parents: b830040
Author: Manoj Govindassamy <ma...@apache.org>
Authored: Tue Jan 2 14:59:36 2018 -0800
Committer: Manoj Govindassamy <ma...@apache.org>
Committed: Fri Jan 12 13:06:57 2018 -0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |  16 +-
 .../hadoop/hdfs/DistributedFileSystem.java      |   8 +
 .../apache/hadoop/hdfs/client/HdfsAdmin.java    |   7 +
 .../hadoop/hdfs/protocol/ClientProtocol.java    |  16 ++
 .../hadoop/hdfs/protocol/OpenFilesIterator.java |  36 +++-
 .../ClientNamenodeProtocolTranslatorPB.java     |  18 +-
 .../hadoop/hdfs/protocolPB/PBHelperClient.java  |  26 +++
 .../src/main/proto/ClientNamenodeProtocol.proto |   7 +
 ...tNamenodeProtocolServerSideTranslatorPB.java |   7 +-
 .../server/blockmanagement/BlockManager.java    |   2 +-
 .../blockmanagement/DatanodeAdminManager.java   |  25 ++-
 .../blockmanagement/DatanodeDescriptor.java     |  24 ++-
 .../federation/router/RouterRpcServer.java      |  10 +-
 .../hdfs/server/namenode/FSNamesystem.java      |  49 ++++-
 .../hdfs/server/namenode/NameNodeRpcServer.java |  10 +-
 .../org/apache/hadoop/hdfs/tools/DFSAdmin.java  |  36 +++-
 .../src/site/markdown/HDFSCommands.md           |   2 +-
 .../apache/hadoop/hdfs/AdminStatesBaseTest.java |  18 +-
 .../apache/hadoop/hdfs/TestDecommission.java    | 177 +++++++++++++++++++
 .../org/apache/hadoop/hdfs/TestHdfsAdmin.java   |   4 +-
 .../blockmanagement/BlockManagerTestUtil.java   |  12 +-
 .../hdfs/server/namenode/TestLeaseManager.java  |  48 ++---
 .../hdfs/server/namenode/TestListOpenFiles.java |  27 ++-
 23 files changed, 522 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee447835/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index ff61384..83c3b94 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -131,6 +131,7 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
 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.ReencryptionStatusIterator;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
@@ -3026,8 +3027,21 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    *
    * @throws IOException
    */
+  @Deprecated
   public RemoteIterator<OpenFileEntry> listOpenFiles() throws IOException {
     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);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee447835/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
index cecd9d1..54b428e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
@@ -85,6 +85,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
 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.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
@@ -2978,10 +2979,17 @@ public class DistributedFileSystem extends FileSystem
    * <p/>
    * This method can only be called by HDFS superusers.
    */
+  @Deprecated
   public RemoteIterator<OpenFileEntry> listOpenFiles() throws IOException {
     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.
    *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee447835/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java
index 9116167..e620039 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java
+++ b/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.ReencryptAction;
 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.security.AccessControlException;
 
@@ -652,8 +653,14 @@ public class HdfsAdmin {
    * <p/>
    * This method can only be called by HDFS superusers.
    */
+  @Deprecated
   public RemoteIterator<OpenFileEntry> listOpenFiles() throws IOException {
     return dfs.listOpenFiles();
   }
 
+  public RemoteIterator<OpenFileEntry> listOpenFiles(
+      EnumSet<OpenFilesType> openFilesTypes) throws IOException {
+    return dfs.listOpenFiles(openFilesTypes);
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee447835/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
index f61ec75..80e053c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
+++ b/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.protocol.HdfsConstants.ReencryptAction;
 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.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
@@ -1669,5 +1670,20 @@ public interface ClientProtocol {
    * @throws IOException
    */
   @Idempotent
+  @Deprecated
   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;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee447835/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/OpenFilesIterator.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/OpenFilesIterator.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/OpenFilesIterator.java
index c24e585..d113d65 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/OpenFilesIterator.java
+++ b/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;
 
 import java.io.IOException;
+import java.util.EnumSet;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -35,20 +36,51 @@ import org.apache.htrace.core.Tracer;
 @InterfaceStability.Evolving
 public class OpenFilesIterator extends
     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 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);
     this.namenode = namenode;
     this.tracer = tracer;
+    this.types = types;
   }
 
   @Override
   public BatchedEntries<OpenFileEntry> makeRequest(Long prevId)
       throws IOException {
     try (TraceScope ignored = tracer.newScope("listOpenFiles")) {
-      return namenode.listOpenFiles(prevId);
+      return namenode.listOpenFiles(prevId, types);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee447835/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
index aef7c1e..c297302 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
@@ -74,6 +74,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 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.OpenFileEntry;
 import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus;
@@ -1846,13 +1847,24 @@ public class ClientNamenodeProtocolTranslatorPB implements
     }
   }
 
+  @Deprecated
   @Override
   public BatchedEntries<OpenFileEntry> listOpenFiles(long prevId)
       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 {
-      ListOpenFilesResponseProto response = rpcProxy.listOpenFiles(null, req);
+      ListOpenFilesResponseProto response =
+          rpcProxy.listOpenFiles(null, req.build());
       List<OpenFileEntry> openFileEntries =
           Lists.newArrayListWithCapacity(response.getEntriesCount());
       for (OpenFilesBatchResponseProto p : response.getEntriesList()) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee447835/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
index c0df325..8d25d2c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
+++ b/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.LocatedBlocks;
 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.OpenFileEntry;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
@@ -128,6 +129,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.GetFsStatsResponseProto;
 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.RollingUpgradeInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SafeModeActionProto;
@@ -3078,4 +3080,28 @@ public class PBHelperClient {
     }
     return ret;
   }
+
+  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;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee447835/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
index 6db6ad0..216821a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
@@ -777,8 +777,14 @@ message GetEditsFromTxidResponseProto {
   required EventsListProto eventsList = 1;
 }
 
+enum OpenFilesTypeProto {
+  ALL_OPEN_FILES = 1;
+  BLOCKING_DECOMMISSION = 2;
+}
+
 message ListOpenFilesRequestProto {
   required int64 id = 1;
+  repeated OpenFilesTypeProto types = 2;
 }
 
 message OpenFilesBatchResponseProto {
@@ -791,6 +797,7 @@ message OpenFilesBatchResponseProto {
 message ListOpenFilesResponseProto {
   repeated OpenFilesBatchResponseProto entries = 1;
   required bool hasMore = 2;
+  repeated OpenFilesTypeProto types = 3;
 }
 
 service ClientNamenodeProtocol {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee447835/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
index dee3f25..d34032d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
+++ b/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.LocatedBlocks;
 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.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
@@ -1807,13 +1808,17 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
   public ListOpenFilesResponseProto listOpenFiles(RpcController controller,
       ListOpenFilesRequestProto req) throws ServiceException {
     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.newBuilder();
       builder.setHasMore(entries.hasMore());
       for (int i = 0; i < entries.size(); i++) {
         builder.addEntries(PBHelperClient.convert(entries.get(i)));
       }
+      builder.addAllTypes(req.getTypesList());
       return builder.build();
     } catch (IOException e) {
       throw new ServiceException(e);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee447835/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index ae04aac..757016b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -2255,7 +2255,7 @@ public class BlockManager implements BlockStatsMXBean {
    * If there were any reconstruction requests that timed out, reap them
    * and put them back into the neededReconstruction queue
    */
-  private void processPendingReconstructions() {
+  void processPendingReconstructions() {
     BlockInfo[] timedOutItems = pendingReconstruction.getTimedOutBlocks();
     if (timedOutItems != null) {
       namesystem.writeLock();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee447835/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeAdminManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeAdminManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeAdminManager.java
index 928036a..e338591 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeAdminManager.java
+++ b/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.hdfs.DFSConfigKeys;
 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.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.Namesystem;
 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.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -649,8 +653,10 @@ public class DatanodeAdminManager {
         boolean pruneReliableBlocks) {
       boolean firstReplicationLog = true;
       // 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;
       // All maintenance and decommission replicas.
       int outOfServiceOnlyReplicas = 0;
@@ -737,15 +743,24 @@ public class DatanodeAdminManager {
         // Update various counts
         lowRedundancyBlocks++;
         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)) {
           outOfServiceOnlyReplicas++;
         }
       }
 
-      datanode.getLeavingServiceStatus().set(lowRedundancyInOpenFiles,
-          lowRedundancyBlocks, outOfServiceOnlyReplicas);
+      datanode.getLeavingServiceStatus().set(lowRedundancyBlocksInOpenFiles,
+          lowRedundancyOpenFiles, lowRedundancyBlocks,
+          outOfServiceOnlyReplicas);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee447835/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
index ee5aca1..e2b52f0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
+++ b/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.LightWeightHashSet;
 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.Time;
 import org.slf4j.Logger;
@@ -795,17 +796,21 @@ public class DatanodeDescriptor extends DatanodeInfo {
   /** Leaving service status. */
   public class LeavingServiceStatus {
     private int underReplicatedBlocks;
+    private int underReplicatedBlocksInOpenFiles;
     private int outOfServiceOnlyReplicas;
-    private int underReplicatedInOpenFiles;
+    private LightWeightHashSet<Long> underReplicatedOpenFiles =
+        new LightWeightLinkedSet<>();
     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()) {
         return;
       }
-      underReplicatedInOpenFiles = underRepInOpenFiles;
+      underReplicatedOpenFiles = underRepInOpenFiles;
       underReplicatedBlocks = underRepBlocks;
+      underReplicatedBlocksInOpenFiles = lowRedundancyBlocksInOpenFiles;
       outOfServiceOnlyReplicas = outOfServiceOnlyRep;
     }
 
@@ -828,7 +833,14 @@ public class DatanodeDescriptor extends DatanodeInfo {
       if (!isDecommissionInProgress() && !isEnteringMaintenance()) {
         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 */
     public synchronized void setStartTime(long time) {
@@ -844,7 +856,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
       }
       return startTime;
     }
-  }  // End of class DecommissioningStatus
+  }  // End of class LeavingServiceStatus
 
   /**
    * Set the flag to indicate if this datanode is disallowed from communicating

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee447835/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
index a8c2257..d3d5959 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
@@ -89,6 +89,7 @@ import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 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.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
@@ -1913,9 +1914,16 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
     return null;
   }
 
+  @Deprecated
   @Override
-  public BatchedEntries<OpenFileEntry> listOpenFiles(long arg0)
+  public BatchedEntries<OpenFileEntry> listOpenFiles(long prevId)
       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);
     return null;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee447835/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 558fef7..97423cb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -90,6 +90,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_KEY;
 import static org.apache.hadoop.hdfs.server.namenode.FSDirStatAndListingOp.*;
 
 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.ECBlockGroupStats;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
@@ -272,6 +273,7 @@ import org.apache.hadoop.hdfs.server.protocol.SlowPeerReports;
 import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 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.io.IOUtils;
 import org.apache.hadoop.io.Text;
@@ -1743,12 +1745,14 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * 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
    * 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 openFilesTypes
    * @throws IOException
    */
-  BatchedListEntries<OpenFileEntry> listOpenFiles(long prevId)
-      throws IOException {
+  BatchedListEntries<OpenFileEntry> listOpenFiles(long prevId,
+      EnumSet<OpenFilesType> openFilesTypes) throws IOException {
     final String operationName = "listOpenFiles";
     checkSuperuserPrivilege();
     checkOperation(OperationCategory.READ);
@@ -1756,7 +1760,16 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     BatchedListEntries<OpenFileEntry> batchedListEntries;
     try {
       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) {
       logAuditEvent(false, operationName, null);
       throw e;
@@ -1767,6 +1780,36 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     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() {
     StringWriter sw = new StringWriter();
     PrintWriter pw = new PrintWriter(sw);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee447835/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
index 283547e..b44aaf1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
@@ -114,6 +114,7 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
 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.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
@@ -1319,11 +1320,18 @@ public class NameNodeRpcServer implements NamenodeProtocols {
     namesystem.metaSave(filename);
   }
 
+  @Deprecated
   @Override // ClientProtocol
   public BatchedEntries<OpenFileEntry> listOpenFiles(long prevId)
       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();
-    return namesystem.listOpenFiles(prevId);
+    return namesystem.listOpenFiles(prevId, openFilesTypes);
   }
 
   @Override // ClientProtocol

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee447835/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
index 9268f38..7f79b1c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
+++ b/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.Collections;
 import java.util.Date;
+import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.LinkedList;
 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.NameNodeProxies;
 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.ClientDatanodeProtocol;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
@@ -462,7 +464,7 @@ public class DFSAdmin extends FsShell {
     "\t[-getDatanodeInfo <datanode_host:ipc_port>]\n" +
     "\t[-metasave filename]\n" +
     "\t[-triggerBlockReport [-incremental] <datanode_host:ipc_port>]\n" +
-    "\t[-listOpenFiles]\n" +
+    "\t[-listOpenFiles [-blockingDecommission]]\n" +
     "\t[-help [cmd]]\n";
 
   /**
@@ -913,8 +915,21 @@ public class DFSAdmin extends FsShell {
    * Usage: hdfs dfsadmin -listOpenFiles
    *
    * @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();
     Configuration dfsConf = dfs.getConf();
     URI dfsUri = dfs.getUri();
@@ -926,9 +941,9 @@ public class DFSAdmin extends FsShell {
           dfsConf, HAUtil.getAddressOfActive(getDFS()), ClientProtocol.class,
           UserGroupInformation.getCurrentUser(), false);
       openFilesRemoteIterator = new OpenFilesIterator(proxy.getProxy(),
-          FsTracer.get(dfsConf));
+          FsTracer.get(dfsConf), openFilesTypes);
     } else {
-      openFilesRemoteIterator = dfs.listOpenFiles();
+      openFilesRemoteIterator = dfs.listOpenFiles(openFilesTypes);
     }
     printOpenFiles(openFilesRemoteIterator);
     return 0;
@@ -1214,9 +1229,11 @@ public class DFSAdmin extends FsShell {
         + "\tIf 'incremental' is specified, it will be an incremental\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"
-        + "\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" +
       "\t\tis specified.\n";
@@ -1964,7 +1981,8 @@ public class DFSAdmin extends FsShell {
       System.err.println("Usage: hdfs dfsadmin"
           + " [-triggerBlockReport [-incremental] <datanode_host:ipc_port>]");
     } else if ("-listOpenFiles".equals(cmd)) {
-      System.err.println("Usage: hdfs dfsadmin [-listOpenFiles]");
+      System.err.println("Usage: hdfs dfsadmin"
+          + " [-listOpenFiles [-blockingDecommission]]");
     } else {
       System.err.println("Usage: hdfs dfsadmin");
       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;
       }
     } else if ("-listOpenFiles".equals(cmd)) {
-      if (argv.length != 1) {
+      if ((argv.length != 1) && (argv.length != 2)) {
         printUsage(cmd);
         return exitCode;
       }
@@ -2205,7 +2223,7 @@ public class DFSAdmin extends FsShell {
       } else if ("-triggerBlockReport".equals(cmd)) {
         exitCode = triggerBlockReport(argv);
       } else if ("-listOpenFiles".equals(cmd)) {
-        exitCode = listOpenFiles();
+        exitCode = listOpenFiles(argv);
       } else if ("-help".equals(cmd)) {
         if (i < argv.length) {
           printHelp(argv[i]);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee447835/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
index ce61fa9..db983e2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
@@ -407,7 +407,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. |
 | `-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. |
-| `-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. |
 
 Runs a HDFS dfsadmin client.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee447835/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/AdminStatesBaseTest.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/AdminStatesBaseTest.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/AdminStatesBaseTest.java
index c0cef19..5d96b7b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/AdminStatesBaseTest.java
+++ b/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,
       boolean setupHostsFile, long[] nodesCapacity,
       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)
-        .nnTopology(MiniDFSNNTopology.simpleFederatedTopology(numNameNodes))
         .numDataNodes(numDatanodes);
+    if (federation) {
+      builder.nnTopology(
+          MiniDFSNNTopology.simpleFederatedTopology(numNameNodes));
+    }
     if (setupHostsFile) {
       builder.setupHostsFile(setupHostsFile);
     }
@@ -413,6 +423,12 @@ public class AdminStatesBaseTest {
     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 {
     cluster = new MiniDFSCluster.Builder(conf)
         .nnTopology(MiniDFSNNTopology.simpleHATopology()).numDataNodes(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee447835/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
index ac14a2a..d82025c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
+++ b/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.assertTrue;
 
+import java.io.ByteArrayOutputStream;
 import java.io.IOException;
+import java.io.PrintStream;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Scanner;
 import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 import com.google.common.base.Supplier;
 import com.google.common.collect.Lists;
+import org.apache.commons.lang.text.StrBuilder;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 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.NameNodeAdapter;
 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.util.ToolRunner;
 import org.apache.log4j.Level;
 import org.junit.Assert;
 import org.junit.Ignore;
@@ -651,6 +660,174 @@ public class TestDecommission extends AdminStatesBaseTest {
     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)
   public void testDecommissionWithOpenFileAndBlockRecovery()
       throws IOException, InterruptedException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee447835/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHdfsAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHdfsAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHdfsAdmin.java
index 685ea8b..3cb10bf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHdfsAdmin.java
+++ b/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.net.URI;
 import java.net.URISyntaxException;
+import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
 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.protocol.BlockStoragePolicy;
 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.junit.After;
 import org.junit.Assert;
@@ -254,7 +256,7 @@ public class TestHdfsAdmin {
     HdfsAdmin hdfsAdmin = new HdfsAdmin(FileSystem.getDefaultUri(conf), conf);
     HashSet<Path> openFiles = new HashSet<>(openFileMap.keySet());
     RemoteIterator<OpenFileEntry> openFilesRemoteItr =
-        hdfsAdmin.listOpenFiles();
+        hdfsAdmin.listOpenFiles(EnumSet.of(OpenFilesType.ALL_OPEN_FILES));
     while (openFilesRemoteItr.hasNext()) {
       String filePath = openFilesRemoteItr.next().getFilePath();
       assertFalse(filePath + " should not be listed under open files!",

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee447835/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java
index 7ee766f..dfb40a6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java
+++ b/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) {
     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
    * given BlockManager.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee447835/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestLeaseManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestLeaseManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestLeaseManager.java
index 55bc7c3..0a8da4b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestLeaseManager.java
+++ b/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(""),
         perm, 0L);
     when(fsDirectory.getRoot()).thenReturn(rootInodeDirectory);
-    verifyINodeLeaseCounts(lm, rootInodeDirectory, 0, 0, 0);
+    verifyINodeLeaseCounts(fsNamesystem, lm, rootInodeDirectory, 0, 0, 0);
 
     for (Long iNodeId : iNodeIds) {
       INodeFile iNodeFile = stubInodeFile(iNodeId);
@@ -215,13 +215,13 @@ public class TestLeaseManager {
       when(fsDirectory.getInode(iNodeId)).thenReturn(iNodeFile);
       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) {
       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
     int scale = 0;
-    testInodeWithLeasesAtScaleImpl(lm, fsDirectory, rootInodeDirectory, scale);
+    testInodeWithLeasesAtScaleImpl(fsNamesystem, lm, fsDirectory,
+        rootInodeDirectory, scale);
 
     for (int workerCount = 1;
          workerCount <= LeaseManager.INODE_FILTER_WORKER_COUNT_MAX / 2;
          workerCount++) {
       // Case 2: Open files count is half of worker task size
       scale = workerCount * LeaseManager.INODE_FILTER_WORKER_TASK_MIN / 2;
-      testInodeWithLeasesAtScaleImpl(lm, fsDirectory,
+      testInodeWithLeasesAtScaleImpl(fsNamesystem, lm, fsDirectory,
           rootInodeDirectory, scale);
 
       // Case 3: Open files count is 1 less of worker task size
       scale = workerCount * LeaseManager.INODE_FILTER_WORKER_TASK_MIN - 1;
-      testInodeWithLeasesAtScaleImpl(lm, fsDirectory,
+      testInodeWithLeasesAtScaleImpl(fsNamesystem, lm, fsDirectory,
           rootInodeDirectory, scale);
 
       // Case 4: Open files count is equal to worker task size
       scale = workerCount * LeaseManager.INODE_FILTER_WORKER_TASK_MIN;
-      testInodeWithLeasesAtScaleImpl(lm, fsDirectory,
+      testInodeWithLeasesAtScaleImpl(fsNamesystem, lm, fsDirectory,
           rootInodeDirectory, scale);
 
       // Case 5: Open files count is 1 more than worker task size
       scale = workerCount * LeaseManager.INODE_FILTER_WORKER_TASK_MIN + 1;
-      testInodeWithLeasesAtScaleImpl(lm, fsDirectory,
+      testInodeWithLeasesAtScaleImpl(fsNamesystem, lm, fsDirectory,
           rootInodeDirectory, scale);
     }
 
     // Case 6: Open files count is way more than worker count
     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<>();
     for (int i = 0; i < scale; i++) {
@@ -293,11 +296,12 @@ public class TestLeaseManager {
       when(fsDirectory.getInode(iNodeId)).thenReturn(iNodeFile);
       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();
-    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,
         leaseManager.getINodeIdWithLeases().size());
     assertEquals(iNodeWithLeaseCount,
@@ -401,6 +405,8 @@ public class TestLeaseManager {
         leaseManager.getINodeWithLeases(ancestorDirectory).size());
     assertEquals(iNodeIdWithLeaseCount,
         leaseManager.getUnderConstructionFiles(0).size());
+    assertEquals(0, (fsNamesystem.getFilesBlockingDecom(0) == null ?
+        0 : fsNamesystem.getFilesBlockingDecom(0).size()));
   }
 
   private Map<String, INode> createINodeTree(INodeDirectory parentDir,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee447835/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestListOpenFiles.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestListOpenFiles.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestListOpenFiles.java
index b290194..cfee7ba 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestListOpenFiles.java
+++ b/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 java.io.IOException;
+import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
 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.protocol.ClientProtocol;
 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.protocol.NamenodeProtocols;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
@@ -95,9 +97,13 @@ public class TestListOpenFiles {
     verifyOpenFiles(openFiles);
 
     BatchedEntries<OpenFileEntry> openFileEntryBatchedEntries =
-        nnRpc.listOpenFiles(0);
+        nnRpc.listOpenFiles(0, EnumSet.of(OpenFilesType.ALL_OPEN_FILES));
     assertTrue("Open files list should be empty!",
         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(
         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());
     OpenFileEntry lastEntry = null;
     BatchedEntries<OpenFileEntry> batchedEntries;
     do {
       if (lastEntry == null) {
-        batchedEntries = nnRpc.listOpenFiles(0);
+        batchedEntries = nnRpc.listOpenFiles(0, openFilesTypes);
       } else {
-        batchedEntries = nnRpc.listOpenFiles(lastEntry.getId());
+        batchedEntries = nnRpc.listOpenFiles(lastEntry.getId(), openFilesTypes);
       }
       assertTrue("Incorrect open files list size!",
           batchedEntries.size() <= BATCH_SIZE);
@@ -146,6 +152,13 @@ public class TestListOpenFiles {
         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,
       int numFilesToCreate) throws IOException {
     HashSet<Path> files = new HashSet<>();
@@ -197,6 +210,8 @@ public class TestListOpenFiles {
             try {
               assertEquals(0, ToolRunner.run(dfsAdmin,
                   new String[] {"-listOpenFiles"}));
+              assertEquals(0, ToolRunner.run(dfsAdmin,
+                  new String[] {"-listOpenFiles", "-blockingDecommission"}));
               // Sleep for some time to avoid
               // flooding logs with listing.
               Thread.sleep(listingIntervalMsec);
@@ -222,6 +237,8 @@ public class TestListOpenFiles {
 
       assertEquals(0, ToolRunner.run(dfsAdmin,
           new String[] {"-listOpenFiles"}));
+      assertEquals(0, ToolRunner.run(dfsAdmin,
+          new String[] {"-listOpenFiles", "-blockingDecommission"}));
       assertFalse("Client Error!", listOpenFilesError.get());
 
       clientThread.join();


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[10/10] hadoop git commit: YARN-7732. Support Generic AM Simulator from SynthGenerator. (Contributed by Young Chen via curino)

Posted by cu...@apache.org.
YARN-7732. Support Generic AM Simulator from SynthGenerator. (Contributed by Young Chen via curino)

(cherry picked from commit 84cea0011ffe510d24cf9f2952944f7a6fe622cf)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/5c6adb3d
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/5c6adb3d
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/5c6adb3d

Branch: refs/heads/branch-3
Commit: 5c6adb3dd34bc9da151591ecccc7b75fd371a234
Parents: ea06ad9
Author: Carlo Curino <cu...@apache.org>
Authored: Tue Feb 20 17:00:34 2018 -0800
Committer: Carlo Curino <cu...@apache.org>
Committed: Tue Feb 20 17:04:20 2018 -0800

----------------------------------------------------------------------
 hadoop-tools/hadoop-sls/pom.xml                 |   2 +
 .../org/apache/hadoop/yarn/sls/SLSRunner.java   | 137 +++---
 .../hadoop/yarn/sls/appmaster/AMSimulator.java  |   2 +-
 .../yarn/sls/appmaster/MRAMSimulator.java       |   7 +-
 .../yarn/sls/appmaster/StreamAMSimulator.java   | 273 +++++++++++
 .../hadoop/yarn/sls/appmaster/package-info.java |  21 +
 .../hadoop/yarn/sls/synthetic/SynthJob.java     | 367 ++++++++------
 .../yarn/sls/synthetic/SynthJobClass.java       | 180 -------
 .../sls/synthetic/SynthTraceJobProducer.java    | 487 ++++++++++++++++---
 .../yarn/sls/synthetic/SynthWorkload.java       | 121 -----
 .../hadoop/yarn/sls/BaseSLSRunnerTest.java      |   2 +-
 .../hadoop/yarn/sls/TestSLSGenericSynth.java    |  76 +++
 .../hadoop/yarn/sls/TestSLSStreamAMSynth.java   |  76 +++
 .../hadoop/yarn/sls/TestSynthJobGeneration.java | 213 +++++++-
 .../yarn/sls/appmaster/TestAMSimulator.java     |   2 +-
 .../src/test/resources/sls-runner.xml           |   4 +
 .../hadoop-sls/src/test/resources/syn.json      |   2 +-
 .../src/test/resources/syn_generic.json         |  54 ++
 .../src/test/resources/syn_stream.json          |  46 ++
 19 files changed, 1430 insertions(+), 642 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5c6adb3d/hadoop-tools/hadoop-sls/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/pom.xml b/hadoop-tools/hadoop-sls/pom.xml
index c123538..cf41cf4 100644
--- a/hadoop-tools/hadoop-sls/pom.xml
+++ b/hadoop-tools/hadoop-sls/pom.xml
@@ -133,6 +133,8 @@
             <exclude>src/test/resources/simulate.info.html.template</exclude>
             <exclude>src/test/resources/track.html.template</exclude>
             <exclude>src/test/resources/syn.json</exclude>
+            <exclude>src/test/resources/syn_generic.json</exclude>
+            <exclude>src/test/resources/syn_stream.json</exclude>
             <exclude>src/test/resources/inputsls.json</exclude>
             <exclude>src/test/resources/nodes.json</exclude>
             <exclude>src/test/resources/exit-invariants.txt</exclude>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5c6adb3d/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java
index 456602f..951c09d 100644
--- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java
+++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java
@@ -47,13 +47,11 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapreduce.TaskType;
 import org.apache.hadoop.metrics2.source.JvmMetrics;
 import org.apache.hadoop.tools.rumen.JobTraceReader;
 import org.apache.hadoop.tools.rumen.LoggedJob;
 import org.apache.hadoop.tools.rumen.LoggedTask;
 import org.apache.hadoop.tools.rumen.LoggedTaskAttempt;
-import org.apache.hadoop.tools.rumen.TaskAttemptInfo;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
@@ -627,89 +625,66 @@ public class SLSRunner extends Configured implements Tool {
     localConf.set("fs.defaultFS", "file:///");
     long baselineTimeMS = 0;
 
-    try {
+    // if we use the nodeFile this could have been not initialized yet.
+    if (stjp == null) {
+      stjp = new SynthTraceJobProducer(getConf(), new Path(inputTraces[0]));
+    }
 
-      // if we use the nodeFile this could have been not initialized yet.
-      if (stjp == null) {
-        stjp = new SynthTraceJobProducer(getConf(), new Path(inputTraces[0]));
-      }
+    SynthJob job = null;
+    // we use stjp, a reference to the job producer instantiated during node
+    // creation
+    while ((job = (SynthJob) stjp.getNextJob()) != null) {
+      // only support MapReduce currently
+      String user = job.getUser();
+      String jobQueue = job.getQueueName();
+      String oldJobId = job.getJobID().toString();
+      long jobStartTimeMS = job.getSubmissionTime();
 
-      SynthJob job = null;
-      // we use stjp, a reference to the job producer instantiated during node
-      // creation
-      while ((job = (SynthJob) stjp.getNextJob()) != null) {
-        // only support MapReduce currently
-        String user = job.getUser();
-        String jobQueue = job.getQueueName();
-        String oldJobId = job.getJobID().toString();
-        long jobStartTimeMS = job.getSubmissionTime();
-
-        // CARLO: Finish time is only used for logging, omit for now
-        long jobFinishTimeMS = -1L;
-
-        if (baselineTimeMS == 0) {
-          baselineTimeMS = jobStartTimeMS;
-        }
-        jobStartTimeMS -= baselineTimeMS;
-        jobFinishTimeMS -= baselineTimeMS;
-        if (jobStartTimeMS < 0) {
-          LOG.warn("Warning: reset job {} start time to 0.", oldJobId);
-          jobFinishTimeMS = jobFinishTimeMS - jobStartTimeMS;
-          jobStartTimeMS = 0;
-        }
-
-        increaseQueueAppNum(jobQueue);
-
-        List<ContainerSimulator> containerList =
-            new ArrayList<ContainerSimulator>();
-        ArrayList<NodeId> keyAsArray = new ArrayList<NodeId>(nmMap.keySet());
-        Random rand = new Random(stjp.getSeed());
-
-        // map tasks
-        for (int i = 0; i < job.getNumberMaps(); i++) {
-          TaskAttemptInfo tai = job.getTaskAttemptInfo(TaskType.MAP, i, 0);
-          RMNode node =
-              nmMap.get(keyAsArray.get(rand.nextInt(keyAsArray.size())))
-                  .getNode();
-          String hostname = "/" + node.getRackName() + "/" + node.getHostName();
-          long containerLifeTime = tai.getRuntime();
-          Resource containerResource =
-              Resource.newInstance((int) tai.getTaskInfo().getTaskMemory(),
-                  (int) tai.getTaskInfo().getTaskVCores());
-          containerList.add(new ContainerSimulator(containerResource,
-              containerLifeTime, hostname, DEFAULT_MAPPER_PRIORITY, "map"));
-        }
+      // CARLO: Finish time is only used for logging, omit for now
+      long jobFinishTimeMS = jobStartTimeMS + job.getDuration();
 
-        // reduce tasks
-        for (int i = 0; i < job.getNumberReduces(); i++) {
-          TaskAttemptInfo tai = job.getTaskAttemptInfo(TaskType.REDUCE, i, 0);
-          RMNode node =
-              nmMap.get(keyAsArray.get(rand.nextInt(keyAsArray.size())))
-                  .getNode();
-          String hostname = "/" + node.getRackName() + "/" + node.getHostName();
-          long containerLifeTime = tai.getRuntime();
-          Resource containerResource =
-              Resource.newInstance((int) tai.getTaskInfo().getTaskMemory(),
-                  (int) tai.getTaskInfo().getTaskVCores());
-          containerList.add(
-              new ContainerSimulator(containerResource, containerLifeTime,
-                  hostname, DEFAULT_REDUCER_PRIORITY, "reduce"));
-        }
+      if (baselineTimeMS == 0) {
+        baselineTimeMS = jobStartTimeMS;
+      }
+      jobStartTimeMS -= baselineTimeMS;
+      jobFinishTimeMS -= baselineTimeMS;
+      if (jobStartTimeMS < 0) {
+        LOG.warn("Warning: reset job {} start time to 0.", oldJobId);
+        jobFinishTimeMS = jobFinishTimeMS - jobStartTimeMS;
+        jobStartTimeMS = 0;
+      }
 
-        ReservationId reservationId = null;
+      increaseQueueAppNum(jobQueue);
+
+      List<ContainerSimulator> containerList =
+          new ArrayList<ContainerSimulator>();
+      ArrayList<NodeId> keyAsArray = new ArrayList<NodeId>(nmMap.keySet());
+      Random rand = new Random(stjp.getSeed());
+
+      for (SynthJob.SynthTask task : job.getTasks()) {
+        RMNode node = nmMap.get(keyAsArray.get(rand.nextInt(keyAsArray.size())))
+            .getNode();
+        String hostname = "/" + node.getRackName() + "/" + node.getHostName();
+        long containerLifeTime = task.getTime();
+        Resource containerResource = Resource
+            .newInstance((int) task.getMemory(), (int) task.getVcores());
+        containerList.add(
+            new ContainerSimulator(containerResource, containerLifeTime,
+                hostname, task.getPriority(), task.getType()));
+      }
 
-        if (job.hasDeadline()) {
-          reservationId =
-              ReservationId.newInstance(this.rm.getStartTime(), AM_ID);
-        }
 
-        runNewAM(SLSUtils.DEFAULT_JOB_TYPE, user, jobQueue, oldJobId,
-            jobStartTimeMS, jobFinishTimeMS, containerList, reservationId,
-            job.getDeadline(), getAMContainerResource(null));
+      ReservationId reservationId = null;
 
+      if(job.hasDeadline()){
+        reservationId = ReservationId
+            .newInstance(this.rm.getStartTime(), AM_ID);
       }
-    } finally {
-      stjp.close();
+
+      runNewAM(job.getType(), user, jobQueue, oldJobId,
+          jobStartTimeMS, jobFinishTimeMS, containerList, reservationId,
+          job.getDeadline(), getAMContainerResource(null),
+          job.getParams());
     }
   }
 
@@ -753,14 +728,14 @@ public class SLSRunner extends Configured implements Tool {
       Resource amContainerResource) {
     runNewAM(jobType, user, jobQueue, oldJobId, jobStartTimeMS,
         jobFinishTimeMS, containerList, null,  -1,
-        amContainerResource);
+        amContainerResource, null);
   }
 
   private void runNewAM(String jobType, String user,
       String jobQueue, String oldJobId, long jobStartTimeMS,
       long jobFinishTimeMS, List<ContainerSimulator> containerList,
-      ReservationId reservationId, long deadline,
-      Resource amContainerResource) {
+      ReservationId reservationId, long deadline, Resource amContainerResource,
+      Map<String, String> params) {
 
     AMSimulator amSim = (AMSimulator) ReflectionUtils.newInstance(
         amClassMap.get(jobType), new Configuration());
@@ -777,7 +752,7 @@ public class SLSRunner extends Configured implements Tool {
       AM_ID++;
       amSim.init(heartbeatInterval, containerList, rm, this, jobStartTimeMS,
           jobFinishTimeMS, user, jobQueue, isTracked, oldJobId,
-          runner.getStartTimeMS(), amContainerResource);
+          runner.getStartTimeMS(), amContainerResource, params);
       if(reservationId != null) {
         // if we have a ReservationId, delegate reservation creation to
         // AMSim (reservation shape is impl specific)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5c6adb3d/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/AMSimulator.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/AMSimulator.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/AMSimulator.java
index 5727b5f..bf85fff 100644
--- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/AMSimulator.java
+++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/AMSimulator.java
@@ -121,7 +121,7 @@ public abstract class AMSimulator extends TaskRunner.Task {
       List<ContainerSimulator> containerList, ResourceManager resourceManager,
       SLSRunner slsRunnner, long startTime, long finishTime, String simUser,
       String simQueue, boolean tracked, String oldApp, long baseTimeMS,
-      Resource amResource) {
+      Resource amResource, Map<String, String> params) {
     super.init(startTime, startTime + 1000000L * heartbeatInterval,
         heartbeatInterval);
     this.user = simUser;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5c6adb3d/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/MRAMSimulator.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/MRAMSimulator.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/MRAMSimulator.java
index 18a155c..6f0f85f 100644
--- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/MRAMSimulator.java
+++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/MRAMSimulator.java
@@ -65,6 +65,9 @@ public class MRAMSimulator extends AMSimulator {
   scheduled when all maps have finished (not support slow-start currently).
   */
 
+  public static final String MAP_TYPE = "map";
+  public static final String REDUCE_TYPE = "reduce";
+
   private static final int PRIORITY_REDUCE = 10;
   private static final int PRIORITY_MAP = 20;
 
@@ -123,10 +126,10 @@ public class MRAMSimulator extends AMSimulator {
       List<ContainerSimulator> containerList, ResourceManager rm, SLSRunner se,
       long traceStartTime, long traceFinishTime, String user, String queue,
       boolean isTracked, String oldAppId, long baselineStartTimeMS,
-      Resource amContainerResource) {
+      Resource amContainerResource, Map<String, String> params) {
     super.init(heartbeatInterval, containerList, rm, se,
         traceStartTime, traceFinishTime, user, queue, isTracked, oldAppId,
-        baselineStartTimeMS, amContainerResource);
+        baselineStartTimeMS, amContainerResource, params);
     amtype = "mapreduce";
 
     // get map/reduce tasks

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5c6adb3d/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/StreamAMSimulator.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/StreamAMSimulator.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/StreamAMSimulator.java
new file mode 100644
index 0000000..b41f5f2
--- /dev/null
+++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/StreamAMSimulator.java
@@ -0,0 +1,273 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.sls.appmaster;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.ReservationId;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
+import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
+import org.apache.hadoop.yarn.sls.SLSRunner;
+import org.apache.hadoop.yarn.sls.scheduler.ContainerSimulator;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * AMSimulator that simulates streaming services - it keeps tasks
+ * running and resubmits them whenever they fail or complete. It finishes
+ * when the specified duration expires.
+ */
+
+@Private
+@Unstable
+public class StreamAMSimulator extends AMSimulator {
+  /*
+  Vocabulary Used:
+  pending -> requests which are NOT yet sent to RM
+  scheduled -> requests which are sent to RM but not yet assigned
+  assigned -> requests which are assigned to a container
+  completed -> request corresponding to which container has completed
+
+  streams are constantly scheduled. If a streaming job is killed, we restart it
+  */
+
+  private static final int PRIORITY_MAP = 20;
+
+  // pending streams
+  private LinkedList<ContainerSimulator> pendingStreams =
+          new LinkedList<>();
+
+  // scheduled streams
+  private LinkedList<ContainerSimulator> scheduledStreams =
+          new LinkedList<ContainerSimulator>();
+
+  // assigned streams
+  private Map<ContainerId, ContainerSimulator> assignedStreams =
+          new HashMap<ContainerId, ContainerSimulator>();
+
+  // all streams
+  private LinkedList<ContainerSimulator> allStreams =
+          new LinkedList<ContainerSimulator>();
+
+  // finished
+  private boolean isFinished = false;
+  private long duration = 0;
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(StreamAMSimulator.class);
+
+  @SuppressWarnings("checkstyle:parameternumber")
+  public void init(int heartbeatInterval,
+      List<ContainerSimulator> containerList, ResourceManager rm, SLSRunner se,
+      long traceStartTime, long traceFinishTime, String user, String queue,
+      boolean isTracked, String oldAppId, long baselineStartTimeMS,
+      Resource amContainerResource, Map<String, String> params) {
+    super.init(heartbeatInterval, containerList, rm, se, traceStartTime,
+        traceFinishTime, user, queue, isTracked, oldAppId, baselineStartTimeMS,
+        amContainerResource, params);
+    amtype = "stream";
+
+    allStreams.addAll(containerList);
+
+    duration = traceFinishTime - traceStartTime;
+
+    LOG.info("Added new job with {} streams, running for {}",
+        allStreams.size(), duration);
+  }
+
+  @Override
+  public synchronized void notifyAMContainerLaunched(Container masterContainer)
+      throws Exception {
+    if (null != masterContainer) {
+      restart();
+      super.notifyAMContainerLaunched(masterContainer);
+    }
+  }
+
+  @Override
+  @SuppressWarnings("unchecked")
+  protected void processResponseQueue() throws Exception {
+    while (!responseQueue.isEmpty()) {
+      AllocateResponse response = responseQueue.take();
+
+      // check completed containers
+      if (!response.getCompletedContainersStatuses().isEmpty()) {
+        for (ContainerStatus cs : response.getCompletedContainersStatuses()) {
+          ContainerId containerId = cs.getContainerId();
+          if(assignedStreams.containsKey(containerId)){
+            // One of our containers completed. Regardless of reason,
+            // we want to maintain our streaming process
+            LOG.debug("Application {} has one streamer finished ({}).", appId,
+                containerId);
+            pendingStreams.add(assignedStreams.remove(containerId));
+          } else if (amContainer.getId().equals(containerId)){
+            // Our am container completed
+            if(cs.getExitStatus() == ContainerExitStatus.SUCCESS){
+              // am container released event (am container completed on success)
+              isAMContainerRunning = false;
+              isFinished = true;
+              LOG.info("Application {} goes to finish.", appId);
+            } else {
+              // am container killed - wait for re allocation
+              LOG.info("Application {}'s AM is "
+                  + "going to be killed. Waiting for rescheduling...", appId);
+              isAMContainerRunning = false;
+            }
+          }
+        }
+      }
+
+      // check finished
+      if (isAMContainerRunning &&
+          (System.currentTimeMillis() - simulateStartTimeMS >= duration)) {
+        LOG.debug("Application {} sends out event to clean up"
+                + " its AM container.", appId);
+        isAMContainerRunning = false;
+        isFinished = true;
+        break;
+      }
+
+      // check allocated containers
+      for (Container container : response.getAllocatedContainers()) {
+        if (!scheduledStreams.isEmpty()) {
+          ContainerSimulator cs = scheduledStreams.remove();
+          LOG.debug("Application {} starts to launch a stream ({}).", appId,
+              container.getId());
+          assignedStreams.put(container.getId(), cs);
+          se.getNmMap().get(container.getNodeId()).addNewContainer(container,
+              cs.getLifeTime());
+        }
+      }
+    }
+  }
+
+  /**
+   * restart running because of the am container killed.
+   */
+  private void restart()
+          throws YarnException, IOException, InterruptedException {
+    // clear
+    isFinished = false;
+    pendingStreams.clear();
+    pendingStreams.addAll(allStreams);
+
+    amContainer = null;
+  }
+
+  private List<ContainerSimulator> mergeLists(List<ContainerSimulator> left,
+      List<ContainerSimulator> right) {
+    List<ContainerSimulator> list = new ArrayList<>();
+    list.addAll(left);
+    list.addAll(right);
+    return list;
+  }
+
+  @Override
+  protected void sendContainerRequest()
+          throws YarnException, IOException, InterruptedException {
+
+    // send out request
+    List<ResourceRequest> ask = new ArrayList<>();
+    List<ContainerId> release = new ArrayList<>();
+    if (!isFinished) {
+      if (!pendingStreams.isEmpty()) {
+        ask = packageRequests(mergeLists(pendingStreams, scheduledStreams),
+            PRIORITY_MAP);
+        LOG.debug("Application {} sends out request for {} streams.",
+            appId, pendingStreams.size());
+        scheduledStreams.addAll(pendingStreams);
+        pendingStreams.clear();
+      }
+    }
+
+    if(isFinished){
+      release.addAll(assignedStreams.keySet());
+      ask.clear();
+    }
+
+    final AllocateRequest request = createAllocateRequest(ask, release);
+    if (totalContainers == 0) {
+      request.setProgress(1.0f);
+    } else {
+      request.setProgress((float) finishedContainers / totalContainers);
+    }
+
+    UserGroupInformation ugi =
+        UserGroupInformation.createRemoteUser(appAttemptId.toString());
+    Token<AMRMTokenIdentifier> token = rm.getRMContext().getRMApps()
+        .get(appAttemptId.getApplicationId())
+        .getRMAppAttempt(appAttemptId).getAMRMToken();
+    ugi.addTokenIdentifier(token.decodeIdentifier());
+    AllocateResponse response = ugi.doAs(
+        new PrivilegedExceptionAction<AllocateResponse>() {
+          @Override
+          public AllocateResponse run() throws Exception {
+            return rm.getApplicationMasterService().allocate(request);
+          }
+        });
+    if (response != null) {
+      responseQueue.put(response);
+    }
+  }
+
+  @Override
+  public void initReservation(
+      ReservationId reservationId, long deadline, long now){
+    // Streaming AM currently doesn't do reservations
+    setReservationRequest(null);
+  }
+
+  @Override
+  protected void checkStop() {
+    if (isFinished) {
+      super.setEndTime(System.currentTimeMillis());
+    }
+  }
+
+  @Override
+  public void lastStep() throws Exception {
+    super.lastStep();
+
+    // clear data structures
+    allStreams.clear();
+    assignedStreams.clear();
+    pendingStreams.clear();
+    scheduledStreams.clear();
+    responseQueue.clear();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5c6adb3d/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/package-info.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/package-info.java
new file mode 100644
index 0000000..ead315b
--- /dev/null
+++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/package-info.java
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Application Master simulators for the SLS.
+ */
+package org.apache.hadoop.yarn.sls.appmaster;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5c6adb3d/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/SynthJob.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/SynthJob.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/SynthJob.java
index 3ed81e1..27156c7 100644
--- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/SynthJob.java
+++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/SynthJob.java
@@ -19,19 +19,25 @@ package org.apache.hadoop.yarn.sls.synthetic;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.commons.math3.distribution.LogNormalDistribution;
 import org.apache.commons.math3.random.JDKRandomGenerator;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.TaskStatus.State;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.JobID;
-import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.TaskType;
-import org.apache.hadoop.tools.rumen.*;
+import org.apache.hadoop.tools.rumen.JobStory;
+import org.apache.hadoop.tools.rumen.MapTaskAttemptInfo;
+import org.apache.hadoop.tools.rumen.ReduceTaskAttemptInfo;
+import org.apache.hadoop.tools.rumen.TaskAttemptInfo;
+import org.apache.hadoop.tools.rumen.TaskInfo;
 import org.apache.hadoop.tools.rumen.Pre21JobHistoryConstants.Values;
+import org.apache.hadoop.yarn.sls.appmaster.MRAMSimulator;
 
-import java.util.Arrays;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
@@ -46,6 +52,9 @@ public class SynthJob implements JobStory {
   @SuppressWarnings("StaticVariableName")
   private static Log LOG = LogFactory.getLog(SynthJob.class);
 
+  private static final long MIN_MEMORY = 1024;
+  private static final long MIN_VCORES = 1;
+
   private final Configuration conf;
   private final int id;
 
@@ -53,75 +62,93 @@ public class SynthJob implements JobStory {
   private static final AtomicInteger sequence = new AtomicInteger(0);
   private final String name;
   private final String queueName;
-  private final SynthJobClass jobClass;
+  private final SynthTraceJobProducer.JobDefinition jobDef;
+
+  private String type;
 
   // job timing
   private final long submitTime;
   private final long duration;
   private final long deadline;
 
-  private final int numMapTasks;
-  private final int numRedTasks;
-  private final long mapMaxMemory;
-  private final long reduceMaxMemory;
-  private final long mapMaxVcores;
-  private final long reduceMaxVcores;
-  private final long[] mapRuntime;
-  private final float[] reduceRuntime;
-  private long totMapRuntime;
-  private long totRedRuntime;
+  private Map<String, String> params;
+
+  private long totalSlotTime = 0;
+
+  // task information
+  private List<SynthTask> tasks = new ArrayList<>();
+  private Map<String, List<SynthTask>> taskByType = new HashMap<>();
+  private Map<String, Integer> taskCounts = new HashMap<>();
+  private Map<String, Long> taskMemory = new HashMap<>();
+  private Map<String, Long> taskVcores = new HashMap<>();
+
+  /**
+   * Nested class used to represent a task instance in a job. Each task
+   * corresponds to one container allocation for the job.
+   */
+  public static final class SynthTask{
+    private String type;
+    private long time;
+    private long maxMemory;
+    private long maxVcores;
+    private int priority;
+
+    private SynthTask(String type, long time, long maxMemory, long maxVcores,
+        int priority){
+      this.type = type;
+      this.time = time;
+      this.maxMemory = maxMemory;
+      this.maxVcores = maxVcores;
+      this.priority = priority;
+    }
+
+    public String getType(){
+      return type;
+    }
 
-  public SynthJob(JDKRandomGenerator rand, Configuration conf,
-      SynthJobClass jobClass, long actualSubmissionTime) {
+    public long getTime(){
+      return time;
+    }
 
-    this.conf = conf;
-    this.jobClass = jobClass;
-
-    this.duration = MILLISECONDS.convert(jobClass.getDur(), SECONDS);
-    this.numMapTasks = jobClass.getMtasks();
-    this.numRedTasks = jobClass.getRtasks();
-
-    // sample memory distributions, correct for sub-minAlloc sizes
-    long tempMapMaxMemory = jobClass.getMapMaxMemory();
-    this.mapMaxMemory = tempMapMaxMemory < MRJobConfig.DEFAULT_MAP_MEMORY_MB
-        ? MRJobConfig.DEFAULT_MAP_MEMORY_MB : tempMapMaxMemory;
-    long tempReduceMaxMemory = jobClass.getReduceMaxMemory();
-    this.reduceMaxMemory =
-            tempReduceMaxMemory < MRJobConfig.DEFAULT_REDUCE_MEMORY_MB
-            ? MRJobConfig.DEFAULT_REDUCE_MEMORY_MB : tempReduceMaxMemory;
-
-    // sample vcores distributions, correct for sub-minAlloc sizes
-    long tempMapMaxVCores = jobClass.getMapMaxVcores();
-    this.mapMaxVcores = tempMapMaxVCores < MRJobConfig.DEFAULT_MAP_CPU_VCORES
-        ? MRJobConfig.DEFAULT_MAP_CPU_VCORES : tempMapMaxVCores;
-    long tempReduceMaxVcores = jobClass.getReduceMaxVcores();
-    this.reduceMaxVcores =
-        tempReduceMaxVcores < MRJobConfig.DEFAULT_REDUCE_CPU_VCORES
-            ? MRJobConfig.DEFAULT_REDUCE_CPU_VCORES : tempReduceMaxVcores;
-
-    if (numMapTasks > 0) {
-      conf.setLong(MRJobConfig.MAP_MEMORY_MB, this.mapMaxMemory);
-      conf.set(MRJobConfig.MAP_JAVA_OPTS,
-          "-Xmx" + (this.mapMaxMemory - 100) + "m");
+    public long getMemory(){
+      return maxMemory;
     }
 
-    if (numRedTasks > 0) {
-      conf.setLong(MRJobConfig.REDUCE_MEMORY_MB, this.reduceMaxMemory);
-      conf.set(MRJobConfig.REDUCE_JAVA_OPTS,
-          "-Xmx" + (this.reduceMaxMemory - 100) + "m");
+    public long getVcores(){
+      return maxVcores;
     }
 
-    boolean hasDeadline =
-        (rand.nextDouble() <= jobClass.jobClass.chance_of_reservation);
+    public int getPriority(){
+      return priority;
+    }
+
+    @Override
+    public String toString(){
+      return String.format("[task]\ttype: %1$-10s\ttime: %2$3s\tmemory: "
+              + "%3$4s\tvcores: %4$2s%n", getType(), getTime(), getMemory(),
+          getVcores());
+    }
+  }
 
-    LogNormalDistribution deadlineFactor =
-        SynthUtils.getLogNormalDist(rand, jobClass.jobClass.deadline_factor_avg,
-            jobClass.jobClass.deadline_factor_stddev);
 
-    double deadlineFactorSample =
-        (deadlineFactor != null) ? deadlineFactor.sample() : -1;
+  protected SynthJob(JDKRandomGenerator rand, Configuration conf,
+      SynthTraceJobProducer.JobDefinition jobDef,
+      String queue, long actualSubmissionTime) {
 
-    this.queueName = jobClass.workload.getQueueName();
+    this.conf = conf;
+    this.jobDef = jobDef;
+
+    this.queueName = queue;
+
+    this.duration = MILLISECONDS.convert(jobDef.duration.getInt(),
+        SECONDS);
+
+    boolean hasDeadline =
+        (rand.nextDouble() <= jobDef.reservation.getDouble());
+
+    double deadlineFactorSample = jobDef.deadline_factor.getDouble();
+
+    this.type = jobDef.type;
 
     this.submitTime = MILLISECONDS.convert(actualSubmissionTime, SECONDS);
 
@@ -129,6 +156,8 @@ public class SynthJob implements JobStory {
         hasDeadline ? MILLISECONDS.convert(actualSubmissionTime, SECONDS)
             + (long) Math.ceil(deadlineFactorSample * duration) : -1;
 
+    this.params = jobDef.params;
+
     conf.set(QUEUE_NAME, queueName);
 
     // name and initialize job randomness
@@ -136,79 +165,166 @@ public class SynthJob implements JobStory {
     rand.setSeed(seed);
     id = sequence.getAndIncrement();
 
-    name = String.format(jobClass.getClassName() + "_%06d", id);
+    name = String.format(jobDef.class_name + "_%06d", id);
     LOG.debug(name + " (" + seed + ")");
 
     LOG.info("JOB TIMING`: job: " + name + " submission:" + submitTime
         + " deadline:" + deadline + " duration:" + duration
         + " deadline-submission: " + (deadline - submitTime));
 
-    // generate map and reduce runtimes
-    mapRuntime = new long[numMapTasks];
-    for (int i = 0; i < numMapTasks; i++) {
-      mapRuntime[i] = jobClass.getMapTimeSample();
-      totMapRuntime += mapRuntime[i];
-    }
-    reduceRuntime = new float[numRedTasks];
-    for (int i = 0; i < numRedTasks; i++) {
-      reduceRuntime[i] = jobClass.getReduceTimeSample();
-      totRedRuntime += (long) Math.ceil(reduceRuntime[i]);
+    // Expand tasks
+    for(SynthTraceJobProducer.TaskDefinition task : jobDef.tasks){
+      int num = task.count.getInt();
+      String taskType = task.type;
+      long memory = task.max_memory.getLong();
+      memory = memory < MIN_MEMORY ? MIN_MEMORY: memory;
+      long vcores = task.max_vcores.getLong();
+      vcores = vcores < MIN_VCORES ? MIN_VCORES  : vcores;
+      int priority = task.priority;
+
+      // Save task information by type
+      taskByType.put(taskType, new ArrayList<>());
+      taskCounts.put(taskType, num);
+      taskMemory.put(taskType, memory);
+      taskVcores.put(taskType, vcores);
+
+      for(int i = 0; i < num; ++i){
+        long time = task.time.getLong();
+        totalSlotTime += time;
+        SynthTask t = new SynthTask(taskType, time, memory, vcores,
+            priority);
+        tasks.add(t);
+        taskByType.get(taskType).add(t);
+      }
     }
+
+  }
+
+  public String getType(){
+    return type;
+  }
+
+  public List<SynthTask> getTasks(){
+    return tasks;
   }
 
   public boolean hasDeadline() {
     return deadline > 0;
   }
 
-  @Override
   public String getName() {
     return name;
   }
 
-  @Override
   public String getUser() {
-    return jobClass.getUserName();
+    return jobDef.user_name;
   }
 
-  @Override
   public JobID getJobID() {
     return new JobID("job_mock_" + name, id);
   }
 
+  public long getSubmissionTime() {
+    return submitTime;
+  }
+
+  public String getQueueName() {
+    return queueName;
+  }
+
   @Override
-  public Values getOutcome() {
-    return Values.SUCCESS;
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    String res = "\nSynthJob [" + jobDef.class_name + "]: \n"
+        + "\tname: " + getName() + "\n"
+        + "\ttype: " + getType() + "\n"
+        + "\tid: " + id + "\n"
+        + "\tqueue: " + getQueueName() + "\n"
+        + "\tsubmission: " + getSubmissionTime() + "\n"
+        + "\tduration: " + getDuration() + "\n"
+        + "\tdeadline: " + getDeadline() + "\n";
+    sb.append(res);
+    int taskno = 0;
+    for(SynthJob.SynthTask t : getTasks()){
+      sb.append("\t");
+      sb.append(taskno);
+      sb.append(": \t");
+      sb.append(t.toString());
+      taskno++;
+    }
+    return sb.toString();
+  }
+
+  public long getTotalSlotTime() {
+    return totalSlotTime;
+  }
+
+  public long getDuration() {
+    return duration;
+  }
+
+  public long getDeadline() {
+    return deadline;
+  }
+
+  public Map<String, String> getParams() {
+    return params;
   }
 
   @Override
-  public long getSubmissionTime() {
-    return submitTime;
+  public boolean equals(Object other) {
+    if (!(other instanceof SynthJob)) {
+      return false;
+    }
+    SynthJob o = (SynthJob) other;
+    return tasks.equals(o.tasks)
+        && submitTime == o.submitTime
+        && type.equals(o.type)
+        && queueName.equals(o.queueName)
+        && jobDef.class_name.equals(o.jobDef.class_name);
+  }
+
+  @Override
+  public int hashCode() {
+    return jobDef.class_name.hashCode()
+        * (int) submitTime * (int) duration;
+  }
+
+
+  @Override
+  public JobConf getJobConf() {
+    return new JobConf(conf);
   }
 
   @Override
   public int getNumberMaps() {
-    return numMapTasks;
+    return taskCounts.get(MRAMSimulator.MAP_TYPE);
   }
 
   @Override
   public int getNumberReduces() {
-    return numRedTasks;
+    return taskCounts.get(MRAMSimulator.REDUCE_TYPE);
+  }
+
+  @Override
+  public InputSplit[] getInputSplits() {
+    throw new UnsupportedOperationException();
   }
 
   @Override
   public TaskInfo getTaskInfo(TaskType taskType, int taskNumber) {
-    switch (taskType) {
+    switch(taskType){
     case MAP:
-      return new TaskInfo(-1, -1, -1, -1, mapMaxMemory, mapMaxVcores);
+      return new TaskInfo(-1, -1, -1, -1,
+          taskMemory.get(MRAMSimulator.MAP_TYPE),
+          taskVcores.get(MRAMSimulator.MAP_TYPE));
     case REDUCE:
-      return new TaskInfo(-1, -1, -1, -1, reduceMaxMemory, reduceMaxVcores);
+      return new TaskInfo(-1, -1, -1, -1,
+          taskMemory.get(MRAMSimulator.REDUCE_TYPE),
+          taskVcores.get(MRAMSimulator.REDUCE_TYPE));
     default:
-      throw new IllegalArgumentException("Not interested");
+      break;
     }
-  }
-
-  @Override
-  public InputSplit[] getInputSplits() {
     throw new UnsupportedOperationException();
   }
 
@@ -218,17 +334,20 @@ public class SynthJob implements JobStory {
     switch (taskType) {
     case MAP:
       return new MapTaskAttemptInfo(State.SUCCEEDED,
-          getTaskInfo(taskType, taskNumber), mapRuntime[taskNumber], null);
-
+          getTaskInfo(taskType, taskNumber),
+          taskByType.get(MRAMSimulator.MAP_TYPE).get(taskNumber).time,
+          null);
     case REDUCE:
       // We assume uniform split between pull/sort/reduce
       // aligned with naive progress reporting assumptions
       return new ReduceTaskAttemptInfo(State.SUCCEEDED,
           getTaskInfo(taskType, taskNumber),
-          (long) Math.round((reduceRuntime[taskNumber] / 3)),
-          (long) Math.round((reduceRuntime[taskNumber] / 3)),
-          (long) Math.round((reduceRuntime[taskNumber] / 3)), null);
-
+          taskByType.get(MRAMSimulator.MAP_TYPE)
+              .get(taskNumber).time / 3,
+          taskByType.get(MRAMSimulator.MAP_TYPE)
+              .get(taskNumber).time / 3,
+          taskByType.get(MRAMSimulator.MAP_TYPE)
+              .get(taskNumber).time / 3, null);
     default:
       break;
     }
@@ -242,65 +361,7 @@ public class SynthJob implements JobStory {
   }
 
   @Override
-  public org.apache.hadoop.mapred.JobConf getJobConf() {
-    return new JobConf(conf);
-  }
-
-  @Override
-  public String getQueueName() {
-    return queueName;
-  }
-
-  @Override
-  public String toString() {
-    return "SynthJob [\n" + "  workload=" + jobClass.getWorkload().getId()
-        + "\n" + "  jobClass="
-        + jobClass.getWorkload().getClassList().indexOf(jobClass) + "\n"
-        + "  conf=" + conf + ",\n" + "  id=" + id + ",\n" + "  name=" + name
-        + ",\n" + "  mapRuntime=" + Arrays.toString(mapRuntime) + ",\n"
-        + "  reduceRuntime=" + Arrays.toString(reduceRuntime) + ",\n"
-        + "  submitTime=" + submitTime + ",\n" + "  numMapTasks=" + numMapTasks
-        + ",\n" + "  numRedTasks=" + numRedTasks + ",\n" + "  mapMaxMemory="
-        + mapMaxMemory + ",\n" + "  reduceMaxMemory=" + reduceMaxMemory + ",\n"
-        + "  queueName=" + queueName + "\n" + "]";
-  }
-
-  public SynthJobClass getJobClass() {
-    return jobClass;
-  }
-
-  public long getTotalSlotTime() {
-    return totMapRuntime + totRedRuntime;
-  }
-
-  public long getDuration() {
-    return duration;
-  }
-
-  public long getDeadline() {
-    return deadline;
-  }
-
-  @Override
-  public boolean equals(Object other) {
-    if (!(other instanceof SynthJob)) {
-      return false;
-    }
-    SynthJob o = (SynthJob) other;
-    return Arrays.equals(mapRuntime, o.mapRuntime)
-        && Arrays.equals(reduceRuntime, o.reduceRuntime)
-        && submitTime == o.submitTime && numMapTasks == o.numMapTasks
-        && numRedTasks == o.numRedTasks && mapMaxMemory == o.mapMaxMemory
-        && reduceMaxMemory == o.reduceMaxMemory
-        && mapMaxVcores == o.mapMaxVcores
-        && reduceMaxVcores == o.reduceMaxVcores && queueName.equals(o.queueName)
-        && jobClass.equals(o.jobClass) && totMapRuntime == o.totMapRuntime
-        && totRedRuntime == o.totRedRuntime;
-  }
-
-  @Override
-  public int hashCode() {
-    // could have a bad distr; investigate if a relevant use case exists
-    return jobClass.hashCode() * (int) submitTime;
+  public Values getOutcome() {
+    return Values.SUCCESS;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5c6adb3d/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/SynthJobClass.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/SynthJobClass.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/SynthJobClass.java
deleted file mode 100644
index 439698f..0000000
--- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/SynthJobClass.java
+++ /dev/null
@@ -1,180 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.yarn.sls.synthetic;
-
-import org.apache.commons.math3.distribution.AbstractRealDistribution;
-import org.apache.commons.math3.distribution.LogNormalDistribution;
-import org.apache.commons.math3.random.JDKRandomGenerator;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.tools.rumen.JobStory;
-import org.apache.hadoop.yarn.sls.synthetic.SynthTraceJobProducer.JobClass;
-import org.apache.hadoop.yarn.sls.synthetic.SynthTraceJobProducer.Trace;
-
-/**
- * This is a class that represent a class of Jobs. It is used to generate an
- * individual job, by picking random durations, task counts, container size,
- * etc.
- */
-public class SynthJobClass {
-
-  private final JDKRandomGenerator rand;
-  private final LogNormalDistribution dur;
-  private final LogNormalDistribution mapRuntime;
-  private final LogNormalDistribution redRuntime;
-  private final LogNormalDistribution mtasks;
-  private final LogNormalDistribution rtasks;
-  private final LogNormalDistribution mapMem;
-  private final LogNormalDistribution redMem;
-  private final LogNormalDistribution mapVcores;
-  private final LogNormalDistribution redVcores;
-
-  private final Trace trace;
-  @SuppressWarnings("VisibilityModifier")
-  protected final SynthWorkload workload;
-  @SuppressWarnings("VisibilityModifier")
-  protected final JobClass jobClass;
-
-  public SynthJobClass(JDKRandomGenerator rand, Trace trace,
-      SynthWorkload workload, int classId) {
-
-    this.trace = trace;
-    this.workload = workload;
-    this.rand = new JDKRandomGenerator();
-    this.rand.setSeed(rand.nextLong());
-    jobClass = trace.workloads.get(workload.getId()).job_classes.get(classId);
-
-    this.dur = SynthUtils.getLogNormalDist(rand, jobClass.dur_avg,
-        jobClass.dur_stddev);
-    this.mapRuntime = SynthUtils.getLogNormalDist(rand, jobClass.mtime_avg,
-        jobClass.mtime_stddev);
-    this.redRuntime = SynthUtils.getLogNormalDist(rand, jobClass.rtime_avg,
-        jobClass.rtime_stddev);
-    this.mtasks = SynthUtils.getLogNormalDist(rand, jobClass.mtasks_avg,
-        jobClass.mtasks_stddev);
-    this.rtasks = SynthUtils.getLogNormalDist(rand, jobClass.rtasks_avg,
-        jobClass.rtasks_stddev);
-
-    this.mapMem = SynthUtils.getLogNormalDist(rand, jobClass.map_max_memory_avg,
-        jobClass.map_max_memory_stddev);
-    this.redMem = SynthUtils.getLogNormalDist(rand,
-        jobClass.reduce_max_memory_avg, jobClass.reduce_max_memory_stddev);
-    this.mapVcores = SynthUtils.getLogNormalDist(rand,
-        jobClass.map_max_vcores_avg, jobClass.map_max_vcores_stddev);
-    this.redVcores = SynthUtils.getLogNormalDist(rand,
-        jobClass.reduce_max_vcores_avg, jobClass.reduce_max_vcores_stddev);
-  }
-
-  public JobStory getJobStory(Configuration conf, long actualSubmissionTime) {
-    return new SynthJob(rand, conf, this, actualSubmissionTime);
-  }
-
-  @Override
-  public String toString() {
-    return "SynthJobClass [workload=" + workload.getName() + ", class="
-        + jobClass.class_name + " job_count=" + jobClass.class_weight + ", dur="
-        + ((dur != null) ? dur.getNumericalMean() : 0) + ", mapRuntime="
-        + ((mapRuntime != null) ? mapRuntime.getNumericalMean() : 0)
-        + ", redRuntime="
-        + ((redRuntime != null) ? redRuntime.getNumericalMean() : 0)
-        + ", mtasks=" + ((mtasks != null) ? mtasks.getNumericalMean() : 0)
-        + ", rtasks=" + ((rtasks != null) ? rtasks.getNumericalMean() : 0)
-        + ", chance_of_reservation=" + jobClass.chance_of_reservation + "]\n";
-
-  }
-
-  public double getClassWeight() {
-    return jobClass.class_weight;
-  }
-
-  public long getDur() {
-    return genLongSample(dur);
-  }
-
-  public int getMtasks() {
-    return genIntSample(mtasks);
-  }
-
-  public int getRtasks() {
-    return genIntSample(rtasks);
-  }
-
-  public long getMapMaxMemory() {
-    return genLongSample(mapMem);
-  }
-
-  public long getReduceMaxMemory() {
-    return genLongSample(redMem);
-  }
-
-  public long getMapMaxVcores() {
-    return genLongSample(mapVcores);
-  }
-
-  public long getReduceMaxVcores() {
-    return genLongSample(redVcores);
-  }
-
-  public SynthWorkload getWorkload() {
-    return workload;
-  }
-
-  public int genIntSample(AbstractRealDistribution dist) {
-    if (dist == null) {
-      return 0;
-    }
-    double baseSample = dist.sample();
-    if (baseSample < 0) {
-      baseSample = 0;
-    }
-    return (int) (Integer.MAX_VALUE & (long) Math.ceil(baseSample));
-  }
-
-  public long genLongSample(AbstractRealDistribution dist) {
-    return dist != null ? (long) Math.ceil(dist.sample()) : 0;
-  }
-
-  @Override
-  public boolean equals(Object other) {
-    if (!(other instanceof SynthJobClass)) {
-      return false;
-    }
-    SynthJobClass o = (SynthJobClass) other;
-    return workload.equals(o.workload);
-  }
-
-  @Override
-  public int hashCode() {
-    return workload.hashCode() * workload.getId();
-  }
-
-  public String getClassName() {
-    return jobClass.class_name;
-  }
-
-  public long getMapTimeSample() {
-    return genLongSample(mapRuntime);
-  }
-
-  public long getReduceTimeSample() {
-    return genLongSample(redRuntime);
-  }
-
-  public String getUserName() {
-    return jobClass.user_name;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5c6adb3d/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/SynthTraceJobProducer.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/SynthTraceJobProducer.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/SynthTraceJobProducer.java
index c89e4e2..09bc9b9 100644
--- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/SynthTraceJobProducer.java
+++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/SynthTraceJobProducer.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.yarn.sls.synthetic;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.commons.math3.distribution.AbstractRealDistribution;
 import org.apache.commons.math3.random.JDKRandomGenerator;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
@@ -26,7 +27,11 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.tools.rumen.JobStory;
 import org.apache.hadoop.tools.rumen.JobStoryProducer;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.hadoop.yarn.sls.appmaster.MRAMSimulator;
+import org.codehaus.jackson.annotate.JsonCreator;
 import org.codehaus.jackson.annotate.JsonProperty;
+import org.codehaus.jackson.map.JsonMappingException;
 import org.codehaus.jackson.map.ObjectMapper;
 
 import javax.xml.bind.annotation.XmlRootElement;
@@ -39,7 +44,7 @@ import static org.codehaus.jackson.map.DeserializationConfig.Feature.FAIL_ON_UNK
 
 /**
  * This is a JobStoryProducer that operates from distribution of different
- * workloads. The .json input file is used to determine how many jobs, which
+ * workloads. The .json input file is used to determine how many weight, which
  * size, number of maps/reducers and their duration, as well as the temporal
  * distributed of submissions. For each parameter we control avg and stdev, and
  * generate values via normal or log-normal distributions.
@@ -55,8 +60,6 @@ public class SynthTraceJobProducer implements JobStoryProducer {
   private final long seed;
 
   private int totalWeight;
-  private final List<Double> weightList;
-  private final Map<Integer, SynthWorkload> workloads;
 
   private final Queue<StoryParams> listStoryParams;
 
@@ -65,6 +68,9 @@ public class SynthTraceJobProducer implements JobStoryProducer {
   public static final String SLS_SYNTHETIC_TRACE_FILE =
       "sls.synthetic" + ".trace_file";
 
+  private final static int DEFAULT_MAPPER_PRIORITY = 20;
+  private final static int DEFAULT_REDUCER_PRIORITY = 10;
+
   public SynthTraceJobProducer(Configuration conf) throws IOException {
     this(conf, new Path(conf.get(SLS_SYNTHETIC_TRACE_FILE)));
   }
@@ -76,8 +82,6 @@ public class SynthTraceJobProducer implements JobStoryProducer {
 
     this.conf = conf;
     this.rand = new JDKRandomGenerator();
-    workloads = new HashMap<Integer, SynthWorkload>();
-    weightList = new ArrayList<Double>();
 
     ObjectMapper mapper = new ObjectMapper();
     mapper.configure(INTERN_FIELD_NAMES, true);
@@ -86,44 +90,132 @@ public class SynthTraceJobProducer implements JobStoryProducer {
     FileSystem ifs = path.getFileSystem(conf);
     FSDataInputStream fileIn = ifs.open(path);
 
+    // Initialize the random generator and the seed
     this.trace = mapper.readValue(fileIn, Trace.class);
-    seed = trace.rand_seed;
-    rand.setSeed(seed);
+    this.seed = trace.rand_seed;
+    this.rand.setSeed(seed);
+    // Initialize the trace
+    this.trace.init(rand);
 
     this.numJobs = new AtomicInteger(trace.num_jobs);
 
-    for (int workloadId = 0; workloadId < trace.workloads
-        .size(); workloadId++) {
-      SynthWorkload workload = new SynthWorkload(workloadId, trace);
-      for (int classId =
-          0; classId < trace.workloads.get(workloadId).job_classes
-              .size(); classId++) {
-        SynthJobClass cls = new SynthJobClass(rand, trace, workload, classId);
-        workload.add(cls);
-      }
-      workloads.put(workloadId, workload);
+    for (Double w : trace.workload_weights) {
+      totalWeight += w;
     }
 
-    for (int i = 0; i < workloads.size(); i++) {
-      double w = workloads.get(i).getWorkloadWeight();
-      totalWeight += w;
-      weightList.add(w);
+    // Initialize our story parameters
+    listStoryParams = createStory();
+
+    LOG.info("Generated " + listStoryParams.size() + " deadlines for "
+        + this.numJobs.get() + " jobs");
+  }
+
+  // StoryParams hold the minimum amount of information needed to completely
+  // specify a job run: job definition, start time, and queue.
+  // This allows us to create "jobs" and then order them according to start time
+  static class StoryParams {
+    // Time the job gets submitted to
+    private long actualSubmissionTime;
+    // The queue the job gets submitted to
+    private String queue;
+    // Definition to construct the job from
+    private JobDefinition jobDef;
+
+    StoryParams(long actualSubmissionTime, String queue, JobDefinition jobDef) {
+      this.actualSubmissionTime = actualSubmissionTime;
+      this.queue = queue;
+      this.jobDef = jobDef;
     }
+  }
 
+
+  private Queue<StoryParams> createStory() {
     // create priority queue to keep start-time sorted
-    listStoryParams =
-        new PriorityQueue<StoryParams>(10, new Comparator<StoryParams>() {
+    Queue<StoryParams> storyQueue =
+        new PriorityQueue<>(this.numJobs.get(), new Comparator<StoryParams>() {
           @Override
           public int compare(StoryParams o1, StoryParams o2) {
             return Math
-                .toIntExact(o2.actualSubmissionTime - o1.actualSubmissionTime);
+                .toIntExact(o1.actualSubmissionTime - o2.actualSubmissionTime);
           }
         });
+    for (int i = 0; i < numJobs.get(); i++) {
+      // Generate a workload
+      Workload wl = trace.generateWorkload();
+      // Save all the parameters needed to completely define a job
+      long actualSubmissionTime = wl.generateSubmissionTime();
+      String queue = wl.queue_name;
+      JobDefinition job = wl.generateJobDefinition();
+      storyQueue.add(new StoryParams(actualSubmissionTime, queue, job));
+    }
+    return storyQueue;
+  }
 
-    // initialize it
-    createStoryParams();
-    LOG.info("Generated " + listStoryParams.size() + " deadlines for "
-        + this.numJobs.get() + " jobs ");
+  @Override
+  public JobStory getNextJob() throws IOException {
+    if (numJobs.decrementAndGet() < 0) {
+      return null;
+    }
+    StoryParams storyParams = listStoryParams.poll();
+    return new SynthJob(rand, conf, storyParams.jobDef, storyParams.queue,
+        storyParams.actualSubmissionTime);
+  }
+
+  @Override
+  public void close(){
+  }
+
+  @Override
+  public String toString() {
+    return "SynthTraceJobProducer [ conf=" + conf + ", numJobs=" + numJobs
+        + ", r=" + rand + ", totalWeight="
+        + totalWeight + ", workloads=" + trace.workloads + "]";
+  }
+
+  public int getNumJobs() {
+    return trace.num_jobs;
+  }
+
+  // Helper to parse and maintain backwards compatibility with
+  // syn json formats
+  private static void validateJobDef(JobDefinition jobDef){
+    if(jobDef.tasks == null) {
+      LOG.info("Detected old JobDefinition format. Converting.");
+      try {
+        jobDef.tasks = new ArrayList<>();
+        jobDef.type = "mapreduce";
+        jobDef.deadline_factor = new Sample(jobDef.deadline_factor_avg,
+            jobDef.deadline_factor_stddev);
+        jobDef.duration = new Sample(jobDef.dur_avg,
+            jobDef.dur_stddev);
+        jobDef.reservation = new Sample(jobDef.chance_of_reservation);
+
+        TaskDefinition map = new TaskDefinition();
+        map.type = MRAMSimulator.MAP_TYPE;
+        map.count = new Sample(jobDef.mtasks_avg, jobDef.mtasks_stddev);
+        map.time = new Sample(jobDef.mtime_avg, jobDef.mtime_stddev);
+        map.max_memory = new Sample((double) jobDef.map_max_memory_avg,
+            jobDef.map_max_memory_stddev);
+        map.max_vcores = new Sample((double) jobDef.map_max_vcores_avg,
+            jobDef.map_max_vcores_stddev);
+        map.priority = DEFAULT_MAPPER_PRIORITY;
+
+        jobDef.tasks.add(map);
+        TaskDefinition reduce = new TaskDefinition();
+        reduce.type = MRAMSimulator.REDUCE_TYPE;
+        reduce.count = new Sample(jobDef.rtasks_avg, jobDef.rtasks_stddev);
+        reduce.time = new Sample(jobDef.rtime_avg, jobDef.rtime_stddev);
+        reduce.max_memory = new Sample((double) jobDef.reduce_max_memory_avg,
+            jobDef.reduce_max_memory_stddev);
+        reduce.max_vcores = new Sample((double) jobDef.reduce_max_vcores_avg,
+            jobDef.reduce_max_vcores_stddev);
+        reduce.priority = DEFAULT_REDUCER_PRIORITY;
+
+        jobDef.tasks.add(reduce);
+      } catch (JsonMappingException e) {
+        LOG.warn("Error converting old JobDefinition format", e);
+      }
+    }
   }
 
   public long getSeed() {
@@ -159,6 +251,25 @@ public class SynthTraceJobProducer implements JobStoryProducer {
     @JsonProperty("workloads")
     List<Workload> workloads;
 
+    List<Double> workload_weights;
+    JDKRandomGenerator rand;
+
+    public void init(JDKRandomGenerator random){
+      this.rand = random;
+      // Pass rand forward
+      for(Workload w : workloads){
+        w.init(rand);
+      }
+      // Initialize workload weights
+      workload_weights = new ArrayList<>();
+      for(Workload w : workloads){
+        workload_weights.add(w.workload_weight);
+      }
+    }
+
+    Workload generateWorkload(){
+      return workloads.get(SynthUtils.getWeighted(workload_weights, rand));
+    }
   }
 
   /**
@@ -174,16 +285,67 @@ public class SynthTraceJobProducer implements JobStoryProducer {
     @JsonProperty("queue_name")
     String queue_name;
     @JsonProperty("job_classes")
-    List<JobClass> job_classes;
+    List<JobDefinition> job_classes;
     @JsonProperty("time_distribution")
     List<TimeSample> time_distribution;
+
+    JDKRandomGenerator rand;
+
+    List<Double> job_weights;
+    List<Double> time_weights;
+
+    public void init(JDKRandomGenerator random){
+      this.rand = random;
+      // Validate and pass rand forward
+      for(JobDefinition def : job_classes){
+        validateJobDef(def);
+        def.init(rand);
+      }
+
+      // Initialize job weights
+      job_weights = new ArrayList<>();
+      job_weights = new ArrayList<>();
+      for(JobDefinition j : job_classes){
+        job_weights.add(j.class_weight);
+      }
+
+      // Initialize time weights
+      time_weights = new ArrayList<>();
+      for(TimeSample ts : time_distribution){
+        time_weights.add(ts.weight);
+      }
+    }
+
+    public long generateSubmissionTime(){
+      int index = SynthUtils.getWeighted(time_weights, rand);
+      // Retrieve the lower and upper bounds for this time "bucket"
+      int start = time_distribution.get(index).time;
+      // Get the beginning of the next time sample (if it exists)
+      index = (index+1)<time_distribution.size() ? index+1 : index;
+      int end = time_distribution.get(index).time;
+      int range = end-start;
+      // Within this time "bucket", uniformly pick a time if our
+      // range is non-zero, otherwise just use the start time of the bucket
+      return start + (range>0 ? rand.nextInt(range) : 0);
+    }
+
+    public JobDefinition generateJobDefinition(){
+      return job_classes.get(SynthUtils.getWeighted(job_weights, rand));
+    }
+
+    @Override
+    public String toString(){
+      return "\nWorkload " + workload_name + ", weight: " + workload_weight
+          + ", queue: " + queue_name + " "
+          + job_classes.toString().replace("\n", "\n\t");
+    }
   }
 
   /**
    * Class used to parse a job class from file.
    */
   @SuppressWarnings({ "membername", "checkstyle:visibilitymodifier" })
-  public static class JobClass {
+  public static class JobDefinition {
 
     @JsonProperty("class_name")
     String class_name;
@@ -194,6 +356,23 @@ public class SynthTraceJobProducer implements JobStoryProducer {
     @JsonProperty("class_weight")
     double class_weight;
 
+    // am type to launch
+    @JsonProperty("type")
+    String type;
+    @JsonProperty("deadline_factor")
+    Sample deadline_factor;
+    @JsonProperty("duration")
+    Sample duration;
+    @JsonProperty("reservation")
+    Sample reservation;
+
+    @JsonProperty("tasks")
+    List<TaskDefinition> tasks;
+
+    @JsonProperty("params")
+    Map<String, String> params;
+
+    // Old JSON fields for backwards compatibility
     // reservation related params
     @JsonProperty("chance_of_reservation")
     double chance_of_reservation;
@@ -246,71 +425,227 @@ public class SynthTraceJobProducer implements JobStoryProducer {
     @JsonProperty("reduce_max_vcores_stddev")
     double reduce_max_vcores_stddev;
 
+    public void init(JDKRandomGenerator rand){
+      deadline_factor.init(rand);
+      duration.init(rand);
+      reservation.init(rand);
+
+      for(TaskDefinition t : tasks){
+        t.count.init(rand);
+        t.time.init(rand);
+        t.max_memory.init(rand);
+        t.max_vcores.init(rand);
+      }
+    }
+
+    @Override
+    public String toString(){
+      return "\nJobDefinition " + class_name + ", weight: " + class_weight
+          + ", type: " + type + " "
+          + tasks.toString().replace("\n", "\n\t");
+    }
   }
 
   /**
-   * This is used to define time-varying probability of a job start-time (e.g.,
-   * to simulate daily patterns).
+   * A task representing a type of container - e.g. "map" in mapreduce
    */
   @SuppressWarnings({ "membername", "checkstyle:visibilitymodifier" })
-  public static class TimeSample {
-    // in sec
+  public static class TaskDefinition {
+
+    @JsonProperty("type")
+    String type;
+    @JsonProperty("count")
+    Sample count;
     @JsonProperty("time")
-    int time;
-    @JsonProperty("weight")
-    double jobs;
+    Sample time;
+    @JsonProperty("max_memory")
+    Sample max_memory;
+    @JsonProperty("max_vcores")
+    Sample max_vcores;
+    @JsonProperty("priority")
+    int priority;
+
+    @Override
+    public String toString(){
+      return "\nTaskDefinition " + type
+          + " Count[" + count + "] Time[" + time + "] Memory[" + max_memory
+          + "] Vcores[" + max_vcores + "] Priority[" + priority + "]";
+    }
   }
 
-  static class StoryParams {
-    private SynthJobClass pickedJobClass;
-    private long actualSubmissionTime;
+  /**
+   * Class used to parse value sample information.
+   */
+  @SuppressWarnings({ "membername", "checkstyle:visibilitymodifier" })
+  public static class Sample {
+    private static final Dist DEFAULT_DIST = Dist.LOGNORM;
+
+    private final double val;
+    private final double std;
+    private final Dist dist;
+    private AbstractRealDistribution dist_instance;
+    private final List<String> discrete;
+    private final List<Double> weights;
+    private final Mode mode;
+
+    private JDKRandomGenerator rand;
+
+    private enum Mode{
+      CONST,
+      DIST,
+      DISC
+    }
 
-    StoryParams(SynthJobClass pickedJobClass, long actualSubmissionTime) {
-      this.pickedJobClass = pickedJobClass;
-      this.actualSubmissionTime = actualSubmissionTime;
+    private enum Dist{
+      LOGNORM,
+      NORM
     }
-  }
 
+    public Sample(Double val) throws JsonMappingException{
+      this(val, null);
+    }
 
-  void createStoryParams() {
+    public Sample(Double val, Double std) throws JsonMappingException{
+      this(val, std, null, null, null);
+    }
 
-    for (int i = 0; i < numJobs.get(); i++) {
-      int workload = SynthUtils.getWeighted(weightList, rand);
-      SynthWorkload pickedWorkload = workloads.get(workload);
-      long jobClass =
-          SynthUtils.getWeighted(pickedWorkload.getWeightList(), rand);
-      SynthJobClass pickedJobClass =
-          pickedWorkload.getClassList().get((int) jobClass);
-      long actualSubmissionTime = pickedWorkload.getBaseSubmissionTime(rand);
-      // long actualSubmissionTime = (i + 1) * 10;
-      listStoryParams
-          .add(new StoryParams(pickedJobClass, actualSubmissionTime));
+    @JsonCreator
+    public Sample(@JsonProperty("val") Double val,
+        @JsonProperty("std") Double std, @JsonProperty("dist") String dist,
+        @JsonProperty("discrete") List<String> discrete,
+        @JsonProperty("weights") List<Double> weights)
+        throws JsonMappingException{
+      // Different Modes
+      // - Constant: val must be specified, all else null. Sampling will
+      // return val.
+      // - Distribution: val, std specified, dist optional (defaults to
+      // LogNormal). Sampling will sample from the appropriate distribution
+      // - Discrete: discrete must be set to a list of strings or numbers,
+      // weights optional (defaults to uniform)
+
+      if(val!=null){
+        if(std==null){
+          // Constant
+          if(dist!=null || discrete!=null || weights!=null){
+            throw new JsonMappingException("Instantiation of " + Sample.class
+                + " failed");
+          }
+          mode = Mode.CONST;
+          this.val = val;
+          this.std = 0;
+          this.dist = null;
+          this.discrete = null;
+          this.weights = null;
+        } else {
+          // Distribution
+          if(discrete!=null || weights != null){
+            throw new JsonMappingException("Instantiation of " + Sample.class
+                + " failed");
+          }
+          mode = Mode.DIST;
+          this.val = val;
+          this.std = std;
+          this.dist = dist!=null ? Dist.valueOf(dist) : DEFAULT_DIST;
+          this.discrete = null;
+          this.weights = null;
+        }
+      } else {
+        // Discrete
+        if(discrete==null){
+          throw new JsonMappingException("Instantiation of " + Sample.class
+              + " failed");
+        }
+        mode = Mode.DISC;
+        this.val = 0;
+        this.std = 0;
+        this.dist = null;
+        this.discrete = discrete;
+        if(weights == null){
+          weights = new ArrayList<>(Collections.nCopies(
+              discrete.size(), 1.0));
+        }
+        if(weights.size() != discrete.size()){
+          throw new JsonMappingException("Instantiation of " + Sample.class
+              + " failed");
+        }
+        this.weights = weights;
+      }
     }
-  }
 
-  @Override
-  public JobStory getNextJob() throws IOException {
-    if (numJobs.decrementAndGet() < 0) {
-      return null;
+    public void init(JDKRandomGenerator random){
+      if(this.rand != null){
+        throw new YarnRuntimeException("init called twice");
+      }
+      this.rand = random;
+      if(mode == Mode.DIST){
+        switch(this.dist){
+        case LOGNORM:
+          this.dist_instance = SynthUtils.getLogNormalDist(rand, val, std);
+          return;
+        case NORM:
+          this.dist_instance = SynthUtils.getNormalDist(rand, val, std);
+          return;
+        default:
+          throw new YarnRuntimeException("Unknown distribution " + dist.name());
+        }
+      }
     }
-    StoryParams storyParams = listStoryParams.poll();
-    return storyParams.pickedJobClass.getJobStory(conf,
-        storyParams.actualSubmissionTime);
-  }
 
-  @Override
-  public void close() {
-  }
+    public int getInt(){
+      return Math.toIntExact(getLong());
+    }
 
-  @Override
-  public String toString() {
-    return "SynthTraceJobProducer [ conf=" + conf + ", numJobs=" + numJobs
-        + ", weightList=" + weightList + ", r=" + rand + ", totalWeight="
-        + totalWeight + ", workloads=" + workloads + "]";
-  }
+    public long getLong(){
+      return Math.round(getDouble());
+    }
+
+    public double getDouble(){
+      return Double.parseDouble(getString());
+    }
+
+    public String getString(){
+      if(this.rand == null){
+        throw new YarnRuntimeException("getValue called without init");
+      }
+      switch(mode){
+      case CONST:
+        return Double.toString(val);
+      case DIST:
+        return Double.toString(dist_instance.sample());
+      case DISC:
+        return this.discrete.get(SynthUtils.getWeighted(this.weights, rand));
+      default:
+        throw new YarnRuntimeException("Unknown sampling mode " + mode.name());
+      }
+    }
+
+    @Override
+    public String toString(){
+      switch(mode){
+      case CONST:
+        return "value: " + Double.toString(val);
+      case DIST:
+        return "value: " + this.val + " std: " + this.std + " dist: "
+            + this.dist.name();
+      case DISC:
+        return "discrete: " + this.discrete + ", weights: " + this.weights;
+      default:
+        throw new YarnRuntimeException("Unknown sampling mode " + mode.name());
+      }
+    }
 
-  public int getNumJobs() {
-    return trace.num_jobs;
   }
 
+  /**
+   * This is used to define time-varying probability of a job start-time (e.g.,
+   * to simulate daily patterns).
+   */
+  @SuppressWarnings({ "membername", "checkstyle:visibilitymodifier" })
+  public static class TimeSample {
+    // in sec
+    @JsonProperty("time")
+    int time;
+    @JsonProperty("weight")
+    double weight;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5c6adb3d/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/SynthWorkload.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/SynthWorkload.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/SynthWorkload.java
deleted file mode 100644
index 9e5fd4e..0000000
--- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/SynthWorkload.java
+++ /dev/null
@@ -1,121 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.yarn.sls.synthetic;
-
-import org.apache.hadoop.yarn.sls.synthetic.SynthTraceJobProducer.Trace;
-
-import java.util.*;
-
-/**
- * This class represent a workload (made up of multiple SynthJobClass(es)). It
- * also stores the temporal distributions of jobs in this workload.
- */
-public class SynthWorkload {
-
-  private final int id;
-  private final List<SynthJobClass> classList;
-  private final Trace trace;
-  private final SortedMap<Integer, Double> timeWeights;
-
-  public SynthWorkload(int identifier, Trace inTrace) {
-    classList = new ArrayList<SynthJobClass>();
-    this.id = identifier;
-    this.trace = inTrace;
-    timeWeights = new TreeMap<Integer, Double>();
-    for (SynthTraceJobProducer.TimeSample ts : trace.workloads
-        .get(id).time_distribution) {
-      timeWeights.put(ts.time, ts.jobs);
-    }
-  }
-
-  public boolean add(SynthJobClass s) {
-    return classList.add(s);
-  }
-
-  public List<Double> getWeightList() {
-    ArrayList<Double> ret = new ArrayList<Double>();
-    for (SynthJobClass s : classList) {
-      ret.add(s.getClassWeight());
-    }
-    return ret;
-  }
-
-  public int getId() {
-    return id;
-  }
-
-  @Override
-  public boolean equals(Object other) {
-    if (!(other instanceof SynthWorkload)) {
-      return false;
-    }
-    // assume ID determines job classes by construction
-    return getId() == ((SynthWorkload) other).getId();
-  }
-
-  @Override
-  public int hashCode() {
-    return getId();
-  }
-
-  @Override
-  public String toString() {
-    return "SynthWorkload " + trace.workloads.get(id).workload_name + "[\n"
-        + classList + "]\n";
-  }
-
-  public String getName() {
-    return trace.workloads.get(id).workload_name;
-  }
-
-  public double getWorkloadWeight() {
-    return trace.workloads.get(id).workload_weight;
-  }
-
-  public String getQueueName() {
-    return trace.workloads.get(id).queue_name;
-  }
-
-  public long getBaseSubmissionTime(Random rand) {
-
-    // pick based on weights the "bucket" for this start time
-    int position = SynthUtils.getWeighted(timeWeights.values(), rand);
-
-    int[] time = new int[timeWeights.keySet().size()];
-    int index = 0;
-    for (Integer i : timeWeights.keySet()) {
-      time[index++] = i;
-    }
-
-    // uniformly pick a time between start and end time of this bucket
-    int startRange = time[position];
-    int endRange = startRange;
-    // if there is no subsequent bucket pick startRange
-    if (position < timeWeights.keySet().size() - 1) {
-      endRange = time[position + 1];
-      return startRange + rand.nextInt((endRange - startRange));
-    } else {
-      return startRange;
-    }
-  }
-
-  public List<SynthJobClass> getClassList() {
-    return classList;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5c6adb3d/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/BaseSLSRunnerTest.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/BaseSLSRunnerTest.java b/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/BaseSLSRunnerTest.java
index 6b369f2..668be14 100644
--- a/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/BaseSLSRunnerTest.java
+++ b/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/BaseSLSRunnerTest.java
@@ -125,7 +125,7 @@ public abstract class BaseSLSRunnerTest {
       if (!exceptionList.isEmpty()) {
         sls.stop();
         Assert.fail("TestSLSRunner catched exception from child thread "
-            + "(TaskRunner.Task): " + exceptionList);
+            + "(TaskRunner.TaskDefinition): " + exceptionList);
         break;
       }
       timeout--;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5c6adb3d/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/TestSLSGenericSynth.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/TestSLSGenericSynth.java b/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/TestSLSGenericSynth.java
new file mode 100644
index 0000000..79ebe21
--- /dev/null
+++ b/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/TestSLSGenericSynth.java
@@ -0,0 +1,76 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.sls;
+
+import net.jcip.annotations.NotThreadSafe;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+import java.util.Arrays;
+import java.util.Collection;
+
+/**
+ * This test performs simple runs of the SLS with the generic syn json format.
+ */
+@RunWith(value = Parameterized.class)
+@NotThreadSafe
+public class TestSLSGenericSynth extends BaseSLSRunnerTest {
+
+  @Parameters(name = "Testing with: {1}, {0}, (nodeFile {3})")
+  public static Collection<Object[]> data() {
+
+    String capScheduler = CapacityScheduler.class.getCanonicalName();
+    String fairScheduler = FairScheduler.class.getCanonicalName();
+    String synthTraceFile = "src/test/resources/syn_generic.json";
+    String nodeFile = "src/test/resources/nodes.json";
+
+    // Test with both schedulers
+    return Arrays.asList(new Object[][] {
+
+        // covering the no nodeFile case
+        {capScheduler, "SYNTH", synthTraceFile, null },
+
+        // covering new commandline and CapacityScheduler
+        {capScheduler, "SYNTH", synthTraceFile, nodeFile },
+
+        // covering FairScheduler
+        {fairScheduler, "SYNTH", synthTraceFile, nodeFile },
+    });
+  }
+
+  @Before
+  public void setup() {
+    ongoingInvariantFile = "src/test/resources/ongoing-invariants.txt";
+    exitInvariantFile = "src/test/resources/exit-invariants.txt";
+  }
+
+  @Test(timeout = 90000)
+  @SuppressWarnings("all")
+  public void testSimulatorRunning() throws Exception {
+    Configuration conf = new Configuration(false);
+    long timeTillShutdownInsec = 20L;
+    runSLS(conf, timeTillShutdownInsec);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5c6adb3d/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/TestSLSStreamAMSynth.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/TestSLSStreamAMSynth.java b/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/TestSLSStreamAMSynth.java
new file mode 100644
index 0000000..a5d30e0
--- /dev/null
+++ b/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/TestSLSStreamAMSynth.java
@@ -0,0 +1,76 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.sls;
+
+import net.jcip.annotations.NotThreadSafe;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+import java.util.Arrays;
+import java.util.Collection;
+
+/**
+ * This test performs simple runs of the SLS with the generic syn json format.
+ */
+@RunWith(value = Parameterized.class)
+@NotThreadSafe
+public class TestSLSStreamAMSynth extends BaseSLSRunnerTest {
+
+  @Parameters(name = "Testing with: {1}, {0}, (nodeFile {3})")
+  public static Collection<Object[]> data() {
+
+    String capScheduler = CapacityScheduler.class.getCanonicalName();
+    String fairScheduler = FairScheduler.class.getCanonicalName();
+    String synthTraceFile = "src/test/resources/syn_stream.json";
+    String nodeFile = "src/test/resources/nodes.json";
+
+    // Test with both schedulers
+    return Arrays.asList(new Object[][] {
+
+        // covering the no nodeFile case
+        {capScheduler, "SYNTH", synthTraceFile, null },
+
+        // covering new commandline and CapacityScheduler
+        {capScheduler, "SYNTH", synthTraceFile, nodeFile },
+
+        // covering FairScheduler
+        {fairScheduler, "SYNTH", synthTraceFile, nodeFile },
+    });
+  }
+
+  @Before
+  public void setup() {
+    ongoingInvariantFile = "src/test/resources/ongoing-invariants.txt";
+    exitInvariantFile = "src/test/resources/exit-invariants.txt";
+  }
+
+  @Test(timeout = 90000)
+  @SuppressWarnings("all")
+  public void testSimulatorRunning() throws Exception {
+    Configuration conf = new Configuration(false);
+    long timeTillShutdownInsec = 20L;
+    runSLS(conf, timeTillShutdownInsec);
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[06/10] hadoop git commit: HDFS-9049. Make Datanode Netty reverse proxy port to be configurable. Contributed by Vinayakumar B.

Posted by cu...@apache.org.
HDFS-9049. Make Datanode Netty reverse proxy port to be configurable. Contributed by Vinayakumar B.

(cherry picked from commit 09efdfe9e13c9695867ce4034aa6ec970c2032f1)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/0802d8af
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/0802d8af
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/0802d8af

Branch: refs/heads/branch-3
Commit: 0802d8afa355d9a0683fdb2e9c4963e8fea8644f
Parents: 082a707
Author: Vinayakumar B <vi...@apache.org>
Authored: Wed Jan 17 14:16:48 2018 +0530
Committer: Vinayakumar B <vi...@apache.org>
Committed: Wed Jan 17 14:28:09 2018 +0530

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java | 2 ++
 .../hadoop/hdfs/server/datanode/web/DatanodeHttpServer.java | 5 ++++-
 .../hadoop-hdfs/src/main/resources/hdfs-default.xml         | 9 +++++++++
 3 files changed, 15 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0802d8af/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index 1058b4a..9c51f25 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
@@ -617,6 +617,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String  DFS_DATANODE_HTTP_ADDRESS_KEY = "dfs.datanode.http.address";
   public static final int     DFS_DATANODE_HTTP_DEFAULT_PORT = 9864;
   public static final String  DFS_DATANODE_HTTP_ADDRESS_DEFAULT = "0.0.0.0:" + DFS_DATANODE_HTTP_DEFAULT_PORT;
+  public static final String  DFS_DATANODE_HTTP_INTERNAL_PROXY_PORT =
+      "dfs.datanode.http.internal-proxy.port";
   public static final String  DFS_DATANODE_MAX_RECEIVER_THREADS_KEY =
       HdfsClientConfigKeys.DeprecatedKeys.DFS_DATANODE_MAX_RECEIVER_THREADS_KEY;
   public static final int     DFS_DATANODE_MAX_RECEIVER_THREADS_DEFAULT = 4096;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0802d8af/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/DatanodeHttpServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/DatanodeHttpServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/DatanodeHttpServer.java
index b51b1fc..2e46b28 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/DatanodeHttpServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/DatanodeHttpServer.java
@@ -72,6 +72,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ADMIN;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HTTPS_ADDRESS_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HTTPS_ADDRESS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HTTP_ADDRESS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HTTP_INTERNAL_PROXY_PORT;
 
 public class DatanodeHttpServer implements Closeable {
   private final HttpServer2 infoServer;
@@ -97,12 +98,14 @@ public class DatanodeHttpServer implements Closeable {
 
     Configuration confForInfoServer = new Configuration(conf);
     confForInfoServer.setInt(HttpServer2.HTTP_MAX_THREADS_KEY, 10);
+    int proxyPort =
+        confForInfoServer.getInt(DFS_DATANODE_HTTP_INTERNAL_PROXY_PORT, 0);
     HttpServer2.Builder builder = new HttpServer2.Builder()
         .setName("datanode")
         .setConf(confForInfoServer)
         .setACL(new AccessControlList(conf.get(DFS_ADMIN, " ")))
         .hostName(getHostnameForSpnegoPrincipal(confForInfoServer))
-        .addEndpoint(URI.create("http://localhost:0"))
+        .addEndpoint(URI.create("http://localhost:" + proxyPort))
         .setFindPort(true);
 
     final boolean xFrameEnabled = conf.getBoolean(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0802d8af/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
index 190a06d..6be11b9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
@@ -146,6 +146,15 @@
 </property>
 
 <property>
+  <name>dfs.datanode.http.internal-proxy.port</name>
+  <value>0</value>
+  <description>
+    The datanode's internal web proxy port.
+    By default it selects a random port available in runtime.
+  </description>
+</property>
+
+<property>
   <name>dfs.datanode.handler.count</name>
   <value>10</value>
   <description>The number of server threads for the datanode.</description>


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[09/10] hadoop git commit: YARN-7732. Support Generic AM Simulator from SynthGenerator. (Contributed by Young Chen via curino)

Posted by cu...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/5c6adb3d/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/TestSynthJobGeneration.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/TestSynthJobGeneration.java b/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/TestSynthJobGeneration.java
index 2b1971a..794cd47 100644
--- a/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/TestSynthJobGeneration.java
+++ b/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/TestSynthJobGeneration.java
@@ -17,20 +17,25 @@
  */
 package org.apache.hadoop.yarn.sls;
 
+import org.apache.commons.math3.random.JDKRandomGenerator;
+import org.codehaus.jackson.map.JsonMappingException;
+import org.codehaus.jackson.map.ObjectMapper;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.TaskType;
-import org.apache.hadoop.tools.rumen.TaskAttemptInfo;
 import org.apache.hadoop.yarn.sls.synthetic.SynthJob;
 import org.apache.hadoop.yarn.sls.synthetic.SynthTraceJobProducer;
-import org.apache.hadoop.mapreduce.MRJobConfig;
-import org.apache.log4j.Logger;
 import org.junit.Assert;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.util.Arrays;
 
 import static org.junit.Assert.assertTrue;
 
+import static org.codehaus.jackson.JsonParser.Feature.INTERN_FIELD_NAMES;
+import static org.codehaus.jackson.map.DeserializationConfig.Feature.FAIL_ON_UNKNOWN_PROPERTIES;
+
 /**
  * Simple test class driving the {@code SynthTraceJobProducer}, and validating
  * jobs produce are within expected range.
@@ -38,10 +43,60 @@ import static org.junit.Assert.assertTrue;
 public class TestSynthJobGeneration {
 
   public final static Logger LOG =
-      Logger.getLogger(TestSynthJobGeneration.class);
+      LoggerFactory.getLogger(TestSynthJobGeneration.class);
 
   @Test
-  public void test() throws IllegalArgumentException, IOException {
+  public void testWorkloadGenerateTime()
+      throws IllegalArgumentException, IOException {
+
+    String workloadJson = "{\"job_classes\": [], \"time_distribution\":["
+        + "{\"time\": 0, \"weight\": 1}, " + "{\"time\": 30, \"weight\": 0},"
+        + "{\"time\": 60, \"weight\": 2}," + "{\"time\": 90, \"weight\": 1}"
+        + "]}";
+
+    ObjectMapper mapper = new ObjectMapper();
+    mapper.configure(INTERN_FIELD_NAMES, true);
+    mapper.configure(FAIL_ON_UNKNOWN_PROPERTIES, false);
+    SynthTraceJobProducer.Workload wl =
+        mapper.readValue(workloadJson, SynthTraceJobProducer.Workload.class);
+
+    JDKRandomGenerator rand = new JDKRandomGenerator();
+    rand.setSeed(0);
+
+    wl.init(rand);
+
+    int bucket0 = 0;
+    int bucket1 = 0;
+    int bucket2 = 0;
+    int bucket3 = 0;
+    for (int i = 0; i < 1000; ++i) {
+      long time = wl.generateSubmissionTime();
+      LOG.info("Generated time " + time);
+      if (time < 30) {
+        bucket0++;
+      } else if (time < 60) {
+        bucket1++;
+      } else if (time < 90) {
+        bucket2++;
+      } else {
+        bucket3++;
+      }
+    }
+
+    Assert.assertTrue(bucket0 > 0);
+    Assert.assertTrue(bucket1 == 0);
+    Assert.assertTrue(bucket2 > 0);
+    Assert.assertTrue(bucket3 > 0);
+    Assert.assertTrue(bucket2 > bucket0);
+    Assert.assertTrue(bucket2 > bucket3);
+
+    LOG.info("bucket0 {}, bucket1 {}, bucket2 {}, bucket3 {}", bucket0, bucket1,
+        bucket2, bucket3);
+
+  }
+
+  @Test
+  public void testMapReduce() throws IllegalArgumentException, IOException {
 
     Configuration conf = new Configuration();
 
@@ -50,47 +105,155 @@ public class TestSynthJobGeneration {
 
     SynthTraceJobProducer stjp = new SynthTraceJobProducer(conf);
 
+    LOG.info(stjp.toString());
+
     SynthJob js = (SynthJob) stjp.getNextJob();
 
     int jobCount = 0;
 
     while (js != null) {
-      LOG.info((jobCount++) + " " + js.getQueueName() + " -- "
-          + js.getJobClass().getClassName() + " (conf: "
-          + js.getJobConf().get(MRJobConfig.QUEUE_NAME) + ") " + " submission: "
-          + js.getSubmissionTime() + ", " + " duration: " + js.getDuration()
-          + " numMaps: " + js.getNumberMaps() + " numReduces: "
-          + js.getNumberReduces());
+      LOG.info(js.toString());
+      validateJob(js);
+      js = (SynthJob) stjp.getNextJob();
+      jobCount++;
+    }
 
+    Assert.assertEquals(stjp.getNumJobs(), jobCount);
+  }
+
+  @Test
+  public void testGeneric() throws IllegalArgumentException, IOException {
+    Configuration conf = new Configuration();
+
+    conf.set(SynthTraceJobProducer.SLS_SYNTHETIC_TRACE_FILE,
+        "src/test/resources/syn_generic.json");
+
+    SynthTraceJobProducer stjp = new SynthTraceJobProducer(conf);
+
+    LOG.info(stjp.toString());
+
+    SynthJob js = (SynthJob) stjp.getNextJob();
+
+    int jobCount = 0;
+
+    while (js != null) {
+      LOG.info(js.toString());
       validateJob(js);
       js = (SynthJob) stjp.getNextJob();
+      jobCount++;
     }
 
     Assert.assertEquals(stjp.getNumJobs(), jobCount);
   }
 
-  private void validateJob(SynthJob js) {
+  @Test
+  public void testStream() throws IllegalArgumentException, IOException {
+    Configuration conf = new Configuration();
 
-    assertTrue(js.getSubmissionTime() > 0);
-    assertTrue(js.getDuration() > 0);
-    assertTrue(js.getNumberMaps() >= 0);
-    assertTrue(js.getNumberReduces() >= 0);
-    assertTrue(js.getNumberMaps() + js.getNumberReduces() > 0);
-    assertTrue(js.getTotalSlotTime() >= 0);
+    conf.set(SynthTraceJobProducer.SLS_SYNTHETIC_TRACE_FILE,
+        "src/test/resources/syn_stream.json");
+
+    SynthTraceJobProducer stjp = new SynthTraceJobProducer(conf);
+
+    LOG.info(stjp.toString());
+
+    SynthJob js = (SynthJob) stjp.getNextJob();
+
+    int jobCount = 0;
+
+    while (js != null) {
+      LOG.info(js.toString());
+      validateJob(js);
+      js = (SynthJob) stjp.getNextJob();
+      jobCount++;
+    }
+
+    Assert.assertEquals(stjp.getNumJobs(), jobCount);
+  }
 
-    for (int i = 0; i < js.getNumberMaps(); i++) {
-      TaskAttemptInfo tai = js.getTaskAttemptInfo(TaskType.MAP, i, 0);
-      assertTrue(tai.getRuntime() > 0);
+  @Test
+  public void testSample() throws IOException {
+
+    ObjectMapper mapper = new ObjectMapper();
+    mapper.configure(INTERN_FIELD_NAMES, true);
+    mapper.configure(FAIL_ON_UNKNOWN_PROPERTIES, false);
+
+    JDKRandomGenerator rand = new JDKRandomGenerator();
+    rand.setSeed(0);
+
+    String valJson = "{\"val\" : 5 }";
+    SynthTraceJobProducer.Sample valSample =
+        mapper.readValue(valJson, SynthTraceJobProducer.Sample.class);
+    valSample.init(rand);
+    int val = valSample.getInt();
+    Assert.assertEquals(5, val);
+
+    String distJson = "{\"val\" : 5, \"std\" : 1 }";
+    SynthTraceJobProducer.Sample distSample =
+        mapper.readValue(distJson, SynthTraceJobProducer.Sample.class);
+    distSample.init(rand);
+    double dist = distSample.getDouble();
+    Assert.assertTrue(dist > 2 && dist < 8);
+
+    String normdistJson = "{\"val\" : 5, \"std\" : 1, \"dist\": \"NORM\" }";
+    SynthTraceJobProducer.Sample normdistSample =
+        mapper.readValue(normdistJson, SynthTraceJobProducer.Sample.class);
+    normdistSample.init(rand);
+    double normdist = normdistSample.getDouble();
+    Assert.assertTrue(normdist > 2 && normdist < 8);
+
+    String discreteJson = "{\"discrete\" : [2, 4, 6, 8]}";
+    SynthTraceJobProducer.Sample discreteSample =
+        mapper.readValue(discreteJson, SynthTraceJobProducer.Sample.class);
+    discreteSample.init(rand);
+    int discrete = discreteSample.getInt();
+    Assert.assertTrue(
+        Arrays.asList(new Integer[] {2, 4, 6, 8}).contains(discrete));
+
+    String discreteWeightsJson =
+        "{\"discrete\" : [2, 4, 6, 8], " + "\"weights\": [0, 0, 0, 1]}";
+    SynthTraceJobProducer.Sample discreteWeightsSample = mapper
+        .readValue(discreteWeightsJson, SynthTraceJobProducer.Sample.class);
+    discreteWeightsSample.init(rand);
+    int discreteWeights = discreteWeightsSample.getInt();
+    Assert.assertEquals(8, discreteWeights);
+
+    String invalidJson = "{\"val\" : 5, \"discrete\" : [2, 4, 6, 8], "
+        + "\"weights\": [0, 0, 0, 1]}";
+    try {
+      mapper.readValue(invalidJson, SynthTraceJobProducer.Sample.class);
+      Assert.fail();
+    } catch (JsonMappingException e) {
+      Assert.assertTrue(e.getMessage().startsWith("Instantiation of"));
     }
 
-    for (int i = 0; i < js.getNumberReduces(); i++) {
-      TaskAttemptInfo tai = js.getTaskAttemptInfo(TaskType.REDUCE, i, 0);
-      assertTrue(tai.getRuntime() > 0);
+    String invalidDistJson =
+        "{\"val\" : 5, \"std\" : 1, " + "\"dist\": \"INVALID\" }";
+    try {
+      mapper.readValue(invalidDistJson, SynthTraceJobProducer.Sample.class);
+      Assert.fail();
+    } catch (JsonMappingException e) {
+      Assert.assertTrue(e.getMessage().startsWith("Instantiation of"));
     }
+  }
+
+  private void validateJob(SynthJob js) {
+
+    assertTrue(js.getSubmissionTime() > 0);
+    assertTrue(js.getDuration() > 0);
+    assertTrue(js.getTotalSlotTime() >= 0);
 
     if (js.hasDeadline()) {
       assertTrue(js.getDeadline() > js.getSubmissionTime() + js.getDuration());
     }
 
+    assertTrue(js.getTasks().size() > 0);
+
+    for (SynthJob.SynthTask t : js.getTasks()) {
+      assertTrue(t.getType() != null);
+      assertTrue(t.getTime() > 0);
+      assertTrue(t.getMemory() > 0);
+      assertTrue(t.getVcores() > 0);
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5c6adb3d/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/appmaster/TestAMSimulator.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/appmaster/TestAMSimulator.java b/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/appmaster/TestAMSimulator.java
index a67845b..bfc7d0c 100644
--- a/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/appmaster/TestAMSimulator.java
+++ b/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/appmaster/TestAMSimulator.java
@@ -139,7 +139,7 @@ public class TestAMSimulator {
     String queue = "default";
     List<ContainerSimulator> containers = new ArrayList<>();
     app.init(1000, containers, rm, null, 0, 1000000L, "user1", queue, true,
-        appId, 0, SLSConfiguration.getAMContainerResource(conf));
+        appId, 0, SLSConfiguration.getAMContainerResource(conf), null);
     app.firstStep();
 
     verifySchedulerMetrics(appId);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5c6adb3d/hadoop-tools/hadoop-sls/src/test/resources/sls-runner.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/test/resources/sls-runner.xml b/hadoop-tools/hadoop-sls/src/test/resources/sls-runner.xml
index 2f076c2..344024a 100644
--- a/hadoop-tools/hadoop-sls/src/test/resources/sls-runner.xml
+++ b/hadoop-tools/hadoop-sls/src/test/resources/sls-runner.xml
@@ -45,6 +45,10 @@
     <name>yarn.sls.am.type.mapreduce</name>
     <value>org.apache.hadoop.yarn.sls.appmaster.MRAMSimulator</value>
   </property>
+  <property>
+    <name>yarn.sls.am.type.stream</name>
+    <value>org.apache.hadoop.yarn.sls.appmaster.StreamAMSimulator</value>
+  </property>
 
   <!-- Containers configuration -->
   <property>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5c6adb3d/hadoop-tools/hadoop-sls/src/test/resources/syn.json
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/test/resources/syn.json b/hadoop-tools/hadoop-sls/src/test/resources/syn.json
index 8479d23..c6e2c92 100644
--- a/hadoop-tools/hadoop-sls/src/test/resources/syn.json
+++ b/hadoop-tools/hadoop-sls/src/test/resources/syn.json
@@ -45,7 +45,7 @@
         },
         {
           "time": 60,
-          "jobs": 0
+          "weight": 0
         }
       ]
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5c6adb3d/hadoop-tools/hadoop-sls/src/test/resources/syn_generic.json
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/test/resources/syn_generic.json b/hadoop-tools/hadoop-sls/src/test/resources/syn_generic.json
new file mode 100644
index 0000000..bde4cd0
--- /dev/null
+++ b/hadoop-tools/hadoop-sls/src/test/resources/syn_generic.json
@@ -0,0 +1,54 @@
+{
+  "description": "tiny jobs workload",
+  "num_nodes": 20,
+  "nodes_per_rack": 4,
+  "num_jobs": 10,
+  "rand_seed": 2,
+  "workloads": [
+    {
+      "workload_name": "tiny-test",
+      "workload_weight": 0.5,
+      "description": "Sort jobs",
+      "queue_name": "sls_queue_1",
+      "job_classes": [
+        {
+          "class_name": "class_1",
+          "user_name": "foobar",
+          "class_weight": 1.0,
+          "type": "mapreduce",
+          "deadline_factor": {"val": 10},
+          "duration": {"val": 60, "std": 5},
+          "reservation": {"val": 0.5},
+          "tasks":[
+            {
+              "type": "map",
+              "priority": 20,
+              "count": { "val": 5, "std": 1},
+              "time": {"val": 10, "std": 2},
+              "max_memory": {"val": 1024},
+              "max_vcores": {"val": 1}
+            },
+            {
+              "type": "reduce",
+              "priority": 10,
+              "count": { "val": 5, "std": 1},
+              "time": {"val": 20, "std": 4},
+              "max_memory": {"val": 2048},
+              "max_vcores": {"val": 2}
+            }
+          ]
+        }
+      ],
+      "time_distribution": [
+        {
+          "time": 1,
+          "weight": 100
+        },
+        {
+          "time": 60,
+          "weight": 0
+        }
+      ]
+    }
+  ]
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5c6adb3d/hadoop-tools/hadoop-sls/src/test/resources/syn_stream.json
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/test/resources/syn_stream.json b/hadoop-tools/hadoop-sls/src/test/resources/syn_stream.json
new file mode 100644
index 0000000..a85065b
--- /dev/null
+++ b/hadoop-tools/hadoop-sls/src/test/resources/syn_stream.json
@@ -0,0 +1,46 @@
+{
+  "description": "stream workload",
+  "num_nodes": 20,
+  "nodes_per_rack": 4,
+  "num_jobs": 5,
+  "rand_seed": 2,
+  "workloads": [
+    {
+      "workload_name": "tiny-test",
+      "workload_weight": 1,
+      "description": "long lived streaming jobs",
+      "queue_name": "sls_queue_1",
+      "job_classes": [
+        {
+          "class_name": "class_1",
+          "user_name": "foobar",
+          "class_weight": 1.0,
+          "type": "stream",
+          "deadline_factor": {"val": 10},
+          "duration": {"val": 30, "std": 5},
+          "reservation": {"val": 0.5},
+          "tasks":[
+            {
+              "type": "stream",
+              "priority": 20,
+              "count": { "val": 2},
+              "time": {"val": 60000},
+              "max_memory": {"val": 4096},
+              "max_vcores": {"val": 4}
+            }
+          ]
+        }
+      ],
+      "time_distribution": [
+        {
+          "time": 1,
+          "weight": 100
+        },
+        {
+          "time": 2,
+          "weight": 0
+        }
+      ]
+    }
+  ]
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[08/10] hadoop git commit: YARN-7798. Refactor SLS Reservation Creation. Contributed by Young Chen.

Posted by cu...@apache.org.
YARN-7798. Refactor SLS Reservation Creation. Contributed by Young Chen.

(cherry picked from commit 16be42d3097c13b17d704e5b6dc8d66bd5ff6d9a)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/ea06ad9e
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/ea06ad9e
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/ea06ad9e

Branch: refs/heads/branch-3
Commit: ea06ad9efa20a615b33f427b73d403a35cfc387d
Parents: 702d280
Author: Yufei Gu <yu...@apache.org>
Authored: Thu Jan 25 13:06:01 2018 -0800
Committer: Carlo Curino <cu...@apache.org>
Committed: Tue Feb 20 17:04:08 2018 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/yarn/sls/SLSRunner.java   | 75 +++++++++-----------
 .../hadoop/yarn/sls/appmaster/AMSimulator.java  | 41 ++++++-----
 .../yarn/sls/appmaster/MRAMSimulator.java       | 74 ++++++++++++++-----
 .../apache/hadoop/yarn/sls/TestSLSRunner.java   |  2 +-
 .../yarn/sls/appmaster/TestAMSimulator.java     |  7 +-
 5 files changed, 119 insertions(+), 80 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea06ad9e/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java
index ad4310f..456602f 100644
--- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java
+++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java
@@ -57,7 +57,6 @@ import org.apache.hadoop.tools.rumen.TaskAttemptInfo;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
-import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.api.records.ReservationId;
@@ -444,7 +443,7 @@ public class SLSRunner extends Configured implements Tool {
 
     for (int i = 0; i < jobCount; i++) {
       runNewAM(amType, user, queue, oldAppId, jobStartTime, jobFinishTime,
-          getTaskContainers(jsonJob), null, getAMContainerResource(jsonJob));
+          getTaskContainers(jsonJob), getAMContainerResource(jsonJob));
     }
   }
 
@@ -607,7 +606,7 @@ public class SLSRunner extends Configured implements Tool {
 
     // Only supports the default job type currently
     runNewAM(SLSUtils.DEFAULT_JOB_TYPE, user, jobQueue, oldJobId,
-        jobStartTimeMS, jobFinishTimeMS, containerList, null,
+        jobStartTimeMS, jobFinishTimeMS, containerList,
         getAMContainerResource(null));
   }
 
@@ -628,10 +627,6 @@ public class SLSRunner extends Configured implements Tool {
     localConf.set("fs.defaultFS", "file:///");
     long baselineTimeMS = 0;
 
-    // reservations use wall clock time, so need to have a reference for that
-    UTCClock clock = new UTCClock();
-    long now = clock.getTime();
-
     try {
 
       // if we use the nodeFile this could have been not initialized yet.
@@ -670,13 +665,12 @@ public class SLSRunner extends Configured implements Tool {
         ArrayList<NodeId> keyAsArray = new ArrayList<NodeId>(nmMap.keySet());
         Random rand = new Random(stjp.getSeed());
 
-        Resource maxMapRes = Resource.newInstance(0, 0);
-        long maxMapDur = 0;
         // map tasks
         for (int i = 0; i < job.getNumberMaps(); i++) {
           TaskAttemptInfo tai = job.getTaskAttemptInfo(TaskType.MAP, i, 0);
-          RMNode node = nmMap
-              .get(keyAsArray.get(rand.nextInt(keyAsArray.size()))).getNode();
+          RMNode node =
+              nmMap.get(keyAsArray.get(rand.nextInt(keyAsArray.size())))
+                  .getNode();
           String hostname = "/" + node.getRackName() + "/" + node.getHostName();
           long containerLifeTime = tai.getRuntime();
           Resource containerResource =
@@ -684,55 +678,39 @@ public class SLSRunner extends Configured implements Tool {
                   (int) tai.getTaskInfo().getTaskVCores());
           containerList.add(new ContainerSimulator(containerResource,
               containerLifeTime, hostname, DEFAULT_MAPPER_PRIORITY, "map"));
-          maxMapRes = Resources.componentwiseMax(maxMapRes, containerResource);
-          maxMapDur =
-              containerLifeTime > maxMapDur ? containerLifeTime : maxMapDur;
-
         }
 
-        Resource maxRedRes = Resource.newInstance(0, 0);
-        long maxRedDur = 0;
         // reduce tasks
         for (int i = 0; i < job.getNumberReduces(); i++) {
           TaskAttemptInfo tai = job.getTaskAttemptInfo(TaskType.REDUCE, i, 0);
-          RMNode node = nmMap
-              .get(keyAsArray.get(rand.nextInt(keyAsArray.size()))).getNode();
+          RMNode node =
+              nmMap.get(keyAsArray.get(rand.nextInt(keyAsArray.size())))
+                  .getNode();
           String hostname = "/" + node.getRackName() + "/" + node.getHostName();
           long containerLifeTime = tai.getRuntime();
           Resource containerResource =
               Resource.newInstance((int) tai.getTaskInfo().getTaskMemory(),
                   (int) tai.getTaskInfo().getTaskVCores());
-          containerList.add(new ContainerSimulator(containerResource,
-              containerLifeTime, hostname, DEFAULT_REDUCER_PRIORITY, "reduce"));
-          maxRedRes = Resources.componentwiseMax(maxRedRes, containerResource);
-          maxRedDur =
-              containerLifeTime > maxRedDur ? containerLifeTime : maxRedDur;
-
+          containerList.add(
+              new ContainerSimulator(containerResource, containerLifeTime,
+                  hostname, DEFAULT_REDUCER_PRIORITY, "reduce"));
         }
 
-        // generating reservations for the jobs that require them
+        ReservationId reservationId = null;
 
-        ReservationSubmissionRequest rr = null;
         if (job.hasDeadline()) {
-          ReservationId reservationId =
+          reservationId =
               ReservationId.newInstance(this.rm.getStartTime(), AM_ID);
-
-          rr = ReservationClientUtil.createMRReservation(reservationId,
-              "reservation_" + AM_ID, maxMapRes, job.getNumberMaps(), maxMapDur,
-              maxRedRes, job.getNumberReduces(), maxRedDur,
-              now + jobStartTimeMS, now + job.getDeadline(),
-              job.getQueueName());
-
         }
 
         runNewAM(SLSUtils.DEFAULT_JOB_TYPE, user, jobQueue, oldJobId,
-            jobStartTimeMS, jobFinishTimeMS, containerList, rr,
-            getAMContainerResource(null));
+            jobStartTimeMS, jobFinishTimeMS, containerList, reservationId,
+            job.getDeadline(), getAMContainerResource(null));
+
       }
     } finally {
       stjp.close();
     }
-
   }
 
   private Resource getAMContainerResource(Map jsonJob) {
@@ -772,7 +750,17 @@ public class SLSRunner extends Configured implements Tool {
   private void runNewAM(String jobType, String user,
       String jobQueue, String oldJobId, long jobStartTimeMS,
       long jobFinishTimeMS, List<ContainerSimulator> containerList,
-      ReservationSubmissionRequest rr, Resource amContainerResource) {
+      Resource amContainerResource) {
+    runNewAM(jobType, user, jobQueue, oldJobId, jobStartTimeMS,
+        jobFinishTimeMS, containerList, null,  -1,
+        amContainerResource);
+  }
+
+  private void runNewAM(String jobType, String user,
+      String jobQueue, String oldJobId, long jobStartTimeMS,
+      long jobFinishTimeMS, List<ContainerSimulator> containerList,
+      ReservationId reservationId, long deadline,
+      Resource amContainerResource) {
 
     AMSimulator amSim = (AMSimulator) ReflectionUtils.newInstance(
         amClassMap.get(jobType), new Configuration());
@@ -787,10 +775,15 @@ public class SLSRunner extends Configured implements Tool {
         oldJobId = Integer.toString(AM_ID);
       }
       AM_ID++;
-
       amSim.init(heartbeatInterval, containerList, rm, this, jobStartTimeMS,
-          jobFinishTimeMS, user, jobQueue, isTracked, oldJobId, rr,
+          jobFinishTimeMS, user, jobQueue, isTracked, oldJobId,
           runner.getStartTimeMS(), amContainerResource);
+      if(reservationId != null) {
+        // if we have a ReservationId, delegate reservation creation to
+        // AMSim (reservation shape is impl specific)
+        UTCClock clock = new UTCClock();
+        amSim.initReservation(reservationId, deadline, clock.getTime());
+      }
       runner.schedule(amSim);
       maxRuntime = Math.max(maxRuntime, jobFinishTimeMS);
       numTasks += containerList.size();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea06ad9e/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/AMSimulator.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/AMSimulator.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/AMSimulator.java
index 72698ea..5727b5f 100644
--- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/AMSimulator.java
+++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/AMSimulator.java
@@ -85,7 +85,7 @@ public abstract class AMSimulator extends TaskRunner.Task {
   protected final BlockingQueue<AllocateResponse> responseQueue;
   private int responseId = 0;
   // user name
-  protected String user;  
+  private String user;
   // queue name
   protected String queue;
   // am type
@@ -105,7 +105,7 @@ public abstract class AMSimulator extends TaskRunner.Task {
   // waiting for AM container
   volatile boolean isAMContainerRunning = false;
   volatile Container amContainer;
-  
+
   private static final Logger LOG = LoggerFactory.getLogger(AMSimulator.class);
 
   private Resource amContainerResource;
@@ -120,9 +120,8 @@ public abstract class AMSimulator extends TaskRunner.Task {
   public void init(int heartbeatInterval,
       List<ContainerSimulator> containerList, ResourceManager resourceManager,
       SLSRunner slsRunnner, long startTime, long finishTime, String simUser,
-      String simQueue, boolean tracked, String oldApp,
-      ReservationSubmissionRequest rr, long baseTimeMS,
-      Resource amContainerResource) {
+      String simQueue, boolean tracked, String oldApp, long baseTimeMS,
+      Resource amResource) {
     super.init(startTime, startTime + 1000000L * heartbeatInterval,
         heartbeatInterval);
     this.user = simUser;
@@ -134,8 +133,7 @@ public abstract class AMSimulator extends TaskRunner.Task {
     this.baselineTimeMS = baseTimeMS;
     this.traceStartTimeMS = startTime;
     this.traceFinishTimeMS = finishTime;
-    this.reservationRequest = rr;
-    this.amContainerResource = amContainerResource;
+    this.amContainerResource = amResource;
   }
 
   /**
@@ -171,6 +169,10 @@ public abstract class AMSimulator extends TaskRunner.Task {
     isAMContainerRunning = true;
   }
 
+  protected void setReservationRequest(ReservationSubmissionRequest rr){
+    this.reservationRequest = rr;
+  }
+
   private ReservationId submitReservationWhenSpecified()
       throws IOException, InterruptedException {
     if (reservationRequest != null) {
@@ -256,7 +258,7 @@ public abstract class AMSimulator extends TaskRunner.Task {
               simulateStartTimeMS, simulateFinishTimeMS);
     }
   }
-  
+
   protected ResourceRequest createResourceRequest(
           Resource resource, String host, int priority, int numContainers) {
     ResourceRequest request = recordFactory
@@ -269,7 +271,7 @@ public abstract class AMSimulator extends TaskRunner.Task {
     request.setPriority(prio);
     return request;
   }
-  
+
   protected AllocateRequest createAllocateRequest(List<ResourceRequest> ask,
       List<ContainerId> toRelease) {
     AllocateRequest allocateRequest =
@@ -279,36 +281,39 @@ public abstract class AMSimulator extends TaskRunner.Task {
     allocateRequest.setReleaseList(toRelease);
     return allocateRequest;
   }
-  
+
   protected AllocateRequest createAllocateRequest(List<ResourceRequest> ask) {
     return createAllocateRequest(ask, new ArrayList<ContainerId>());
   }
 
   protected abstract void processResponseQueue() throws Exception;
-  
+
   protected abstract void sendContainerRequest() throws Exception;
-  
+
+  public abstract void initReservation(
+      ReservationId reservationId, long deadline, long now);
+
   protected abstract void checkStop();
-  
+
   private void submitApp(ReservationId reservationId)
           throws YarnException, InterruptedException, IOException {
     // ask for new application
     GetNewApplicationRequest newAppRequest =
         Records.newRecord(GetNewApplicationRequest.class);
-    GetNewApplicationResponse newAppResponse = 
+    GetNewApplicationResponse newAppResponse =
         rm.getClientRMService().getNewApplication(newAppRequest);
     appId = newAppResponse.getApplicationId();
-    
+
     // submit the application
     final SubmitApplicationRequest subAppRequest =
         Records.newRecord(SubmitApplicationRequest.class);
-    ApplicationSubmissionContext appSubContext = 
+    ApplicationSubmissionContext appSubContext =
         Records.newRecord(ApplicationSubmissionContext.class);
     appSubContext.setApplicationId(appId);
     appSubContext.setMaxAppAttempts(1);
     appSubContext.setQueue(queue);
     appSubContext.setPriority(Priority.newInstance(0));
-    ContainerLaunchContext conLauContext = 
+    ContainerLaunchContext conLauContext =
         Records.newRecord(ContainerLaunchContext.class);
     conLauContext.setApplicationACLs(new HashMap<>());
     conLauContext.setCommands(new ArrayList<>());
@@ -379,7 +384,7 @@ public abstract class AMSimulator extends TaskRunner.Task {
       }
     }
   }
-  
+
   protected List<ResourceRequest> packageRequests(
           List<ContainerSimulator> csList, int priority) {
     // create requests

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea06ad9e/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/MRAMSimulator.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/MRAMSimulator.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/MRAMSimulator.java
index 21bf054..18a155c 100644
--- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/MRAMSimulator.java
+++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/MRAMSimulator.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.yarn.sls.appmaster;
 import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.List;
@@ -34,6 +35,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest;
 import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -42,8 +44,10 @@ import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
+import org.apache.hadoop.yarn.sls.ReservationClientUtil;
 import org.apache.hadoop.yarn.sls.scheduler.ContainerSimulator;
 import org.apache.hadoop.yarn.sls.SLSRunner;
+import org.apache.hadoop.yarn.util.resource.Resources;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -51,51 +55,51 @@ import org.slf4j.LoggerFactory;
 @Unstable
 public class MRAMSimulator extends AMSimulator {
   /*
-  Vocabulary Used: 
+  Vocabulary Used:
   pending -> requests which are NOT yet sent to RM
   scheduled -> requests which are sent to RM but not yet assigned
   assigned -> requests which are assigned to a container
   completed -> request corresponding to which container has completed
-  
+
   Maps are scheduled as soon as their requests are received. Reduces are
   scheduled when all maps have finished (not support slow-start currently).
   */
-  
+
   private static final int PRIORITY_REDUCE = 10;
   private static final int PRIORITY_MAP = 20;
 
   // pending maps
   private LinkedList<ContainerSimulator> pendingMaps =
           new LinkedList<>();
-  
+
   // pending failed maps
   private LinkedList<ContainerSimulator> pendingFailedMaps =
           new LinkedList<ContainerSimulator>();
-  
+
   // scheduled maps
   private LinkedList<ContainerSimulator> scheduledMaps =
           new LinkedList<ContainerSimulator>();
-  
+
   // assigned maps
   private Map<ContainerId, ContainerSimulator> assignedMaps =
           new HashMap<ContainerId, ContainerSimulator>();
-  
+
   // reduces which are not yet scheduled
   private LinkedList<ContainerSimulator> pendingReduces =
           new LinkedList<ContainerSimulator>();
-  
+
   // pending failed reduces
   private LinkedList<ContainerSimulator> pendingFailedReduces =
           new LinkedList<ContainerSimulator>();
- 
+
   // scheduled reduces
   private LinkedList<ContainerSimulator> scheduledReduces =
           new LinkedList<ContainerSimulator>();
-  
+
   // assigned reduces
   private Map<ContainerId, ContainerSimulator> assignedReduces =
           new HashMap<ContainerId, ContainerSimulator>();
-  
+
   // all maps & reduces
   private LinkedList<ContainerSimulator> allMaps =
           new LinkedList<ContainerSimulator>();
@@ -117,14 +121,14 @@ public class MRAMSimulator extends AMSimulator {
   @SuppressWarnings("checkstyle:parameternumber")
   public void init(int heartbeatInterval,
       List<ContainerSimulator> containerList, ResourceManager rm, SLSRunner se,
-      long traceStartTime, long traceFinishTime, String user, String queue, 
-      boolean isTracked, String oldAppId, ReservationSubmissionRequest rr,
-      long baselineStartTimeMS, Resource amContainerResource) {
+      long traceStartTime, long traceFinishTime, String user, String queue,
+      boolean isTracked, String oldAppId, long baselineStartTimeMS,
+      Resource amContainerResource) {
     super.init(heartbeatInterval, containerList, rm, se,
         traceStartTime, traceFinishTime, user, queue, isTracked, oldAppId,
-        rr, baselineStartTimeMS, amContainerResource);
+        baselineStartTimeMS, amContainerResource);
     amtype = "mapreduce";
-    
+
     // get map/reduce tasks
     for (ContainerSimulator cs : containerList) {
       if (cs.getType().equals("map")) {
@@ -202,7 +206,7 @@ public class MRAMSimulator extends AMSimulator {
           }
         }
       }
-      
+
       // check finished
       if (isAMContainerRunning &&
               (mapFinished >= mapTotal) &&
@@ -234,7 +238,7 @@ public class MRAMSimulator extends AMSimulator {
       }
     }
   }
-  
+
   /**
    * restart running because of the am container killed
    */
@@ -322,7 +326,7 @@ public class MRAMSimulator extends AMSimulator {
     if (ask == null) {
       ask = new ArrayList<>();
     }
-    
+
     final AllocateRequest request = createAllocateRequest(ask);
     if (totalContainers == 0) {
       request.setProgress(1.0f);
@@ -349,6 +353,38 @@ public class MRAMSimulator extends AMSimulator {
   }
 
   @Override
+  public void initReservation(ReservationId reservationId, long deadline,
+      long now) {
+
+    Resource mapRes = getMaxResource(allMaps);
+    long mapDur = getMaxDuration(allMaps);
+    Resource redRes = getMaxResource(allReduces);
+    long redDur = getMaxDuration(allReduces);
+
+    ReservationSubmissionRequest rr = ReservationClientUtil.
+        createMRReservation(reservationId,
+            "reservation_" + reservationId.getId(), mapRes, allMaps.size(),
+            mapDur, redRes, allReduces.size(), redDur, now + traceStartTimeMS,
+            now + deadline, queue);
+
+    setReservationRequest(rr);
+  }
+
+  // Helper to compute the component-wise maximum resource used by any container
+  private Resource getMaxResource(Collection<ContainerSimulator> containers) {
+    return containers.parallelStream()
+        .map(ContainerSimulator::getResource)
+        .reduce(Resource.newInstance(0, 0), Resources::componentwiseMax);
+  }
+
+  // Helper to compute the maximum resource used by any map container
+  private long getMaxDuration(Collection<ContainerSimulator> containers) {
+    return containers.parallelStream()
+        .mapToLong(ContainerSimulator::getLifeTime)
+        .reduce(0L, Long::max);
+  }
+
+  @Override
   protected void checkStop() {
     if (isFinished) {
       super.setEndTime(System.currentTimeMillis());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea06ad9e/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/TestSLSRunner.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/TestSLSRunner.java b/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/TestSLSRunner.java
index 567f0d9..abb3b5e 100644
--- a/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/TestSLSRunner.java
+++ b/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/TestSLSRunner.java
@@ -78,7 +78,7 @@ public class TestSLSRunner extends BaseSLSRunnerTest {
     exitInvariantFile = "src/test/resources/exit-invariants.txt";
   }
 
-  @Test(timeout = 60000)
+  @Test(timeout = 90000)
   @SuppressWarnings("all")
   public void testSimulatorRunning() throws Exception {
     Configuration conf = new Configuration(false);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ea06ad9e/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/appmaster/TestAMSimulator.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/appmaster/TestAMSimulator.java b/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/appmaster/TestAMSimulator.java
index 02dc26e..a67845b 100644
--- a/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/appmaster/TestAMSimulator.java
+++ b/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/appmaster/TestAMSimulator.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.yarn.sls.appmaster;
 
 import com.codahale.metrics.MetricRegistry;
 import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
@@ -90,6 +91,10 @@ public class TestAMSimulator {
     }
 
     @Override
+    public void initReservation(ReservationId id, long deadline, long now){
+    }
+
+    @Override
     protected void checkStop() {
     }
   }
@@ -134,7 +139,7 @@ public class TestAMSimulator {
     String queue = "default";
     List<ContainerSimulator> containers = new ArrayList<>();
     app.init(1000, containers, rm, null, 0, 1000000L, "user1", queue, true,
-        appId, null, 0, SLSConfiguration.getAMContainerResource(conf));
+        appId, 0, SLSConfiguration.getAMContainerResource(conf));
     app.firstStep();
 
     verifySchedulerMetrics(appId);


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[02/10] hadoop git commit: HDFS-11848. Enhance dfsadmin listOpenFiles command to list files under a given path. Contributed by Yiqun Lin.

Posted by cu...@apache.org.
HDFS-11848. Enhance dfsadmin listOpenFiles command to list files under a given path. Contributed by Yiqun Lin.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/d3fbcd92
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/d3fbcd92
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/d3fbcd92

Branch: refs/heads/branch-3
Commit: d3fbcd92fe53192a319683b9ac72179cb28bd978
Parents: ee44783
Author: Yiqun Lin <yq...@apache.org>
Authored: Sat Jan 6 14:31:08 2018 +0800
Committer: Manoj Govindassamy <ma...@apache.org>
Committed: Fri Jan 12 13:52:27 2018 -0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/DFSClient.java  | 36 +++++++++-
 .../hadoop/hdfs/DistributedFileSystem.java      |  4 +-
 .../apache/hadoop/hdfs/client/HdfsAdmin.java    |  4 +-
 .../hadoop/hdfs/protocol/ClientProtocol.java    |  5 +-
 .../hadoop/hdfs/protocol/OpenFilesIterator.java | 10 ++-
 .../ClientNamenodeProtocolTranslatorPB.java     |  8 ++-
 .../src/main/proto/ClientNamenodeProtocol.proto |  1 +
 ...tNamenodeProtocolServerSideTranslatorPB.java |  2 +-
 .../federation/router/RouterRpcServer.java      |  6 +-
 .../hdfs/server/namenode/FSNamesystem.java      | 27 +++++---
 .../hdfs/server/namenode/LeaseManager.java      | 25 +++++--
 .../hdfs/server/namenode/NameNodeRpcServer.java |  8 ++-
 .../org/apache/hadoop/hdfs/tools/DFSAdmin.java  | 23 +++++--
 .../src/site/markdown/HDFSCommands.md           |  4 +-
 .../apache/hadoop/hdfs/TestDecommission.java    | 38 ++++++++++-
 .../org/apache/hadoop/hdfs/TestHdfsAdmin.java   |  4 +-
 .../hdfs/server/namenode/TestLeaseManager.java  |  8 ++-
 .../hdfs/server/namenode/TestListOpenFiles.java | 20 ++++--
 .../apache/hadoop/hdfs/tools/TestDFSAdmin.java  | 69 +++++++++++++++++++-
 19 files changed, 248 insertions(+), 54 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3fbcd92/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index 83c3b94..c20e2a0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -3030,11 +3030,26 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   @Deprecated
   public RemoteIterator<OpenFileEntry> listOpenFiles() throws IOException {
     checkOpen();
-    return listOpenFiles(EnumSet.of(OpenFilesType.ALL_OPEN_FILES));
+    return listOpenFiles(EnumSet.of(OpenFilesType.ALL_OPEN_FILES),
+        OpenFilesIterator.FILTER_PATH_DEFAULT);
   }
 
   /**
-   * Get a remote iterator to the open files list by type, managed by NameNode.
+   * Get a remote iterator to the open files list by path,
+   * managed by NameNode.
+   *
+   * @param path
+   * @throws IOException
+   */
+  public RemoteIterator<OpenFileEntry> listOpenFiles(String path)
+      throws IOException {
+    checkOpen();
+    return listOpenFiles(EnumSet.of(OpenFilesType.ALL_OPEN_FILES), path);
+  }
+
+  /**
+   * Get a remote iterator to the open files list by type,
+   * managed by NameNode.
    *
    * @param openFilesTypes
    * @throws IOException
@@ -3042,6 +3057,21 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   public RemoteIterator<OpenFileEntry> listOpenFiles(
       EnumSet<OpenFilesType> openFilesTypes) throws IOException {
     checkOpen();
-    return new OpenFilesIterator(namenode, tracer, openFilesTypes);
+    return listOpenFiles(openFilesTypes,
+        OpenFilesIterator.FILTER_PATH_DEFAULT);
+  }
+
+  /**
+   * Get a remote iterator to the open files list by type and path,
+   * managed by NameNode.
+   *
+   * @param openFilesTypes
+   * @param path
+   * @throws IOException
+   */
+  public RemoteIterator<OpenFileEntry> listOpenFiles(
+      EnumSet<OpenFilesType> openFilesTypes, String path) throws IOException {
+    checkOpen();
+    return new OpenFilesIterator(namenode, tracer, openFilesTypes, path);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3fbcd92/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
index 54b428e..369a5bd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
@@ -2985,8 +2985,8 @@ public class DistributedFileSystem extends FileSystem
   }
 
   public RemoteIterator<OpenFileEntry> listOpenFiles(
-      EnumSet<OpenFilesType> openFilesTypes) throws IOException {
-    return dfs.listOpenFiles(openFilesTypes);
+      EnumSet<OpenFilesType> openFilesTypes, String path) throws IOException {
+    return dfs.listOpenFiles(openFilesTypes, path);
   }
 
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3fbcd92/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java
index e620039..2c0659a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java
@@ -659,8 +659,8 @@ public class HdfsAdmin {
   }
 
   public RemoteIterator<OpenFileEntry> listOpenFiles(
-      EnumSet<OpenFilesType> openFilesTypes) throws IOException {
-    return dfs.listOpenFiles(openFilesTypes);
+      EnumSet<OpenFilesType> openFilesTypes, String path) throws IOException {
+    return dfs.listOpenFiles(openFilesTypes, path);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3fbcd92/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
index 80e053c..24f0321 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
@@ -1680,10 +1680,11 @@ public interface ClientProtocol {
    * the the list across batches are not atomic.
    *
    * @param prevId the cursor INode id.
-   * @param openFilesTypes types to filter the open files
+   * @param openFilesTypes types to filter the open files.
+   * @param path path to filter the open files.
    * @throws IOException
    */
   @Idempotent
   BatchedEntries<OpenFileEntry> listOpenFiles(long prevId,
-      EnumSet<OpenFilesType> openFilesTypes) throws IOException;
+      EnumSet<OpenFilesType> openFilesTypes, String path) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3fbcd92/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/OpenFilesIterator.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/OpenFilesIterator.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/OpenFilesIterator.java
index d113d65..c2b3781 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/OpenFilesIterator.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/OpenFilesIterator.java
@@ -37,6 +37,9 @@ import org.apache.htrace.core.Tracer;
 public class OpenFilesIterator extends
     BatchedRemoteIterator<Long, OpenFileEntry> {
 
+  /** No path to be filtered by default. */
+  public static final String FILTER_PATH_DEFAULT = "/";
+
   /**
    * Open file types to filter the results.
    */
@@ -67,20 +70,23 @@ public class OpenFilesIterator extends
   private final ClientProtocol namenode;
   private final Tracer tracer;
   private final EnumSet<OpenFilesType> types;
+  /** List files filtered by given path. */
+  private String path;
 
   public OpenFilesIterator(ClientProtocol namenode, Tracer tracer,
-      EnumSet<OpenFilesType> types) {
+      EnumSet<OpenFilesType> types, String path) {
     super(HdfsConstants.GRANDFATHER_INODE_ID);
     this.namenode = namenode;
     this.tracer = tracer;
     this.types = types;
+    this.path = path;
   }
 
   @Override
   public BatchedEntries<OpenFileEntry> makeRequest(Long prevId)
       throws IOException {
     try (TraceScope ignored = tracer.newScope("listOpenFiles")) {
-      return namenode.listOpenFiles(prevId, types);
+      return namenode.listOpenFiles(prevId, types, path);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3fbcd92/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
index c297302..b843e29 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
@@ -77,6 +77,7 @@ 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.OpenFileEntry;
+import org.apache.hadoop.hdfs.protocol.OpenFilesIterator;
 import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
@@ -1851,17 +1852,20 @@ public class ClientNamenodeProtocolTranslatorPB implements
   @Override
   public BatchedEntries<OpenFileEntry> listOpenFiles(long prevId)
       throws IOException {
-    return listOpenFiles(prevId, EnumSet.of(OpenFilesType.ALL_OPEN_FILES));
+    return listOpenFiles(prevId, EnumSet.of(OpenFilesType.ALL_OPEN_FILES),
+        OpenFilesIterator.FILTER_PATH_DEFAULT);
   }
 
   @Override
   public BatchedEntries<OpenFileEntry> listOpenFiles(long prevId,
-      EnumSet<OpenFilesType> openFilesTypes) throws IOException {
+      EnumSet<OpenFilesType> openFilesTypes, String path) throws IOException {
     ListOpenFilesRequestProto.Builder req =
         ListOpenFilesRequestProto.newBuilder().setId(prevId);
     if (openFilesTypes != null) {
       req.addAllTypes(PBHelperClient.convertOpenFileTypes(openFilesTypes));
     }
+    req.setPath(path);
+
     try {
       ListOpenFilesResponseProto response =
           rpcProxy.listOpenFiles(null, req.build());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3fbcd92/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
index 216821a..5c30f2c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
@@ -785,6 +785,7 @@ enum OpenFilesTypeProto {
 message ListOpenFilesRequestProto {
   required int64 id = 1;
   repeated OpenFilesTypeProto types = 2;
+  optional string path = 3;
 }
 
 message OpenFilesBatchResponseProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3fbcd92/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
index d34032d..d31007b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
@@ -1811,7 +1811,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       EnumSet<OpenFilesType> openFilesTypes =
           PBHelperClient.convertOpenFileTypes(req.getTypesList());
       BatchedEntries<OpenFileEntry> entries = server.listOpenFiles(req.getId(),
-          openFilesTypes);
+          openFilesTypes, req.getPath());
       ListOpenFilesResponseProto.Builder builder =
           ListOpenFilesResponseProto.newBuilder();
       builder.setHasMore(entries.hasMore());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3fbcd92/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
index d3d5959..4c317a8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
@@ -89,6 +89,7 @@ import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
+import org.apache.hadoop.hdfs.protocol.OpenFilesIterator;
 import org.apache.hadoop.hdfs.protocol.OpenFilesIterator.OpenFilesType;
 import org.apache.hadoop.hdfs.protocol.ReplicatedBlockStats;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
@@ -1918,12 +1919,13 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
   @Override
   public BatchedEntries<OpenFileEntry> listOpenFiles(long prevId)
       throws IOException {
-    return listOpenFiles(prevId, EnumSet.of(OpenFilesType.ALL_OPEN_FILES));
+    return listOpenFiles(prevId, EnumSet.of(OpenFilesType.ALL_OPEN_FILES),
+        OpenFilesIterator.FILTER_PATH_DEFAULT);
   }
 
   @Override
   public BatchedEntries<OpenFileEntry> listOpenFiles(long prevId,
-      EnumSet<OpenFilesType> openFilesTypes) throws IOException {
+      EnumSet<OpenFilesType> openFilesTypes, String path) throws IOException {
     checkOperation(OperationCategory.READ, false);
     return null;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3fbcd92/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 97423cb..ae57599 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -1748,11 +1748,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * TODO: HDFS-12969 - to report open files by type.
    *
    * @param prevId the cursor INode id.
-   * @param openFilesTypes
+   * @param openFilesTypes types to filter the open files.
+   * @param path path to filter the open files.
    * @throws IOException
    */
   BatchedListEntries<OpenFileEntry> listOpenFiles(long prevId,
-      EnumSet<OpenFilesType> openFilesTypes) throws IOException {
+      EnumSet<OpenFilesType> openFilesTypes, String path) throws IOException {
     final String operationName = "listOpenFiles";
     checkSuperuserPrivilege();
     checkOperation(OperationCategory.READ);
@@ -1761,10 +1762,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     try {
       checkOperation(OperationCategory.READ);
       if(openFilesTypes.contains(OpenFilesType.ALL_OPEN_FILES)) {
-        batchedListEntries = leaseManager.getUnderConstructionFiles(prevId);
+        batchedListEntries = leaseManager.getUnderConstructionFiles(prevId,
+            path);
       } else {
         if(openFilesTypes.contains(OpenFilesType.BLOCKING_DECOMMISSION)) {
-          batchedListEntries = getFilesBlockingDecom(prevId);
+          batchedListEntries = getFilesBlockingDecom(prevId, path);
         } else {
           throw new IllegalArgumentException("Unknown OpenFileType: "
               + openFilesTypes);
@@ -1780,7 +1782,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     return batchedListEntries;
   }
 
-  public BatchedListEntries<OpenFileEntry> getFilesBlockingDecom(long prevId) {
+  public BatchedListEntries<OpenFileEntry> getFilesBlockingDecom(long prevId,
+      String path) {
     assert hasReadLock();
     final List<OpenFileEntry> openFileEntries = Lists.newArrayList();
     LightWeightHashSet<Long> openFileIds = new LightWeightHashSet<>();
@@ -1798,10 +1801,16 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         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()));
+
+        String fullPathName = inodeFile.getFullPathName();
+        if (org.apache.commons.lang.StringUtils.isEmpty(path)
+            || fullPathName.startsWith(path)) {
+          openFileEntries.add(new OpenFileEntry(inodeFile.getId(),
+              inodeFile.getFullPathName(),
+              inodeFile.getFileUnderConstructionFeature().getClientName(),
+              inodeFile.getFileUnderConstructionFeature().getClientMachine()));
+        }
+
         if (openFileIds.size() >= this.maxListOpenFilesResponses) {
           return new BatchedListEntries<>(openFileEntries, true);
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3fbcd92/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
index 45699cb..29b4fe1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
@@ -39,10 +39,12 @@ import java.util.concurrent.Future;
 import com.google.common.collect.Lists;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
+import org.apache.hadoop.hdfs.protocol.OpenFilesIterator;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.util.Daemon;
@@ -258,6 +260,12 @@ public class LeaseManager {
     return iipSet;
   }
 
+  public BatchedListEntries<OpenFileEntry> getUnderConstructionFiles(
+      final long prevId) throws IOException {
+    return getUnderConstructionFiles(prevId,
+        OpenFilesIterator.FILTER_PATH_DEFAULT);
+  }
+
   /**
    * Get a batch of under construction files from the currently active leases.
    * File INodeID is the cursor used to fetch new batch of results and the
@@ -270,7 +278,7 @@ public class LeaseManager {
    * @throws IOException
    */
   public BatchedListEntries<OpenFileEntry> getUnderConstructionFiles(
-      final long prevId) throws IOException {
+      final long prevId, final String path) throws IOException {
     assert fsnamesystem.hasReadLock();
     SortedMap<Long, Lease> remainingLeases;
     synchronized (this) {
@@ -283,6 +291,7 @@ public class LeaseManager {
         Lists.newArrayListWithExpectedSize(numResponses);
 
     int count = 0;
+    String fullPathName = null;
     for (Long inodeId: inodeIds) {
       final INodeFile inodeFile =
           fsnamesystem.getFSDirectory().getInode(inodeId).asFile();
@@ -291,11 +300,15 @@ public class LeaseManager {
             + " is not under construction but has lease.");
         continue;
       }
-      openFileEntries.add(new OpenFileEntry(
-          inodeFile.getId(), inodeFile.getFullPathName(),
-          inodeFile.getFileUnderConstructionFeature().getClientName(),
-          inodeFile.getFileUnderConstructionFeature().getClientMachine()));
-      count++;
+
+      fullPathName = inodeFile.getFullPathName();
+      if (StringUtils.isEmpty(path) || fullPathName.startsWith(path)) {
+        openFileEntries.add(new OpenFileEntry(inodeFile.getId(), fullPathName,
+            inodeFile.getFileUnderConstructionFeature().getClientName(),
+            inodeFile.getFileUnderConstructionFeature().getClientMachine()));
+        count++;
+      }
+
       if (count >= numResponses) {
         break;
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3fbcd92/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
index b44aaf1..432df34 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
@@ -114,6 +114,7 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
+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.QuotaExceededException;
@@ -1324,14 +1325,15 @@ public class NameNodeRpcServer implements NamenodeProtocols {
   @Override // ClientProtocol
   public BatchedEntries<OpenFileEntry> listOpenFiles(long prevId)
       throws IOException {
-    return listOpenFiles(prevId, EnumSet.of(OpenFilesType.ALL_OPEN_FILES));
+    return listOpenFiles(prevId, EnumSet.of(OpenFilesType.ALL_OPEN_FILES),
+        OpenFilesIterator.FILTER_PATH_DEFAULT);
   }
 
   @Override // ClientProtocol
   public BatchedEntries<OpenFileEntry> listOpenFiles(long prevId,
-      EnumSet<OpenFilesType> openFilesTypes) throws IOException {
+      EnumSet<OpenFilesType> openFilesTypes, String path) throws IOException {
     checkNNStartup();
-    return namesystem.listOpenFiles(prevId, openFilesTypes);
+    return namesystem.listOpenFiles(prevId, openFilesTypes, path);
   }
 
   @Override // ClientProtocol

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3fbcd92/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
index 7f79b1c..a3e1bc1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
@@ -464,7 +464,7 @@ public class DFSAdmin extends FsShell {
     "\t[-getDatanodeInfo <datanode_host:ipc_port>]\n" +
     "\t[-metasave filename]\n" +
     "\t[-triggerBlockReport [-incremental] <datanode_host:ipc_port>]\n" +
-    "\t[-listOpenFiles [-blockingDecommission]]\n" +
+    "\t[-listOpenFiles [-blockingDecommission] [-path <path>]]\n" +
     "\t[-help [cmd]]\n";
 
   /**
@@ -918,16 +918,29 @@ public class DFSAdmin extends FsShell {
    * @param argv
    */
   public int listOpenFiles(String[] argv) throws IOException {
+    String path = null;
     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);
       }
+
+      path = StringUtils.popOptionWithArgument("-path", args);
     }
     if (types.isEmpty()) {
       types.add(OpenFilesType.ALL_OPEN_FILES);
     }
+
+    if (path != null) {
+      path = path.trim();
+      if (path.length() == 0) {
+        path = OpenFilesIterator.FILTER_PATH_DEFAULT;
+      }
+    } else {
+      path = OpenFilesIterator.FILTER_PATH_DEFAULT;
+    }
+
     EnumSet<OpenFilesType> openFilesTypes = EnumSet.copyOf(types);
 
     DistributedFileSystem dfs = getDFS();
@@ -941,9 +954,9 @@ public class DFSAdmin extends FsShell {
           dfsConf, HAUtil.getAddressOfActive(getDFS()), ClientProtocol.class,
           UserGroupInformation.getCurrentUser(), false);
       openFilesRemoteIterator = new OpenFilesIterator(proxy.getProxy(),
-          FsTracer.get(dfsConf), openFilesTypes);
+          FsTracer.get(dfsConf), openFilesTypes, path);
     } else {
-      openFilesRemoteIterator = dfs.listOpenFiles(openFilesTypes);
+      openFilesRemoteIterator = dfs.listOpenFiles(openFilesTypes, path);
     }
     printOpenFiles(openFilesRemoteIterator);
     return 0;
@@ -1982,7 +1995,7 @@ public class DFSAdmin extends FsShell {
           + " [-triggerBlockReport [-incremental] <datanode_host:ipc_port>]");
     } else if ("-listOpenFiles".equals(cmd)) {
       System.err.println("Usage: hdfs dfsadmin"
-          + " [-listOpenFiles [-blockingDecommission]]");
+          + " [-listOpenFiles [-blockingDecommission] [-path <path>]]");
     } else {
       System.err.println("Usage: hdfs dfsadmin");
       System.err.println("Note: Administrative commands can only be run as the HDFS superuser.");
@@ -2137,7 +2150,7 @@ public class DFSAdmin extends FsShell {
         return exitCode;
       }
     } else if ("-listOpenFiles".equals(cmd)) {
-      if ((argv.length != 1) && (argv.length != 2)) {
+      if ((argv.length > 4)) {
         printUsage(cmd);
         return exitCode;
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3fbcd92/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
index db983e2..03661f3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
@@ -370,7 +370,7 @@ Usage:
         hdfs dfsadmin [-getDatanodeInfo <datanode_host:ipc_port>]
         hdfs dfsadmin [-metasave filename]
         hdfs dfsadmin [-triggerBlockReport [-incremental] <datanode_host:ipc_port>]
-        hdfs dfsadmin [-listOpenFiles]
+        hdfs dfsadmin [-listOpenFiles [-blockingDecommission] [-path <path>]]
         hdfs dfsadmin [-help [cmd]]
 
 | COMMAND\_OPTION | Description |
@@ -407,7 +407,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. |
 | `-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. |
-| `-listOpenFiles` `[-blockingDecommission]` | List all open files currently managed by the NameNode along with client name and client machine accessing them. |
+| `-listOpenFiles` `[-blockingDecommission]` `[-path <path>]` | List all open files currently managed by the NameNode along with client name and client machine accessing them. Open files list will be filtered by given type and path. |
 | `-help` [cmd] | Displays help for the given command or all commands if none is specified. |
 
 Runs a HDFS dfsadmin client.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3fbcd92/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
index d82025c..c0a595b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
@@ -710,13 +710,49 @@ public class TestDecommission extends AdminStatesBaseTest {
         @Override
         public Boolean get() {
           try {
+            boolean result1 = false;
+            boolean result2 = false;
             toolOut.reset();
             assertEquals(0, ToolRunner.run(dfsAdmin,
                 new String[]{"-listOpenFiles", "-blockingDecommission"}));
             toolOut.flush();
-            return verifyOpenFilesListing(
+            result1 = verifyOpenFilesListing(
                 "dfsadmin -listOpenFiles -blockingDecommission",
                 closedFileSet, openFilesMap, toolOut, maxOpenFiles);
+
+            // test -blockingDecommission with option -path
+            if (openFilesMap.size() > 0) {
+              String firstOpenFile = null;
+              // Construct a new open-file and close-file map.
+              // Pick the first open file into new open-file map, remaining
+              //  open files move into close-files map.
+              HashMap<Path, FSDataOutputStream> newOpenFilesMap =
+                  new HashMap<>();
+              HashSet<Path> newClosedFileSet = new HashSet<>();
+              for (Map.Entry<Path, FSDataOutputStream> entry : openFilesMap
+                  .entrySet()) {
+                if (firstOpenFile == null) {
+                  newOpenFilesMap.put(entry.getKey(), entry.getValue());
+                  firstOpenFile = entry.getKey().toString();
+                } else {
+                  newClosedFileSet.add(entry.getKey());
+                }
+              }
+
+              toolOut.reset();
+              assertEquals(0,
+                  ToolRunner.run(dfsAdmin, new String[] {"-listOpenFiles",
+                      "-blockingDecommission", "-path", firstOpenFile}));
+              toolOut.flush();
+              result2 = verifyOpenFilesListing(
+                  "dfsadmin -listOpenFiles -blockingDecommission -path"
+                      + firstOpenFile,
+                  newClosedFileSet, newOpenFilesMap, toolOut, 1);
+            } else {
+              result2 = true;
+            }
+
+            return result1 && result2;
           } catch (Exception e) {
             LOG.warn("Unexpected exception: " + e);
           }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3fbcd92/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHdfsAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHdfsAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHdfsAdmin.java
index 3cb10bf..cc32a3c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHdfsAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHdfsAdmin.java
@@ -42,6 +42,7 @@ import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.hdfs.client.HdfsAdmin;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
+import org.apache.hadoop.hdfs.protocol.OpenFilesIterator;
 import org.apache.hadoop.hdfs.protocol.OpenFilesIterator.OpenFilesType;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.junit.After;
@@ -256,7 +257,8 @@ public class TestHdfsAdmin {
     HdfsAdmin hdfsAdmin = new HdfsAdmin(FileSystem.getDefaultUri(conf), conf);
     HashSet<Path> openFiles = new HashSet<>(openFileMap.keySet());
     RemoteIterator<OpenFileEntry> openFilesRemoteItr =
-        hdfsAdmin.listOpenFiles(EnumSet.of(OpenFilesType.ALL_OPEN_FILES));
+        hdfsAdmin.listOpenFiles(EnumSet.of(OpenFilesType.ALL_OPEN_FILES),
+            OpenFilesIterator.FILTER_PATH_DEFAULT);
     while (openFilesRemoteItr.hasNext()) {
       String filePath = openFilesRemoteItr.next().getFilePath();
       assertFalse(filePath + " should not be listed under open files!",

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3fbcd92/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestLeaseManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestLeaseManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestLeaseManager.java
index 0a8da4b..ccd908b6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestLeaseManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestLeaseManager.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
+import org.apache.hadoop.hdfs.protocol.OpenFilesIterator;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
@@ -405,8 +406,11 @@ public class TestLeaseManager {
         leaseManager.getINodeWithLeases(ancestorDirectory).size());
     assertEquals(iNodeIdWithLeaseCount,
         leaseManager.getUnderConstructionFiles(0).size());
-    assertEquals(0, (fsNamesystem.getFilesBlockingDecom(0) == null ?
-        0 : fsNamesystem.getFilesBlockingDecom(0).size()));
+    assertEquals(0,
+        (fsNamesystem.getFilesBlockingDecom(0,
+            OpenFilesIterator.FILTER_PATH_DEFAULT) == null ? 0
+                : fsNamesystem.getFilesBlockingDecom(0,
+                    OpenFilesIterator.FILTER_PATH_DEFAULT).size()));
   }
 
   private Map<String, INode> createINodeTree(INodeDirectory parentDir,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3fbcd92/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestListOpenFiles.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestListOpenFiles.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestListOpenFiles.java
index cfee7ba..70550d5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestListOpenFiles.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestListOpenFiles.java
@@ -44,6 +44,7 @@ import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
+import org.apache.hadoop.hdfs.protocol.OpenFilesIterator;
 import org.apache.hadoop.hdfs.protocol.OpenFilesIterator.OpenFilesType;
 import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
@@ -97,11 +98,13 @@ public class TestListOpenFiles {
     verifyOpenFiles(openFiles);
 
     BatchedEntries<OpenFileEntry> openFileEntryBatchedEntries =
-        nnRpc.listOpenFiles(0, EnumSet.of(OpenFilesType.ALL_OPEN_FILES));
+        nnRpc.listOpenFiles(0, EnumSet.of(OpenFilesType.ALL_OPEN_FILES),
+            OpenFilesIterator.FILTER_PATH_DEFAULT);
     assertTrue("Open files list should be empty!",
         openFileEntryBatchedEntries.size() == 0);
     BatchedEntries<OpenFileEntry> openFilesBlockingDecomEntries =
-        nnRpc.listOpenFiles(0, EnumSet.of(OpenFilesType.BLOCKING_DECOMMISSION));
+        nnRpc.listOpenFiles(0, EnumSet.of(OpenFilesType.BLOCKING_DECOMMISSION),
+            OpenFilesIterator.FILTER_PATH_DEFAULT);
     assertTrue("Open files list blocking decommission should be empty!",
         openFilesBlockingDecomEntries.size() == 0);
 
@@ -128,15 +131,16 @@ public class TestListOpenFiles {
   }
 
   private void verifyOpenFiles(Map<Path, FSDataOutputStream> openFiles,
-      EnumSet<OpenFilesType> openFilesTypes) throws IOException {
+      EnumSet<OpenFilesType> openFilesTypes, String path) throws IOException {
     HashSet<Path> remainingFiles = new HashSet<>(openFiles.keySet());
     OpenFileEntry lastEntry = null;
     BatchedEntries<OpenFileEntry> batchedEntries;
     do {
       if (lastEntry == null) {
-        batchedEntries = nnRpc.listOpenFiles(0, openFilesTypes);
+        batchedEntries = nnRpc.listOpenFiles(0, openFilesTypes, path);
       } else {
-        batchedEntries = nnRpc.listOpenFiles(lastEntry.getId(), openFilesTypes);
+        batchedEntries = nnRpc.listOpenFiles(lastEntry.getId(),
+            openFilesTypes, path);
       }
       assertTrue("Incorrect open files list size!",
           batchedEntries.size() <= BATCH_SIZE);
@@ -154,9 +158,11 @@ public class TestListOpenFiles {
 
   private void verifyOpenFiles(Map<Path, FSDataOutputStream> openFiles)
       throws IOException {
-    verifyOpenFiles(openFiles, EnumSet.of(OpenFilesType.ALL_OPEN_FILES));
+    verifyOpenFiles(openFiles, EnumSet.of(OpenFilesType.ALL_OPEN_FILES),
+        OpenFilesIterator.FILTER_PATH_DEFAULT);
     verifyOpenFiles(new HashMap<>(),
-        EnumSet.of(OpenFilesType.BLOCKING_DECOMMISSION));
+        EnumSet.of(OpenFilesType.BLOCKING_DECOMMISSION),
+        OpenFilesIterator.FILTER_PATH_DEFAULT);
   }
 
   private Set<Path> createFiles(FileSystem fileSystem, String fileNamePrefix,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d3fbcd92/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java
index 6a01de2..7237c88 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java
@@ -725,6 +725,67 @@ public class TestDFSAdmin {
             new String[]{"-listOpenFiles"}));
         verifyOpenFilesListing(closedFileSet, openFilesMap);
       }
+
+      // test -listOpenFiles command with option <path>
+      openFilesMap.clear();
+      Path file;
+      HashMap<Path, FSDataOutputStream> openFiles1 = new HashMap<>();
+      HashMap<Path, FSDataOutputStream> openFiles2 = new HashMap<>();
+      for (int i = 0; i < numOpenFiles; i++) {
+        if (i % 2 == 0) {
+          file = new Path(new Path("/tmp/files/a"), "open-file-" + i);
+        } else {
+          file = new Path(new Path("/tmp/files/b"), "open-file-" + i);
+        }
+
+        DFSTestUtil.createFile(fs, file, fileLength, replFactor, 12345L);
+        FSDataOutputStream outputStream = fs.append(file);
+
+        if (i % 2 == 0) {
+          openFiles1.put(file, outputStream);
+        } else {
+          openFiles2.put(file, outputStream);
+        }
+        openFilesMap.put(file, outputStream);
+      }
+
+      resetStream();
+      // list all open files
+      assertEquals(0,
+          ToolRunner.run(dfsAdmin, new String[] {"-listOpenFiles"}));
+      verifyOpenFilesListing(null, openFilesMap);
+
+      resetStream();
+      // list open files under directory path /tmp/files/a
+      assertEquals(0, ToolRunner.run(dfsAdmin,
+          new String[] {"-listOpenFiles", "-path", "/tmp/files/a"}));
+      verifyOpenFilesListing(null, openFiles1);
+
+      resetStream();
+      // list open files without input path
+      assertEquals(-1, ToolRunner.run(dfsAdmin,
+          new String[] {"-listOpenFiles", "-path"}));
+      // verify the error
+      String outStr = scanIntoString(err);
+      assertTrue(outStr.contains("listOpenFiles: option"
+          + " -path requires 1 argument"));
+
+      resetStream();
+      // list open files with empty path
+      assertEquals(0, ToolRunner.run(dfsAdmin,
+          new String[] {"-listOpenFiles", "-path", ""}));
+      // all the open files will be listed
+      verifyOpenFilesListing(null, openFilesMap);
+
+      resetStream();
+      // list invalid path file
+      assertEquals(0, ToolRunner.run(dfsAdmin,
+          new String[] {"-listOpenFiles", "-path", "/invalid_path"}));
+      outStr = scanIntoString(out);
+      for (Path openFilePath : openFilesMap.keySet()) {
+        assertThat(outStr, not(containsString(openFilePath.toString())));
+      }
+      DFSTestUtil.closeOpenFiles(openFilesMap, openFilesMap.size());
     }
   }
 
@@ -732,9 +793,13 @@ public class TestDFSAdmin {
       HashMap<Path, FSDataOutputStream> openFilesMap) {
     final String outStr = scanIntoString(out);
     LOG.info("dfsadmin -listOpenFiles output: \n" + out);
-    for (Path closedFilePath : closedFileSet) {
-      assertThat(outStr, not(containsString(closedFilePath.toString() + "\n")));
+    if (closedFileSet != null) {
+      for (Path closedFilePath : closedFileSet) {
+        assertThat(outStr,
+            not(containsString(closedFilePath.toString() + "\n")));
+      }
     }
+
     for (Path openFilePath : openFilesMap.keySet()) {
       assertThat(outStr, is(containsString(openFilePath.toString() + "\n")));
     }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[07/10] hadoop git commit: Revert "HADOOP-15027. AliyunOSS: Support multi-thread pre-read to improve sequential read from Hadoop to Aliyun OSS performance. (Contributed by Jinhu Wu)"

Posted by cu...@apache.org.
Revert "HADOOP-15027. AliyunOSS: Support multi-thread pre-read to improve sequential read from Hadoop to Aliyun OSS performance. (Contributed by Jinhu Wu)"

This reverts commit 082a707bae4bb97444a34c00eecd62975807388d.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/702d280c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/702d280c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/702d280c

Branch: refs/heads/branch-3
Commit: 702d280c853fa67c9fcb10959b47cc19f03a98a7
Parents: 0802d8a
Author: Jason Lowe <jl...@apache.org>
Authored: Wed Jan 17 10:11:24 2018 -0600
Committer: Jason Lowe <jl...@apache.org>
Committed: Wed Jan 17 10:11:24 2018 -0600

----------------------------------------------------------------------
 .../dev-support/findbugs-exclude.xml            |   8 -
 .../fs/aliyun/oss/AliyunOSSFileReaderTask.java  | 109 --------------
 .../fs/aliyun/oss/AliyunOSSFileSystem.java      |  31 +---
 .../fs/aliyun/oss/AliyunOSSInputStream.java     | 149 ++++++-------------
 .../hadoop/fs/aliyun/oss/AliyunOSSUtils.java    |  12 --
 .../apache/hadoop/fs/aliyun/oss/Constants.java  |  13 +-
 .../apache/hadoop/fs/aliyun/oss/ReadBuffer.java |  86 -----------
 .../fs/aliyun/oss/TestAliyunOSSInputStream.java |  49 ------
 8 files changed, 50 insertions(+), 407 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/702d280c/hadoop-tools/hadoop-aliyun/dev-support/findbugs-exclude.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/dev-support/findbugs-exclude.xml b/hadoop-tools/hadoop-aliyun/dev-support/findbugs-exclude.xml
index c55f8e3..40d78d0 100644
--- a/hadoop-tools/hadoop-aliyun/dev-support/findbugs-exclude.xml
+++ b/hadoop-tools/hadoop-aliyun/dev-support/findbugs-exclude.xml
@@ -15,12 +15,4 @@
    limitations under the License.
 -->
 <FindBugsFilter>
-    <!-- Disable FindBugs warning and return the buffer to caller directly.
-         It is convenient and efficient because we do not need to copy the buffer
-    -->
-    <Match>
-        <Class name="org.apache.hadoop.fs.aliyun.oss.ReadBuffer" />
-        <Method name="getBuffer" />
-        <Bug pattern="EI_EXPOSE_REP" />
-    </Match>
 </FindBugsFilter>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/702d280c/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileReaderTask.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileReaderTask.java b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileReaderTask.java
deleted file mode 100644
index e5bfc2c..0000000
--- a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileReaderTask.java
+++ /dev/null
@@ -1,109 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * <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.fs.aliyun.oss;
-
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.io.retry.RetryPolicies;
-import org.apache.hadoop.io.retry.RetryPolicy;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-
-/**
- * Used by {@link AliyunOSSInputStream} as an task that submitted
- * to the thread pool.
- * Each AliyunOSSFileReaderTask reads one part of the file so that
- * we can accelerate the sequential read.
- */
-public class AliyunOSSFileReaderTask implements Runnable {
-  public static final Logger LOG =
-      LoggerFactory.getLogger(AliyunOSSFileReaderTask.class);
-
-  private String key;
-  private AliyunOSSFileSystemStore store;
-  private ReadBuffer readBuffer;
-  private static final int MAX_RETRIES = 3;
-  private RetryPolicy retryPolicy;
-
-  public AliyunOSSFileReaderTask(String key, AliyunOSSFileSystemStore store,
-      ReadBuffer readBuffer) {
-    this.key = key;
-    this.store = store;
-    this.readBuffer = readBuffer;
-    RetryPolicy defaultPolicy =
-        RetryPolicies.retryUpToMaximumCountWithFixedSleep(
-            MAX_RETRIES, 3, TimeUnit.SECONDS);
-    Map<Class<? extends Exception>, RetryPolicy> policies = new HashMap<>();
-    policies.put(IOException.class, defaultPolicy);
-    policies.put(IndexOutOfBoundsException.class,
-        RetryPolicies.TRY_ONCE_THEN_FAIL);
-    policies.put(NullPointerException.class,
-        RetryPolicies.TRY_ONCE_THEN_FAIL);
-
-    this.retryPolicy = RetryPolicies.retryByException(defaultPolicy, policies);
-  }
-
-  @Override
-  public void run() {
-    int retries = 0;
-    readBuffer.lock();
-    try {
-      while (true) {
-        try (InputStream in = store.retrieve(
-            key, readBuffer.getByteStart(), readBuffer.getByteEnd())) {
-          IOUtils.readFully(in, readBuffer.getBuffer(),
-              0, readBuffer.getBuffer().length);
-          readBuffer.setStatus(ReadBuffer.STATUS.SUCCESS);
-          break;
-        } catch (Exception e) {
-          LOG.warn("Exception thrown when retrieve key: "
-              + this.key + ", exception: " + e);
-          try {
-            RetryPolicy.RetryAction rc = retryPolicy.shouldRetry(
-                e, retries++, 0, true);
-            if (rc.action == RetryPolicy.RetryAction.RetryDecision.RETRY) {
-              Thread.sleep(rc.delayMillis);
-            } else {
-              //should not retry
-              break;
-            }
-          } catch (Exception ex) {
-            //FAIL
-            LOG.warn("Exception thrown when call shouldRetry, exception " + ex);
-            break;
-          }
-        }
-      }
-
-      if (readBuffer.getStatus() != ReadBuffer.STATUS.SUCCESS) {
-        readBuffer.setStatus(ReadBuffer.STATUS.ERROR);
-      }
-
-      //notify main thread which wait for this buffer
-      readBuffer.signalAll();
-    } finally {
-      readBuffer.unlock();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/702d280c/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystem.java b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystem.java
index afff223..41d475d 100644
--- a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystem.java
+++ b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystem.java
@@ -24,9 +24,7 @@ import java.net.URI;
 import java.util.ArrayList;
 import java.util.EnumSet;
 import java.util.List;
-import java.util.concurrent.TimeUnit;
 
-import com.google.common.util.concurrent.ListeningExecutorService;
 import org.apache.commons.collections.CollectionUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
@@ -43,14 +41,12 @@ import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.fs.PathIOException;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.util.BlockingThreadPoolExecutorService;
 import org.apache.hadoop.util.Progressable;
 
 import com.aliyun.oss.model.OSSObjectSummary;
 import com.aliyun.oss.model.ObjectListing;
 import com.aliyun.oss.model.ObjectMetadata;
 
-import org.apache.hadoop.util.SemaphoredDelegatingExecutor;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -69,9 +65,6 @@ public class AliyunOSSFileSystem extends FileSystem {
   private Path workingDir;
   private AliyunOSSFileSystemStore store;
   private int maxKeys;
-  private int maxReadAheadPartNumber;
-  private ListeningExecutorService boundedThreadPool;
-
   private static final PathFilter DEFAULT_FILTER = new PathFilter() {
     @Override
     public boolean accept(Path file) {
@@ -89,7 +82,6 @@ public class AliyunOSSFileSystem extends FileSystem {
   public void close() throws IOException {
     try {
       store.close();
-      boundedThreadPool.shutdown();
     } finally {
       super.close();
     }
@@ -317,24 +309,10 @@ public class AliyunOSSFileSystem extends FileSystem {
     store = new AliyunOSSFileSystemStore();
     store.initialize(name, conf, statistics);
     maxKeys = conf.getInt(MAX_PAGING_KEYS_KEY, MAX_PAGING_KEYS_DEFAULT);
-
-    int threadNum = AliyunOSSUtils.intPositiveOption(conf,
-        Constants.MULTIPART_DOWNLOAD_THREAD_NUMBER_KEY,
-        Constants.MULTIPART_DOWNLOAD_THREAD_NUMBER_DEFAULT);
-
-    int totalTasks = AliyunOSSUtils.intPositiveOption(conf,
-        Constants.MAX_TOTAL_TASKS_KEY, Constants.MAX_TOTAL_TASKS_DEFAULT);
-
-    maxReadAheadPartNumber = AliyunOSSUtils.intPositiveOption(conf,
-        Constants.MULTIPART_DOWNLOAD_AHEAD_PART_MAX_NUM_KEY,
-        Constants.MULTIPART_DOWNLOAD_AHEAD_PART_MAX_NUM_DEFAULT);
-
-    this.boundedThreadPool = BlockingThreadPoolExecutorService.newInstance(
-        threadNum, totalTasks, 60L, TimeUnit.SECONDS, "oss-read-shared");
     setConf(conf);
   }
 
-/**
+  /**
    * Turn a path (relative or otherwise) into an OSS key.
    *
    * @param path the path of the file.
@@ -545,11 +523,8 @@ public class AliyunOSSFileSystem extends FileSystem {
           " because it is a directory");
     }
 
-    return new FSDataInputStream(new AliyunOSSInputStream(getConf(),
-        new SemaphoredDelegatingExecutor(
-            boundedThreadPool, maxReadAheadPartNumber, true),
-        maxReadAheadPartNumber, store, pathToKey(path), fileStatus.getLen(),
-        statistics));
+    return new FSDataInputStream(new AliyunOSSInputStream(getConf(), store,
+        pathToKey(path), fileStatus.getLen(), statistics));
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/702d280c/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSInputStream.java b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSInputStream.java
index 494ac53..72ba619 100644
--- a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSInputStream.java
+++ b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSInputStream.java
@@ -20,11 +20,8 @@ package org.apache.hadoop.fs.aliyun.oss;
 
 import java.io.EOFException;
 import java.io.IOException;
-import java.util.ArrayDeque;
-import java.util.Queue;
-import java.util.concurrent.ExecutorService;
+import java.io.InputStream;
 
-import com.google.common.util.concurrent.MoreExecutors;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -46,33 +43,20 @@ public class AliyunOSSInputStream extends FSInputStream {
   private final String key;
   private Statistics statistics;
   private boolean closed;
+  private InputStream wrappedStream = null;
   private long contentLength;
   private long position;
   private long partRemaining;
-  private byte[] buffer;
-  private int maxReadAheadPartNumber;
-  private long expectNextPos;
-  private long lastByteStart;
-
-  private ExecutorService readAheadExecutorService;
-  private Queue<ReadBuffer> readBufferQueue = new ArrayDeque<>();
 
   public AliyunOSSInputStream(Configuration conf,
-      ExecutorService readAheadExecutorService, int maxReadAheadPartNumber,
       AliyunOSSFileSystemStore store, String key, Long contentLength,
       Statistics statistics) throws IOException {
-    this.readAheadExecutorService =
-        MoreExecutors.listeningDecorator(readAheadExecutorService);
     this.store = store;
     this.key = key;
     this.statistics = statistics;
     this.contentLength = contentLength;
     downloadPartSize = conf.getLong(MULTIPART_DOWNLOAD_SIZE_KEY,
         MULTIPART_DOWNLOAD_SIZE_DEFAULT);
-    this.maxReadAheadPartNumber = maxReadAheadPartNumber;
-
-    this.expectNextPos = 0;
-    this.lastByteStart = -1;
     reopen(0);
     closed = false;
   }
@@ -98,81 +82,15 @@ public class AliyunOSSInputStream extends FSInputStream {
       partSize = downloadPartSize;
     }
 
-    if (this.buffer != null) {
+    if (wrappedStream != null) {
       if (LOG.isDebugEnabled()) {
         LOG.debug("Aborting old stream to open at pos " + pos);
       }
-      this.buffer = null;
-    }
-
-    boolean isRandomIO = true;
-    if (pos == this.expectNextPos) {
-      isRandomIO = false;
-    } else {
-      //new seek, remove cache buffers if its byteStart is not equal to pos
-      while (readBufferQueue.size() != 0) {
-        if (readBufferQueue.element().getByteStart() != pos) {
-          readBufferQueue.poll();
-        } else {
-          break;
-        }
-      }
-    }
-
-    this.expectNextPos = pos + partSize;
-
-    int currentSize = readBufferQueue.size();
-    if (currentSize == 0) {
-      //init lastByteStart to pos - partSize, used by for loop below
-      lastByteStart = pos - partSize;
-    } else {
-      ReadBuffer[] readBuffers = readBufferQueue.toArray(
-          new ReadBuffer[currentSize]);
-      lastByteStart = readBuffers[currentSize - 1].getByteStart();
+      wrappedStream.close();
     }
 
-    int maxLen = this.maxReadAheadPartNumber - currentSize;
-    for (int i = 0; i < maxLen && i < (currentSize + 1) * 2; i++) {
-      if (lastByteStart + partSize * (i + 1) > contentLength) {
-        break;
-      }
-
-      long byteStart = lastByteStart + partSize * (i + 1);
-      long byteEnd = byteStart + partSize -1;
-      if (byteEnd >= contentLength) {
-        byteEnd = contentLength - 1;
-      }
-
-      ReadBuffer readBuffer = new ReadBuffer(byteStart, byteEnd);
-      if (readBuffer.getBuffer().length == 0) {
-        //EOF
-        readBuffer.setStatus(ReadBuffer.STATUS.SUCCESS);
-      } else {
-        this.readAheadExecutorService.execute(
-            new AliyunOSSFileReaderTask(key, store, readBuffer));
-      }
-      readBufferQueue.add(readBuffer);
-      if (isRandomIO) {
-        break;
-      }
-    }
-
-    ReadBuffer readBuffer = readBufferQueue.poll();
-    readBuffer.lock();
-    try {
-      readBuffer.await(ReadBuffer.STATUS.INIT);
-      if (readBuffer.getStatus() == ReadBuffer.STATUS.ERROR) {
-        this.buffer = null;
-      } else {
-        this.buffer = readBuffer.getBuffer();
-      }
-    } catch (InterruptedException e) {
-      LOG.warn("interrupted when wait a read buffer");
-    } finally {
-      readBuffer.unlock();
-    }
-
-    if (this.buffer == null) {
+    wrappedStream = store.retrieve(key, pos, pos + partSize -1);
+    if (wrappedStream == null) {
       throw new IOException("Null IO stream");
     }
     position = pos;
@@ -187,10 +105,18 @@ public class AliyunOSSInputStream extends FSInputStream {
       reopen(position);
     }
 
+    int tries = MAX_RETRIES;
+    boolean retry;
     int byteRead = -1;
-    if (partRemaining != 0) {
-      byteRead = this.buffer[this.buffer.length - (int)partRemaining] & 0xFF;
-    }
+    do {
+      retry = false;
+      try {
+        byteRead = wrappedStream.read();
+      } catch (Exception e) {
+        handleReadException(e, --tries);
+        retry = true;
+      }
+    } while (retry);
     if (byteRead >= 0) {
       position++;
       partRemaining--;
@@ -235,18 +161,21 @@ public class AliyunOSSInputStream extends FSInputStream {
         reopen(position);
       }
 
-      int bytes = 0;
-      for (int i = this.buffer.length - (int)partRemaining;
-           i < this.buffer.length; i++) {
-        buf[off + bytesRead] = this.buffer[i];
-        bytes++;
-        bytesRead++;
-        if (off + bytesRead >= len) {
-          break;
+      int tries = MAX_RETRIES;
+      boolean retry;
+      int bytes = -1;
+      do {
+        retry = false;
+        try {
+          bytes = wrappedStream.read(buf, off + bytesRead, len - bytesRead);
+        } catch (Exception e) {
+          handleReadException(e, --tries);
+          retry = true;
         }
-      }
+      } while (retry);
 
       if (bytes > 0) {
+        bytesRead += bytes;
         position += bytes;
         partRemaining -= bytes;
       } else if (partRemaining != 0) {
@@ -273,7 +202,9 @@ public class AliyunOSSInputStream extends FSInputStream {
       return;
     }
     closed = true;
-    this.buffer = null;
+    if (wrappedStream != null) {
+      wrappedStream.close();
+    }
   }
 
   @Override
@@ -294,6 +225,7 @@ public class AliyunOSSInputStream extends FSInputStream {
       return;
     } else if (pos > position && pos < position + partRemaining) {
       long len = pos - position;
+      AliyunOSSUtils.skipFully(wrappedStream, len);
       position = pos;
       partRemaining -= len;
     } else {
@@ -313,7 +245,18 @@ public class AliyunOSSInputStream extends FSInputStream {
     return false;
   }
 
-  public long getExpectNextPos() {
-    return this.expectNextPos;
+  private void handleReadException(Exception e, int tries) throws IOException{
+    if (tries == 0) {
+      throw new IOException(e);
+    }
+
+    LOG.warn("Some exceptions occurred in oss connection, try to reopen oss" +
+        " connection at position '" + position + "', " + e.getMessage());
+    try {
+      Thread.sleep(100);
+    } catch (InterruptedException e2) {
+      LOG.warn(e2.getMessage());
+    }
+    reopen(position);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/702d280c/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSUtils.java b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSUtils.java
index 1a21608..fdf72e4 100644
--- a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSUtils.java
+++ b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSUtils.java
@@ -40,18 +40,6 @@ final public class AliyunOSSUtils {
   private AliyunOSSUtils() {
   }
 
-  public static int intPositiveOption(
-      Configuration conf, String key, int defVal) {
-    int v = conf.getInt(key, defVal);
-    if (v <= 0) {
-      LOG.warn(key + " is configured to " + v
-          + ", will use default value: " + defVal);
-      v = defVal;
-    }
-
-    return v;
-  }
-
   /**
    * Used to get password from configuration.
    *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/702d280c/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/Constants.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/Constants.java b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/Constants.java
index 410adc9..dd71842 100644
--- a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/Constants.java
+++ b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/Constants.java
@@ -97,18 +97,7 @@ public final class Constants {
   public static final String MULTIPART_DOWNLOAD_SIZE_KEY =
       "fs.oss.multipart.download.size";
 
-  public static final long MULTIPART_DOWNLOAD_SIZE_DEFAULT = 512 * 1024;
-
-  public static final String MULTIPART_DOWNLOAD_THREAD_NUMBER_KEY =
-      "fs.oss.multipart.download.threads";
-  public static final int MULTIPART_DOWNLOAD_THREAD_NUMBER_DEFAULT = 10;
-
-  public static final String MAX_TOTAL_TASKS_KEY = "fs.oss.max.total.tasks";
-  public static final int MAX_TOTAL_TASKS_DEFAULT = 128;
-
-  public static final String MULTIPART_DOWNLOAD_AHEAD_PART_MAX_NUM_KEY =
-      "fs.oss.multipart.download.ahead.part.max.number";
-  public static final int MULTIPART_DOWNLOAD_AHEAD_PART_MAX_NUM_DEFAULT = 4;
+  public static final long MULTIPART_DOWNLOAD_SIZE_DEFAULT = 100 * 1024;
 
   // Comma separated list of directories
   public static final String BUFFER_DIR_KEY = "fs.oss.buffer.dir";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/702d280c/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/ReadBuffer.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/ReadBuffer.java b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/ReadBuffer.java
deleted file mode 100644
index 46bb5bf..0000000
--- a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/ReadBuffer.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.fs.aliyun.oss;
-
-import java.util.concurrent.locks.Condition;
-import java.util.concurrent.locks.ReentrantLock;
-
-/**
- * This class is used by {@link AliyunOSSInputStream}
- * and {@link AliyunOSSFileReaderTask} to buffer data that read from oss.
- */
-public class ReadBuffer {
-  enum STATUS {
-    INIT, SUCCESS, ERROR
-  }
-  private final ReentrantLock lock = new ReentrantLock();
-
-  private Condition readyCondition = lock.newCondition();
-
-  private byte[] buffer;
-  private STATUS status;
-  private long byteStart;
-  private long byteEnd;
-
-  public ReadBuffer(long byteStart, long byteEnd) {
-    this.buffer = new byte[(int)(byteEnd - byteStart) + 1];
-
-    this.status = STATUS.INIT;
-    this.byteStart = byteStart;
-    this.byteEnd = byteEnd;
-  }
-
-  public void lock() {
-    lock.lock();
-  }
-
-  public void unlock() {
-    lock.unlock();
-  }
-
-  public void await(STATUS waitStatus) throws InterruptedException {
-    while (this.status == waitStatus) {
-      readyCondition.await();
-    }
-  }
-
-  public void signalAll() {
-    readyCondition.signalAll();
-  }
-
-  public byte[] getBuffer() {
-    return buffer;
-  }
-
-  public STATUS getStatus() {
-    return status;
-  }
-
-  public void setStatus(STATUS status) {
-    this.status = status;
-  }
-
-  public long getByteStart() {
-    return byteStart;
-  }
-
-  public long getByteEnd() {
-    return byteEnd;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/702d280c/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSInputStream.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSInputStream.java
index 66068c6..10c4edd 100644
--- a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSInputStream.java
+++ b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSInputStream.java
@@ -35,7 +35,6 @@ import org.slf4j.LoggerFactory;
 
 import java.util.Random;
 
-import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
 /**
@@ -109,54 +108,6 @@ public class TestAliyunOSSInputStream {
   }
 
   @Test
-  public void testSequentialAndRandomRead() throws Exception {
-    Path smallSeekFile = setPath("/test/smallSeekFile.txt");
-    long size = 5 * 1024 * 1024;
-
-    ContractTestUtils.generateTestFile(this.fs, smallSeekFile, size, 256, 255);
-    LOG.info("5MB file created: smallSeekFile.txt");
-
-    FSDataInputStream fsDataInputStream = this.fs.open(smallSeekFile);
-    AliyunOSSInputStream in =
-        (AliyunOSSInputStream)fsDataInputStream.getWrappedStream();
-    assertTrue("expected position at:" + 0 + ", but got:"
-        + fsDataInputStream.getPos(), fsDataInputStream.getPos() == 0);
-
-    assertTrue("expected position at:"
-            + Constants.MULTIPART_DOWNLOAD_SIZE_DEFAULT + ", but got:"
-            + in.getExpectNextPos(),
-        in.getExpectNextPos() == Constants.MULTIPART_DOWNLOAD_SIZE_DEFAULT);
-    fsDataInputStream.seek(4 * 1024 * 1024);
-    assertTrue("expected position at:" + 4 * 1024 * 1024
-            + Constants.MULTIPART_DOWNLOAD_SIZE_DEFAULT + ", but got:"
-            + in.getExpectNextPos(),
-        in.getExpectNextPos() == 4 * 1024 * 1024
-            + Constants.MULTIPART_DOWNLOAD_SIZE_DEFAULT);
-    IOUtils.closeStream(fsDataInputStream);
-  }
-
-  @Test
-  public void testOSSFileReaderTask() throws Exception {
-    Path smallSeekFile = setPath("/test/smallSeekFileOSSFileReader.txt");
-    long size = 5 * 1024 * 1024;
-
-    ContractTestUtils.generateTestFile(this.fs, smallSeekFile, size, 256, 255);
-    LOG.info("5MB file created: smallSeekFileOSSFileReader.txt");
-    ReadBuffer readBuffer = new ReadBuffer(12, 24);
-    AliyunOSSFileReaderTask task = new AliyunOSSFileReaderTask("1",
-        ((AliyunOSSFileSystem)this.fs).getStore(), readBuffer);
-    //NullPointerException, fail
-    task.run();
-    assertEquals(readBuffer.getStatus(), ReadBuffer.STATUS.ERROR);
-    //OK
-    task = new AliyunOSSFileReaderTask(
-        "test/test/smallSeekFileOSSFileReader.txt",
-        ((AliyunOSSFileSystem)this.fs).getStore(), readBuffer);
-    task.run();
-    assertEquals(readBuffer.getStatus(), ReadBuffer.STATUS.SUCCESS);
-  }
-
-  @Test
   public void testReadFile() throws Exception {
     final int bufLen = 256;
     final int sizeFlag = 5;


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[03/10] hadoop git commit: HDFS-12919. RBF: Support erasure coding methods in RouterRpcServer. Contributed by Inigo Goiri.

Posted by cu...@apache.org.
HDFS-12919. RBF: Support erasure coding methods in RouterRpcServer. Contributed by Inigo Goiri.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/82741091
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/82741091
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/82741091

Branch: refs/heads/branch-3
Commit: 82741091a78d7ce62c240ec3e7f81a3a9a3fee36
Parents: d3fbcd9
Author: Inigo Goiri <in...@apache.org>
Authored: Mon Jan 15 12:21:24 2018 -0800
Committer: Inigo Goiri <in...@apache.org>
Committed: Mon Jan 15 12:21:24 2018 -0800

----------------------------------------------------------------------
 .../AddErasureCodingPolicyResponse.java         |  24 ++
 .../server/federation/router/ErasureCoding.java | 198 +++++++++++++++
 .../federation/router/RouterRpcClient.java      |  65 ++++-
 .../federation/router/RouterRpcServer.java      | 247 ++++++++++---------
 .../server/federation/RouterDFSCluster.java     |  12 +-
 .../server/federation/router/TestRouterRpc.java | 191 +++++++++++++-
 6 files changed, 603 insertions(+), 134 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/82741091/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/AddErasureCodingPolicyResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/AddErasureCodingPolicyResponse.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/AddErasureCodingPolicyResponse.java
index 2e8d081..dc77a47 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/AddErasureCodingPolicyResponse.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/AddErasureCodingPolicyResponse.java
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hdfs.protocol;
 
+import org.apache.commons.lang.builder.EqualsBuilder;
+import org.apache.commons.lang.builder.HashCodeBuilder;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 
 /**
@@ -65,4 +67,26 @@ public class AddErasureCodingPolicyResponse {
           + "error message is " + getErrorMsg();
     }
   }
+
+  @Override
+  public boolean equals(Object o) {
+    if (o instanceof AddErasureCodingPolicyResponse) {
+      AddErasureCodingPolicyResponse other = (AddErasureCodingPolicyResponse) o;
+      return new EqualsBuilder()
+          .append(policy, other.policy)
+          .append(succeed, other.succeed)
+          .append(errorMsg, other.errorMsg)
+          .isEquals();
+    }
+    return false;
+  }
+
+  @Override
+  public int hashCode() {
+    return new HashCodeBuilder(303855623, 582626729)
+        .append(policy)
+        .append(succeed)
+        .append(errorMsg)
+        .toHashCode();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/82741091/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ErasureCoding.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ErasureCoding.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ErasureCoding.java
new file mode 100644
index 0000000..d2b2d50
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ErasureCoding.java
@@ -0,0 +1,198 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.federation.router;
+
+import static org.apache.hadoop.hdfs.server.federation.router.RouterRpcServer.merge;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse;
+import org.apache.hadoop.hdfs.protocol.ECBlockGroupStats;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicyInfo;
+import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver;
+import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamespaceInfo;
+import org.apache.hadoop.hdfs.server.federation.resolver.RemoteLocation;
+import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
+
+/**
+ * Module that implements all the RPC calls in
+ * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol} related to
+ * Erasure Coding in the {@link RouterRpcServer}.
+ */
+public class ErasureCoding {
+
+  /** RPC server to receive client calls. */
+  private final RouterRpcServer rpcServer;
+  /** RPC clients to connect to the Namenodes. */
+  private final RouterRpcClient rpcClient;
+  /** Interface to identify the active NN for a nameservice or blockpool ID. */
+  private final ActiveNamenodeResolver namenodeResolver;
+
+
+  public ErasureCoding(RouterRpcServer server) {
+    this.rpcServer = server;
+    this.rpcClient =  this.rpcServer.getRPCClient();
+    this.namenodeResolver = this.rpcClient.getNamenodeResolver();
+  }
+
+  public ErasureCodingPolicyInfo[] getErasureCodingPolicies()
+      throws IOException {
+    rpcServer.checkOperation(OperationCategory.READ);
+
+    RemoteMethod method = new RemoteMethod("getErasureCodingPolicies");
+    Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    Map<FederationNamespaceInfo, ErasureCodingPolicyInfo[]> ret =
+        rpcClient.invokeConcurrent(
+            nss, method, true, false, ErasureCodingPolicyInfo[].class);
+    return merge(ret, ErasureCodingPolicyInfo.class);
+  }
+
+  public Map<String, String> getErasureCodingCodecs() throws IOException {
+    rpcServer.checkOperation(OperationCategory.READ);
+
+    RemoteMethod method = new RemoteMethod("getErasureCodingCodecs");
+    Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    @SuppressWarnings("rawtypes")
+    Map<FederationNamespaceInfo, Map> retCodecs =
+        rpcClient.invokeConcurrent(
+            nss, method, true, false, Map.class);
+
+    Map<String, String> ret = new HashMap<>();
+    Object obj = retCodecs;
+    @SuppressWarnings("unchecked")
+    Map<FederationNamespaceInfo, Map<String, String>> results =
+        (Map<FederationNamespaceInfo, Map<String, String>>)obj;
+    Collection<Map<String, String>> allCodecs = results.values();
+    for (Map<String, String> codecs : allCodecs) {
+      ret.putAll(codecs);
+    }
+
+    return ret;
+  }
+
+  public AddErasureCodingPolicyResponse[] addErasureCodingPolicies(
+      ErasureCodingPolicy[] policies) throws IOException {
+    rpcServer.checkOperation(OperationCategory.WRITE);
+
+    RemoteMethod method = new RemoteMethod("addErasureCodingPolicies",
+        new Class<?>[] {ErasureCodingPolicy[].class}, new Object[] {policies});
+    Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    Map<FederationNamespaceInfo, AddErasureCodingPolicyResponse[]> ret =
+        rpcClient.invokeConcurrent(
+            nss, method, true, false, AddErasureCodingPolicyResponse[].class);
+
+    return merge(ret, AddErasureCodingPolicyResponse.class);
+  }
+
+  public void removeErasureCodingPolicy(String ecPolicyName)
+      throws IOException {
+    rpcServer.checkOperation(OperationCategory.WRITE);
+
+    RemoteMethod method = new RemoteMethod("removeErasureCodingPolicy",
+        new Class<?>[] {String.class}, ecPolicyName);
+    Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    rpcClient.invokeConcurrent(nss, method, true, false);
+  }
+
+  public void disableErasureCodingPolicy(String ecPolicyName)
+      throws IOException {
+    rpcServer.checkOperation(OperationCategory.WRITE);
+
+    RemoteMethod method = new RemoteMethod("disableErasureCodingPolicy",
+        new Class<?>[] {String.class}, ecPolicyName);
+    Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    rpcClient.invokeConcurrent(nss, method, true, false);
+  }
+
+  public void enableErasureCodingPolicy(String ecPolicyName)
+      throws IOException {
+    rpcServer.checkOperation(OperationCategory.WRITE);
+
+    RemoteMethod method = new RemoteMethod("enableErasureCodingPolicy",
+        new Class<?>[] {String.class}, ecPolicyName);
+    Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    rpcClient.invokeConcurrent(nss, method, true, false);
+  }
+
+  public ErasureCodingPolicy getErasureCodingPolicy(String src)
+      throws IOException {
+    rpcServer.checkOperation(OperationCategory.READ);
+
+    final List<RemoteLocation> locations =
+        rpcServer.getLocationsForPath(src, true);
+    RemoteMethod remoteMethod = new RemoteMethod("getErasureCodingPolicy",
+        new Class<?>[] {String.class}, new RemoteParam());
+    ErasureCodingPolicy ret = rpcClient.invokeSequential(
+        locations, remoteMethod, null, null);
+    return ret;
+  }
+
+  public void setErasureCodingPolicy(String src, String ecPolicyName)
+      throws IOException {
+    rpcServer.checkOperation(OperationCategory.WRITE);
+
+    final List<RemoteLocation> locations =
+        rpcServer.getLocationsForPath(src, true);
+    RemoteMethod remoteMethod = new RemoteMethod("setErasureCodingPolicy",
+        new Class<?>[] {String.class, String.class},
+        new RemoteParam(), ecPolicyName);
+    rpcClient.invokeSequential(locations, remoteMethod, null, null);
+  }
+
+  public void unsetErasureCodingPolicy(String src) throws IOException {
+    rpcServer.checkOperation(OperationCategory.WRITE);
+
+    final List<RemoteLocation> locations =
+        rpcServer.getLocationsForPath(src, true);
+    RemoteMethod remoteMethod = new RemoteMethod("unsetErasureCodingPolicy",
+        new Class<?>[] {String.class}, new RemoteParam());
+    rpcClient.invokeSequential(locations, remoteMethod, null, null);
+  }
+
+  public ECBlockGroupStats getECBlockGroupStats() throws IOException {
+    rpcServer.checkOperation(OperationCategory.READ);
+
+    RemoteMethod method = new RemoteMethod("getECBlockGroupStats");
+    Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
+    Map<FederationNamespaceInfo, ECBlockGroupStats> allStats =
+        rpcClient.invokeConcurrent(
+            nss, method, true, false, ECBlockGroupStats.class);
+
+    // Merge the stats from all the namespaces
+    long lowRedundancyBlockGroups = 0;
+    long corruptBlockGroups = 0;
+    long missingBlockGroups = 0;
+    long bytesInFutureBlockGroups = 0;
+    long pendingDeletionBlocks = 0;
+    for (ECBlockGroupStats stats : allStats.values()) {
+      lowRedundancyBlockGroups += stats.getLowRedundancyBlockGroups();
+      corruptBlockGroups += stats.getCorruptBlockGroups();
+      missingBlockGroups += stats.getMissingBlockGroups();
+      bytesInFutureBlockGroups += stats.getBytesInFutureBlockGroups();
+      pendingDeletionBlocks += stats.getPendingDeletionBlocks();
+    }
+    return new ECBlockGroupStats(lowRedundancyBlockGroups, corruptBlockGroups,
+        missingBlockGroups, bytesInFutureBlockGroups, pendingDeletionBlocks);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/82741091/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java
index cac3713..4209a49 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java
@@ -148,6 +148,14 @@ public class RouterRpcClient {
   }
 
   /**
+   * Get the active namenode resolver used by this client.
+   * @return Active namenode resolver.
+   */
+  public ActiveNamenodeResolver getNamenodeResolver() {
+    return this.namenodeResolver;
+  }
+
+  /**
    * Shutdown the client.
    */
   public void shutdown() {
@@ -617,9 +625,9 @@ public class RouterRpcClient {
    * @throws IOException if the success condition is not met, return the first
    *                     remote exception generated.
    */
-  public Object invokeSequential(
+  public <T> T invokeSequential(
       final List<? extends RemoteLocationContext> locations,
-      final RemoteMethod remoteMethod, Class<?> expectedResultClass,
+      final RemoteMethod remoteMethod, Class<T> expectedResultClass,
       Object expectedResultValue) throws IOException {
 
     final UserGroupInformation ugi = RouterRpcServer.getRemoteUser();
@@ -639,7 +647,9 @@ public class RouterRpcClient {
         if (isExpectedClass(expectedResultClass, result) &&
             isExpectedValue(expectedResultValue, result)) {
           // Valid result, stop here
-          return result;
+          @SuppressWarnings("unchecked")
+          T ret = (T)result;
+          return ret;
         }
         if (firstResult == null) {
           firstResult = result;
@@ -669,7 +679,9 @@ public class RouterRpcClient {
       throw firstThrownException;
     }
     // Return the last result, whether it is the value we are looking for or a
-    return firstResult;
+    @SuppressWarnings("unchecked")
+    T ret = (T)firstResult;
+    return ret;
   }
 
   /**
@@ -709,6 +721,28 @@ public class RouterRpcClient {
   }
 
   /**
+   * Invoke multiple concurrent proxy calls to different clients. Returns an
+   * array of results.
+   *
+   * Re-throws exceptions generated by the remote RPC call as either
+   * RemoteException or IOException.
+   *
+   * @param <T> The type of the remote location.
+   * @param locations List of remote locations to call concurrently.
+   * @param method The remote method and parameters to invoke.
+   * @param requireResponse If true an exception will be thrown if all calls do
+   *          not complete. If false exceptions are ignored and all data results
+   *          successfully received are returned.
+   * @param standby If the requests should go to the standby namenodes too.
+   * @throws IOException If all the calls throw an exception.
+   */
+  public <T extends RemoteLocationContext, R> void invokeConcurrent(
+      final Collection<T> locations, final RemoteMethod method,
+      boolean requireResponse, boolean standby) throws IOException {
+    invokeConcurrent(locations, method, requireResponse, standby, void.class);
+  }
+
+  /**
    * Invokes multiple concurrent proxy calls to different clients. Returns an
    * array of results.
    *
@@ -716,20 +750,24 @@ public class RouterRpcClient {
    * RemoteException or IOException.
    *
    * @param <T> The type of the remote location.
+   * @param <R> The type of the remote method return.
    * @param locations List of remote locations to call concurrently.
    * @param method The remote method and parameters to invoke.
    * @param requireResponse If true an exception will be thrown if all calls do
    *          not complete. If false exceptions are ignored and all data results
    *          successfully received are returned.
    * @param standby If the requests should go to the standby namenodes too.
+   * @param clazz Type of the remote return type.
    * @return Result of invoking the method per subcluster: nsId -> result.
    * @throws IOException If requiredResponse=true and any of the calls throw an
    *           exception.
    */
-  public <T extends RemoteLocationContext> Map<T, Object> invokeConcurrent(
+  public <T extends RemoteLocationContext, R> Map<T, R> invokeConcurrent(
       final Collection<T> locations, final RemoteMethod method,
-      boolean requireResponse, boolean standby) throws IOException {
-    return invokeConcurrent(locations, method, requireResponse, standby, -1);
+      boolean requireResponse, boolean standby, Class<R> clazz)
+          throws IOException {
+    return invokeConcurrent(
+        locations, method, requireResponse, standby, -1, clazz);
   }
 
   /**
@@ -739,6 +777,8 @@ public class RouterRpcClient {
    * Re-throws exceptions generated by the remote RPC call as either
    * RemoteException or IOException.
    *
+   * @param <T> The type of the remote location.
+   * @param <R> The type of the remote method return.
    * @param locations List of remote locations to call concurrently.
    * @param method The remote method and parameters to invoke.
    * @param requireResponse If true an exception will be thrown if all calls do
@@ -746,14 +786,15 @@ public class RouterRpcClient {
    *          successfully received are returned.
    * @param standby If the requests should go to the standby namenodes too.
    * @param timeOutMs Timeout for each individual call.
+   * @param clazz Type of the remote return type.
    * @return Result of invoking the method per subcluster: nsId -> result.
    * @throws IOException If requiredResponse=true and any of the calls throw an
    *           exception.
    */
   @SuppressWarnings("unchecked")
-  public <T extends RemoteLocationContext> Map<T, Object> invokeConcurrent(
+  public <T extends RemoteLocationContext, R> Map<T, R> invokeConcurrent(
       final Collection<T> locations, final RemoteMethod method,
-      boolean requireResponse, boolean standby, long timeOutMs)
+      boolean requireResponse, boolean standby, long timeOutMs, Class<R> clazz)
           throws IOException {
 
     final UserGroupInformation ugi = RouterRpcServer.getRemoteUser();
@@ -767,7 +808,7 @@ public class RouterRpcClient {
           getNamenodesForNameservice(ns);
       Object[] paramList = method.getParams(location);
       Object result = invokeMethod(ugi, namenodes, m, paramList);
-      return Collections.singletonMap(location, result);
+      return Collections.singletonMap(location, clazz.cast(result));
     }
 
     List<T> orderedLocations = new LinkedList<>();
@@ -817,14 +858,14 @@ public class RouterRpcClient {
       } else {
         futures = executorService.invokeAll(callables);
       }
-      Map<T, Object> results = new TreeMap<>();
+      Map<T, R> results = new TreeMap<>();
       Map<T, IOException> exceptions = new TreeMap<>();
       for (int i=0; i<futures.size(); i++) {
         T location = orderedLocations.get(i);
         try {
           Future<Object> future = futures.get(i);
           Object result = future.get();
-          results.put(location, result);
+          results.put(location, clazz.cast(result));
         } catch (CancellationException ce) {
           T loc = orderedLocations.get(i);
           String msg =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/82741091/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
index 4c317a8..8e62009 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
@@ -28,12 +28,14 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ROUTER_READER_QUEUE_SIZE_
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.lang.reflect.Array;
 import java.net.InetSocketAddress;
 import java.util.Collection;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
@@ -179,6 +181,10 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
   /** Category of the operation that a thread is executing. */
   private final ThreadLocal<OperationCategory> opCategory = new ThreadLocal<>();
 
+  // Modules implementing groups of RPC calls
+  /** Erasure coding calls. */
+  private final ErasureCoding erasureCoding;
+
 
   /**
    * Construct a router RPC server.
@@ -275,6 +281,9 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
     // Create the client
     this.rpcClient = new RouterRpcClient(this.conf, this.router.getRouterId(),
         this.namenodeResolver, this.rpcMonitor);
+
+    // Initialize modules
+    this.erasureCoding = new ErasureCoding(this);
   }
 
   @Override
@@ -360,7 +369,7 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
    *                          client requests.
    * @throws UnsupportedOperationException If the operation is not supported.
    */
-  private void checkOperation(OperationCategory op, boolean supported)
+  protected void checkOperation(OperationCategory op, boolean supported)
       throws StandbyException, UnsupportedOperationException {
     checkOperation(op);
 
@@ -382,7 +391,7 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
    * @throws StandbyException If the Router is in safe mode and cannot serve
    *                          client requests.
    */
-  private void checkOperation(OperationCategory op) throws StandbyException {
+  protected void checkOperation(OperationCategory op) throws StandbyException {
     // Log the function we are currently calling.
     if (rpcMonitor != null) {
       rpcMonitor.startOp();
@@ -942,8 +951,9 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
     RemoteMethod method = new RemoteMethod("getListing",
         new Class<?>[] {String.class, startAfter.getClass(), boolean.class},
         new RemoteParam(), startAfter, needLocation);
-    Map<RemoteLocation, Object> listings =
-        rpcClient.invokeConcurrent(locations, method, false, false);
+    Map<RemoteLocation, DirectoryListing> listings =
+        rpcClient.invokeConcurrent(
+            locations, method, false, false, DirectoryListing.class);
 
     Map<String, HdfsFileStatus> nnListing = new TreeMap<>();
     int totalRemainingEntries = 0;
@@ -952,9 +962,10 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
     if (listings != null) {
       // Check the subcluster listing with the smallest name
       String lastName = null;
-      for (Entry<RemoteLocation, Object> entry : listings.entrySet()) {
+      for (Entry<RemoteLocation, DirectoryListing> entry :
+          listings.entrySet()) {
         RemoteLocation location = entry.getKey();
-        DirectoryListing listing = (DirectoryListing) entry.getValue();
+        DirectoryListing listing = entry.getValue();
         if (listing == null) {
           LOG.debug("Cannot get listing from {}", location);
         } else {
@@ -1078,11 +1089,10 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
 
     RemoteMethod method = new RemoteMethod("getStats");
     Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
-    Map<FederationNamespaceInfo, Object> results =
-        rpcClient.invokeConcurrent(nss, method, true, false);
+    Map<FederationNamespaceInfo, long[]> results =
+        rpcClient.invokeConcurrent(nss, method, true, false, long[].class);
     long[] combinedData = new long[STATS_ARRAY_LENGTH];
-    for (Object o : results.values()) {
-      long[] data = (long[]) o;
+    for (long[] data : results.values()) {
       for (int i = 0; i < combinedData.length && i < data.length; i++) {
         if (data[i] >= 0) {
           combinedData[i] += data[i];
@@ -1115,11 +1125,13 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
         new Class<?>[] {DatanodeReportType.class}, type);
 
     Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
-    Map<FederationNamespaceInfo, Object> results =
-        rpcClient.invokeConcurrent(nss, method, true, false, timeOutMs);
-    for (Entry<FederationNamespaceInfo, Object> entry : results.entrySet()) {
+    Map<FederationNamespaceInfo, DatanodeInfo[]> results =
+        rpcClient.invokeConcurrent(
+            nss, method, true, false, timeOutMs, DatanodeInfo[].class);
+    for (Entry<FederationNamespaceInfo, DatanodeInfo[]> entry :
+        results.entrySet()) {
       FederationNamespaceInfo ns = entry.getKey();
-      DatanodeInfo[] result = (DatanodeInfo[]) entry.getValue();
+      DatanodeInfo[] result = entry.getValue();
       for (DatanodeInfo node : result) {
         String nodeId = node.getXferAddr();
         if (!datanodesMap.containsKey(nodeId)) {
@@ -1149,10 +1161,10 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
     RemoteMethod method = new RemoteMethod("getDatanodeStorageReport",
         new Class<?>[] {DatanodeReportType.class}, type);
     Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
-    Map<FederationNamespaceInfo, Object> results =
-        rpcClient.invokeConcurrent(nss, method, true, false);
-    for (Object r : results.values()) {
-      DatanodeStorageReport[] result = (DatanodeStorageReport[]) r;
+    Map<FederationNamespaceInfo, DatanodeStorageReport[]> results =
+        rpcClient.invokeConcurrent(
+            nss, method, true, false, DatanodeStorageReport[].class);
+    for (DatanodeStorageReport[] result : results.values()) {
       for (DatanodeStorageReport node : result) {
         String nodeId = node.getDatanodeInfo().getXferAddr();
         if (!datanodesMap.containsKey(nodeId)) {
@@ -1180,17 +1192,14 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
         new Class<?>[] {SafeModeAction.class, boolean.class},
         action, isChecked);
     Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
-    Map<FederationNamespaceInfo, Object> results =
-        rpcClient.invokeConcurrent(nss, method, true, true);
+    Map<FederationNamespaceInfo, Boolean> results =
+        rpcClient.invokeConcurrent(nss, method, true, true, boolean.class);
 
     // We only report true if all the name space are in safe mode
     int numSafemode = 0;
-    for (Object result : results.values()) {
-      if (result instanceof Boolean) {
-        boolean safemode = (boolean) result;
-        if (safemode) {
-          numSafemode++;
-        }
+    for (boolean safemode : results.values()) {
+      if (safemode) {
+        numSafemode++;
       }
     }
     return numSafemode == results.size();
@@ -1203,18 +1212,14 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
     RemoteMethod method = new RemoteMethod("restoreFailedStorage",
         new Class<?>[] {String.class}, arg);
     final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
-    Map<FederationNamespaceInfo, Object> ret =
-        rpcClient.invokeConcurrent(nss, method, true, false);
+    Map<FederationNamespaceInfo, Boolean> ret =
+        rpcClient.invokeConcurrent(nss, method, true, false, boolean.class);
 
     boolean success = true;
-    Object obj = ret;
-    @SuppressWarnings("unchecked")
-    Map<FederationNamespaceInfo, Boolean> results =
-        (Map<FederationNamespaceInfo, Boolean>)obj;
-    Collection<Boolean> sucesses = results.values();
-    for (boolean s : sucesses) {
+    for (boolean s : ret.values()) {
       if (!s) {
         success = false;
+        break;
       }
     }
     return success;
@@ -1227,18 +1232,14 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
     RemoteMethod method = new RemoteMethod("saveNamespace",
         new Class<?>[] {Long.class, Long.class}, timeWindow, txGap);
     final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
-    Map<FederationNamespaceInfo, Object> ret =
-        rpcClient.invokeConcurrent(nss, method, true, false);
+    Map<FederationNamespaceInfo, Boolean> ret =
+        rpcClient.invokeConcurrent(nss, method, true, false, boolean.class);
 
     boolean success = true;
-    Object obj = ret;
-    @SuppressWarnings("unchecked")
-    Map<FederationNamespaceInfo, Boolean> results =
-        (Map<FederationNamespaceInfo, Boolean>)obj;
-    Collection<Boolean> sucesses = results.values();
-    for (boolean s : sucesses) {
+    for (boolean s : ret.values()) {
       if (!s) {
         success = false;
+        break;
       }
     }
     return success;
@@ -1250,17 +1251,12 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
 
     RemoteMethod method = new RemoteMethod("rollEdits", new Class<?>[] {});
     final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
-    Map<FederationNamespaceInfo, Object> ret =
-        rpcClient.invokeConcurrent(nss, method, true, false);
+    Map<FederationNamespaceInfo, Long> ret =
+        rpcClient.invokeConcurrent(nss, method, true, false, long.class);
 
     // Return the maximum txid
     long txid = 0;
-    Object obj = ret;
-    @SuppressWarnings("unchecked")
-    Map<FederationNamespaceInfo, Long> results =
-        (Map<FederationNamespaceInfo, Long>)obj;
-    Collection<Long> txids = results.values();
-    for (long t : txids) {
+    for (long t : ret.values()) {
       if (t > txid) {
         txid = t;
       }
@@ -1295,17 +1291,13 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
     RemoteMethod method = new RemoteMethod("rollingUpgrade",
         new Class<?>[] {RollingUpgradeAction.class}, action);
     final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
-    Map<FederationNamespaceInfo, Object> ret =
-        rpcClient.invokeConcurrent(nss, method, true, false);
+    Map<FederationNamespaceInfo, RollingUpgradeInfo> ret =
+        rpcClient.invokeConcurrent(
+            nss, method, true, false, RollingUpgradeInfo.class);
 
     // Return the first rolling upgrade info
     RollingUpgradeInfo info = null;
-    Object obj = ret;
-    @SuppressWarnings("unchecked")
-    Map<FederationNamespaceInfo, RollingUpgradeInfo> results =
-        (Map<FederationNamespaceInfo, RollingUpgradeInfo>)obj;
-    Collection<RollingUpgradeInfo> infos = results.values();
-    for (RollingUpgradeInfo infoNs : infos) {
+    for (RollingUpgradeInfo infoNs : ret.values()) {
       if (info == null && infoNs != null) {
         info = infoNs;
       }
@@ -1357,10 +1349,9 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
       final List<RemoteLocation> locations = getLocationsForPath(path, false);
       RemoteMethod method = new RemoteMethod("getContentSummary",
           new Class<?>[] {String.class}, new RemoteParam());
-      @SuppressWarnings("unchecked")
-      Map<String, ContentSummary> results =
-          (Map<String, ContentSummary>) ((Object)rpcClient.invokeConcurrent(
-              locations, method, false, false));
+      Map<RemoteLocation, ContentSummary> results =
+          rpcClient.invokeConcurrent(
+              locations, method, false, false, ContentSummary.class);
       summaries.addAll(results.values());
     } catch (FileNotFoundException e) {
       notFoundException = e;
@@ -1746,17 +1737,12 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
     RemoteMethod method = new RemoteMethod(
         "getCurrentEditLogTxid", new Class<?>[] {});
     final Set<FederationNamespaceInfo> nss = namenodeResolver.getNamespaces();
-    Map<FederationNamespaceInfo, Object> ret =
-        rpcClient.invokeConcurrent(nss, method, true, false);
+    Map<FederationNamespaceInfo, Long> ret =
+        rpcClient.invokeConcurrent(nss, method, true, false, long.class);
 
     // Return the maximum txid
     long txid = 0;
-    Object obj = ret;
-    @SuppressWarnings("unchecked")
-    Map<FederationNamespaceInfo, Long> results =
-        (Map<FederationNamespaceInfo, Long>)obj;
-    Collection<Long> txids = results.values();
-    for (long t : txids) {
+    for (long t : ret.values()) {
       if (t > txid) {
         txid = t;
       }
@@ -1789,31 +1775,6 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
     checkOperation(OperationCategory.WRITE, false);
   }
 
-  @Override
-  public ErasureCodingPolicyInfo[] getErasureCodingPolicies()
-      throws IOException {
-    checkOperation(OperationCategory.READ, false);
-    return null;
-  }
-
-  @Override // ClientProtocol
-  public ErasureCodingPolicy getErasureCodingPolicy(String src)
-      throws IOException {
-    checkOperation(OperationCategory.READ, false);
-    return null;
-  }
-
-  @Override // ClientProtocol
-  public void setErasureCodingPolicy(String src, String ecPolicyName)
-      throws IOException {
-    checkOperation(OperationCategory.WRITE, false);
-  }
-
-  @Override // ClientProtocol
-  public void unsetErasureCodingPolicy(String src) throws IOException {
-    checkOperation(OperationCategory.WRITE, false);
-  }
-
   @Override // ClientProtocol
   public void setQuota(String path, long namespaceQuota, long storagespaceQuota,
       StorageType type) throws IOException {
@@ -1875,38 +1836,61 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
     return null;
   }
 
-  @Override
+  @Override // ClientProtocol
+  public ErasureCodingPolicyInfo[] getErasureCodingPolicies()
+      throws IOException {
+    return erasureCoding.getErasureCodingPolicies();
+  }
+
+  @Override // ClientProtocol
+  public Map<String, String> getErasureCodingCodecs() throws IOException {
+    return erasureCoding.getErasureCodingCodecs();
+  }
+
+  @Override // ClientProtocol
   public AddErasureCodingPolicyResponse[] addErasureCodingPolicies(
       ErasureCodingPolicy[] policies) throws IOException {
-    checkOperation(OperationCategory.WRITE, false);
-    return null;
+    return erasureCoding.addErasureCodingPolicies(policies);
   }
 
-  @Override
-  public void removeErasureCodingPolicy(String arg0) throws IOException {
-    checkOperation(OperationCategory.WRITE, false);
+  @Override // ClientProtocol
+  public void removeErasureCodingPolicy(String ecPolicyName)
+      throws IOException {
+    erasureCoding.removeErasureCodingPolicy(ecPolicyName);
   }
 
-  @Override
-  public void disableErasureCodingPolicy(String arg0) throws IOException {
-    checkOperation(OperationCategory.WRITE, false);
+  @Override // ClientProtocol
+  public void disableErasureCodingPolicy(String ecPolicyName)
+      throws IOException {
+    erasureCoding.disableErasureCodingPolicy(ecPolicyName);
   }
 
-  @Override
-  public void enableErasureCodingPolicy(String arg0) throws IOException {
-    checkOperation(OperationCategory.WRITE, false);
+  @Override // ClientProtocol
+  public void enableErasureCodingPolicy(String ecPolicyName)
+      throws IOException {
+    erasureCoding.enableErasureCodingPolicy(ecPolicyName);
   }
 
-  @Override
-  public ECBlockGroupStats getECBlockGroupStats() throws IOException {
-    checkOperation(OperationCategory.READ, false);
-    return null;
+  @Override // ClientProtocol
+  public ErasureCodingPolicy getErasureCodingPolicy(String src)
+      throws IOException {
+    return erasureCoding.getErasureCodingPolicy(src);
+  }
+
+  @Override // ClientProtocol
+  public void setErasureCodingPolicy(String src, String ecPolicyName)
+      throws IOException {
+    erasureCoding.setErasureCodingPolicy(src, ecPolicyName);
+  }
+
+  @Override // ClientProtocol
+  public void unsetErasureCodingPolicy(String src) throws IOException {
+    erasureCoding.unsetErasureCodingPolicy(src);
   }
 
   @Override
-  public Map<String, String> getErasureCodingCodecs() throws IOException {
-    checkOperation(OperationCategory.READ, false);
-    return null;
+  public ECBlockGroupStats getECBlockGroupStats() throws IOException {
+    return erasureCoding.getECBlockGroupStats();
   }
 
   @Override
@@ -1974,7 +1958,7 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
    * @return Prioritized list of locations in the federated cluster.
    * @throws IOException If the location for this path cannot be determined.
    */
-  private List<RemoteLocation> getLocationsForPath(
+  protected List<RemoteLocation> getLocationsForPath(
       String path, boolean failIfLocked) throws IOException {
     try {
       // Check the location for this path
@@ -2097,4 +2081,37 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
     UserGroupInformation ugi = Server.getRemoteUser();
     return (ugi != null) ? ugi : UserGroupInformation.getCurrentUser();
   }
+
+  /**
+   * Merge the outputs from multiple namespaces.
+   * @param map Namespace -> Output array.
+   * @param clazz Class of the values.
+   * @return Array with the outputs.
+   */
+  protected static <T> T[] merge(
+      Map<FederationNamespaceInfo, T[]> map, Class<T> clazz) {
+
+    // Put all results into a set to avoid repeats
+    Set<T> ret = new LinkedHashSet<>();
+    for (T[] values : map.values()) {
+      for (T val : values) {
+        ret.add(val);
+      }
+    }
+
+    return toArray(ret, clazz);
+  }
+
+  /**
+   * Convert a set of values into an array.
+   * @param set Input set.
+   * @param clazz Class of the values.
+   * @return Array with the values in set.
+   */
+  private static <T> T[] toArray(Set<T> set, Class<T> clazz) {
+    @SuppressWarnings("unchecked")
+    T[] combinedData = (T[]) Array.newInstance(clazz, set.size());
+    combinedData = set.toArray(combinedData);
+    return combinedData;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/82741091/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterDFSCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterDFSCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterDFSCluster.java
index 7424499..8f8bd3e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterDFSCluster.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/RouterDFSCluster.java
@@ -109,6 +109,8 @@ public class RouterDFSCluster {
   private List<RouterContext> routers;
   /** If the Namenodes are in high availability.*/
   private boolean highAvailability;
+  /** Number of datanodes per nameservice. */
+  private int numDatanodesPerNameservice = 2;
 
   /** Mini cluster. */
   private MiniDFSCluster cluster;
@@ -356,8 +358,8 @@ public class RouterDFSCluster {
         DEFAULT_HEARTBEAT_INTERVAL_MS, DEFAULT_CACHE_INTERVAL_MS);
   }
 
-  public RouterDFSCluster(boolean ha, int numNameservices, int numNamnodes) {
-    this(ha, numNameservices, numNamnodes,
+  public RouterDFSCluster(boolean ha, int numNameservices, int numNamenodes) {
+    this(ha, numNameservices, numNamenodes,
         DEFAULT_HEARTBEAT_INTERVAL_MS, DEFAULT_CACHE_INTERVAL_MS);
   }
 
@@ -531,6 +533,10 @@ public class RouterDFSCluster {
     }
   }
 
+  public void setNumDatanodesPerNameservice(int num) {
+    this.numDatanodesPerNameservice = num;
+  }
+
   public String getNameservicesKey() {
     StringBuilder sb = new StringBuilder();
     for (String nsId : this.nameservices) {
@@ -658,7 +664,7 @@ public class RouterDFSCluster {
         nnConf.addResource(overrideConf);
       }
       cluster = new MiniDFSCluster.Builder(nnConf)
-          .numDataNodes(nameservices.size()*2)
+          .numDataNodes(nameservices.size() * numDatanodesPerNameservice)
           .nnTopology(topology)
           .build();
       cluster.waitActive();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/82741091/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpc.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpc.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpc.java
index af506c9..6a8c0e1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpc.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpc.java
@@ -24,19 +24,24 @@ import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.delet
 import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.getFileStatus;
 import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.verifyFileExists;
 import static org.apache.hadoop.hdfs.server.federation.RouterDFSCluster.TEST_STRING;
+import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 import java.lang.reflect.Method;
 import java.net.URISyntaxException;
 import java.util.Arrays;
+import java.util.Comparator;
 import java.util.EnumSet;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
 import java.util.Random;
 import java.util.Set;
 import java.util.TreeSet;
@@ -53,10 +58,15 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
+import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
+import org.apache.hadoop.hdfs.protocol.ECBlockGroupStats;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicyInfo;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicyState;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@@ -68,12 +78,18 @@ import org.apache.hadoop.hdfs.server.federation.RouterDFSCluster.RouterContext;
 import org.apache.hadoop.hdfs.server.federation.resolver.FileSubclusterResolver;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
 import org.apache.hadoop.io.EnumSetWritable;
+import org.apache.hadoop.io.erasurecode.ECSchema;
+import org.apache.hadoop.io.erasurecode.ErasureCodeConstants;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.service.Service.STATE;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Maps;
 
 /**
  * The the RPC interface of the {@link Router} implemented by
@@ -81,6 +97,20 @@ import org.junit.Test;
  */
 public class TestRouterRpc {
 
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestRouterRpc.class);
+
+  private static final Comparator<ErasureCodingPolicyInfo> EC_POLICY_CMP =
+      new Comparator<ErasureCodingPolicyInfo>() {
+        public int compare(
+            ErasureCodingPolicyInfo ec0,
+            ErasureCodingPolicyInfo ec1) {
+          String name0 = ec0.getPolicy().getName();
+          String name1 = ec1.getPolicy().getName();
+          return name0.compareTo(name1);
+        }
+      };
+
   /** Federated HDFS cluster. */
   private static RouterDFSCluster cluster;
 
@@ -111,6 +141,8 @@ public class TestRouterRpc {
   @BeforeClass
   public static void globalSetUp() throws Exception {
     cluster = new RouterDFSCluster(false, 2);
+    // We need 6 DNs to test Erasure Coding with RS-6-3-64k
+    cluster.setNumDatanodesPerNameservice(6);
 
     // Start NNs and DNs and wait until ready
     cluster.startCluster();
@@ -144,9 +176,9 @@ public class TestRouterRpc {
     // Wait to ensure NN has fully created its test directories
     Thread.sleep(100);
 
-    // Pick a NS, namenode and router for this test
+    // Default namenode and random router for this test
     this.router = cluster.getRandomRouter();
-    this.ns = cluster.getRandomNameservice();
+    this.ns = cluster.getNameservices().get(0);
     this.namenode = cluster.getNamenode(ns, null);
 
     // Handles to the ClientProtocol interface
@@ -481,7 +513,7 @@ public class TestRouterRpc {
       for (int i = 0; i < data.length; i++) {
         individualData[i] += data[i];
       }
-      assert(data.length == combinedData.length);
+      assertEquals(data.length, combinedData.length);
     }
 
     for (int i = 0; i < combinedData.length && i < individualData.length; i++) {
@@ -489,7 +521,9 @@ public class TestRouterRpc {
         // Skip available storage as this fluctuates in mini cluster
         continue;
       }
-      assertEquals(combinedData[i], individualData[i]);
+      assertEquals("Stats for " + i + " don't match: " +
+          combinedData[i] + "!=" + individualData[i],
+          combinedData[i], individualData[i]);
     }
   }
 
@@ -866,4 +900,153 @@ public class TestRouterRpc {
 
     assertEquals(routerFailure.getClass(), nnFailure.getClass());
   }
+
+  @Test
+  public void testErasureCoding() throws IOException {
+
+    LOG.info("List the available erasurce coding policies");
+    ErasureCodingPolicyInfo[] policies = checkErasureCodingPolicies();
+    for (ErasureCodingPolicyInfo policy : policies) {
+      LOG.info("  {}", policy);
+    }
+
+    LOG.info("List the erasure coding codecs");
+    Map<String, String> codecsRouter = routerProtocol.getErasureCodingCodecs();
+    Map<String, String> codecsNamenode = nnProtocol.getErasureCodingCodecs();
+    assertTrue(Maps.difference(codecsRouter, codecsNamenode).areEqual());
+    for (Entry<String, String> entry : codecsRouter.entrySet()) {
+      LOG.info("  {}: {}", entry.getKey(), entry.getValue());
+    }
+
+    LOG.info("Create a testing directory via the router at the root level");
+    String dirPath = "/testec";
+    String filePath1 = dirPath + "/testfile1";
+    FsPermission permission = new FsPermission("755");
+    routerProtocol.mkdirs(dirPath, permission, false);
+    createFile(routerFS, filePath1, 32);
+    assertTrue(verifyFileExists(routerFS, filePath1));
+    DFSClient file1Protocol = getFileDFSClient(filePath1);
+
+    LOG.info("The policy for the new file should not be set");
+    assertNull(routerProtocol.getErasureCodingPolicy(filePath1));
+    assertNull(file1Protocol.getErasureCodingPolicy(filePath1));
+
+    String policyName = "RS-6-3-1024k";
+    LOG.info("Set policy \"{}\" for \"{}\"", policyName, dirPath);
+    routerProtocol.setErasureCodingPolicy(dirPath, policyName);
+
+    String filePath2 = dirPath + "/testfile2";
+    LOG.info("Create {} in the path with the new EC policy", filePath2);
+    createFile(routerFS, filePath2, 32);
+    assertTrue(verifyFileExists(routerFS, filePath2));
+    DFSClient file2Protocol = getFileDFSClient(filePath2);
+
+    LOG.info("Check that the policy is set for {}", filePath2);
+    ErasureCodingPolicy policyRouter1 =
+        routerProtocol.getErasureCodingPolicy(filePath2);
+    ErasureCodingPolicy policyNamenode1 =
+        file2Protocol.getErasureCodingPolicy(filePath2);
+    assertNotNull(policyRouter1);
+    assertEquals(policyName, policyRouter1.getName());
+    assertEquals(policyName, policyNamenode1.getName());
+
+    LOG.info("Create a new erasure coding policy");
+    String newPolicyName = "RS-6-3-128k";
+    ECSchema ecSchema = new ECSchema(ErasureCodeConstants.RS_CODEC_NAME, 6, 3);
+    ErasureCodingPolicy ecPolicy = new ErasureCodingPolicy(
+        newPolicyName,
+        ecSchema,
+        128 * 1024,
+        (byte) -1);
+    ErasureCodingPolicy[] newPolicies = new ErasureCodingPolicy[] {
+        ecPolicy
+    };
+    AddErasureCodingPolicyResponse[] responses =
+        routerProtocol.addErasureCodingPolicies(newPolicies);
+    assertEquals(1, responses.length);
+    assertTrue(responses[0].isSucceed());
+    routerProtocol.disableErasureCodingPolicy(newPolicyName);
+
+    LOG.info("The new policy should be there and disabled");
+    policies = checkErasureCodingPolicies();
+    boolean found = false;
+    for (ErasureCodingPolicyInfo policy : policies) {
+      LOG.info("  {}" + policy);
+      if (policy.getPolicy().getName().equals(newPolicyName)) {
+        found = true;
+        assertEquals(ErasureCodingPolicyState.DISABLED, policy.getState());
+        break;
+      }
+    }
+    assertTrue(found);
+
+    LOG.info("Set the test folder to use the new policy");
+    routerProtocol.enableErasureCodingPolicy(newPolicyName);
+    routerProtocol.setErasureCodingPolicy(dirPath, newPolicyName);
+
+    LOG.info("Create a file in the path with the new EC policy");
+    String filePath3 = dirPath + "/testfile3";
+    createFile(routerFS, filePath3, 32);
+    assertTrue(verifyFileExists(routerFS, filePath3));
+    DFSClient file3Protocol = getFileDFSClient(filePath3);
+
+    ErasureCodingPolicy policyRouterFile3 =
+        routerProtocol.getErasureCodingPolicy(filePath3);
+    assertEquals(newPolicyName, policyRouterFile3.getName());
+    ErasureCodingPolicy policyNamenodeFile3 =
+        file3Protocol.getErasureCodingPolicy(filePath3);
+    assertEquals(newPolicyName, policyNamenodeFile3.getName());
+
+    LOG.info("Remove the policy and check the one for the test folder");
+    routerProtocol.removeErasureCodingPolicy(newPolicyName);
+    ErasureCodingPolicy policyRouter3 =
+        routerProtocol.getErasureCodingPolicy(filePath3);
+    assertEquals(newPolicyName, policyRouter3.getName());
+    ErasureCodingPolicy policyNamenode3 =
+        file3Protocol.getErasureCodingPolicy(filePath3);
+    assertEquals(newPolicyName, policyNamenode3.getName());
+
+    LOG.info("Check the stats");
+    ECBlockGroupStats statsRouter = routerProtocol.getECBlockGroupStats();
+    ECBlockGroupStats statsNamenode = nnProtocol.getECBlockGroupStats();
+    assertEquals(statsNamenode.toString(), statsRouter.toString());
+  }
+
+  /**
+   * Check the erasure coding policies in the Router and the Namenode.
+   * @return The erasure coding policies.
+   */
+  private ErasureCodingPolicyInfo[] checkErasureCodingPolicies()
+      throws IOException {
+    ErasureCodingPolicyInfo[] policiesRouter =
+        routerProtocol.getErasureCodingPolicies();
+    assertNotNull(policiesRouter);
+    ErasureCodingPolicyInfo[] policiesNamenode =
+        nnProtocol.getErasureCodingPolicies();
+    Arrays.sort(policiesRouter, EC_POLICY_CMP);
+    Arrays.sort(policiesNamenode, EC_POLICY_CMP);
+    assertArrayEquals(policiesRouter, policiesNamenode);
+    return policiesRouter;
+  }
+
+  /**
+   * Find the Namenode for a particular file and return the DFSClient.
+   * @param path Path of the file to check.
+   * @return The DFSClient to the Namenode holding the file.
+   */
+  private DFSClient getFileDFSClient(final String path) {
+    for (String nsId : cluster.getNameservices()) {
+      LOG.info("Checking {} for {}", nsId, path);
+      NamenodeContext nn = cluster.getNamenode(nsId, null);
+      try {
+        DFSClient nnClientProtocol = nn.getClient();
+        if (nnClientProtocol.getFileInfo(path) != null) {
+          return nnClientProtocol;
+        }
+      } catch (Exception ignore) {
+        // ignore
+      }
+    }
+    return null;
+  }
 }
\ No newline at end of file


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[05/10] hadoop git commit: HADOOP-15027. AliyunOSS: Support multi-thread pre-read to improve sequential read from Hadoop to Aliyun OSS performance. (Contributed by Jinhu Wu)

Posted by cu...@apache.org.
HADOOP-15027. AliyunOSS: Support multi-thread pre-read to improve sequential read from Hadoop to Aliyun OSS performance. (Contributed by Jinhu Wu)

(cherry picked from commit 9195a6e302028ed3921d1016ac2fa5754f06ebf0)
(cherry picked from commit 55142849db02a9191db0dd6f4e1401ff19ec242a)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/082a707b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/082a707b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/082a707b

Branch: refs/heads/branch-3
Commit: 082a707bae4bb97444a34c00eecd62975807388d
Parents: db8345f
Author: Sammi Chen <sa...@intel.com>
Authored: Wed Jan 17 15:55:59 2018 +0800
Committer: Sammi Chen <sa...@intel.com>
Committed: Wed Jan 17 16:16:03 2018 +0800

----------------------------------------------------------------------
 .../dev-support/findbugs-exclude.xml            |   8 +
 .../fs/aliyun/oss/AliyunOSSFileReaderTask.java  | 109 ++++++++++++++
 .../fs/aliyun/oss/AliyunOSSFileSystem.java      |  31 +++-
 .../fs/aliyun/oss/AliyunOSSInputStream.java     | 149 +++++++++++++------
 .../hadoop/fs/aliyun/oss/AliyunOSSUtils.java    |  12 ++
 .../apache/hadoop/fs/aliyun/oss/Constants.java  |  13 +-
 .../apache/hadoop/fs/aliyun/oss/ReadBuffer.java |  86 +++++++++++
 .../fs/aliyun/oss/TestAliyunOSSInputStream.java |  49 ++++++
 8 files changed, 407 insertions(+), 50 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/082a707b/hadoop-tools/hadoop-aliyun/dev-support/findbugs-exclude.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/dev-support/findbugs-exclude.xml b/hadoop-tools/hadoop-aliyun/dev-support/findbugs-exclude.xml
index 40d78d0..c55f8e3 100644
--- a/hadoop-tools/hadoop-aliyun/dev-support/findbugs-exclude.xml
+++ b/hadoop-tools/hadoop-aliyun/dev-support/findbugs-exclude.xml
@@ -15,4 +15,12 @@
    limitations under the License.
 -->
 <FindBugsFilter>
+    <!-- Disable FindBugs warning and return the buffer to caller directly.
+         It is convenient and efficient because we do not need to copy the buffer
+    -->
+    <Match>
+        <Class name="org.apache.hadoop.fs.aliyun.oss.ReadBuffer" />
+        <Method name="getBuffer" />
+        <Bug pattern="EI_EXPOSE_REP" />
+    </Match>
 </FindBugsFilter>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/082a707b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileReaderTask.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileReaderTask.java b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileReaderTask.java
new file mode 100644
index 0000000..e5bfc2c
--- /dev/null
+++ b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileReaderTask.java
@@ -0,0 +1,109 @@
+/**
+ * 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.fs.aliyun.oss;
+
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.retry.RetryPolicies;
+import org.apache.hadoop.io.retry.RetryPolicy;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Used by {@link AliyunOSSInputStream} as an task that submitted
+ * to the thread pool.
+ * Each AliyunOSSFileReaderTask reads one part of the file so that
+ * we can accelerate the sequential read.
+ */
+public class AliyunOSSFileReaderTask implements Runnable {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(AliyunOSSFileReaderTask.class);
+
+  private String key;
+  private AliyunOSSFileSystemStore store;
+  private ReadBuffer readBuffer;
+  private static final int MAX_RETRIES = 3;
+  private RetryPolicy retryPolicy;
+
+  public AliyunOSSFileReaderTask(String key, AliyunOSSFileSystemStore store,
+      ReadBuffer readBuffer) {
+    this.key = key;
+    this.store = store;
+    this.readBuffer = readBuffer;
+    RetryPolicy defaultPolicy =
+        RetryPolicies.retryUpToMaximumCountWithFixedSleep(
+            MAX_RETRIES, 3, TimeUnit.SECONDS);
+    Map<Class<? extends Exception>, RetryPolicy> policies = new HashMap<>();
+    policies.put(IOException.class, defaultPolicy);
+    policies.put(IndexOutOfBoundsException.class,
+        RetryPolicies.TRY_ONCE_THEN_FAIL);
+    policies.put(NullPointerException.class,
+        RetryPolicies.TRY_ONCE_THEN_FAIL);
+
+    this.retryPolicy = RetryPolicies.retryByException(defaultPolicy, policies);
+  }
+
+  @Override
+  public void run() {
+    int retries = 0;
+    readBuffer.lock();
+    try {
+      while (true) {
+        try (InputStream in = store.retrieve(
+            key, readBuffer.getByteStart(), readBuffer.getByteEnd())) {
+          IOUtils.readFully(in, readBuffer.getBuffer(),
+              0, readBuffer.getBuffer().length);
+          readBuffer.setStatus(ReadBuffer.STATUS.SUCCESS);
+          break;
+        } catch (Exception e) {
+          LOG.warn("Exception thrown when retrieve key: "
+              + this.key + ", exception: " + e);
+          try {
+            RetryPolicy.RetryAction rc = retryPolicy.shouldRetry(
+                e, retries++, 0, true);
+            if (rc.action == RetryPolicy.RetryAction.RetryDecision.RETRY) {
+              Thread.sleep(rc.delayMillis);
+            } else {
+              //should not retry
+              break;
+            }
+          } catch (Exception ex) {
+            //FAIL
+            LOG.warn("Exception thrown when call shouldRetry, exception " + ex);
+            break;
+          }
+        }
+      }
+
+      if (readBuffer.getStatus() != ReadBuffer.STATUS.SUCCESS) {
+        readBuffer.setStatus(ReadBuffer.STATUS.ERROR);
+      }
+
+      //notify main thread which wait for this buffer
+      readBuffer.signalAll();
+    } finally {
+      readBuffer.unlock();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/082a707b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystem.java b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystem.java
index 41d475d..afff223 100644
--- a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystem.java
+++ b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystem.java
@@ -24,7 +24,9 @@ import java.net.URI;
 import java.util.ArrayList;
 import java.util.EnumSet;
 import java.util.List;
+import java.util.concurrent.TimeUnit;
 
+import com.google.common.util.concurrent.ListeningExecutorService;
 import org.apache.commons.collections.CollectionUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
@@ -41,12 +43,14 @@ import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.fs.PathIOException;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.util.BlockingThreadPoolExecutorService;
 import org.apache.hadoop.util.Progressable;
 
 import com.aliyun.oss.model.OSSObjectSummary;
 import com.aliyun.oss.model.ObjectListing;
 import com.aliyun.oss.model.ObjectMetadata;
 
+import org.apache.hadoop.util.SemaphoredDelegatingExecutor;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -65,6 +69,9 @@ public class AliyunOSSFileSystem extends FileSystem {
   private Path workingDir;
   private AliyunOSSFileSystemStore store;
   private int maxKeys;
+  private int maxReadAheadPartNumber;
+  private ListeningExecutorService boundedThreadPool;
+
   private static final PathFilter DEFAULT_FILTER = new PathFilter() {
     @Override
     public boolean accept(Path file) {
@@ -82,6 +89,7 @@ public class AliyunOSSFileSystem extends FileSystem {
   public void close() throws IOException {
     try {
       store.close();
+      boundedThreadPool.shutdown();
     } finally {
       super.close();
     }
@@ -309,10 +317,24 @@ public class AliyunOSSFileSystem extends FileSystem {
     store = new AliyunOSSFileSystemStore();
     store.initialize(name, conf, statistics);
     maxKeys = conf.getInt(MAX_PAGING_KEYS_KEY, MAX_PAGING_KEYS_DEFAULT);
+
+    int threadNum = AliyunOSSUtils.intPositiveOption(conf,
+        Constants.MULTIPART_DOWNLOAD_THREAD_NUMBER_KEY,
+        Constants.MULTIPART_DOWNLOAD_THREAD_NUMBER_DEFAULT);
+
+    int totalTasks = AliyunOSSUtils.intPositiveOption(conf,
+        Constants.MAX_TOTAL_TASKS_KEY, Constants.MAX_TOTAL_TASKS_DEFAULT);
+
+    maxReadAheadPartNumber = AliyunOSSUtils.intPositiveOption(conf,
+        Constants.MULTIPART_DOWNLOAD_AHEAD_PART_MAX_NUM_KEY,
+        Constants.MULTIPART_DOWNLOAD_AHEAD_PART_MAX_NUM_DEFAULT);
+
+    this.boundedThreadPool = BlockingThreadPoolExecutorService.newInstance(
+        threadNum, totalTasks, 60L, TimeUnit.SECONDS, "oss-read-shared");
     setConf(conf);
   }
 
-  /**
+/**
    * Turn a path (relative or otherwise) into an OSS key.
    *
    * @param path the path of the file.
@@ -523,8 +545,11 @@ public class AliyunOSSFileSystem extends FileSystem {
           " because it is a directory");
     }
 
-    return new FSDataInputStream(new AliyunOSSInputStream(getConf(), store,
-        pathToKey(path), fileStatus.getLen(), statistics));
+    return new FSDataInputStream(new AliyunOSSInputStream(getConf(),
+        new SemaphoredDelegatingExecutor(
+            boundedThreadPool, maxReadAheadPartNumber, true),
+        maxReadAheadPartNumber, store, pathToKey(path), fileStatus.getLen(),
+        statistics));
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/082a707b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSInputStream.java b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSInputStream.java
index 72ba619..494ac53 100644
--- a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSInputStream.java
+++ b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSInputStream.java
@@ -20,8 +20,11 @@ package org.apache.hadoop.fs.aliyun.oss;
 
 import java.io.EOFException;
 import java.io.IOException;
-import java.io.InputStream;
+import java.util.ArrayDeque;
+import java.util.Queue;
+import java.util.concurrent.ExecutorService;
 
+import com.google.common.util.concurrent.MoreExecutors;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -43,20 +46,33 @@ public class AliyunOSSInputStream extends FSInputStream {
   private final String key;
   private Statistics statistics;
   private boolean closed;
-  private InputStream wrappedStream = null;
   private long contentLength;
   private long position;
   private long partRemaining;
+  private byte[] buffer;
+  private int maxReadAheadPartNumber;
+  private long expectNextPos;
+  private long lastByteStart;
+
+  private ExecutorService readAheadExecutorService;
+  private Queue<ReadBuffer> readBufferQueue = new ArrayDeque<>();
 
   public AliyunOSSInputStream(Configuration conf,
+      ExecutorService readAheadExecutorService, int maxReadAheadPartNumber,
       AliyunOSSFileSystemStore store, String key, Long contentLength,
       Statistics statistics) throws IOException {
+    this.readAheadExecutorService =
+        MoreExecutors.listeningDecorator(readAheadExecutorService);
     this.store = store;
     this.key = key;
     this.statistics = statistics;
     this.contentLength = contentLength;
     downloadPartSize = conf.getLong(MULTIPART_DOWNLOAD_SIZE_KEY,
         MULTIPART_DOWNLOAD_SIZE_DEFAULT);
+    this.maxReadAheadPartNumber = maxReadAheadPartNumber;
+
+    this.expectNextPos = 0;
+    this.lastByteStart = -1;
     reopen(0);
     closed = false;
   }
@@ -82,15 +98,81 @@ public class AliyunOSSInputStream extends FSInputStream {
       partSize = downloadPartSize;
     }
 
-    if (wrappedStream != null) {
+    if (this.buffer != null) {
       if (LOG.isDebugEnabled()) {
         LOG.debug("Aborting old stream to open at pos " + pos);
       }
-      wrappedStream.close();
+      this.buffer = null;
+    }
+
+    boolean isRandomIO = true;
+    if (pos == this.expectNextPos) {
+      isRandomIO = false;
+    } else {
+      //new seek, remove cache buffers if its byteStart is not equal to pos
+      while (readBufferQueue.size() != 0) {
+        if (readBufferQueue.element().getByteStart() != pos) {
+          readBufferQueue.poll();
+        } else {
+          break;
+        }
+      }
+    }
+
+    this.expectNextPos = pos + partSize;
+
+    int currentSize = readBufferQueue.size();
+    if (currentSize == 0) {
+      //init lastByteStart to pos - partSize, used by for loop below
+      lastByteStart = pos - partSize;
+    } else {
+      ReadBuffer[] readBuffers = readBufferQueue.toArray(
+          new ReadBuffer[currentSize]);
+      lastByteStart = readBuffers[currentSize - 1].getByteStart();
     }
 
-    wrappedStream = store.retrieve(key, pos, pos + partSize -1);
-    if (wrappedStream == null) {
+    int maxLen = this.maxReadAheadPartNumber - currentSize;
+    for (int i = 0; i < maxLen && i < (currentSize + 1) * 2; i++) {
+      if (lastByteStart + partSize * (i + 1) > contentLength) {
+        break;
+      }
+
+      long byteStart = lastByteStart + partSize * (i + 1);
+      long byteEnd = byteStart + partSize -1;
+      if (byteEnd >= contentLength) {
+        byteEnd = contentLength - 1;
+      }
+
+      ReadBuffer readBuffer = new ReadBuffer(byteStart, byteEnd);
+      if (readBuffer.getBuffer().length == 0) {
+        //EOF
+        readBuffer.setStatus(ReadBuffer.STATUS.SUCCESS);
+      } else {
+        this.readAheadExecutorService.execute(
+            new AliyunOSSFileReaderTask(key, store, readBuffer));
+      }
+      readBufferQueue.add(readBuffer);
+      if (isRandomIO) {
+        break;
+      }
+    }
+
+    ReadBuffer readBuffer = readBufferQueue.poll();
+    readBuffer.lock();
+    try {
+      readBuffer.await(ReadBuffer.STATUS.INIT);
+      if (readBuffer.getStatus() == ReadBuffer.STATUS.ERROR) {
+        this.buffer = null;
+      } else {
+        this.buffer = readBuffer.getBuffer();
+      }
+    } catch (InterruptedException e) {
+      LOG.warn("interrupted when wait a read buffer");
+    } finally {
+      readBuffer.unlock();
+    }
+
+    if (this.buffer == null) {
       throw new IOException("Null IO stream");
     }
     position = pos;
@@ -105,18 +187,10 @@ public class AliyunOSSInputStream extends FSInputStream {
       reopen(position);
     }
 
-    int tries = MAX_RETRIES;
-    boolean retry;
     int byteRead = -1;
-    do {
-      retry = false;
-      try {
-        byteRead = wrappedStream.read();
-      } catch (Exception e) {
-        handleReadException(e, --tries);
-        retry = true;
-      }
-    } while (retry);
+    if (partRemaining != 0) {
+      byteRead = this.buffer[this.buffer.length - (int)partRemaining] & 0xFF;
+    }
     if (byteRead >= 0) {
       position++;
       partRemaining--;
@@ -161,21 +235,18 @@ public class AliyunOSSInputStream extends FSInputStream {
         reopen(position);
       }
 
-      int tries = MAX_RETRIES;
-      boolean retry;
-      int bytes = -1;
-      do {
-        retry = false;
-        try {
-          bytes = wrappedStream.read(buf, off + bytesRead, len - bytesRead);
-        } catch (Exception e) {
-          handleReadException(e, --tries);
-          retry = true;
+      int bytes = 0;
+      for (int i = this.buffer.length - (int)partRemaining;
+           i < this.buffer.length; i++) {
+        buf[off + bytesRead] = this.buffer[i];
+        bytes++;
+        bytesRead++;
+        if (off + bytesRead >= len) {
+          break;
         }
-      } while (retry);
+      }
 
       if (bytes > 0) {
-        bytesRead += bytes;
         position += bytes;
         partRemaining -= bytes;
       } else if (partRemaining != 0) {
@@ -202,9 +273,7 @@ public class AliyunOSSInputStream extends FSInputStream {
       return;
     }
     closed = true;
-    if (wrappedStream != null) {
-      wrappedStream.close();
-    }
+    this.buffer = null;
   }
 
   @Override
@@ -225,7 +294,6 @@ public class AliyunOSSInputStream extends FSInputStream {
       return;
     } else if (pos > position && pos < position + partRemaining) {
       long len = pos - position;
-      AliyunOSSUtils.skipFully(wrappedStream, len);
       position = pos;
       partRemaining -= len;
     } else {
@@ -245,18 +313,7 @@ public class AliyunOSSInputStream extends FSInputStream {
     return false;
   }
 
-  private void handleReadException(Exception e, int tries) throws IOException{
-    if (tries == 0) {
-      throw new IOException(e);
-    }
-
-    LOG.warn("Some exceptions occurred in oss connection, try to reopen oss" +
-        " connection at position '" + position + "', " + e.getMessage());
-    try {
-      Thread.sleep(100);
-    } catch (InterruptedException e2) {
-      LOG.warn(e2.getMessage());
-    }
-    reopen(position);
+  public long getExpectNextPos() {
+    return this.expectNextPos;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/082a707b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSUtils.java b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSUtils.java
index fdf72e4..1a21608 100644
--- a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSUtils.java
+++ b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSUtils.java
@@ -40,6 +40,18 @@ final public class AliyunOSSUtils {
   private AliyunOSSUtils() {
   }
 
+  public static int intPositiveOption(
+      Configuration conf, String key, int defVal) {
+    int v = conf.getInt(key, defVal);
+    if (v <= 0) {
+      LOG.warn(key + " is configured to " + v
+          + ", will use default value: " + defVal);
+      v = defVal;
+    }
+
+    return v;
+  }
+
   /**
    * Used to get password from configuration.
    *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/082a707b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/Constants.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/Constants.java b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/Constants.java
index dd71842..410adc9 100644
--- a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/Constants.java
+++ b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/Constants.java
@@ -97,7 +97,18 @@ public final class Constants {
   public static final String MULTIPART_DOWNLOAD_SIZE_KEY =
       "fs.oss.multipart.download.size";
 
-  public static final long MULTIPART_DOWNLOAD_SIZE_DEFAULT = 100 * 1024;
+  public static final long MULTIPART_DOWNLOAD_SIZE_DEFAULT = 512 * 1024;
+
+  public static final String MULTIPART_DOWNLOAD_THREAD_NUMBER_KEY =
+      "fs.oss.multipart.download.threads";
+  public static final int MULTIPART_DOWNLOAD_THREAD_NUMBER_DEFAULT = 10;
+
+  public static final String MAX_TOTAL_TASKS_KEY = "fs.oss.max.total.tasks";
+  public static final int MAX_TOTAL_TASKS_DEFAULT = 128;
+
+  public static final String MULTIPART_DOWNLOAD_AHEAD_PART_MAX_NUM_KEY =
+      "fs.oss.multipart.download.ahead.part.max.number";
+  public static final int MULTIPART_DOWNLOAD_AHEAD_PART_MAX_NUM_DEFAULT = 4;
 
   // Comma separated list of directories
   public static final String BUFFER_DIR_KEY = "fs.oss.buffer.dir";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/082a707b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/ReadBuffer.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/ReadBuffer.java b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/ReadBuffer.java
new file mode 100644
index 0000000..46bb5bf
--- /dev/null
+++ b/hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/ReadBuffer.java
@@ -0,0 +1,86 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.aliyun.oss;
+
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * This class is used by {@link AliyunOSSInputStream}
+ * and {@link AliyunOSSFileReaderTask} to buffer data that read from oss.
+ */
+public class ReadBuffer {
+  enum STATUS {
+    INIT, SUCCESS, ERROR
+  }
+  private final ReentrantLock lock = new ReentrantLock();
+
+  private Condition readyCondition = lock.newCondition();
+
+  private byte[] buffer;
+  private STATUS status;
+  private long byteStart;
+  private long byteEnd;
+
+  public ReadBuffer(long byteStart, long byteEnd) {
+    this.buffer = new byte[(int)(byteEnd - byteStart) + 1];
+
+    this.status = STATUS.INIT;
+    this.byteStart = byteStart;
+    this.byteEnd = byteEnd;
+  }
+
+  public void lock() {
+    lock.lock();
+  }
+
+  public void unlock() {
+    lock.unlock();
+  }
+
+  public void await(STATUS waitStatus) throws InterruptedException {
+    while (this.status == waitStatus) {
+      readyCondition.await();
+    }
+  }
+
+  public void signalAll() {
+    readyCondition.signalAll();
+  }
+
+  public byte[] getBuffer() {
+    return buffer;
+  }
+
+  public STATUS getStatus() {
+    return status;
+  }
+
+  public void setStatus(STATUS status) {
+    this.status = status;
+  }
+
+  public long getByteStart() {
+    return byteStart;
+  }
+
+  public long getByteEnd() {
+    return byteEnd;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/082a707b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSInputStream.java b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSInputStream.java
index 10c4edd..66068c6 100644
--- a/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSInputStream.java
+++ b/hadoop-tools/hadoop-aliyun/src/test/java/org/apache/hadoop/fs/aliyun/oss/TestAliyunOSSInputStream.java
@@ -35,6 +35,7 @@ import org.slf4j.LoggerFactory;
 
 import java.util.Random;
 
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
 /**
@@ -108,6 +109,54 @@ public class TestAliyunOSSInputStream {
   }
 
   @Test
+  public void testSequentialAndRandomRead() throws Exception {
+    Path smallSeekFile = setPath("/test/smallSeekFile.txt");
+    long size = 5 * 1024 * 1024;
+
+    ContractTestUtils.generateTestFile(this.fs, smallSeekFile, size, 256, 255);
+    LOG.info("5MB file created: smallSeekFile.txt");
+
+    FSDataInputStream fsDataInputStream = this.fs.open(smallSeekFile);
+    AliyunOSSInputStream in =
+        (AliyunOSSInputStream)fsDataInputStream.getWrappedStream();
+    assertTrue("expected position at:" + 0 + ", but got:"
+        + fsDataInputStream.getPos(), fsDataInputStream.getPos() == 0);
+
+    assertTrue("expected position at:"
+            + Constants.MULTIPART_DOWNLOAD_SIZE_DEFAULT + ", but got:"
+            + in.getExpectNextPos(),
+        in.getExpectNextPos() == Constants.MULTIPART_DOWNLOAD_SIZE_DEFAULT);
+    fsDataInputStream.seek(4 * 1024 * 1024);
+    assertTrue("expected position at:" + 4 * 1024 * 1024
+            + Constants.MULTIPART_DOWNLOAD_SIZE_DEFAULT + ", but got:"
+            + in.getExpectNextPos(),
+        in.getExpectNextPos() == 4 * 1024 * 1024
+            + Constants.MULTIPART_DOWNLOAD_SIZE_DEFAULT);
+    IOUtils.closeStream(fsDataInputStream);
+  }
+
+  @Test
+  public void testOSSFileReaderTask() throws Exception {
+    Path smallSeekFile = setPath("/test/smallSeekFileOSSFileReader.txt");
+    long size = 5 * 1024 * 1024;
+
+    ContractTestUtils.generateTestFile(this.fs, smallSeekFile, size, 256, 255);
+    LOG.info("5MB file created: smallSeekFileOSSFileReader.txt");
+    ReadBuffer readBuffer = new ReadBuffer(12, 24);
+    AliyunOSSFileReaderTask task = new AliyunOSSFileReaderTask("1",
+        ((AliyunOSSFileSystem)this.fs).getStore(), readBuffer);
+    //NullPointerException, fail
+    task.run();
+    assertEquals(readBuffer.getStatus(), ReadBuffer.STATUS.ERROR);
+    //OK
+    task = new AliyunOSSFileReaderTask(
+        "test/test/smallSeekFileOSSFileReader.txt",
+        ((AliyunOSSFileSystem)this.fs).getStore(), readBuffer);
+    task.run();
+    assertEquals(readBuffer.getStatus(), ReadBuffer.STATUS.SUCCESS);
+  }
+
+  @Test
   public void testReadFile() throws Exception {
     final int bufLen = 256;
     final int sizeFlag = 5;


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[04/10] hadoop git commit: HDFS-13004. TestLeaseRecoveryStriped.testLeaseRecovery is failing when safeLength is 0MB or larger than the test file. (Zsolt Venczel via lei)

Posted by cu...@apache.org.
HDFS-13004. TestLeaseRecoveryStriped.testLeaseRecovery is failing when safeLength is 0MB or larger than the test file. (Zsolt Venczel via lei)

(cherry picked from commit 3bd9ea63df769345a9d02a404cfb61323a4cd7e3)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/db8345fa
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/db8345fa
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/db8345fa

Branch: refs/heads/branch-3
Commit: db8345fa9cd124728d935f725525e2626438b4c1
Parents: 8274109
Author: Lei Xu <le...@apache.org>
Authored: Tue Jan 16 15:15:11 2018 -0800
Committer: Lei Xu <le...@apache.org>
Committed: Tue Jan 16 15:19:09 2018 -0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/StripedFileTestUtil.java |  7 ++++---
 .../org/apache/hadoop/hdfs/TestLeaseRecoveryStriped.java | 11 +++++++----
 2 files changed, 11 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/db8345fa/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java
index 08bf20a..13ca390 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java
@@ -363,11 +363,12 @@ public class StripedFileTestUtil {
     List<List<LocatedBlock>> blockGroupList = new ArrayList<>();
     LocatedBlocks lbs = dfs.getClient().getLocatedBlocks(srcPath.toString(), 0L,
         Long.MAX_VALUE);
-    int expectedNumGroup = 0;
+
     if (length > 0) {
-      expectedNumGroup = (length - 1) / blkGroupSize + 1;
+      int expectedNumGroup = (length - 1) / blkGroupSize + 1;
+
+      assertEquals(expectedNumGroup, lbs.getLocatedBlocks().size());
     }
-    assertEquals(expectedNumGroup, lbs.getLocatedBlocks().size());
 
     final ErasureCodingPolicy ecPolicy = dfs.getErasureCodingPolicy(srcPath);
     final int cellSize = ecPolicy.getCellSize();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/db8345fa/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecoveryStriped.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecoveryStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecoveryStriped.java
index 36ac8b3..d74f193 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecoveryStriped.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecoveryStriped.java
@@ -85,6 +85,7 @@ public class TestLeaseRecoveryStriped {
   private Configuration conf;
   private final Path dir = new Path("/" + this.getClass().getSimpleName());
   final Path p = new Path(dir, "testfile");
+  private final int testFileLength = (stripesPerBlock - 1) * stripeSize;
 
   @Before
   public void setup() throws IOException {
@@ -191,17 +192,20 @@ public class TestLeaseRecoveryStriped {
 
   private void runTest(int[] blockLengths, long safeLength) throws Exception {
     writePartialBlocks(blockLengths);
+
+    int checkDataLength = Math.min(testFileLength, (int)safeLength);
+
     recoverLease();
 
     List<Long> oldGS = new ArrayList<>();
     oldGS.add(1001L);
-    StripedFileTestUtil.checkData(dfs, p, (int)safeLength,
+    StripedFileTestUtil.checkData(dfs, p, checkDataLength,
         new ArrayList<DatanodeInfo>(), oldGS, blockGroupSize);
     // After recovery, storages are reported by primary DN. we should verify
     // storages reported by blockReport.
     cluster.restartNameNode(true);
     cluster.waitFirstBRCompleted(0, 10000);
-    StripedFileTestUtil.checkData(dfs, p, (int)safeLength,
+    StripedFileTestUtil.checkData(dfs, p, checkDataLength,
         new ArrayList<DatanodeInfo>(), oldGS, blockGroupSize);
   }
 
@@ -219,12 +223,11 @@ public class TestLeaseRecoveryStriped {
     final FSDataOutputStream out = dfs.create(p);
     final DFSStripedOutputStream stripedOut = (DFSStripedOutputStream) out
         .getWrappedStream();
-    int length = (stripesPerBlock - 1) * stripeSize;
     int[] posToKill = getPosToKill(blockLengths);
     int checkingPos = nextCheckingPos(posToKill, 0);
     Set<Integer> stoppedStreamerIndexes = new HashSet<>();
     try {
-      for (int pos = 0; pos < length; pos++) {
+      for (int pos = 0; pos < testFileLength; pos++) {
         out.write(StripedFileTestUtil.getByte(pos));
         if (pos == checkingPos) {
           for (int index : getIndexToStop(posToKill, pos)) {


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org