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 na...@apache.org on 2017/12/26 22:43:57 UTC

[01/50] [abbrv] hadoop git commit: HDFS-12685. [READ] FsVolumeImpl exception when scanning Provided storage volume [Forced Update!]

Repository: hadoop
Updated Branches:
  refs/heads/yarn-3409 9e4aa217f -> 84ebbfbfb (forced update)


HDFS-12685. [READ] FsVolumeImpl exception when scanning Provided storage volume


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

Branch: refs/heads/yarn-3409
Commit: cc933cba77c147153e463415fc192cee2d53a1ef
Parents: 4d59dab
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Thu Nov 30 10:11:12 2017 -0800
Committer: Chris Douglas <cd...@apache.org>
Committed: Fri Dec 15 17:51:40 2017 -0800

----------------------------------------------------------------------
 .../impl/TextFileRegionAliasMap.java            |  3 +-
 .../hdfs/server/datanode/DirectoryScanner.java  |  3 +-
 .../server/datanode/fsdataset/FsVolumeSpi.java  | 40 ++++++++++----------
 .../fsdataset/impl/ProvidedVolumeImpl.java      |  4 +-
 .../fsdataset/impl/TestProvidedImpl.java        | 19 ++++++----
 5 files changed, 37 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc933cba/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TextFileRegionAliasMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TextFileRegionAliasMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TextFileRegionAliasMap.java
index 80f48c1..bd04d60 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TextFileRegionAliasMap.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TextFileRegionAliasMap.java
@@ -439,7 +439,8 @@ public class TextFileRegionAliasMap
 
   @Override
   public void refresh() throws IOException {
-    //nothing to do;
+    throw new UnsupportedOperationException(
+        "Refresh not supported by " + getClass());
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc933cba/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
index 8fb8551..ab9743c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
@@ -515,7 +515,8 @@ public class DirectoryScanner implements Runnable {
    *
    * @return a map of sorted arrays of block information
    */
-  private Map<String, ScanInfo[]> getDiskReport() {
+  @VisibleForTesting
+  public Map<String, ScanInfo[]> getDiskReport() {
     ScanInfoPerBlockPool list = new ScanInfoPerBlockPool();
     ScanInfoPerBlockPool[] dirReports = null;
     // First get list of data directories

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc933cba/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java
index 15e71f0..20a153d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java
@@ -296,8 +296,23 @@ public interface FsVolumeSpi
      */
     public ScanInfo(long blockId, File blockFile, File metaFile,
         FsVolumeSpi vol) {
-      this(blockId, blockFile, metaFile, vol, null,
-          (blockFile != null) ? blockFile.length() : 0);
+      this.blockId = blockId;
+      String condensedVolPath =
+          (vol == null || vol.getBaseURI() == null) ? null :
+              getCondensedPath(new File(vol.getBaseURI()).getAbsolutePath());
+      this.blockSuffix = blockFile == null ? null :
+              getSuffix(blockFile, condensedVolPath);
+      this.blockLength = (blockFile != null) ? blockFile.length() : 0;
+      if (metaFile == null) {
+        this.metaSuffix = null;
+      } else if (blockFile == null) {
+        this.metaSuffix = getSuffix(metaFile, condensedVolPath);
+      } else {
+        this.metaSuffix = getSuffix(metaFile,
+            condensedVolPath + blockSuffix);
+      }
+      this.volume = vol;
+      this.fileRegion = null;
     }
 
     /**
@@ -305,31 +320,18 @@ public interface FsVolumeSpi
      * the block data and meta-data files.
      *
      * @param blockId the block ID
-     * @param blockFile the path to the block data file
-     * @param metaFile the path to the block meta-data file
      * @param vol the volume that contains the block
      * @param fileRegion the file region (for provided blocks)
      * @param length the length of the block data
      */
-    public ScanInfo(long blockId, File blockFile, File metaFile,
-        FsVolumeSpi vol, FileRegion fileRegion, long length) {
+    public ScanInfo(long blockId, FsVolumeSpi vol, FileRegion fileRegion,
+        long length) {
       this.blockId = blockId;
-      String condensedVolPath =
-          (vol == null || vol.getBaseURI() == null) ? null :
-            getCondensedPath(new File(vol.getBaseURI()).getAbsolutePath());
-      this.blockSuffix = blockFile == null ? null :
-        getSuffix(blockFile, condensedVolPath);
       this.blockLength = length;
-      if (metaFile == null) {
-        this.metaSuffix = null;
-      } else if (blockFile == null) {
-        this.metaSuffix = getSuffix(metaFile, condensedVolPath);
-      } else {
-        this.metaSuffix = getSuffix(metaFile,
-            condensedVolPath + blockSuffix);
-      }
       this.volume = vol;
       this.fileRegion = fileRegion;
+      this.blockSuffix = null;
+      this.metaSuffix = null;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc933cba/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeImpl.java
index 65487f9..ab59fa5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeImpl.java
@@ -226,9 +226,7 @@ public class ProvidedVolumeImpl extends FsVolumeImpl {
         reportCompiler.throttle();
         FileRegion region = iter.next();
         if (region.getBlockPoolId().equals(bpid)) {
-          LOG.info("Adding ScanInfo for blkid " +
-              region.getBlock().getBlockId());
-          report.add(new ScanInfo(region.getBlock().getBlockId(), null, null,
+          report.add(new ScanInfo(region.getBlock().getBlockId(),
               providedVolume, region, region.getLength()));
         }
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cc933cba/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestProvidedImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestProvidedImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestProvidedImpl.java
index 52112f7..4190730 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestProvidedImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestProvidedImpl.java
@@ -61,6 +61,7 @@ import org.apache.hadoop.hdfs.server.datanode.BlockScanner;
 import org.apache.hadoop.hdfs.server.datanode.DNConf;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataStorage;
+import org.apache.hadoop.hdfs.server.datanode.DirectoryScanner;
 import org.apache.hadoop.hdfs.server.datanode.ProvidedReplica;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
 import org.apache.hadoop.hdfs.server.datanode.ShortCircuitRegistry;
@@ -231,14 +232,6 @@ public class TestProvidedImpl {
     public void refresh() throws IOException {
       // do nothing!
     }
-
-    public void setMinBlkId(int minId) {
-      this.minId = minId;
-    }
-
-    public void setBlockCount(int numBlocks) {
-      this.numBlocks = numBlocks;
-    }
   }
 
   private static Storage.StorageDirectory createLocalStorageDirectory(
@@ -606,4 +599,14 @@ public class TestProvidedImpl {
       }
     }
   }
+
+  @Test
+  public void testScannerWithProvidedVolumes() throws Exception {
+    DirectoryScanner scanner = new DirectoryScanner(datanode, dataset, conf);
+    Map<String, FsVolumeSpi.ScanInfo[]> report = scanner.getDiskReport();
+    // no blocks should be reported for the Provided volume as long as
+    // the directoryScanner is disabled.
+    assertEquals(0, report.get(BLOCK_POOL_IDS[CHOSEN_BP_ID]).length);
+  }
+
 }


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


[04/50] [abbrv] hadoop git commit: HDFS-12912. [READ] Fix configuration and implementation of LevelDB-based alias maps

Posted by na...@apache.org.
HDFS-12912. [READ] Fix configuration and implementation of LevelDB-based alias maps


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

Branch: refs/heads/yarn-3409
Commit: 80c3fec3a13c41051daaae42e5c9a9fedf5c7ee7
Parents: c89b29b
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Wed Dec 13 13:39:21 2017 -0800
Committer: Chris Douglas <cd...@apache.org>
Committed: Fri Dec 15 17:51:41 2017 -0800

----------------------------------------------------------------------
 .../hdfs/server/aliasmap/InMemoryAliasMap.java  | 42 ++++++++++----------
 .../aliasmap/InMemoryLevelDBAliasMapServer.java |  9 +++--
 .../impl/LevelDBFileRegionAliasMap.java         |  5 +++
 .../src/site/markdown/HdfsProvidedStorage.md    |  4 +-
 .../server/aliasmap/ITestInMemoryAliasMap.java  |  9 +++--
 .../server/aliasmap/TestInMemoryAliasMap.java   |  2 +-
 .../impl/TestInMemoryLevelDBAliasMapClient.java |  2 +
 .../impl/TestLevelDbMockAliasMapClient.java     |  2 +-
 .../TestNameNodeProvidedImplementation.java     |  2 +
 9 files changed, 45 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/80c3fec3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryAliasMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryAliasMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryAliasMap.java
index 3d9eeea..142a040 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryAliasMap.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryAliasMap.java
@@ -59,6 +59,7 @@ public class InMemoryAliasMap implements InMemoryAliasMapProtocol,
 
   private final DB levelDb;
   private Configuration conf;
+  private String blockPoolID;
 
   @Override
   public void setConf(Configuration conf) {
@@ -79,32 +80,38 @@ public class InMemoryAliasMap implements InMemoryAliasMapProtocol,
         .toString();
   }
 
-  public static @Nonnull InMemoryAliasMap init(Configuration conf)
-      throws IOException {
+  public static @Nonnull InMemoryAliasMap init(Configuration conf,
+      String blockPoolID) throws IOException {
     Options options = new Options();
     options.createIfMissing(true);
     String directory =
         conf.get(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_LEVELDB_DIR);
     LOG.info("Attempting to load InMemoryAliasMap from \"{}\"", directory);
-    File path = new File(directory);
-    if (!path.exists()) {
+    File levelDBpath;
+    if (blockPoolID != null) {
+      levelDBpath = new File(directory, blockPoolID);
+    } else {
+      levelDBpath = new File(directory);
+    }
+    if (!levelDBpath.exists()) {
       String error = createPathErrorMessage(directory);
       throw new IOException(error);
     }
-    DB levelDb = JniDBFactory.factory.open(path, options);
-    InMemoryAliasMap aliasMap = new InMemoryAliasMap(levelDb);
+    DB levelDb = JniDBFactory.factory.open(levelDBpath, options);
+    InMemoryAliasMap aliasMap = new InMemoryAliasMap(levelDb, blockPoolID);
     aliasMap.setConf(conf);
     return aliasMap;
   }
 
   @VisibleForTesting
-  InMemoryAliasMap(DB levelDb) {
+  InMemoryAliasMap(DB levelDb, String blockPoolID) {
     this.levelDb = levelDb;
+    this.blockPoolID = blockPoolID;
   }
 
   @Override
   public IterationResult list(Optional<Block> marker) throws IOException {
-    return withIterator((DBIterator iterator) -> {
+    try (DBIterator iterator = levelDb.iterator()) {
       Integer batchSize =
           conf.getInt(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_BATCH_SIZE,
               DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_BATCH_SIZE_DEFAULT);
@@ -130,8 +137,7 @@ public class InMemoryAliasMap implements InMemoryAliasMapProtocol,
       } else {
         return new IterationResult(batch, Optional.empty());
       }
-
-    });
+    }
   }
 
   public @Nonnull Optional<ProvidedStorageLocation> read(@Nonnull Block block)
@@ -159,7 +165,7 @@ public class InMemoryAliasMap implements InMemoryAliasMapProtocol,
 
   @Override
   public String getBlockPoolId() {
-    return null;
+    return blockPoolID;
   }
 
   public void close() throws IOException {
@@ -202,21 +208,15 @@ public class InMemoryAliasMap implements InMemoryAliasMapProtocol,
     return blockOutputStream.toByteArray();
   }
 
-  private IterationResult withIterator(
-      CheckedFunction<DBIterator, IterationResult> func) throws IOException {
-    try (DBIterator iterator = levelDb.iterator()) {
-      return func.apply(iterator);
-    }
-  }
-
   /**
    * CheckedFunction is akin to {@link java.util.function.Function} but
    * specifies an IOException.
-   * @param <T> Argument type.
+   * @param <T1> First argument type.
+   * @param <T2> Second argument type.
    * @param <R> Return type.
    */
   @FunctionalInterface
-  public interface CheckedFunction<T, R> {
-    R apply(T t) throws IOException;
+  public interface CheckedFunction2<T1, T2, R> {
+    R apply(T1 t1, T2 t2) throws IOException;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/80c3fec3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryLevelDBAliasMapServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryLevelDBAliasMapServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryLevelDBAliasMapServer.java
index a229ae7..4edc9a2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryLevelDBAliasMapServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryLevelDBAliasMapServer.java
@@ -38,7 +38,7 @@ import java.util.Optional;
 
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS_DEFAULT;
 import static org.apache.hadoop.hdfs.protocol.proto.AliasMapProtocolProtos.*;
-import static org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMap.CheckedFunction;
+import static org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMap.CheckedFunction2;
 
 /**
  * InMemoryLevelDBAliasMapServer is the entry point from the Namenode into
@@ -51,14 +51,15 @@ public class InMemoryLevelDBAliasMapServer implements InMemoryAliasMapProtocol,
 
   private static final Logger LOG = LoggerFactory
       .getLogger(InMemoryLevelDBAliasMapServer.class);
-  private final CheckedFunction<Configuration, InMemoryAliasMap> initFun;
+  private final CheckedFunction2<Configuration, String, InMemoryAliasMap>
+      initFun;
   private RPC.Server aliasMapServer;
   private Configuration conf;
   private InMemoryAliasMap aliasMap;
   private String blockPoolId;
 
   public InMemoryLevelDBAliasMapServer(
-      CheckedFunction<Configuration, InMemoryAliasMap> initFun,
+          CheckedFunction2<Configuration, String, InMemoryAliasMap> initFun,
       String blockPoolId) {
     this.initFun = initFun;
     this.blockPoolId = blockPoolId;
@@ -127,7 +128,7 @@ public class InMemoryLevelDBAliasMapServer implements InMemoryAliasMapProtocol,
   public void setConf(Configuration conf) {
     this.conf = conf;
     try {
-      this.aliasMap = initFun.apply(conf);
+      this.aliasMap = initFun.apply(conf, blockPoolId);
     } catch (IOException e) {
       throw new RuntimeException(e);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/80c3fec3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/LevelDBFileRegionAliasMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/LevelDBFileRegionAliasMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/LevelDBFileRegionAliasMap.java
index adad79a..6afe6bb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/LevelDBFileRegionAliasMap.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/LevelDBFileRegionAliasMap.java
@@ -113,6 +113,11 @@ public class LevelDBFileRegionAliasMap
     } else {
       dbFile = new File(levelDBPath);
     }
+    if (createIfMissing && !dbFile.exists()) {
+      if (!dbFile.mkdirs()) {
+        throw new IOException("Unable to create " + dbFile);
+      }
+    }
     return factory.open(dbFile, options);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/80c3fec3/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsProvidedStorage.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsProvidedStorage.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsProvidedStorage.md
index 7455044..01e7076 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsProvidedStorage.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsProvidedStorage.md
@@ -134,7 +134,7 @@ hadoop org.apache.hadoop.hdfs.server.namenode.FileSystemImage \
 Assign ownership based on a custom `UGIResolver`, in LevelDB:
 ```
 hadoop org.apache.hadoop.hdfs.server.namenode.FileSystemImage \
-  -Ddfs.provided.aliasmap.leveldb.path=file:///path/to/leveldb/map/dingos.db \
+  -Ddfs.provided.aliasmap.leveldb.path=/path/to/leveldb/map/dingos.db \
   -b org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.LevelDBFileRegionAliasMap \
   -o file:///tmp/name \
   -u CustomResolver \
@@ -180,7 +180,7 @@ Datanodes contact this alias map using the `org.apache.hadoop.hdfs.server.aliasm
 
   <property>
     <name>dfs.provided.aliasmap.inmemory.leveldb.dir</name>
-    <value>file:///path/to/leveldb/map/dingos.db</value>
+    <value>/path/to/leveldb/map/dingos.db</value>
     <description>
       The directory where the leveldb files will be kept
     </description>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/80c3fec3/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/aliasmap/ITestInMemoryAliasMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/aliasmap/ITestInMemoryAliasMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/aliasmap/ITestInMemoryAliasMap.java
index 6f1ff3e..432aefb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/aliasmap/ITestInMemoryAliasMap.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/aliasmap/ITestInMemoryAliasMap.java
@@ -44,14 +44,17 @@ import java.util.Optional;
 public class ITestInMemoryAliasMap {
   private InMemoryAliasMap aliasMap;
   private File tempDirectory;
+  private static String bpid = "bpid-0";
 
   @Before
   public void setUp() throws Exception {
     Configuration conf = new Configuration();
-    tempDirectory = Files.createTempDirectory("seagull").toFile();
+    File temp = Files.createTempDirectory("seagull").toFile();
+    tempDirectory = new File(temp, bpid);
+    tempDirectory.mkdirs();
     conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_LEVELDB_DIR,
-        tempDirectory.getAbsolutePath());
-    aliasMap = InMemoryAliasMap.init(conf);
+        temp.getAbsolutePath());
+    aliasMap = InMemoryAliasMap.init(conf, bpid);
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/hadoop/blob/80c3fec3/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/aliasmap/TestInMemoryAliasMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/aliasmap/TestInMemoryAliasMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/aliasmap/TestInMemoryAliasMap.java
index f699055..0f57247 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/aliasmap/TestInMemoryAliasMap.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/aliasmap/TestInMemoryAliasMap.java
@@ -39,7 +39,7 @@ public class TestInMemoryAliasMap {
         nonExistingDirectory);
 
     assertThatExceptionOfType(IOException.class)
-        .isThrownBy(() -> InMemoryAliasMap.init(conf)).withMessage(
+        .isThrownBy(() -> InMemoryAliasMap.init(conf, "bpid")).withMessage(
             InMemoryAliasMap.createPathErrorMessage(nonExistingDirectory));
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/80c3fec3/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestInMemoryLevelDBAliasMapClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestInMemoryLevelDBAliasMapClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestInMemoryLevelDBAliasMapClient.java
index a388199..61a1558 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestInMemoryLevelDBAliasMapClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestInMemoryLevelDBAliasMapClient.java
@@ -66,6 +66,8 @@ public class TestInMemoryLevelDBAliasMapClient {
     conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS,
         "localhost:" + port);
     tempDir = Files.createTempDir();
+    File levelDBDir = new File(tempDir, BPID);
+    levelDBDir.mkdirs();
     conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_LEVELDB_DIR,
         tempDir.getAbsolutePath());
     levelDBAliasMapServer =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/80c3fec3/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestLevelDbMockAliasMapClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestLevelDbMockAliasMapClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestLevelDbMockAliasMapClient.java
index 8212b28..534bc36 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestLevelDbMockAliasMapClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TestLevelDbMockAliasMapClient.java
@@ -54,7 +54,7 @@ public class TestLevelDbMockAliasMapClient {
     aliasMapMock = mock(InMemoryAliasMap.class);
     when(aliasMapMock.getBlockPoolId()).thenReturn(bpid);
     levelDBAliasMapServer = new InMemoryLevelDBAliasMapServer(
-        config -> aliasMapMock, bpid);
+        (config, blockPoolID) -> aliasMapMock, bpid);
     conf = new Configuration();
     int port = 9877;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/80c3fec3/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java
index 2917a34..1023616 100644
--- a/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java
+++ b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java
@@ -784,6 +784,8 @@ public class TestNameNodeProvidedImplementation {
         "localhost:32445");
     File tempDirectory =
         Files.createTempDirectory("in-memory-alias-map").toFile();
+    File leveDBPath = new File(tempDirectory, bpid);
+    leveDBPath.mkdirs();
     conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_LEVELDB_DIR,
         tempDirectory.getAbsolutePath());
     conf.setBoolean(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_ENABLED, true);


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


[22/50] [abbrv] hadoop git commit: HDFS-12930. Remove the extra space in HdfsImageViewer.md. Contributed by Rahul Pathak.

Posted by na...@apache.org.
HDFS-12930. Remove the extra space in HdfsImageViewer.md. Contributed by Rahul Pathak.


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

Branch: refs/heads/yarn-3409
Commit: 25a36b74528678f56c63be643c76d819d6f07840
Parents: c7499f2
Author: Yiqun Lin <yq...@apache.org>
Authored: Tue Dec 19 11:23:16 2017 +0800
Committer: Yiqun Lin <yq...@apache.org>
Committed: Tue Dec 19 11:23:16 2017 +0800

----------------------------------------------------------------------
 .../hadoop-hdfs/src/site/markdown/HdfsImageViewer.md               | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/25a36b74/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsImageViewer.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsImageViewer.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsImageViewer.md
index 9baadc0..bd3a797 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsImageViewer.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsImageViewer.md
@@ -99,7 +99,7 @@ The Web processor now supports the following operations:
 * [GETACLSTATUS](./WebHDFS.html#Get_ACL_Status)
 * [GETXATTRS](./WebHDFS.html#Get_an_XAttr)
 * [LISTXATTRS](./WebHDFS.html#List_all_XAttrs)
-* [CONTENTSUMMARY] (./WebHDFS.html#Get_Content_Summary_of_a_Directory)
+* [CONTENTSUMMARY](./WebHDFS.html#Get_Content_Summary_of_a_Directory)
 
 ### XML Processor
 


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


[36/50] [abbrv] hadoop git commit: YARN-7616. Map YARN application status to Service Status more accurately. (Contributed by Gour Saha)

Posted by na...@apache.org.
YARN-7616. Map YARN application status to Service Status more accurately.  (Contributed by Gour Saha)


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

Branch: refs/heads/yarn-3409
Commit: 41b581012a83a17db785343362c718363e13e8f5
Parents: 94a2ac6
Author: Eric Yang <ey...@apache.org>
Authored: Tue Dec 19 19:14:45 2017 -0500
Committer: Eric Yang <ey...@apache.org>
Committed: Tue Dec 19 19:14:45 2017 -0500

----------------------------------------------------------------------
 .../hadoop/yarn/service/ServiceMaster.java      | 37 ++++++++++++++
 .../hadoop/yarn/service/ServiceScheduler.java   |  4 ++
 .../yarn/service/client/ServiceClient.java      | 26 ++++++----
 .../yarn/service/component/Component.java       | 53 ++++++++++++++++++--
 .../component/instance/ComponentInstance.java   |  6 +--
 .../yarn/service/TestYarnNativeServices.java    | 34 +++++++++++--
 6 files changed, 137 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/41b58101/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceMaster.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceMaster.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceMaster.java
index 1283604..75cc9c5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceMaster.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceMaster.java
@@ -44,6 +44,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
 import org.apache.hadoop.yarn.security.client.ClientToAMTokenSecretManager;
+import org.apache.hadoop.yarn.service.api.records.ServiceState;
 import org.apache.hadoop.yarn.service.exceptions.BadClusterStateException;
 import org.apache.hadoop.yarn.service.monitor.ServiceMonitor;
 import org.apache.hadoop.yarn.service.utils.ServiceApiUtil;
@@ -237,6 +238,7 @@ public class ServiceMaster extends CompositeService {
       SliderFileSystem fs) throws IOException {
     context.service = ServiceApiUtil
         .loadServiceFrom(fs, new Path(serviceDefPath));
+    context.service.setState(ServiceState.ACCEPTED);
     LOG.info(context.service.toString());
   }
 
@@ -257,6 +259,41 @@ public class ServiceMaster extends CompositeService {
     super.serviceStop();
   }
 
+  // This method should be called whenever there is an increment or decrement
+  // of a READY state component of a service
+  public static synchronized void checkAndUpdateServiceState(
+      ServiceScheduler scheduler, boolean isIncrement) {
+    ServiceState curState = scheduler.getApp().getState();
+    if (!isIncrement) {
+      // set it to STARTED every time a component moves out of STABLE state
+      scheduler.getApp().setState(ServiceState.STARTED);
+    } else {
+      // otherwise check the state of all components
+      boolean isStable = true;
+      for (org.apache.hadoop.yarn.service.api.records.Component comp : scheduler
+          .getApp().getComponents()) {
+        if (comp.getState() !=
+            org.apache.hadoop.yarn.service.api.records.ComponentState.STABLE) {
+          isStable = false;
+          break;
+        }
+      }
+      if (isStable) {
+        scheduler.getApp().setState(ServiceState.STABLE);
+      } else {
+        // mark new state as started only if current state is stable, otherwise
+        // leave it as is
+        if (curState == ServiceState.STABLE) {
+          scheduler.getApp().setState(ServiceState.STARTED);
+        }
+      }
+    }
+    if (curState != scheduler.getApp().getState()) {
+      LOG.info("Service state changed from {} -> {}", curState,
+          scheduler.getApp().getState());
+    }
+  }
+
   private void printSystemEnv() {
     for (Map.Entry<String, String> envs : System.getenv().entrySet()) {
       LOG.info("{} = {}", envs.getKey(), envs.getValue());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/41b58101/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceScheduler.java
index 2697050..45cdd28 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceScheduler.java
@@ -58,6 +58,7 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.service.api.ServiceApiConstants;
 import org.apache.hadoop.yarn.service.api.records.Service;
+import org.apache.hadoop.yarn.service.api.records.ServiceState;
 import org.apache.hadoop.yarn.service.api.records.ConfigFile;
 import org.apache.hadoop.yarn.service.component.instance.ComponentInstance;
 import org.apache.hadoop.yarn.service.component.instance.ComponentInstanceEvent;
@@ -284,6 +285,9 @@ public class ServiceScheduler extends CompositeService {
     }
     registerServiceInstance(context.attemptId, app);
 
+    // Since AM has been started and registered, the service is in STARTED state
+    app.setState(ServiceState.STARTED);
+
     // recover components based on containers sent from RM
     recoverComponents(response);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/41b58101/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/ServiceClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/ServiceClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/ServiceClient.java
index 81c56d2..d1ccc4f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/ServiceClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/client/ServiceClient.java
@@ -268,7 +268,8 @@ public class ServiceClient extends AppAdminClient implements SliderExitCodes,
       long ret = orig - Long.parseLong(newNumber.substring(1));
       if (ret < 0) {
         LOG.warn(MessageFormat.format(
-            "[COMPONENT {}]: component count goes to negative ({}{} = {}), reset it to 0.",
+            "[COMPONENT {0}]: component count goes to negative ({1}{2} = {3}),"
+                + " ignore and reset it to 0.",
             component.getName(), orig, newNumber, ret));
         ret = 0;
       }
@@ -878,18 +879,23 @@ public class ServiceClient extends AppAdminClient implements SliderExitCodes,
     return newTimeout;
   }
 
-  public ServiceState convertState(FinalApplicationStatus status) {
-    switch (status) {
-    case UNDEFINED:
+  public ServiceState convertState(YarnApplicationState state) {
+    switch (state) {
+    case NEW:
+    case NEW_SAVING:
+    case SUBMITTED:
+    case ACCEPTED:
       return ServiceState.ACCEPTED;
-    case FAILED:
+    case RUNNING:
+      return ServiceState.STARTED;
+    case FINISHED:
     case KILLED:
-      return ServiceState.FAILED;
-    case ENDED:
-    case SUCCEEDED:
       return ServiceState.STOPPED;
+    case FAILED:
+      return ServiceState.FAILED;
+    default:
+      return ServiceState.ACCEPTED;
     }
-    return ServiceState.ACCEPTED;
   }
 
   public String getStatusString(String appId)
@@ -917,7 +923,7 @@ public class ServiceClient extends AppAdminClient implements SliderExitCodes,
     ApplicationReport appReport = yarnClient.getApplicationReport(currentAppId);
     Service appSpec = new Service();
     appSpec.setName(serviceName);
-    appSpec.setState(convertState(appReport.getFinalApplicationStatus()));
+    appSpec.setState(convertState(appReport.getYarnApplicationState()));
     ApplicationTimeout lifetime =
         appReport.getApplicationTimeouts().get(ApplicationTimeoutType.LIFETIME);
     if (lifetime != null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/41b58101/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/Component.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/Component.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/Component.java
index 9c5cbae..a84c1b1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/Component.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/Component.java
@@ -31,7 +31,9 @@ import org.apache.hadoop.yarn.service.component.instance.ComponentInstanceId;
 import org.apache.hadoop.yarn.service.ContainerFailureTracker;
 import org.apache.hadoop.yarn.service.ServiceContext;
 import org.apache.hadoop.yarn.service.ServiceScheduler;
+import org.apache.hadoop.yarn.service.api.records.ServiceState;
 import org.apache.hadoop.yarn.service.component.instance.ComponentInstanceEvent;
+import org.apache.hadoop.yarn.service.ServiceMaster;
 import org.apache.hadoop.yarn.service.ServiceMetrics;
 import org.apache.hadoop.yarn.service.provider.ProviderUtils;
 import org.apache.hadoop.yarn.state.InvalidStateTransitionException;
@@ -209,6 +211,7 @@ public class Component implements EventHandler<ComponentEvent> {
         component.createNumCompInstances(delta);
         component.componentSpec.setState(
             org.apache.hadoop.yarn.service.api.records.ComponentState.FLEXING);
+        component.getScheduler().getApp().setState(ServiceState.STARTED);
         return FLEXING;
       } else if (delta < 0){
         delta = 0 - delta;
@@ -229,14 +232,11 @@ public class Component implements EventHandler<ComponentEvent> {
           component.instanceIdCounter.decrementAndGet();
           instance.destroy();
         }
-        component.componentSpec.setState(
-            org.apache.hadoop.yarn.service.api.records.ComponentState.STABLE);
+        checkAndUpdateComponentState(component, false);
         return STABLE;
       } else {
         LOG.info("[FLEX COMPONENT " + component.getName() + "]: already has " +
             event.getDesired() + " instances, ignoring");
-        component.componentSpec.setState(
-            org.apache.hadoop.yarn.service.api.records.ComponentState.STABLE);
         return STABLE;
       }
     }
@@ -289,7 +289,7 @@ public class Component implements EventHandler<ComponentEvent> {
 
   private static ComponentState checkIfStable(Component component) {
     // if desired == running
-    if (component.componentMetrics.containersRunning.value() == component
+    if (component.componentMetrics.containersReady.value() == component
         .getComponentSpec().getNumberOfContainers()) {
       component.componentSpec.setState(
           org.apache.hadoop.yarn.service.api.records.ComponentState.STABLE);
@@ -301,6 +301,46 @@ public class Component implements EventHandler<ComponentEvent> {
     }
   }
 
+  // This method should be called whenever there is an increment or decrement
+  // of a READY state container of a component
+  public static synchronized void checkAndUpdateComponentState(
+      Component component, boolean isIncrement) {
+    org.apache.hadoop.yarn.service.api.records.ComponentState curState =
+        component.componentSpec.getState();
+    if (isIncrement) {
+      // check if all containers are in READY state
+      if (component.componentMetrics.containersReady
+          .value() == component.componentMetrics.containersDesired.value()) {
+        component.componentSpec.setState(
+            org.apache.hadoop.yarn.service.api.records.ComponentState.STABLE);
+        if (curState != component.componentSpec.getState()) {
+          LOG.info("[COMPONENT {}] state changed from {} -> {}",
+              component.componentSpec.getName(), curState,
+              component.componentSpec.getState());
+        }
+        // component state change will trigger re-check of service state
+        ServiceMaster.checkAndUpdateServiceState(component.scheduler,
+            isIncrement);
+      }
+    } else {
+      // container moving out of READY state could be because of FLEX down so
+      // still need to verify the count before changing the component state
+      if (component.componentMetrics.containersReady
+          .value() < component.componentMetrics.containersDesired.value()) {
+        component.componentSpec.setState(
+            org.apache.hadoop.yarn.service.api.records.ComponentState.FLEXING);
+        if (curState != component.componentSpec.getState()) {
+          LOG.info("[COMPONENT {}] state changed from {} -> {}",
+              component.componentSpec.getName(), curState,
+              component.componentSpec.getState());
+        }
+        // component state change will trigger re-check of service state
+        ServiceMaster.checkAndUpdateServiceState(component.scheduler,
+            isIncrement);
+      }
+    }
+  }
+
   private static class ContainerCompletedTransition extends BaseTransition {
     @Override
     public void transition(Component component, ComponentEvent event) {
@@ -310,6 +350,7 @@ public class Component implements EventHandler<ComponentEvent> {
               STOP).setStatus(event.getStatus()));
       component.componentSpec.setState(
           org.apache.hadoop.yarn.service.api.records.ComponentState.FLEXING);
+      component.getScheduler().getApp().setState(ServiceState.STARTED);
     }
   }
 
@@ -472,11 +513,13 @@ public class Component implements EventHandler<ComponentEvent> {
   public void incContainersReady() {
     componentMetrics.containersReady.incr();
     scheduler.getServiceMetrics().containersReady.incr();
+    checkAndUpdateComponentState(this, true);
   }
 
   public void decContainersReady() {
     componentMetrics.containersReady.decr();
     scheduler.getServiceMetrics().containersReady.decr();
+    checkAndUpdateComponentState(this, false);
   }
 
   public int getNumReadyInstances() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/41b58101/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/instance/ComponentInstance.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/instance/ComponentInstance.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/instance/ComponentInstance.java
index 31fa5c7..0e3e11b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/instance/ComponentInstance.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/instance/ComponentInstance.java
@@ -147,7 +147,6 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
               new ContainerStatusRetriever(compInstance.scheduler,
                   event.getContainerId(), compInstance), 0, 1,
               TimeUnit.SECONDS);
-      compInstance.component.incRunningContainers();
       long containerStartTime = System.currentTimeMillis();
       try {
         ContainerTokenIdentifier containerTokenIdentifier = BuilderUtils
@@ -171,6 +170,7 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
       compInstance.containerSpec = container;
       compInstance.getCompSpec().addContainer(container);
       compInstance.containerStartedTime = containerStartTime;
+      compInstance.component.incRunningContainers();
 
       if (compInstance.timelineServiceEnabled) {
         compInstance.serviceTimelinePublisher
@@ -183,8 +183,8 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
     @Override
     public void transition(ComponentInstance compInstance,
         ComponentInstanceEvent event) {
-      compInstance.component.incContainersReady();
       compInstance.containerSpec.setState(ContainerState.READY);
+      compInstance.component.incContainersReady();
       if (compInstance.timelineServiceEnabled) {
         compInstance.serviceTimelinePublisher
             .componentInstanceBecomeReady(compInstance.containerSpec);
@@ -196,8 +196,8 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
     @Override
     public void transition(ComponentInstance compInstance,
         ComponentInstanceEvent event) {
-      compInstance.component.decContainersReady();
       compInstance.containerSpec.setState(ContainerState.RUNNING_BUT_UNREADY);
+      compInstance.component.decContainersReady();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/41b58101/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestYarnNativeServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestYarnNativeServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestYarnNativeServices.java
index 1c517d9..debab8b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestYarnNativeServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/TestYarnNativeServices.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.yarn.api.records.*;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.service.api.records.Service;
+import org.apache.hadoop.yarn.service.api.records.ServiceState;
 import org.apache.hadoop.yarn.service.api.records.Component;
 import org.apache.hadoop.yarn.service.api.records.Container;
 import org.apache.hadoop.yarn.service.api.records.ContainerState;
@@ -90,25 +91,25 @@ public class TestYarnNativeServices extends ServiceTestUtils {
     // check app.json is persisted.
     Assert.assertTrue(
         getFS().exists(new Path(appDir, exampleApp.getName() + ".json")));
-    waitForAllCompToBeReady(client, exampleApp);
+    waitForServiceToBeStable(client, exampleApp);
 
     // Flex two components, each from 2 container to 3 containers.
     flexComponents(client, exampleApp, 3L);
     // wait for flex to be completed, increase from 2 to 3 containers.
-    waitForAllCompToBeReady(client, exampleApp);
+    waitForServiceToBeStable(client, exampleApp);
     // check all instances name for each component are in sequential order.
     checkCompInstancesInOrder(client, exampleApp);
 
     // flex down to 1
     flexComponents(client, exampleApp, 1L);
-    waitForAllCompToBeReady(client, exampleApp);
+    waitForServiceToBeStable(client, exampleApp);
     checkCompInstancesInOrder(client, exampleApp);
 
     // check component dir and registry are cleaned up.
 
     // flex up again to 2
     flexComponents(client, exampleApp, 2L);
-    waitForAllCompToBeReady(client, exampleApp);
+    waitForServiceToBeStable(client, exampleApp);
     checkCompInstancesInOrder(client, exampleApp);
 
     // stop the service
@@ -145,7 +146,7 @@ public class TestYarnNativeServices extends ServiceTestUtils {
     exampleApp.addComponent(compb);
 
     client.actionCreate(exampleApp);
-    waitForAllCompToBeReady(client, exampleApp);
+    waitForServiceToBeStable(client, exampleApp);
 
     // check that containers for compa are launched before containers for compb
     checkContainerLaunchDependencies(client, exampleApp, "compa", "compb");
@@ -372,6 +373,29 @@ public class TestYarnNativeServices extends ServiceTestUtils {
     return allContainers;
   }
 
+  /**
+   * Wait until service state becomes stable. A service is stable when all
+   * requested containers of all components are running and in ready state.
+   *
+   * @param client
+   * @param exampleApp
+   * @throws TimeoutException
+   * @throws InterruptedException
+   */
+  private void waitForServiceToBeStable(ServiceClient client,
+      Service exampleApp) throws TimeoutException, InterruptedException {
+    GenericTestUtils.waitFor(() -> {
+      try {
+        Service retrievedApp = client.getStatus(exampleApp.getName());
+        System.out.println(retrievedApp);
+        return retrievedApp.getState() == ServiceState.STABLE;
+      } catch (Exception e) {
+        e.printStackTrace();
+        return false;
+      }
+    }, 2000, 200000);
+  }
+
   private ServiceClient createClient() throws Exception {
     ServiceClient client = new ServiceClient() {
       @Override protected Path addJarResource(String appName,


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


[17/50] [abbrv] hadoop git commit: HDFS-12818. Support multiple storages in DataNodeCluster / SimulatedFSDataset. Contributed by Erik Krogen.

Posted by na...@apache.org.
HDFS-12818. Support multiple storages in DataNodeCluster / SimulatedFSDataset. Contributed by Erik Krogen.

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

Branch: refs/heads/yarn-3409
Commit: 94576b17fbc19c440efafb6c3322f53ec78a5b55
Parents: 0010089
Author: Erik Krogen <ek...@linkedin.com>
Authored: Mon Dec 18 11:36:22 2017 -0800
Committer: Konstantin V Shvachko <sh...@apache.org>
Committed: Mon Dec 18 11:36:22 2017 -0800

----------------------------------------------------------------------
 .../server/datanode/SimulatedFSDataset.java     | 308 +++++++++++++------
 .../server/datanode/TestSimulatedFSDataset.java | 147 +++++----
 ...tSimulatedFSDatasetWithMultipleStorages.java |  50 +++
 3 files changed, 352 insertions(+), 153 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/94576b17/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
index c31df4c..987ba97 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
@@ -23,8 +23,8 @@ import java.io.InputStream;
 import java.io.OutputStream;
 import java.net.URI;
 import java.nio.channels.ClosedChannelException;
+import java.util.ArrayList;
 import java.util.Collection;
-import java.util.Collections;
 import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.List;
@@ -37,11 +37,13 @@ import javax.management.NotCompliantMBeanException;
 import javax.management.ObjectName;
 import javax.management.StandardMBean;
 
+import com.google.common.math.LongMath;
 import org.apache.commons.lang.ArrayUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.DF;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.server.datanode.checker.VolumeCheckResult;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetImplTestUtils;
 import org.apache.hadoop.util.AutoCloseableLock;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.Block;
@@ -88,6 +90,7 @@ import org.apache.hadoop.util.DataChecksum;
  */
 public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
   public final static int BYTE_MASK = 0xff;
+  private final static int DEFAULT_NUM_SIMULATED_DATA_DIRS = 1;
   static class Factory extends FsDatasetSpi.Factory<SimulatedFSDataset> {
     @Override
     public SimulatedFSDataset newInstance(DataNode datanode,
@@ -100,10 +103,42 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
       return true;
     }
   }
-  
+
+  /**
+   * Used to change the default number of data storages and to mark the
+   * FSDataset as simulated.
+   */
+  static class TestUtilsFactory
+      extends FsDatasetTestUtils.Factory<FsDatasetTestUtils> {
+    @Override
+    public FsDatasetTestUtils newInstance(DataNode datanode) {
+      return new FsDatasetImplTestUtils(datanode) {
+        @Override
+        public int getDefaultNumOfDataDirs() {
+          return DEFAULT_NUM_SIMULATED_DATA_DIRS;
+        }
+      };
+    }
+
+    @Override
+    public boolean isSimulated() {
+      return true;
+    }
+
+    @Override
+    public int getDefaultNumOfDataDirs() {
+      return DEFAULT_NUM_SIMULATED_DATA_DIRS;
+    }
+
+  }
+
   public static void setFactory(Configuration conf) {
     conf.set(DFSConfigKeys.DFS_DATANODE_FSDATASET_FACTORY_KEY,
         Factory.class.getName());
+    conf.setClass("org.apache.hadoop.hdfs.server.datanode." +
+            "SimulatedFSDatasetTestUtilsFactory",
+        TestUtilsFactory.class, FsDatasetTestUtils.Factory.class
+    );
   }
 
   public static byte simulatedByte(Block b, long offsetInBlk) {
@@ -151,7 +186,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
       if (theBlock.getNumBytes() < 0) {
         theBlock.setNumBytes(0);
       }
-      if (!storage.alloc(bpid, theBlock.getNumBytes())) { 
+      if (!getStorage(theBlock).alloc(bpid, theBlock.getNumBytes())) {
         // expected length - actual length may
         // be more - we find out at finalize
         DataNode.LOG.warn("Lack of free storage on a block alloc");
@@ -169,7 +204,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
     
     @Override
     public String getStorageUuid() {
-      return storage.getStorageUuid();
+      return getStorage(theBlock).getStorageUuid();
     }
 
     @Override
@@ -226,12 +261,12 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
       // adjust if necessary
       long extraLen = finalSize - theBlock.getNumBytes();
       if (extraLen > 0) {
-        if (!storage.alloc(bpid,extraLen)) {
+        if (!getStorage(theBlock).alloc(bpid, extraLen)) {
           DataNode.LOG.warn("Lack of free storage on a block alloc");
           throw new IOException("Creating block, no free space available");
         }
       } else {
-        storage.free(bpid, -extraLen);
+        getStorage(theBlock).free(bpid, -extraLen);
       }
       theBlock.setNumBytes(finalSize);  
 
@@ -271,7 +306,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
       } else {
         SimulatedOutputStream crcStream = new SimulatedOutputStream();
         return new ReplicaOutputStreams(oStream, crcStream, requestedChecksum,
-            volume, fileIoProvider);
+            getStorage(theBlock).getVolume(), fileIoProvider);
       }
     }
 
@@ -368,6 +403,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
    */
   private static class SimulatedBPStorage {
     private long used;    // in bytes
+    private final Map<Block, BInfo> blockMap = new TreeMap<>();
     
     long getUsed() {
       return used;
@@ -381,6 +417,10 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
       used -= amount;
     }
     
+    Map<Block, BInfo> getBlockMap() {
+      return blockMap;
+    }
+
     SimulatedBPStorage() {
       used = 0;   
     }
@@ -392,10 +432,11 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
    */
   private static class SimulatedStorage {
     private final Map<String, SimulatedBPStorage> map =
-      new HashMap<String, SimulatedBPStorage>();
+        new ConcurrentHashMap<>();
 
     private final long capacity;  // in bytes
     private final DatanodeStorage dnStorage;
+    private final SimulatedVolume volume;
     
     synchronized long getFree() {
       return capacity - getUsed();
@@ -433,11 +474,15 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
       getBPStorage(bpid).free(amount);
     }
     
-    SimulatedStorage(long cap, DatanodeStorage.State state) {
+    SimulatedStorage(long cap, DatanodeStorage.State state,
+        FileIoProvider fileIoProvider, Configuration conf) {
       capacity = cap;
       dnStorage = new DatanodeStorage(
           "SimulatedStorage-" + DatanodeStorage.generateUuid(),
           state, StorageType.DEFAULT);
+      DataNodeVolumeMetrics volumeMetrics =
+          DataNodeVolumeMetrics.create(conf, dnStorage.getStorageID());
+      this.volume = new SimulatedVolume(this, fileIoProvider, volumeMetrics);
     }
     
     synchronized void addBlockPool(String bpid) {
@@ -473,6 +518,18 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
           false, getCapacity(), getUsed(), getFree(),
           map.get(bpid).getUsed(), 0L);
     }
+
+    SimulatedVolume getVolume() {
+      return volume;
+    }
+
+    Map<Block, BInfo> getBlockMap(String bpid) throws IOException {
+      SimulatedBPStorage bpStorage = map.get(bpid);
+      if (bpStorage == null) {
+        throw new IOException("Nonexistent block pool: " + bpid);
+      }
+      return bpStorage.getBlockMap();
+    }
   }
   
   static class SimulatedVolume implements FsVolumeSpi {
@@ -601,10 +658,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
     }
   }
 
-  private final Map<String, Map<Block, BInfo>> blockMap
-      = new ConcurrentHashMap<String, Map<Block,BInfo>>();
-  private final SimulatedStorage storage;
-  private final SimulatedVolume volume;
+  private final List<SimulatedStorage> storages;
   private final String datanodeUuid;
   private final DataNode datanode;
   
@@ -615,27 +669,30 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
 
   public SimulatedFSDataset(DataNode datanode, DataStorage storage, Configuration conf) {
     this.datanode = datanode;
-    if (storage != null) {
+    int storageCount;
+    if (storage != null && storage.getNumStorageDirs() > 0) {
+      storageCount = storage.getNumStorageDirs();
       for (int i = 0; i < storage.getNumStorageDirs(); ++i) {
         DataStorage.createStorageID(storage.getStorageDir(i), false, conf);
       }
       this.datanodeUuid = storage.getDatanodeUuid();
     } else {
+      storageCount = DataNode.getStorageLocations(conf).size();
       this.datanodeUuid = "SimulatedDatanode-" + DataNode.generateUuid();
     }
 
     registerMBean(datanodeUuid);
     this.fileIoProvider = new FileIoProvider(conf, datanode);
-    this.storage = new SimulatedStorage(
-        conf.getLong(CONFIG_PROPERTY_CAPACITY, DEFAULT_CAPACITY),
-        conf.getEnum(CONFIG_PROPERTY_STATE, DEFAULT_STATE));
-
-    // TODO: per volume id or path
-    DataNodeVolumeMetrics volumeMetrics = DataNodeVolumeMetrics.create(conf,
-        datanodeUuid);
-    this.volume = new SimulatedVolume(this.storage, this.fileIoProvider,
-        volumeMetrics);
+
     this.datasetLock = new AutoCloseableLock();
+
+    this.storages = new ArrayList<>();
+    for (int i = 0; i < storageCount; i++) {
+      this.storages.add(new SimulatedStorage(
+          conf.getLong(CONFIG_PROPERTY_CAPACITY, DEFAULT_CAPACITY),
+          conf.getEnum(CONFIG_PROPERTY_STATE, DEFAULT_STATE),
+          fileIoProvider, conf));
+    }
   }
 
   public synchronized void injectBlocks(String bpid,
@@ -651,33 +708,50 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
           throw new IOException("Block already exists in  block list");
         }
       }
-      Map<Block, BInfo> map = blockMap.get(bpid);
-      if (map == null) {
-        map = new TreeMap<>();
-        blockMap.put(bpid, map);
+
+      for (SimulatedStorage storage : storages) {
+        storage.addBlockPool(bpid);
       }
-      
+
       for (Block b: injectBlocks) {
         BInfo binfo = new BInfo(bpid, b, false);
-        map.put(binfo.theBlock, binfo);
+        getBlockMap(b, bpid).put(binfo.theBlock, binfo);
       }
     }
   }
+
+  /** Get the storage that a given block lives within. */
+  private SimulatedStorage getStorage(Block b) {
+    return storages.get(LongMath.mod(b.getBlockId(), storages.size()));
+  }
   
-  /** Get a map for a given block pool Id */
-  private Map<Block, BInfo> getMap(String bpid) throws IOException {
-    final Map<Block, BInfo> map = blockMap.get(bpid);
-    if (map == null) {
-      throw new IOException("Non existent blockpool " + bpid);
-    }
-    return map;
+  /**
+   * Get the block map that a given block lives within, assuming it is within
+   * block pool bpid.
+   * @param b The block to look for
+   * @param bpid The block pool that contains b
+   * @return The block map (non-null)
+   * @throws IOException if bpid does not exist
+   */
+  private Map<Block, BInfo> getBlockMap(Block b, String bpid)
+      throws IOException {
+    return getStorage(b).getBlockMap(bpid);
+  }
+
+  /**
+   * Get the block map that a given block lives within.
+   * @param b The extended block to look for
+   * @return The block map (non-null)
+   * @throws IOException if b is in a nonexistent block pool
+   */
+  private Map<Block, BInfo> getBlockMap(ExtendedBlock b) throws IOException {
+    return getBlockMap(b.getLocalBlock(), b.getBlockPoolId());
   }
 
   @Override // FsDatasetSpi
   public synchronized void finalizeBlock(ExtendedBlock b, boolean fsyncDir)
       throws IOException {
-    final Map<Block, BInfo> map = getMap(b.getBlockPoolId());
-    BInfo binfo = map.get(b.getLocalBlock());
+    BInfo binfo = getBlockMap(b).get(b.getLocalBlock());
     if (binfo == null) {
       throw new IOException("Finalizing a non existing block " + b);
     }
@@ -687,20 +761,21 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
   @Override // FsDatasetSpi
   public synchronized void unfinalizeBlock(ExtendedBlock b) throws IOException{
     if (isValidRbw(b)) {
-      final Map<Block, BInfo> map = getMap(b.getBlockPoolId());
-      map.remove(b.getLocalBlock());
+      getBlockMap(b).remove(b.getLocalBlock());
     }
   }
 
-  synchronized BlockListAsLongs getBlockReport(String bpid) {
+  synchronized BlockListAsLongs getBlockReport(String bpid,
+      SimulatedStorage storage) {
     BlockListAsLongs.Builder report = BlockListAsLongs.builder();
-    final Map<Block, BInfo> map = blockMap.get(bpid);
-    if (map != null) {
-      for (BInfo b : map.values()) {
+    try {
+      for (BInfo b : storage.getBlockMap(bpid).values()) {
         if (b.isFinalized()) {
           report.add(b);
         }
       }
+    } catch (IOException ioe) {
+      DataNode.LOG.error("Exception while getting block reports", ioe);
     }
     return report.build();
   }
@@ -708,7 +783,11 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
   @Override
   public synchronized Map<DatanodeStorage, BlockListAsLongs> getBlockReports(
       String bpid) {
-    return Collections.singletonMap(storage.getDnStorage(), getBlockReport(bpid));
+    Map<DatanodeStorage, BlockListAsLongs> blockReports = new HashMap<>();
+    for (SimulatedStorage storage : storages) {
+      blockReports.put(storage.getDnStorage(), getBlockReport(bpid, storage));
+    }
+    return blockReports;
   }
 
   @Override // FsDatasetSpi
@@ -718,27 +797,49 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
 
   @Override // FSDatasetMBean
   public long getCapacity() {
-    return storage.getCapacity();
+    long total = 0;
+    for (SimulatedStorage storage : storages) {
+      total += storage.getCapacity();
+    }
+    return total;
   }
 
   @Override // FSDatasetMBean
   public long getDfsUsed() {
-    return storage.getUsed();
+    long total = 0;
+    for (SimulatedStorage storage : storages) {
+      total += storage.getUsed();
+    }
+    return total;
   }
 
   @Override // FSDatasetMBean
   public long getBlockPoolUsed(String bpid) throws IOException {
-    return storage.getBlockPoolUsed(bpid);
+    long total = 0;
+    for (SimulatedStorage storage : storages) {
+      total += storage.getBlockPoolUsed(bpid);
+    }
+    return total;
   }
   
   @Override // FSDatasetMBean
   public long getRemaining() {
-    return storage.getFree();
+
+    long total = 0;
+    for (SimulatedStorage storage : storages) {
+      total += storage.getFree();
+    }
+    return total;
   }
 
   @Override // FSDatasetMBean
   public int getNumFailedVolumes() {
-    return storage.getNumFailedVolumes();
+
+    int total = 0;
+    for (SimulatedStorage storage : storages) {
+      total += storage.getNumFailedVolumes();
+    }
+    return total;
   }
 
   @Override // FSDatasetMBean
@@ -803,8 +904,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
 
   @Override // FsDatasetSpi
   public synchronized long getLength(ExtendedBlock b) throws IOException {
-    final Map<Block, BInfo> map = getMap(b.getBlockPoolId());
-    BInfo binfo = map.get(b.getLocalBlock());
+    BInfo binfo = getBlockMap(b).get(b.getLocalBlock());
     if (binfo == null) {
       throw new IOException("Finalizing a non existing block " + b);
     }
@@ -814,34 +914,38 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
   @Override
   @Deprecated
   public Replica getReplica(String bpid, long blockId) {
-    final Map<Block, BInfo> map = blockMap.get(bpid);
-    if (map != null) {
-      return map.get(new Block(blockId));
+    Block b = new Block(blockId);
+    try {
+      return getBlockMap(b, bpid).get(b);
+    } catch (IOException ioe) {
+      return null;
     }
-    return null;
   }
 
   @Override 
   public synchronized String getReplicaString(String bpid, long blockId) {
     Replica r = null;
-    final Map<Block, BInfo> map = blockMap.get(bpid);
-    if (map != null) {
-      r = map.get(new Block(blockId));
+    try {
+      Block b = new Block(blockId);
+      r = getBlockMap(b, bpid).get(b);
+    } catch (IOException ioe) {
+      // Ignore
     }
     return r == null? "null": r.toString();
   }
 
   @Override // FsDatasetSpi
   public Block getStoredBlock(String bpid, long blkid) throws IOException {
-    final Map<Block, BInfo> map = blockMap.get(bpid);
-    if (map != null) {
-      BInfo binfo = map.get(new Block(blkid));
+    Block b = new Block(blkid);
+    try {
+      BInfo binfo = getBlockMap(b, bpid).get(b);
       if (binfo == null) {
         return null;
       }
       return new Block(blkid, binfo.getGenerationStamp(), binfo.getNumBytes());
+    } catch (IOException ioe) {
+      return null;
     }
-    return null;
   }
 
   @Override // FsDatasetSpi
@@ -851,18 +955,18 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
     if (invalidBlks == null) {
       return;
     }
-    final Map<Block, BInfo> map = getMap(bpid);
     for (Block b: invalidBlks) {
       if (b == null) {
         continue;
       }
+      Map<Block, BInfo> map = getBlockMap(b, bpid);
       BInfo binfo = map.get(b);
       if (binfo == null) {
         error = true;
         DataNode.LOG.warn("Invalidate: Missing block");
         continue;
       }
-      storage.free(bpid, binfo.getNumBytes());
+      getStorage(b).free(bpid, binfo.getNumBytes());
       map.remove(b);
       if (datanode != null) {
         datanode.notifyNamenodeDeletedBlock(new ExtendedBlock(bpid, b),
@@ -892,8 +996,11 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
   }
 
   private BInfo getBInfo(final ExtendedBlock b) {
-    final Map<Block, BInfo> map = blockMap.get(b.getBlockPoolId());
-    return map == null? null: map.get(b.getLocalBlock());
+    try {
+      return getBlockMap(b).get(b.getLocalBlock());
+    } catch (IOException ioe) {
+      return null;
+    }
   }
 
   @Override // {@link FsDatasetSpi}
@@ -957,8 +1064,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
   @Override // FsDatasetSpi
   public synchronized ReplicaHandler append(
       ExtendedBlock b, long newGS, long expectedBlockLen) throws IOException {
-    final Map<Block, BInfo> map = getMap(b.getBlockPoolId());
-    BInfo binfo = map.get(b.getLocalBlock());
+    BInfo binfo = getBlockMap(b).get(b.getLocalBlock());
     if (binfo == null || !binfo.isFinalized()) {
       throw new ReplicaNotFoundException("Block " + b
           + " is not valid, and cannot be appended to.");
@@ -970,7 +1076,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
   @Override // FsDatasetSpi
   public synchronized ReplicaHandler recoverAppend(
       ExtendedBlock b, long newGS, long expectedBlockLen) throws IOException {
-    final Map<Block, BInfo> map = getMap(b.getBlockPoolId());
+    final Map<Block, BInfo> map = getBlockMap(b);
     BInfo binfo = map.get(b.getLocalBlock());
     if (binfo == null) {
       throw new ReplicaNotFoundException("Block " + b
@@ -988,7 +1094,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
   @Override // FsDatasetSpi
   public Replica recoverClose(ExtendedBlock b, long newGS, long expectedBlockLen)
       throws IOException {
-    final Map<Block, BInfo> map = getMap(b.getBlockPoolId());
+    final Map<Block, BInfo> map = getBlockMap(b);
     BInfo binfo = map.get(b.getLocalBlock());
     if (binfo == null) {
       throw new ReplicaNotFoundException("Block " + b
@@ -1007,7 +1113,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
   public synchronized ReplicaHandler recoverRbw(
       ExtendedBlock b, long newGS, long minBytesRcvd, long maxBytesRcvd)
       throws IOException {
-    final Map<Block, BInfo> map = getMap(b.getBlockPoolId());
+    final Map<Block, BInfo> map = getBlockMap(b);
     BInfo binfo = map.get(b.getLocalBlock());
     if ( binfo == null) {
       throw new ReplicaNotFoundException("Block " + b
@@ -1042,16 +1148,14 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
       throw new ReplicaAlreadyExistsException("Block " + b +
           " is being written, and cannot be written to.");
     }
-    final Map<Block, BInfo> map = getMap(b.getBlockPoolId());
     BInfo binfo = new BInfo(b.getBlockPoolId(), b.getLocalBlock(), true);
-    map.put(binfo.theBlock, binfo);
+    getBlockMap(b).put(binfo.theBlock, binfo);
     return new ReplicaHandler(binfo, null);
   }
 
   protected synchronized InputStream getBlockInputStream(ExtendedBlock b)
       throws IOException {
-    final Map<Block, BInfo> map = getMap(b.getBlockPoolId());
-    BInfo binfo = map.get(b.getLocalBlock());
+    BInfo binfo = getBlockMap(b).get(b.getLocalBlock());
     if (binfo == null) {
       throw new IOException("No such Block " + b );  
     }
@@ -1077,8 +1181,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
   @Override // FsDatasetSpi
   public synchronized LengthInputStream getMetaDataInputStream(ExtendedBlock b
       ) throws IOException {
-    final Map<Block, BInfo> map = getMap(b.getBlockPoolId());
-    BInfo binfo = map.get(b.getLocalBlock());
+    BInfo binfo = getBlockMap(b).get(b.getLocalBlock());
     if (binfo == null) {
       throw new IOException("No such Block " + b );  
     }
@@ -1266,8 +1369,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
   public ReplicaRecoveryInfo initReplicaRecovery(RecoveringBlock rBlock)
   throws IOException {
     ExtendedBlock b = rBlock.getBlock();
-    final Map<Block, BInfo> map = getMap(b.getBlockPoolId());
-    BInfo binfo = map.get(b.getLocalBlock());
+    BInfo binfo = getBlockMap(b).get(b.getLocalBlock());
     if (binfo == null) {
       throw new IOException("No such Block " + b );  
     }
@@ -1282,7 +1384,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
                                         long recoveryId,
                                         long newBlockId,
                                         long newlength) throws IOException {
-    return getMap(oldBlock.getBlockPoolId()).get(oldBlock.getLocalBlock());
+    return getBInfo(oldBlock);
   }
 
   @Override // FsDatasetSpi
@@ -1292,15 +1394,16 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
 
   @Override // FsDatasetSpi
   public void addBlockPool(String bpid, Configuration conf) {
-    Map<Block, BInfo> map = new TreeMap<>();
-    blockMap.put(bpid, map);
-    storage.addBlockPool(bpid);
+    for (SimulatedStorage storage : storages) {
+      storage.addBlockPool(bpid);
+    }
   }
   
   @Override // FsDatasetSpi
   public void shutdownBlockPool(String bpid) {
-    blockMap.remove(bpid);
-    storage.removeBlockPool(bpid);
+    for (SimulatedStorage storage : storages) {
+      storage.removeBlockPool(bpid);
+    }
   }
   
   @Override // FsDatasetSpi
@@ -1311,11 +1414,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
   @Override
   public ReplicaInPipeline convertTemporaryToRbw(ExtendedBlock temporary)
       throws IOException {
-    final Map<Block, BInfo> map = blockMap.get(temporary.getBlockPoolId());
-    if (map == null) {
-      throw new IOException("Block pool not found, temporary=" + temporary);
-    }
-    final BInfo r = map.get(temporary.getLocalBlock());
+    final BInfo r = getBlockMap(temporary).get(temporary.getLocalBlock());
     if (r == null) {
       throw new IOException("Block not found, temporary=" + temporary);
     } else if (r.isFinalized()) {
@@ -1359,7 +1458,11 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
 
   @Override
   public FsVolumeReferences getFsVolumeReferences() {
-    return new FsVolumeReferences(Collections.singletonList(volume));
+    List<SimulatedVolume> volumes = new ArrayList<>();
+    for (SimulatedStorage storage : storages) {
+      volumes.add(storage.getVolume());
+    }
+    return new FsVolumeReferences(volumes);
   }
 
   @Override
@@ -1371,14 +1474,21 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
 
   @Override
   public DatanodeStorage getStorage(final String storageUuid) {
-    return storageUuid.equals(storage.getStorageUuid()) ?
-        storage.dnStorage :
-        null;
+    for (SimulatedStorage storage : storages) {
+      if (storageUuid.equals(storage.getStorageUuid())) {
+        return storage.getDnStorage();
+      }
+    }
+    return null;
   }
 
   @Override
   public StorageReport[] getStorageReports(String bpid) {
-    return new StorageReport[] {storage.getStorageReport(bpid)};
+    List<StorageReport> reports = new ArrayList<>();
+    for (SimulatedStorage storage : storages) {
+      reports.add(storage.getStorageReport(bpid));
+    }
+    return reports.toArray(new StorageReport[0]);
   }
 
   @Override
@@ -1393,7 +1503,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
 
   @Override
   public FsVolumeSpi getVolume(ExtendedBlock b) {
-    return volume;
+    return getStorage(b.getLocalBlock()).getVolume();
   }
 
   @Override
@@ -1428,12 +1538,12 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
   
   @Override
   public void setPinning(ExtendedBlock b) throws IOException {
-    blockMap.get(b.getBlockPoolId()).get(b.getLocalBlock()).pinned = true;
+    getBlockMap(b).get(b.getLocalBlock()).pinned = true;
   }
   
   @Override
   public boolean getPinning(ExtendedBlock b) throws IOException {
-    return blockMap.get(b.getBlockPoolId()).get(b.getLocalBlock()).pinned;
+    return getBlockMap(b).get(b.getLocalBlock()).pinned;
   }
   
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94576b17/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDataset.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDataset.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDataset.java
index 4775fc7..dde9ad5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDataset.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDataset.java
@@ -26,20 +26,19 @@ import java.io.DataInputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
+import java.util.Map;
 
-import org.apache.commons.lang.exception.ExceptionUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.server.blockmanagement.SequentialBlockIdGenerator;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaOutputStreams;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetFactory;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.util.DataChecksum;
-import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -53,6 +52,16 @@ public class TestSimulatedFSDataset {
   static final int BLOCK_LENGTH_MULTIPLIER = 79;
   static final long FIRST_BLK_ID = 1;
 
+  private final int storageCount;
+
+  public TestSimulatedFSDataset() {
+    this(1);
+  }
+
+  protected TestSimulatedFSDataset(int storageCount) {
+    this.storageCount = storageCount;
+  }
+
   @Before
   public void setUp() throws Exception {
     conf = new HdfsConfiguration();
@@ -187,43 +196,28 @@ public class TestSimulatedFSDataset {
 
   @Test
   public void testGetBlockReport() throws IOException {
-    SimulatedFSDataset fsdataset = getSimulatedFSDataset(); 
-    BlockListAsLongs blockReport = fsdataset.getBlockReport(bpid);
-    assertEquals(0, blockReport.getNumberOfBlocks());
+    final SimulatedFSDataset fsdataset = getSimulatedFSDataset();
+    assertBlockReportCountAndSize(fsdataset, 0);
     addSomeBlocks(fsdataset);
-    blockReport = fsdataset.getBlockReport(bpid);
-    assertEquals(NUMBLOCKS, blockReport.getNumberOfBlocks());
-    for (Block b: blockReport) {
-      assertNotNull(b);
-      assertEquals(blockIdToLen(b.getBlockId()), b.getNumBytes());
-    }
+    assertBlockReportCountAndSize(fsdataset, NUMBLOCKS);
+    assertBlockLengthInBlockReports(fsdataset);
   }
   
   @Test
   public void testInjectionEmpty() throws IOException {
     SimulatedFSDataset fsdataset = getSimulatedFSDataset(); 
-    BlockListAsLongs blockReport = fsdataset.getBlockReport(bpid);
-    assertEquals(0, blockReport.getNumberOfBlocks());
+    assertBlockReportCountAndSize(fsdataset, 0);
     int bytesAdded = addSomeBlocks(fsdataset);
-    blockReport = fsdataset.getBlockReport(bpid);
-    assertEquals(NUMBLOCKS, blockReport.getNumberOfBlocks());
-    for (Block b: blockReport) {
-      assertNotNull(b);
-      assertEquals(blockIdToLen(b.getBlockId()), b.getNumBytes());
-    }
+    assertBlockReportCountAndSize(fsdataset, NUMBLOCKS);
+    assertBlockLengthInBlockReports(fsdataset);
     
     // Inject blocks into an empty fsdataset
     //  - injecting the blocks we got above.
     SimulatedFSDataset sfsdataset = getSimulatedFSDataset();
-    sfsdataset.injectBlocks(bpid, blockReport);
-    blockReport = sfsdataset.getBlockReport(bpid);
-    assertEquals(NUMBLOCKS, blockReport.getNumberOfBlocks());
-    for (Block b: blockReport) {
-      assertNotNull(b);
-      assertEquals(blockIdToLen(b.getBlockId()), b.getNumBytes());
-      assertEquals(blockIdToLen(b.getBlockId()), sfsdataset
-          .getLength(new ExtendedBlock(bpid, b)));
-    }
+    injectBlocksFromBlockReport(fsdataset, sfsdataset);
+    assertBlockReportCountAndSize(fsdataset, NUMBLOCKS);
+    assertBlockLengthInBlockReports(fsdataset, sfsdataset);
+
     assertEquals(bytesAdded, sfsdataset.getDfsUsed());
     assertEquals(sfsdataset.getCapacity()-bytesAdded, sfsdataset.getRemaining());
   }
@@ -231,16 +225,10 @@ public class TestSimulatedFSDataset {
   @Test
   public void testInjectionNonEmpty() throws IOException {
     SimulatedFSDataset fsdataset = getSimulatedFSDataset(); 
-    BlockListAsLongs blockReport = fsdataset.getBlockReport(bpid);
-    assertEquals(0, blockReport.getNumberOfBlocks());
+    assertBlockReportCountAndSize(fsdataset, 0);
     int bytesAdded = addSomeBlocks(fsdataset);
-    blockReport = fsdataset.getBlockReport(bpid);
-    assertEquals(NUMBLOCKS, blockReport.getNumberOfBlocks());
-    for (Block b: blockReport) {
-      assertNotNull(b);
-      assertEquals(blockIdToLen(b.getBlockId()), b.getNumBytes());
-    }
-    fsdataset = null;
+    assertBlockReportCountAndSize(fsdataset, NUMBLOCKS);
+    assertBlockLengthInBlockReports(fsdataset);
     
     // Inject blocks into an non-empty fsdataset
     //  - injecting the blocks we got above.
@@ -248,19 +236,10 @@ public class TestSimulatedFSDataset {
     // Add come blocks whose block ids do not conflict with
     // the ones we are going to inject.
     bytesAdded += addSomeBlocks(sfsdataset, NUMBLOCKS+1, false);
-    sfsdataset.getBlockReport(bpid);
-    assertEquals(NUMBLOCKS, blockReport.getNumberOfBlocks());
-    sfsdataset.getBlockReport(bpid);
-    assertEquals(NUMBLOCKS, blockReport.getNumberOfBlocks());
-    sfsdataset.injectBlocks(bpid, blockReport);
-    blockReport = sfsdataset.getBlockReport(bpid);
-    assertEquals(NUMBLOCKS*2, blockReport.getNumberOfBlocks());
-    for (Block b: blockReport) {
-      assertNotNull(b);
-      assertEquals(blockIdToLen(b.getBlockId()), b.getNumBytes());
-      assertEquals(blockIdToLen(b.getBlockId()), sfsdataset
-          .getLength(new ExtendedBlock(bpid, b)));
-    }
+    assertBlockReportCountAndSize(sfsdataset, NUMBLOCKS);
+    injectBlocksFromBlockReport(fsdataset, sfsdataset);
+    assertBlockReportCountAndSize(sfsdataset, NUMBLOCKS * 2);
+    assertBlockLengthInBlockReports(fsdataset, sfsdataset);
     assertEquals(bytesAdded, sfsdataset.getDfsUsed());
     assertEquals(sfsdataset.getCapacity()-bytesAdded,  sfsdataset.getRemaining());
     
@@ -270,7 +249,7 @@ public class TestSimulatedFSDataset {
     try {
       sfsdataset = getSimulatedFSDataset();
       sfsdataset.addBlockPool(bpid, conf);
-      sfsdataset.injectBlocks(bpid, blockReport);
+      injectBlocksFromBlockReport(fsdataset, sfsdataset);
       assertTrue("Expected an IO exception", false);
     } catch (IOException e) {
       // ok - as expected
@@ -334,8 +313,68 @@ public class TestSimulatedFSDataset {
       assertTrue(fsdataset.isValidBlock(new ExtendedBlock(bpid, b)));
     }
   }
-  
-  private SimulatedFSDataset getSimulatedFSDataset() {
+
+  /**
+   * Inject all of the blocks returned from sourceFSDataset's block reports
+   * into destinationFSDataset.
+   */
+  private void injectBlocksFromBlockReport(SimulatedFSDataset sourceFSDataset,
+      SimulatedFSDataset destinationFSDataset) throws IOException {
+    for (Map.Entry<DatanodeStorage, BlockListAsLongs> ent :
+        sourceFSDataset.getBlockReports(bpid).entrySet()) {
+      destinationFSDataset.injectBlocks(bpid, ent.getValue());
+    }
+  }
+
+  /**
+   * Assert that the number of block reports returned from fsdataset matches
+   * {@code storageCount}, and that the total number of blocks is equal to
+   * expectedBlockCount.
+   */
+  private void assertBlockReportCountAndSize(SimulatedFSDataset fsdataset,
+      int expectedBlockCount) {
+    Map<DatanodeStorage, BlockListAsLongs> blockReportMap =
+        fsdataset.getBlockReports(bpid);
+    assertEquals(storageCount, blockReportMap.size());
+    int totalCount = 0;
+    for (Map.Entry<DatanodeStorage, BlockListAsLongs> ent :
+        blockReportMap.entrySet()) {
+      totalCount += ent.getValue().getNumberOfBlocks();
+    }
+    assertEquals(expectedBlockCount, totalCount);
+  }
+
+  /**
+   * Convenience method to call {@link #assertBlockLengthInBlockReports(
+   * SimulatedFSDataset,SimulatedFSDataset)} with a null second parameter.
+   */
+  private void assertBlockLengthInBlockReports(SimulatedFSDataset fsdataset)
+      throws IOException {
+    assertBlockLengthInBlockReports(fsdataset, null);
+  }
+
+  /**
+   * Assert that, for all of the blocks in the block report(s) returned from
+   * fsdataset, they are not null and their length matches the expectation.
+   * If otherFSDataset is non-null, additionally confirm that its idea of the
+   * length of the block matches as well.
+   */
+  private void assertBlockLengthInBlockReports(SimulatedFSDataset fsdataset,
+      SimulatedFSDataset otherFSDataset) throws IOException {
+    for (Map.Entry<DatanodeStorage, BlockListAsLongs> ent :
+        fsdataset.getBlockReports(bpid).entrySet()) {
+      for (Block b : ent.getValue()) {
+        assertNotNull(b);
+        assertEquals(blockIdToLen(b.getBlockId()), b.getNumBytes());
+        if (otherFSDataset != null) {
+          assertEquals(blockIdToLen(b.getBlockId()), otherFSDataset
+              .getLength(new ExtendedBlock(bpid, b)));
+        }
+      }
+    }
+  }
+
+  protected SimulatedFSDataset getSimulatedFSDataset() {
     SimulatedFSDataset fsdataset = new SimulatedFSDataset(null, conf);
     fsdataset.addBlockPool(bpid, conf);
     return fsdataset;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94576b17/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDatasetWithMultipleStorages.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDatasetWithMultipleStorages.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDatasetWithMultipleStorages.java
new file mode 100644
index 0000000..b31ae98
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDatasetWithMultipleStorages.java
@@ -0,0 +1,50 @@
+/**
+ * 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.datanode;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
+import static org.junit.Assert.assertEquals;
+
+
+/**
+ * Test that the {@link SimulatedFSDataset} works correctly when configured
+ * with multiple storages.
+ */
+public class TestSimulatedFSDatasetWithMultipleStorages
+    extends TestSimulatedFSDataset {
+
+  public TestSimulatedFSDatasetWithMultipleStorages() {
+    super(2);
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    super.setUp();
+    conf.set(DFS_DATANODE_DATA_DIR_KEY, "data1,data2");
+  }
+
+  @Test
+  public void testMultipleStoragesConfigured() {
+    SimulatedFSDataset fsDataset = getSimulatedFSDataset();
+    assertEquals(2, fsDataset.getStorageReports(bpid).length);
+  }
+
+}


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


[33/50] [abbrv] hadoop git commit: YARN-7662. [ATSv2] Define new set of configurations for reader and collectors to bind (Rohith Sharma K S via Varun Saxena)

Posted by na...@apache.org.
YARN-7662. [ATSv2] Define new set of configurations for reader and collectors to bind (Rohith Sharma K S via Varun Saxena)


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

Branch: refs/heads/yarn-3409
Commit: c0aeb666a4d43aac196569d9ec6768d62139d2b9
Parents: fe5b057
Author: Varun Saxena <va...@apache.org>
Authored: Tue Dec 19 22:29:24 2017 +0530
Committer: Varun Saxena <va...@apache.org>
Committed: Tue Dec 19 22:29:24 2017 +0530

----------------------------------------------------------------------
 .../hadoop/yarn/conf/YarnConfiguration.java     | 45 +++++++++++++++++++-
 .../hadoop/yarn/webapp/util/WebAppUtils.java    | 39 ++++++++++++++---
 .../src/main/resources/yarn-default.xml         | 24 +++++++++++
 .../security/TestTimelineAuthFilterForV2.java   |  3 +-
 .../AbstractTimelineReaderHBaseTestBase.java    |  2 +-
 .../collector/NodeTimelineCollectorManager.java | 23 +++++++---
 .../reader/TimelineReaderServer.java            | 25 ++++++++---
 .../reader/TestTimelineReaderServer.java        |  6 +--
 .../reader/TestTimelineReaderWebServices.java   |  2 +-
 .../src/site/markdown/TimelineServiceV2.md      |  6 +--
 10 files changed, 145 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c0aeb666/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
index e57f988..1b6bd0e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
@@ -2389,6 +2389,9 @@ public class YarnConfiguration extends Configuration {
   /**
    * Settings for timeline service v2.0.
    */
+  public static final String TIMELINE_SERVICE_READER_PREFIX =
+      TIMELINE_SERVICE_PREFIX + "reader.";
+
   public static final String TIMELINE_SERVICE_WRITER_CLASS =
       TIMELINE_SERVICE_PREFIX + "writer.class";
 
@@ -2397,7 +2400,7 @@ public class YarnConfiguration extends Configuration {
           + ".storage.HBaseTimelineWriterImpl";
 
   public static final String TIMELINE_SERVICE_READER_CLASS =
-      TIMELINE_SERVICE_PREFIX + "reader.class";
+      TIMELINE_SERVICE_READER_PREFIX + "class";
 
   public static final String DEFAULT_TIMELINE_SERVICE_READER_CLASS =
       "org.apache.hadoop.yarn.server.timelineservice.storage" +
@@ -3422,6 +3425,46 @@ public class YarnConfiguration extends Configuration {
   public static final String TIMELINE_XFS_OPTIONS =
       TIMELINE_XFS_PREFIX + "xframe-options";
 
+  /**
+   * Settings for timeline reader.
+   */
+  public static final String TIMELINE_SERVICE_READER_BIND_HOST =
+      TIMELINE_SERVICE_READER_PREFIX + "bind-host";
+
+  public static final String TIMELINE_SERVICE_READER_WEBAPP_ADDRESS =
+      TIMELINE_SERVICE_READER_PREFIX + "webapp.address";
+  public static final String DEFAULT_TIMELINE_SERVICE_READER_WEBAPP_ADDRESS =
+      DEFAULT_TIMELINE_SERVICE_WEBAPP_ADDRESS;
+
+  public static final String TIMELINE_SERVICE_READER_WEBAPP_HTTPS_ADDRESS =
+      TIMELINE_SERVICE_READER_PREFIX + "webapp.https.address";
+  public static final String
+      DEFAULT_TIMELINE_SERVICE_READER_WEBAPP_HTTPS_ADDRESS =
+      DEFAULT_TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS;
+
+  /**
+   * Marked collector properties as Private since it run as auxillary service.
+   */
+  public static final String TIMELINE_SERVICE_COLLECTOR_PREFIX =
+      TIMELINE_SERVICE_PREFIX + "collector.";
+
+  @Private
+  public static final String TIMELINE_SERVICE_COLLECTOR_BIND_HOST =
+      TIMELINE_SERVICE_COLLECTOR_PREFIX + "bind-host";
+
+  @Private
+  public static final String TIMELINE_SERVICE_COLLECTOR_WEBAPP_ADDRESS =
+      TIMELINE_SERVICE_COLLECTOR_PREFIX + "webapp.address";
+  public static final String DEFAULT_TIMELINE_SERVICE_COLLECTOR_WEBAPP_ADDRESS =
+      DEFAULT_TIMELINE_SERVICE_WEBAPP_ADDRESS;
+
+  @Private
+  public static final String TIMELINE_SERVICE_COLLECTOR_WEBAPP_HTTPS_ADDRESS =
+      TIMELINE_SERVICE_COLLECTOR_PREFIX + "webapp.https.address";
+  public static final String
+      DEFAULT_TIMELINE_SERVICE_COLLECTOR_WEBAPP_HTTPS_ADDRESS =
+      DEFAULT_TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS;
+
   public YarnConfiguration() {
     super();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c0aeb666/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/util/WebAppUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/util/WebAppUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/util/WebAppUtils.java
index 446f0a1..e62bf10 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/util/WebAppUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/util/WebAppUtils.java
@@ -314,16 +314,41 @@ public class WebAppUtils {
   }
 
   public static String getAHSWebAppURLWithoutScheme(Configuration conf) {
-    return getTimelineReaderWebAppURL(conf);
-  }
-
-  public static String getTimelineReaderWebAppURL(Configuration conf) {
     if (YarnConfiguration.useHttps(conf)) {
       return conf.get(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS,
-        YarnConfiguration.DEFAULT_TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS);
+          YarnConfiguration.DEFAULT_TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS);
     } else {
       return conf.get(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS,
-        YarnConfiguration.DEFAULT_TIMELINE_SERVICE_WEBAPP_ADDRESS);
+          YarnConfiguration.DEFAULT_TIMELINE_SERVICE_WEBAPP_ADDRESS);
+    }
+  }
+
+  public static String getTimelineReaderWebAppURLWithoutScheme(
+      Configuration conf) {
+    if (YarnConfiguration.useHttps(conf)) {
+      return conf
+          .get(YarnConfiguration.TIMELINE_SERVICE_READER_WEBAPP_HTTPS_ADDRESS,
+              YarnConfiguration.
+                  DEFAULT_TIMELINE_SERVICE_READER_WEBAPP_HTTPS_ADDRESS);
+    } else {
+      return conf.get(YarnConfiguration.TIMELINE_SERVICE_READER_WEBAPP_ADDRESS,
+          YarnConfiguration.
+              DEFAULT_TIMELINE_SERVICE_READER_WEBAPP_ADDRESS);
+    }
+  }
+
+  public static String getTimelineCollectorWebAppURLWithoutScheme(
+      Configuration conf) {
+    if (YarnConfiguration.useHttps(conf)) {
+      return conf.get(
+          YarnConfiguration.TIMELINE_SERVICE_COLLECTOR_WEBAPP_HTTPS_ADDRESS,
+          YarnConfiguration.
+              DEFAULT_TIMELINE_SERVICE_COLLECTOR_WEBAPP_HTTPS_ADDRESS);
+    } else {
+      return conf
+          .get(YarnConfiguration.TIMELINE_SERVICE_COLLECTOR_WEBAPP_ADDRESS,
+              YarnConfiguration.
+                  DEFAULT_TIMELINE_SERVICE_COLLECTOR_WEBAPP_ADDRESS);
     }
   }
 
@@ -342,7 +367,7 @@ public class WebAppUtils {
       return schemePrefix + url;
     }
   }
-  
+
   public static String getRunningLogURL(
       String nodeHttpAddress, String containerId, String user) {
     if (nodeHttpAddress == null || nodeHttpAddress.isEmpty() ||

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c0aeb666/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
index 192f62e..d450eca 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
@@ -3599,4 +3599,28 @@
     <value>0,1</value>
   </property>
 
+  <property>
+    <description>The http address of the timeline reader web application.</description>
+    <name>yarn.timeline-service.reader.webapp.address</name>
+    <value>${yarn.timeline-service.webapp.address}</value>
+  </property>
+
+  <property>
+    <description>The https address of the timeline reader web application.</description>
+    <name>yarn.timeline-service.reader.webapp.https.address</name>
+    <value>${yarn.timeline-service.webapp.https.address}</value>
+  </property>
+
+  <property>
+    <description>
+      The actual address timeline reader will bind to. If this optional address is
+      set, the reader server will bind to this address and the port specified in
+      yarn.timeline-service.reader.webapp.address.
+      This is most useful for making the service listen to all interfaces by setting to
+      0.0.0.0.
+    </description>
+    <name>yarn.timeline-service.reader.bind-host</name>
+    <value></value>
+  </property>
+
 </configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c0aeb666/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/security/TestTimelineAuthFilterForV2.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/security/TestTimelineAuthFilterForV2.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/security/TestTimelineAuthFilterForV2.java
index 75f17fb..bb511d8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/security/TestTimelineAuthFilterForV2.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/security/TestTimelineAuthFilterForV2.java
@@ -162,7 +162,8 @@ public class TestTimelineAuthFilterForV2 {
       conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f);
       conf.setClass(YarnConfiguration.TIMELINE_SERVICE_WRITER_CLASS,
           FileSystemTimelineWriterImpl.class, TimelineWriter.class);
-      conf.set(YarnConfiguration.TIMELINE_SERVICE_BIND_HOST, "localhost");
+      conf.set(YarnConfiguration.TIMELINE_SERVICE_READER_BIND_HOST,
+          "localhost");
       conf.set(FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_DIR_ROOT,
           TEST_ROOT_DIR.getAbsolutePath());
       conf.set("hadoop.proxyuser.HTTP.hosts", "*");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c0aeb666/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/AbstractTimelineReaderHBaseTestBase.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/AbstractTimelineReaderHBaseTestBase.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/AbstractTimelineReaderHBaseTestBase.java
index 3519c3f..471fb6c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/AbstractTimelineReaderHBaseTestBase.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/AbstractTimelineReaderHBaseTestBase.java
@@ -79,7 +79,7 @@ public abstract class AbstractTimelineReaderHBaseTestBase {
       Configuration config = util.getConfiguration();
       config.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
       config.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f);
-      config.set(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS,
+      config.set(YarnConfiguration.TIMELINE_SERVICE_READER_WEBAPP_ADDRESS,
           "localhost:0");
       config.set(YarnConfiguration.RM_CLUSTER_ID, "cluster1");
       config.set(YarnConfiguration.TIMELINE_SERVICE_READER_CLASS,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c0aeb666/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/NodeTimelineCollectorManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/NodeTimelineCollectorManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/NodeTimelineCollectorManager.java
index 68a68f0..696f4a3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/NodeTimelineCollectorManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/NodeTimelineCollectorManager.java
@@ -145,10 +145,9 @@ public class NodeTimelineCollectorManager extends TimelineCollectorManager {
 
   private void doSecureLogin() throws IOException {
     Configuration conf = getConfig();
-    InetSocketAddress addr = NetUtils.createSocketAddr(conf.getTrimmed(
-        YarnConfiguration.TIMELINE_SERVICE_BIND_HOST,
-            YarnConfiguration.DEFAULT_TIMELINE_SERVICE_BIND_HOST), 0,
-                YarnConfiguration.TIMELINE_SERVICE_BIND_HOST);
+    String webAppURLWithoutScheme =
+        WebAppUtils.getTimelineCollectorWebAppURLWithoutScheme(conf);
+    InetSocketAddress addr = NetUtils.createSocketAddr(webAppURLWithoutScheme);
     SecurityUtil.login(conf, YarnConfiguration.TIMELINE_SERVICE_KEYTAB,
         YarnConfiguration.TIMELINE_SERVICE_PRINCIPAL, addr.getHostName());
   }
@@ -277,8 +276,20 @@ public class NodeTimelineCollectorManager extends TimelineCollectorManager {
         initializers, defaultInitializers, tokenMgrService);
     TimelineServerUtils.setTimelineFilters(
         conf, initializers, defaultInitializers);
-    String bindAddress = conf.get(YarnConfiguration.TIMELINE_SERVICE_BIND_HOST,
-        YarnConfiguration.DEFAULT_TIMELINE_SERVICE_BIND_HOST) + ":0";
+
+    String bindAddress = null;
+    String host =
+        conf.getTrimmed(YarnConfiguration.TIMELINE_SERVICE_COLLECTOR_BIND_HOST);
+    if (host == null || host.isEmpty()) {
+      // if collector bind-host is not set, fall back to
+      // timeline-service.bind-host to maintain compatibility
+      bindAddress =
+          conf.get(YarnConfiguration.DEFAULT_TIMELINE_SERVICE_BIND_HOST,
+              YarnConfiguration.DEFAULT_TIMELINE_SERVICE_BIND_HOST) + ":0";
+    } else {
+      bindAddress = host + ":0";
+    }
+
     try {
       HttpServer2.Builder builder = new HttpServer2.Builder()
           .setName("timeline")

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c0aeb666/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderServer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderServer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderServer.java
index 5c049ea..3cc24ea 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderServer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderServer.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.http.HttpServer2;
+import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.HttpCrossOriginFilterInitializer;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.service.CompositeService;
@@ -63,6 +64,8 @@ public class TimelineReaderServer extends CompositeService {
 
   private HttpServer2 readerWebServer;
   private TimelineReaderManager timelineReaderManager;
+  private String webAppURLWithoutScheme;
+
 
   public TimelineReaderServer() {
     super(TimelineReaderServer.class.getName());
@@ -73,10 +76,10 @@ public class TimelineReaderServer extends CompositeService {
     if (!YarnConfiguration.timelineServiceV2Enabled(conf)) {
       throw new YarnException("timeline service v.2 is not enabled");
     }
-    InetSocketAddress bindAddr = conf.getSocketAddr(
-        YarnConfiguration.TIMELINE_SERVICE_ADDRESS,
-            YarnConfiguration.DEFAULT_TIMELINE_SERVICE_ADDRESS,
-                YarnConfiguration.DEFAULT_TIMELINE_SERVICE_PORT);
+    webAppURLWithoutScheme =
+        WebAppUtils.getTimelineReaderWebAppURLWithoutScheme(conf);
+    InetSocketAddress bindAddr =
+        NetUtils.createSocketAddr(webAppURLWithoutScheme);
     // Login from keytab if security is enabled.
     try {
       SecurityUtil.login(conf, YarnConfiguration.TIMELINE_SERVICE_KEYTAB,
@@ -170,9 +173,17 @@ public class TimelineReaderServer extends CompositeService {
   private void startTimelineReaderWebApp() {
     Configuration conf = getConfig();
     addFilters(conf);
-    String bindAddress = WebAppUtils.getWebAppBindURL(conf,
-        YarnConfiguration.TIMELINE_SERVICE_BIND_HOST,
-        WebAppUtils.getTimelineReaderWebAppURL(conf));
+
+    String hostProperty = YarnConfiguration.TIMELINE_SERVICE_READER_BIND_HOST;
+    String host = conf.getTrimmed(hostProperty);
+    if (host == null || host.isEmpty()) {
+      // if reader bind-host is not set, fall back to timeline-service.bind-host
+      // to maintain compatibility
+      hostProperty = YarnConfiguration.TIMELINE_SERVICE_BIND_HOST;
+    }
+    String bindAddress = WebAppUtils
+        .getWebAppBindURL(conf, hostProperty, webAppURLWithoutScheme);
+
     LOG.info("Instantiating TimelineReaderWebApp at " + bindAddress);
     try {
       HttpServer2.Builder builder = new HttpServer2.Builder()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c0aeb666/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderServer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderServer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderServer.java
index bb96f37..6fc46cc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderServer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderServer.java
@@ -37,7 +37,7 @@ public class TestTimelineReaderServer {
     Configuration config = new YarnConfiguration();
     config.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
     config.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f);
-    config.set(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS,
+    config.set(YarnConfiguration.TIMELINE_SERVICE_READER_WEBAPP_ADDRESS,
         "localhost:0");
     config.setClass(YarnConfiguration.TIMELINE_SERVICE_READER_CLASS,
         FileSystemTimelineReaderImpl.class, TimelineReader.class);
@@ -61,7 +61,7 @@ public class TestTimelineReaderServer {
     Configuration conf = new YarnConfiguration();
     conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
     conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f);
-    conf.set(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS,
+    conf.set(YarnConfiguration.TIMELINE_SERVICE_READER_WEBAPP_ADDRESS,
         "localhost:0");
     conf.set(YarnConfiguration.TIMELINE_SERVICE_READER_CLASS,
         Object.class.getName());
@@ -75,7 +75,7 @@ public class TestTimelineReaderServer {
     Configuration conf = new YarnConfiguration();
     conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
     conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f);
-    conf.set(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS,
+    conf.set(YarnConfiguration.TIMELINE_SERVICE_READER_WEBAPP_ADDRESS,
         "localhost:0");
     conf.set(YarnConfiguration.TIMELINE_SERVICE_READER_CLASS,
         nonexistentTimelineReaderClass);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c0aeb666/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServices.java
index f760834..03939ad 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServices.java
@@ -84,7 +84,7 @@ public class TestTimelineReaderWebServices {
       Configuration config = new YarnConfiguration();
       config.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
       config.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f);
-      config.set(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS,
+      config.set(YarnConfiguration.TIMELINE_SERVICE_READER_WEBAPP_ADDRESS,
           "localhost:0");
       config.set(YarnConfiguration.RM_CLUSTER_ID, "cluster1");
       config.setClass(YarnConfiguration.TIMELINE_SERVICE_READER_CLASS,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c0aeb666/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md
index 7c51ce0..022f76d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md
@@ -133,9 +133,9 @@ New configuration parameters that are introduced with v.2 are marked bold.
 | Configuration Property | Description |
 |:---- |:---- |
 | `yarn.timeline-service.hostname` | The hostname of the Timeline service web application. Defaults to `0.0.0.0` |
-| `yarn.timeline-service.address` | Address for the Timeline server to start the RPC server. Defaults to `${yarn.timeline-service.hostname}:10200`. |
-| `yarn.timeline-service.webapp.address` | The http address of the Timeline service web application. Defaults to `${yarn.timeline-service.hostname}:8188`. |
-| `yarn.timeline-service.webapp.https.address` | The https address of the Timeline service web application. Defaults to `${yarn.timeline-service.hostname}:8190`. |
+| `yarn.timeline-service.reader.webapp.address` | The http address of the Timeline Reader web application. Defaults to `${yarn.timeline-service.hostname}:8188`. |
+| `yarn.timeline-service.reader.webapp.https.address` | The https address of the Timeline Reader web application. Defaults to `${yarn.timeline-service.hostname}:8190`. |
+| `yarn.timeline-service.reader.bind-host` | The actual address the timeline reader will bind to. If this optional address is set, reader server will bind to this address and the port specified in yarn.timeline-service.reader.webapp.address. This is most useful for making the service listen on all interfaces by setting to 0.0.0.0. |
 | **`yarn.timeline-service.hbase.configuration.file`** | Optional URL to an hbase-site.xml configuration file to be used to connect to the timeline-service hbase cluster. If empty or not specified, then the HBase configuration will be loaded from the classpath. When specified the values in the specified configuration file will override those from the ones that are present on the classpath. Defaults to `null`. |
 | **`yarn.timeline-service.writer.flush-interval-seconds`** | The setting that controls how often the timeline collector flushes the timeline writer. Defaults to `60`. |
 | **`yarn.timeline-service.app-collector.linger-period.ms`** | Time period till which the application collector will be alive in NM, after the  application master container finishes. Defaults to `1000` (1 second). |


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


[29/50] [abbrv] hadoop git commit: Add 2.8.3 release jdiff files.

Posted by na...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/a7f8caf5/hadoop-common-project/hadoop-common/dev-support/jdiff/Apache_Hadoop_Common_2.8.3.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/dev-support/jdiff/Apache_Hadoop_Common_2.8.3.xml b/hadoop-common-project/hadoop-common/dev-support/jdiff/Apache_Hadoop_Common_2.8.3.xml
new file mode 100644
index 0000000..bd7e69c
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/dev-support/jdiff/Apache_Hadoop_Common_2.8.3.xml
@@ -0,0 +1,38433 @@
+<?xml version="1.0" encoding="iso-8859-1" standalone="no"?>
+<!-- Generated by the JDiff Javadoc doclet -->
+<!-- (http://www.jdiff.org) -->
+<!-- on Tue Dec 05 05:00:41 UTC 2017 -->
+
+<api
+  xmlns:xsi='http://www.w3.org/2001/XMLSchema-instance'
+  xsi:noNamespaceSchemaLocation='api.xsd'
+  name="Apache Hadoop Common 2.8.3"
+  jdversion="1.0.9">
+
+<!--  Command line arguments =  -doclet org.apache.hadoop.classification.tools.IncludePublicAnnotationsJDiffDoclet -docletpath /build/source/hadoop-common-project/hadoop-common/target/hadoop-annotations.jar:/build/source/hadoop-common-project/hadoop-common/target/jdiff.jar -verbose -classpath /build/source/hadoop-common-project/hadoop-common/target/classes:/build/source/hadoop-common-project/hadoop-annotations/target/hadoop-annotations-2.8.3.jar:/usr/lib/jvm/java-7-openjdk-amd64/lib/tools.jar:/maven/com/google/guava/guava/11.0.2/guava-11.0.2.jar:/maven/commons-cli/commons-cli/1.2/commons-cli-1.2.jar:/maven/org/apache/commons/commons-math3/3.1.1/commons-math3-3.1.1.jar:/maven/xmlenc/xmlenc/0.52/xmlenc-0.52.jar:/maven/org/apache/httpcomponents/httpclient/4.5.2/httpclient-4.5.2.jar:/maven/org/apache/httpcomponents/httpcore/4.4.4/httpcore-4.4.4.jar:/maven/commons-codec/commons-codec/1.4/commons-codec-1.4.jar:/maven/commons-io/commons-io/2.4/commons-io-2.4.jar:/maven/commons-net/commons-
 net/3.1/commons-net-3.1.jar:/maven/commons-collections/commons-collections/3.2.2/commons-collections-3.2.2.jar:/maven/javax/servlet/servlet-api/2.5/servlet-api-2.5.jar:/maven/org/mortbay/jetty/jetty/6.1.26/jetty-6.1.26.jar:/maven/org/mortbay/jetty/jetty-util/6.1.26/jetty-util-6.1.26.jar:/maven/org/mortbay/jetty/jetty-sslengine/6.1.26/jetty-sslengine-6.1.26.jar:/maven/com/sun/jersey/jersey-core/1.9/jersey-core-1.9.jar:/maven/com/sun/jersey/jersey-json/1.9/jersey-json-1.9.jar:/maven/org/codehaus/jettison/jettison/1.1/jettison-1.1.jar:/maven/com/sun/xml/bind/jaxb-impl/2.2.3-1/jaxb-impl-2.2.3-1.jar:/maven/javax/xml/bind/jaxb-api/2.2.2/jaxb-api-2.2.2.jar:/maven/javax/xml/stream/stax-api/1.0-2/stax-api-1.0-2.jar:/maven/javax/activation/activation/1.1/activation-1.1.jar:/maven/org/codehaus/jackson/jackson-jaxrs/1.9.13/jackson-jaxrs-1.9.13.jar:/maven/org/codehaus/jackson/jackson-xc/1.9.13/jackson-xc-1.9.13.jar:/maven/com/sun/jersey/jersey-server/1.9/jersey-server-1.9.jar:/maven/asm/asm/3.2/
 asm-3.2.jar:/maven/commons-logging/commons-logging/1.1.3/commons-logging-1.1.3.jar:/maven/log4j/log4j/1.2.17/log4j-1.2.17.jar:/maven/net/java/dev/jets3t/jets3t/0.9.0/jets3t-0.9.0.jar:/maven/com/jamesmurty/utils/java-xmlbuilder/0.4/java-xmlbuilder-0.4.jar:/maven/commons-lang/commons-lang/2.6/commons-lang-2.6.jar:/maven/commons-configuration/commons-configuration/1.6/commons-configuration-1.6.jar:/maven/commons-digester/commons-digester/1.8/commons-digester-1.8.jar:/maven/commons-beanutils/commons-beanutils/1.7.0/commons-beanutils-1.7.0.jar:/maven/commons-beanutils/commons-beanutils-core/1.8.0/commons-beanutils-core-1.8.0.jar:/maven/org/slf4j/slf4j-api/1.7.10/slf4j-api-1.7.10.jar:/maven/org/codehaus/jackson/jackson-core-asl/1.9.13/jackson-core-asl-1.9.13.jar:/maven/org/codehaus/jackson/jackson-mapper-asl/1.9.13/jackson-mapper-asl-1.9.13.jar:/maven/org/apache/avro/avro/1.7.4/avro-1.7.4.jar:/maven/com/thoughtworks/paranamer/paranamer/2.3/paranamer-2.3.jar:/maven/org/xerial/snappy/snappy
 -java/1.0.4.1/snappy-java-1.0.4.1.jar:/maven/org/apache/ant/ant/1.8.1/ant-1.8.1.jar:/maven/org/apache/ant/ant-launcher/1.8.1/ant-launcher-1.8.1.jar:/maven/com/google/protobuf/protobuf-java/2.5.0/protobuf-java-2.5.0.jar:/maven/com/google/code/gson/gson/2.2.4/gson-2.2.4.jar:/build/source/hadoop-common-project/hadoop-auth/target/hadoop-auth-2.8.3.jar:/maven/com/nimbusds/nimbus-jose-jwt/3.9/nimbus-jose-jwt-3.9.jar:/maven/net/jcip/jcip-annotations/1.0/jcip-annotations-1.0.jar:/maven/net/minidev/json-smart/1.1.1/json-smart-1.1.1.jar:/maven/org/apache/directory/server/apacheds-kerberos-codec/2.0.0-M15/apacheds-kerberos-codec-2.0.0-M15.jar:/maven/org/apache/directory/server/apacheds-i18n/2.0.0-M15/apacheds-i18n-2.0.0-M15.jar:/maven/org/apache/directory/api/api-asn1-api/1.0.0-M20/api-asn1-api-1.0.0-M20.jar:/maven/org/apache/directory/api/api-util/1.0.0-M20/api-util-1.0.0-M20.jar:/maven/org/apache/curator/curator-framework/2.7.1/curator-framework-2.7.1.jar:/maven/org/apache/directory/api/api-
 i18n/1.0.0-M20/api-i18n-1.0.0-M20.jar:/maven/org/apache/directory/api/api-ldap-model/1.0.0-M20/api-ldap-model-1.0.0-M20.jar:/maven/org/apache/mina/mina-core/2.0.0-M5/mina-core-2.0.0-M5.jar:/maven/net/sf/ehcache/ehcache-core/2.4.4/ehcache-core-2.4.4.jar:/maven/antlr/antlr/2.7.7/antlr-2.7.7.jar:/maven/org/apache/directory/api/api-asn1-ber/1.0.0-M20/api-asn1-ber-1.0.0-M20.jar:/maven/com/jcraft/jsch/0.1.54/jsch-0.1.54.jar:/maven/org/apache/curator/curator-client/2.7.1/curator-client-2.7.1.jar:/maven/org/apache/curator/curator-recipes/2.7.1/curator-recipes-2.7.1.jar:/maven/com/google/code/findbugs/jsr305/3.0.0/jsr305-3.0.0.jar:/maven/org/apache/htrace/htrace-core4/4.0.1-incubating/htrace-core4-4.0.1-incubating.jar:/maven/org/apache/zookeeper/zookeeper/3.4.6/zookeeper-3.4.6.jar:/maven/io/netty/netty/3.6.2.Final/netty-3.6.2.Final.jar:/maven/org/apache/commons/commons-compress/1.4.1/commons-compress-1.4.1.jar:/maven/org/tukaani/xz/1.0/xz-1.0.jar -sourcepath /build/source/hadoop-common-proje
 ct/hadoop-common/src/main/java -apidir /build/source/hadoop-common-project/hadoop-common/target/site/jdiff/xml -apiname Apache Hadoop Common 2.8.3 -->
+<package name="org.apache.hadoop">
+  <!-- start class org.apache.hadoop.HadoopIllegalArgumentException -->
+  <class name="HadoopIllegalArgumentException" extends="java.lang.IllegalArgumentException"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="HadoopIllegalArgumentException" type="java.lang.String"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Constructs exception with the specified detail message. 
+ @param message detailed message.]]>
+      </doc>
+    </constructor>
+    <doc>
+    <![CDATA[Indicates that a method has been passed illegal or invalid argument. This
+ exception is thrown instead of IllegalArgumentException to differentiate the
+ exception thrown in Hadoop implementation from the one thrown in JDK.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.HadoopIllegalArgumentException -->
+</package>
+<package name="org.apache.hadoop.conf">
+  <!-- start interface org.apache.hadoop.conf.Configurable -->
+  <interface name="Configurable"    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <method name="setConf"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <doc>
+      <![CDATA[Set the configuration to be used by this object.]]>
+      </doc>
+    </method>
+    <method name="getConf" return="org.apache.hadoop.conf.Configuration"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Return the configuration used by this object.]]>
+      </doc>
+    </method>
+    <doc>
+    <![CDATA[Something that may be configured with a {@link Configuration}.]]>
+    </doc>
+  </interface>
+  <!-- end interface org.apache.hadoop.conf.Configurable -->
+  <!-- start class org.apache.hadoop.conf.Configuration -->
+  <class name="Configuration" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="java.lang.Iterable"/>
+    <implements name="org.apache.hadoop.io.Writable"/>
+    <constructor name="Configuration"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[A new configuration.]]>
+      </doc>
+    </constructor>
+    <constructor name="Configuration" type="boolean"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[A new configuration where the behavior of reading from the default 
+ resources can be turned off.
+ 
+ If the parameter {@code loadDefaults} is false, the new instance
+ will not load resources from the default files. 
+ @param loadDefaults specifies whether to load from the default files]]>
+      </doc>
+    </constructor>
+    <constructor name="Configuration" type="org.apache.hadoop.conf.Configuration"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[A new configuration with the same settings cloned from another.
+ 
+ @param other the configuration from which to clone settings.]]>
+      </doc>
+    </constructor>
+    <method name="addDeprecations"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="deltas" type="org.apache.hadoop.conf.Configuration.DeprecationDelta[]"/>
+      <doc>
+      <![CDATA[Adds a set of deprecated keys to the global deprecations.
+
+ This method is lockless.  It works by means of creating a new
+ DeprecationContext based on the old one, and then atomically swapping in
+ the new context.  If someone else updated the context in between us reading
+ the old context and swapping in the new one, we try again until we win the
+ race.
+
+ @param deltas   The deprecations to add.]]>
+      </doc>
+    </method>
+    <method name="addDeprecation"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="use {@link #addDeprecation(String key, String newKey,
+      String customMessage)} instead">
+      <param name="key" type="java.lang.String"/>
+      <param name="newKeys" type="java.lang.String[]"/>
+      <param name="customMessage" type="java.lang.String"/>
+      <doc>
+      <![CDATA[Adds the deprecated key to the global deprecation map.
+ It does not override any existing entries in the deprecation map.
+ This is to be used only by the developers in order to add deprecation of
+ keys, and attempts to call this method after loading resources once,
+ would lead to <tt>UnsupportedOperationException</tt>
+ 
+ If a key is deprecated in favor of multiple keys, they are all treated as 
+ aliases of each other, and setting any one of them resets all the others 
+ to the new value.
+
+ If you have multiple deprecation entries to add, it is more efficient to
+ use #addDeprecations(DeprecationDelta[] deltas) instead.
+ 
+ @param key
+ @param newKeys
+ @param customMessage
+ @deprecated use {@link #addDeprecation(String key, String newKey,
+      String customMessage)} instead]]>
+      </doc>
+    </method>
+    <method name="addDeprecation"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="key" type="java.lang.String"/>
+      <param name="newKey" type="java.lang.String"/>
+      <param name="customMessage" type="java.lang.String"/>
+      <doc>
+      <![CDATA[Adds the deprecated key to the global deprecation map.
+ It does not override any existing entries in the deprecation map.
+ This is to be used only by the developers in order to add deprecation of
+ keys, and attempts to call this method after loading resources once,
+ would lead to <tt>UnsupportedOperationException</tt>
+ 
+ If you have multiple deprecation entries to add, it is more efficient to
+ use #addDeprecations(DeprecationDelta[] deltas) instead.
+
+ @param key
+ @param newKey
+ @param customMessage]]>
+      </doc>
+    </method>
+    <method name="addDeprecation"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="use {@link #addDeprecation(String key, String newKey)} instead">
+      <param name="key" type="java.lang.String"/>
+      <param name="newKeys" type="java.lang.String[]"/>
+      <doc>
+      <![CDATA[Adds the deprecated key to the global deprecation map when no custom
+ message is provided.
+ It does not override any existing entries in the deprecation map.
+ This is to be used only by the developers in order to add deprecation of
+ keys, and attempts to call this method after loading resources once,
+ would lead to <tt>UnsupportedOperationException</tt>
+ 
+ If a key is deprecated in favor of multiple keys, they are all treated as 
+ aliases of each other, and setting any one of them resets all the others 
+ to the new value.
+ 
+ If you have multiple deprecation entries to add, it is more efficient to
+ use #addDeprecations(DeprecationDelta[] deltas) instead.
+
+ @param key Key that is to be deprecated
+ @param newKeys list of keys that take up the values of deprecated key
+ @deprecated use {@link #addDeprecation(String key, String newKey)} instead]]>
+      </doc>
+    </method>
+    <method name="addDeprecation"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="key" type="java.lang.String"/>
+      <param name="newKey" type="java.lang.String"/>
+      <doc>
+      <![CDATA[Adds the deprecated key to the global deprecation map when no custom
+ message is provided.
+ It does not override any existing entries in the deprecation map.
+ This is to be used only by the developers in order to add deprecation of
+ keys, and attempts to call this method after loading resources once,
+ would lead to <tt>UnsupportedOperationException</tt>
+ 
+ If you have multiple deprecation entries to add, it is more efficient to
+ use #addDeprecations(DeprecationDelta[] deltas) instead.
+
+ @param key Key that is to be deprecated
+ @param newKey key that takes up the value of deprecated key]]>
+      </doc>
+    </method>
+    <method name="isDeprecated" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="key" type="java.lang.String"/>
+      <doc>
+      <![CDATA[checks whether the given <code>key</code> is deprecated.
+ 
+ @param key the parameter which is to be checked for deprecation
+ @return <code>true</code> if the key is deprecated and 
+         <code>false</code> otherwise.]]>
+      </doc>
+    </method>
+    <method name="setDeprecatedProperties"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Sets all deprecated properties that are not currently set but have a
+ corresponding new property that is set. Useful for iterating the
+ properties when all deprecated properties for currently set properties
+ need to be present.]]>
+      </doc>
+    </method>
+    <method name="reloadExistingConfigurations"
+      abstract="false" native="false" synchronized="true"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Reload existing configuration instances.]]>
+      </doc>
+    </method>
+    <method name="addDefaultResource"
+      abstract="false" native="false" synchronized="true"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <doc>
+      <![CDATA[Add a default resource. Resources are loaded in the order of the resources 
+ added.
+ @param name file name. File should be present in the classpath.]]>
+      </doc>
+    </method>
+    <method name="setRestrictSystemPropertiesDefault"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="val" type="boolean"/>
+    </method>
+    <method name="setRestrictSystemProperties"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="val" type="boolean"/>
+    </method>
+    <method name="addResource"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <doc>
+      <![CDATA[Add a configuration resource. 
+ 
+ The properties of this resource will override properties of previously 
+ added resources, unless they were marked <a href="#Final">final</a>. 
+ 
+ @param name resource to be added, the classpath is examined for a file 
+             with that name.]]>
+      </doc>
+    </method>
+    <method name="addResource"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <param name="restrictedParser" type="boolean"/>
+    </method>
+    <method name="addResource"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="url" type="java.net.URL"/>
+      <doc>
+      <![CDATA[Add a configuration resource. 
+ 
+ The properties of this resource will override properties of previously 
+ added resources, unless they were marked <a href="#Final">final</a>. 
+ 
+ @param url url of the resource to be added, the local filesystem is 
+            examined directly to find the resource, without referring to 
+            the classpath.]]>
+      </doc>
+    </method>
+    <method name="addResource"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="url" type="java.net.URL"/>
+      <param name="restrictedParser" type="boolean"/>
+    </method>
+    <method name="addResource"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="file" type="org.apache.hadoop.fs.Path"/>
+      <doc>
+      <![CDATA[Add a configuration resource. 
+ 
+ The properties of this resource will override properties of previously 
+ added resources, unless they were marked <a href="#Final">final</a>. 
+ 
+ @param file file-path of resource to be added, the local filesystem is
+             examined directly to find the resource, without referring to 
+             the classpath.]]>
+      </doc>
+    </method>
+    <method name="addResource"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="file" type="org.apache.hadoop.fs.Path"/>
+      <param name="restrictedParser" type="boolean"/>
+    </method>
+    <method name="addResource"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="in" type="java.io.InputStream"/>
+      <doc>
+      <![CDATA[Add a configuration resource. 
+ 
+ The properties of this resource will override properties of previously 
+ added resources, unless they were marked <a href="#Final">final</a>. 
+ 
+ WARNING: The contents of the InputStream will be cached, by this method. 
+ So use this sparingly because it does increase the memory consumption.
+ 
+ @param in InputStream to deserialize the object from. In will be read from
+ when a get or set is called next.  After it is read the stream will be
+ closed.]]>
+      </doc>
+    </method>
+    <method name="addResource"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="in" type="java.io.InputStream"/>
+      <param name="restrictedParser" type="boolean"/>
+    </method>
+    <method name="addResource"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="in" type="java.io.InputStream"/>
+      <param name="name" type="java.lang.String"/>
+      <doc>
+      <![CDATA[Add a configuration resource. 
+ 
+ The properties of this resource will override properties of previously 
+ added resources, unless they were marked <a href="#Final">final</a>. 
+ 
+ @param in InputStream to deserialize the object from.
+ @param name the name of the resource because InputStream.toString is not
+ very descriptive some times.]]>
+      </doc>
+    </method>
+    <method name="addResource"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="in" type="java.io.InputStream"/>
+      <param name="name" type="java.lang.String"/>
+      <param name="restrictedParser" type="boolean"/>
+    </method>
+    <method name="addResource"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <doc>
+      <![CDATA[Add a configuration resource.
+
+ The properties of this resource will override properties of previously
+ added resources, unless they were marked <a href="#Final">final</a>.
+
+ @param conf Configuration object from which to load properties]]>
+      </doc>
+    </method>
+    <method name="reloadConfiguration"
+      abstract="false" native="false" synchronized="true"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Reload configuration from previously added resources.
+
+ This method will clear all the configuration read from the added 
+ resources, and final parameters. This will make the resources to 
+ be read again before accessing the values. Values that are added
+ via set methods will overlay values read from the resources.]]>
+      </doc>
+    </method>
+    <method name="get" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <doc>
+      <![CDATA[Get the value of the <code>name</code> property, <code>null</code> if
+ no such property exists. If the key is deprecated, it returns the value of
+ the first key which replaces the deprecated key and is not null.
+ 
+ Values are processed for <a href="#VariableExpansion">variable expansion</a> 
+ before being returned. 
+ 
+ @param name the property name, will be trimmed before get value.
+ @return the value of the <code>name</code> or its replacing property, 
+         or null if no such property exists.]]>
+      </doc>
+    </method>
+    <method name="setAllowNullValueProperties"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="val" type="boolean"/>
+      <doc>
+      <![CDATA[Set Configuration to allow keys without values during setup.  Intended
+ for use during testing.
+
+ @param val If true, will allow Configuration to store keys without values]]>
+      </doc>
+    </method>
+    <method name="setRestrictSystemProps"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="val" type="boolean"/>
+    </method>
+    <method name="onlyKeyExists" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <doc>
+      <![CDATA[Return existence of the <code>name</code> property, but only for
+ names which have no valid value, usually non-existent or commented
+ out in XML.
+
+ @param name the property name
+ @return true if the property <code>name</code> exists without value]]>
+      </doc>
+    </method>
+    <method name="getTrimmed" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <doc>
+      <![CDATA[Get the value of the <code>name</code> property as a trimmed <code>String</code>, 
+ <code>null</code> if no such property exists. 
+ If the key is deprecated, it returns the value of
+ the first key which replaces the deprecated key and is not null
+ 
+ Values are processed for <a href="#VariableExpansion">variable expansion</a> 
+ before being returned. 
+ 
+ @param name the property name.
+ @return the value of the <code>name</code> or its replacing property, 
+         or null if no such property exists.]]>
+      </doc>
+    </method>
+    <method name="getTrimmed" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <param name="defaultValue" type="java.lang.String"/>
+      <doc>
+      <![CDATA[Get the value of the <code>name</code> property as a trimmed <code>String</code>, 
+ <code>defaultValue</code> if no such property exists. 
+ See @{Configuration#getTrimmed} for more details.
+ 
+ @param name          the property name.
+ @param defaultValue  the property default value.
+ @return              the value of the <code>name</code> or defaultValue
+                      if it is not set.]]>
+      </doc>
+    </method>
+    <method name="getRaw" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <doc>
+      <![CDATA[Get the value of the <code>name</code> property, without doing
+ <a href="#VariableExpansion">variable expansion</a>.If the key is 
+ deprecated, it returns the value of the first key which replaces 
+ the deprecated key and is not null.
+ 
+ @param name the property name.
+ @return the value of the <code>name</code> property or 
+         its replacing property and null if no such property exists.]]>
+      </doc>
+    </method>
+    <method name="set"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <param name="value" type="java.lang.String"/>
+      <doc>
+      <![CDATA[Set the <code>value</code> of the <code>name</code> property. If 
+ <code>name</code> is deprecated or there is a deprecated name associated to it,
+ it sets the value to both names. Name will be trimmed before put into
+ configuration.
+ 
+ @param name property name.
+ @param value property value.]]>
+      </doc>
+    </method>
+    <method name="set"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <param name="value" type="java.lang.String"/>
+      <param name="source" type="java.lang.String"/>
+      <doc>
+      <![CDATA[Set the <code>value</code> of the <code>name</code> property. If 
+ <code>name</code> is deprecated, it also sets the <code>value</code> to
+ the keys that replace the deprecated key. Name will be trimmed before put
+ into configuration.
+
+ @param name property name.
+ @param value property value.
+ @param source the place that this configuration value came from 
+ (For debugging).
+ @throws IllegalArgumentException when the value or name is null.]]>
+      </doc>
+    </method>
+    <method name="unset"
+      abstract="false" native="false" synchronized="true"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <doc>
+      <![CDATA[Unset a previously set property.]]>
+      </doc>
+    </method>
+    <method name="setIfUnset"
+      abstract="false" native="false" synchronized="true"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <param name="value" type="java.lang.String"/>
+      <doc>
+      <![CDATA[Sets a property if it is currently unset.
+ @param name the property name
+ @param value the new value]]>
+      </doc>
+    </method>
+    <method name="get" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <param name="defaultValue" type="java.lang.String"/>
+      <doc>
+      <![CDATA[Get the value of the <code>name</code>. If the key is deprecated,
+ it returns the value of the first key which replaces the deprecated key
+ and is not null.
+ If no such property exists,
+ then <code>defaultValue</code> is returned.
+ 
+ @param name property name, will be trimmed before get value.
+ @param defaultValue default value.
+ @return property value, or <code>defaultValue</code> if the property 
+         doesn't exist.]]>
+      </doc>
+    </method>
+    <method name="getInt" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <param name="defaultValue" type="int"/>
+      <doc>
+      <![CDATA[Get the value of the <code>name</code> property as an <code>int</code>.
+   
+ If no such property exists, the provided default value is returned,
+ or if the specified value is not a valid <code>int</code>,
+ then an error is thrown.
+ 
+ @param name property name.
+ @param defaultValue default value.
+ @throws NumberFormatException when the value is invalid
+ @return property value as an <code>int</code>, 
+         or <code>defaultValue</code>.]]>
+      </doc>
+    </method>
+    <method name="getInts" return="int[]"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <doc>
+      <![CDATA[Get the value of the <code>name</code> property as a set of comma-delimited
+ <code>int</code> values.
+ 
+ If no such property exists, an empty array is returned.
+ 
+ @param name property name
+ @return property value interpreted as an array of comma-delimited
+         <code>int</code> values]]>
+      </doc>
+    </method>
+    <method name="setInt"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <param name="value" type="int"/>
+      <doc>
+      <![CDATA[Set the value of the <code>name</code> property to an <code>int</code>.
+ 
+ @param name property name.
+ @param value <code>int</code> value of the property.]]>
+      </doc>
+    </method>
+    <method name="getLong" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <param name="defaultValue" type="long"/>
+      <doc>
+      <![CDATA[Get the value of the <code>name</code> property as a <code>long</code>.  
+ If no such property exists, the provided default value is returned,
+ or if the specified value is not a valid <code>long</code>,
+ then an error is thrown.
+ 
+ @param name property name.
+ @param defaultValue default value.
+ @throws NumberFormatException when the value is invalid
+ @return property value as a <code>long</code>, 
+         or <code>defaultValue</code>.]]>
+      </doc>
+    </method>
+    <method name="getLongBytes" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <param name="defaultValue" type="long"/>
+      <doc>
+      <![CDATA[Get the value of the <code>name</code> property as a <code>long</code> or
+ human readable format. If no such property exists, the provided default
+ value is returned, or if the specified value is not a valid
+ <code>long</code> or human readable format, then an error is thrown. You
+ can use the following suffix (case insensitive): k(kilo), m(mega), g(giga),
+ t(tera), p(peta), e(exa)
+
+ @param name property name.
+ @param defaultValue default value.
+ @throws NumberFormatException when the value is invalid
+ @return property value as a <code>long</code>,
+         or <code>defaultValue</code>.]]>
+      </doc>
+    </method>
+    <method name="setLong"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <param name="value" type="long"/>
+      <doc>
+      <![CDATA[Set the value of the <code>name</code> property to a <code>long</code>.
+ 
+ @param name property name.
+ @param value <code>long</code> value of the property.]]>
+      </doc>
+    </method>
+    <method name="getFloat" return="float"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <param name="defaultValue" type="float"/>
+      <doc>
+      <![CDATA[Get the value of the <code>name</code> property as a <code>float</code>.  
+ If no such property exists, the provided default value is returned,
+ or if the specified value is not a valid <code>float</code>,
+ then an error is thrown.
+
+ @param name property name.
+ @param defaultValue default value.
+ @throws NumberFormatException when the value is invalid
+ @return property value as a <code>float</code>, 
+         or <code>defaultValue</code>.]]>
+      </doc>
+    </method>
+    <method name="setFloat"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <param name="value" type="float"/>
+      <doc>
+      <![CDATA[Set the value of the <code>name</code> property to a <code>float</code>.
+ 
+ @param name property name.
+ @param value property value.]]>
+      </doc>
+    </method>
+    <method name="getDouble" return="double"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <param name="defaultValue" type="double"/>
+      <doc>
+      <![CDATA[Get the value of the <code>name</code> property as a <code>double</code>.  
+ If no such property exists, the provided default value is returned,
+ or if the specified value is not a valid <code>double</code>,
+ then an error is thrown.
+
+ @param name property name.
+ @param defaultValue default value.
+ @throws NumberFormatException when the value is invalid
+ @return property value as a <code>double</code>, 
+         or <code>defaultValue</code>.]]>
+      </doc>
+    </method>
+    <method name="setDouble"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <param name="value" type="double"/>
+      <doc>
+      <![CDATA[Set the value of the <code>name</code> property to a <code>double</code>.
+ 
+ @param name property name.
+ @param value property value.]]>
+      </doc>
+    </method>
+    <method name="getBoolean" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <param name="defaultValue" type="boolean"/>
+      <doc>
+      <![CDATA[Get the value of the <code>name</code> property as a <code>boolean</code>.  
+ If no such property is specified, or if the specified value is not a valid
+ <code>boolean</code>, then <code>defaultValue</code> is returned.
+ 
+ @param name property name.
+ @param defaultValue default value.
+ @return property value as a <code>boolean</code>, 
+         or <code>defaultValue</code>.]]>
+      </doc>
+    </method>
+    <method name="setBoolean"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <param name="value" type="boolean"/>
+      <doc>
+      <![CDATA[Set the value of the <code>name</code> property to a <code>boolean</code>.
+ 
+ @param name property name.
+ @param value <code>boolean</code> value of the property.]]>
+      </doc>
+    </method>
+    <method name="setBooleanIfUnset"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <param name="value" type="boolean"/>
+      <doc>
+      <![CDATA[Set the given property, if it is currently unset.
+ @param name property name
+ @param value new value]]>
+      </doc>
+    </method>
+    <method name="setEnum"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <param name="value" type="T"/>
+      <doc>
+      <![CDATA[Set the value of the <code>name</code> property to the given type. This
+ is equivalent to <code>set(&lt;name&gt;, value.toString())</code>.
+ @param name property name
+ @param value new value]]>
+      </doc>
+    </method>
+    <method name="getEnum" return="T"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <param name="defaultValue" type="T"/>
+      <doc>
+      <![CDATA[Return value matching this enumerated type.
+ Note that the returned value is trimmed by this method.
+ @param name Property name
+ @param defaultValue Value returned if no mapping exists
+ @throws IllegalArgumentException If mapping is illegal for the type
+ provided]]>
+      </doc>
+    </method>
+    <method name="setTimeDuration"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <param name="value" type="long"/>
+      <param name="unit" type="java.util.concurrent.TimeUnit"/>
+      <doc>
+      <![CDATA[Set the value of <code>name</code> to the given time duration. This
+ is equivalent to <code>set(&lt;name&gt;, value + &lt;time suffix&gt;)</code>.
+ @param name Property name
+ @param value Time duration
+ @param unit Unit of time]]>
+      </doc>
+    </method>
+    <method name="getTimeDuration" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <param name="defaultValue" type="long"/>
+      <param name="unit" type="java.util.concurrent.TimeUnit"/>
+      <doc>
+      <![CDATA[Return time duration in the given time unit. Valid units are encoded in
+ properties as suffixes: nanoseconds (ns), microseconds (us), milliseconds
+ (ms), seconds (s), minutes (m), hours (h), and days (d).
+ @param name Property name
+ @param defaultValue Value returned if no mapping exists.
+ @param unit Unit to convert the stored property, if it exists.
+ @throws NumberFormatException If the property stripped of its unit is not
+         a number]]>
+      </doc>
+    </method>
+    <method name="getTimeDurations" return="long[]"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <param name="unit" type="java.util.concurrent.TimeUnit"/>
+    </method>
+    <method name="getPattern" return="java.util.regex.Pattern"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <param name="defaultValue" type="java.util.regex.Pattern"/>
+      <doc>
+      <![CDATA[Get the value of the <code>name</code> property as a <code>Pattern</code>.
+ If no such property is specified, or if the specified value is not a valid
+ <code>Pattern</code>, then <code>DefaultValue</code> is returned.
+ Note that the returned value is NOT trimmed by this method.
+
+ @param name property name
+ @param defaultValue default value
+ @return property value as a compiled Pattern, or defaultValue]]>
+      </doc>
+    </method>
+    <method name="setPattern"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <param name="pattern" type="java.util.regex.Pattern"/>
+      <doc>
+      <![CDATA[Set the given property to <code>Pattern</code>.
+ If the pattern is passed as null, sets the empty pattern which results in
+ further calls to getPattern(...) returning the default value.
+
+ @param name property name
+ @param pattern new value]]>
+      </doc>
+    </method>
+    <method name="getPropertySources" return="java.lang.String[]"
+      abstract="false" native="false" synchronized="true"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <doc>
+      <![CDATA[Gets information about why a property was set.  Typically this is the 
+ path to the resource objects (file, URL, etc.) the property came from, but
+ it can also indicate that it was set programatically, or because of the
+ command line.
+
+ @param name - The property name to get the source of.
+ @return null - If the property or its source wasn't found. Otherwise, 
+ returns a list of the sources of the resource.  The older sources are
+ the first ones in the list.  So for example if a configuration is set from
+ the command line, and then written out to a file that is read back in the
+ first entry would indicate that it was set from the command line, while
+ the second one would indicate the file that the new configuration was read
+ in from.]]>
+      </doc>
+    </method>
+    <method name="getRange" return="org.apache.hadoop.conf.Configuration.IntegerRanges"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <param name="defaultValue" type="java.lang.String"/>
+      <doc>
+      <![CDATA[Parse the given attribute as a set of integer ranges
+ @param name the attribute name
+ @param defaultValue the default value if it is not set
+ @return a new set of ranges from the configured value]]>
+      </doc>
+    </method>
+    <method name="getStringCollection" return="java.util.Collection"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <doc>
+      <![CDATA[Get the comma delimited values of the <code>name</code> property as 
+ a collection of <code>String</code>s.  
+ If no such property is specified then empty collection is returned.
+ <p>
+ This is an optimized version of {@link #getStrings(String)}
+ 
+ @param name property name.
+ @return property value as a collection of <code>String</code>s.]]>
+      </doc>
+    </method>
+    <method name="getStrings" return="java.lang.String[]"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <doc>
+      <![CDATA[Get the comma delimited values of the <code>name</code> property as 
+ an array of <code>String</code>s.  
+ If no such property is specified then <code>null</code> is returned.
+ 
+ @param name property name.
+ @return property value as an array of <code>String</code>s, 
+         or <code>null</code>.]]>
+      </doc>
+    </method>
+    <method name="getStrings" return="java.lang.String[]"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <param name="defaultValue" type="java.lang.String[]"/>
+      <doc>
+      <![CDATA[Get the comma delimited values of the <code>name</code> property as 
+ an array of <code>String</code>s.  
+ If no such property is specified then default value is returned.
+ 
+ @param name property name.
+ @param defaultValue The default value
+ @return property value as an array of <code>String</code>s, 
+         or default value.]]>
+      </doc>
+    </method>
+    <method name="getTrimmedStringCollection" return="java.util.Collection"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <doc>
+      <![CDATA[Get the comma delimited values of the <code>name</code> property as 
+ a collection of <code>String</code>s, trimmed of the leading and trailing whitespace.  
+ If no such property is specified then empty <code>Collection</code> is returned.
+
+ @param name property name.
+ @return property value as a collection of <code>String</code>s, or empty <code>Collection</code>]]>
+      </doc>
+    </method>
+    <method name="getTrimmedStrings" return="java.lang.String[]"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <doc>
+      <![CDATA[Get the comma delimited values of the <code>name</code> property as 
+ an array of <code>String</code>s, trimmed of the leading and trailing whitespace.
+ If no such property is specified then an empty array is returned.
+ 
+ @param name property name.
+ @return property value as an array of trimmed <code>String</code>s, 
+         or empty array.]]>
+      </doc>
+    </method>
+    <method name="getTrimmedStrings" return="java.lang.String[]"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <param name="defaultValue" type="java.lang.String[]"/>
+      <doc>
+      <![CDATA[Get the comma delimited values of the <code>name</code> property as 
+ an array of <code>String</code>s, trimmed of the leading and trailing whitespace.
+ If no such property is specified then default value is returned.
+ 
+ @param name property name.
+ @param defaultValue The default value
+ @return property value as an array of trimmed <code>String</code>s, 
+         or default value.]]>
+      </doc>
+    </method>
+    <method name="setStrings"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <param name="values" type="java.lang.String[]"/>
+      <doc>
+      <![CDATA[Set the array of string values for the <code>name</code> property as 
+ as comma delimited values.  
+ 
+ @param name property name.
+ @param values The values]]>
+      </doc>
+    </method>
+    <method name="getPassword" return="char[]"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Get the value for a known password configuration element.
+ In order to enable the elimination of clear text passwords in config,
+ this method attempts to resolve the property name as an alias through
+ the CredentialProvider API and conditionally fallsback to config.
+ @param name property name
+ @return password]]>
+      </doc>
+    </method>
+    <method name="getPasswordFromCredentialProviders" return="char[]"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Try and resolve the provided element name as a credential provider
+ alias.
+ @param name alias of the provisioned credential
+ @return password or null if not found
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getPasswordFromConfig" return="char[]"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <doc>
+      <![CDATA[Fallback to clear text passwords in configuration.
+ @param name
+ @return clear text password or null]]>
+      </doc>
+    </method>
+    <method name="getSocketAddr" return="java.net.InetSocketAddress"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="hostProperty" type="java.lang.String"/>
+      <param name="addressProperty" type="java.lang.String"/>
+      <param name="defaultAddressValue" type="java.lang.String"/>
+      <param name="defaultPort" type="int"/>
+      <doc>
+      <![CDATA[Get the socket address for <code>hostProperty</code> as a
+ <code>InetSocketAddress</code>. If <code>hostProperty</code> is
+ <code>null</code>, <code>addressProperty</code> will be used. This
+ is useful for cases where we want to differentiate between host
+ bind address and address clients should use to establish connection.
+
+ @param hostProperty bind host property name.
+ @param addressProperty address property name.
+ @param defaultAddressValue the default value
+ @param defaultPort the default port
+ @return InetSocketAddress]]>
+      </doc>
+    </method>
+    <method name="getSocketAddr" return="java.net.InetSocketAddress"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <param name="defaultAddress" type="java.lang.String"/>
+      <param name="defaultPort" type="int"/>
+      <doc>
+      <![CDATA[Get the socket address for <code>name</code> property as a
+ <code>InetSocketAddress</code>.
+ @param name property name.
+ @param defaultAddress the default value
+ @param defaultPort the default port
+ @return InetSocketAddress]]>
+      </doc>
+    </method>
+    <method name="setSocketAddr"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <param name="addr" type="java.net.InetSocketAddress"/>
+      <doc>
+      <![CDATA[Set the socket address for the <code>name</code> property as
+ a <code>host:port</code>.]]>
+      </doc>
+    </method>
+    <method name="updateConnectAddr" return="java.net.InetSocketAddress"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="hostProperty" type="java.lang.String"/>
+      <param name="addressProperty" type="java.lang.String"/>
+      <param name="defaultAddressValue" type="java.lang.String"/>
+      <param name="addr" type="java.net.InetSocketAddress"/>
+      <doc>
+      <![CDATA[Set the socket address a client can use to connect for the
+ <code>name</code> property as a <code>host:port</code>.  The wildcard
+ address is replaced with the local host's address. If the host and address
+ properties are configured the host component of the address will be combined
+ with the port component of the addr to generate the address.  This is to allow
+ optional control over which host name is used in multi-home bind-host
+ cases where a host can have multiple names
+ @param hostProperty the bind-host configuration name
+ @param addressProperty the service address configuration name
+ @param defaultAddressValue the service default address configuration value
+ @param addr InetSocketAddress of the service listener
+ @return InetSocketAddress for clients to connect]]>
+      </doc>
+    </method>
+    <method name="updateConnectAddr" return="java.net.InetSocketAddress"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <param name="addr" type="java.net.InetSocketAddress"/>
+      <doc>
+      <![CDATA[Set the socket address a client can use to connect for the
+ <code>name</code> property as a <code>host:port</code>.  The wildcard
+ address is replaced with the local host's address.
+ @param name property name.
+ @param addr InetSocketAddress of a listener to store in the given property
+ @return InetSocketAddress for clients to connect]]>
+      </doc>
+    </method>
+    <method name="getClassByName" return="java.lang.Class"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <exception name="ClassNotFoundException" type="java.lang.ClassNotFoundException"/>
+      <doc>
+      <![CDATA[Load a class by name.
+ 
+ @param name the class name.
+ @return the class object.
+ @throws ClassNotFoundException if the class is not found.]]>
+      </doc>
+    </method>
+    <method name="getClassByNameOrNull" return="java.lang.Class"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <doc>
+      <![CDATA[Load a class by name, returning null rather than throwing an exception
+ if it couldn't be loaded. This is to avoid the overhead of creating
+ an exception.
+ 
+ @param name the class name
+ @return the class object, or null if it could not be found.]]>
+      </doc>
+    </method>
+    <method name="getClasses" return="java.lang.Class[]"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <param name="defaultValue" type="java.lang.Class[]"/>
+      <doc>
+      <![CDATA[Get the value of the <code>name</code> property
+ as an array of <code>Class</code>.
+ The value of the property specifies a list of comma separated class names.  
+ If no such property is specified, then <code>defaultValue</code> is 
+ returned.
+ 
+ @param name the property name.
+ @param defaultValue default value.
+ @return property value as a <code>Class[]</code>, 
+         or <code>defaultValue</code>.]]>
+      </doc>
+    </method>
+    <method name="getClass" return="java.lang.Class"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <param name="defaultValue" type="java.lang.Class"/>
+      <doc>
+      <![CDATA[Get the value of the <code>name</code> property as a <code>Class</code>.  
+ If no such property is specified, then <code>defaultValue</code> is 
+ returned.
+ 
+ @param name the class name.
+ @param defaultValue default value.
+ @return property value as a <code>Class</code>, 
+         or <code>defaultValue</code>.]]>
+      </doc>
+    </method>
+    <method name="getClass" return="java.lang.Class"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <param name="defaultValue" type="java.lang.Class"/>
+      <param name="xface" type="java.lang.Class"/>
+      <doc>
+      <![CDATA[Get the value of the <code>name</code> property as a <code>Class</code>
+ implementing the interface specified by <code>xface</code>.
+   
+ If no such property is specified, then <code>defaultValue</code> is 
+ returned.
+ 
+ An exception is thrown if the returned class does not implement the named
+ interface. 
+ 
+ @param name the class name.
+ @param defaultValue default value.
+ @param xface the interface implemented by the named class.
+ @return property value as a <code>Class</code>, 
+         or <code>defaultValue</code>.]]>
+      </doc>
+    </method>
+    <method name="getInstances" return="java.util.List"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <param name="xface" type="java.lang.Class"/>
+      <doc>
+      <![CDATA[Get the value of the <code>name</code> property as a <code>List</code>
+ of objects implementing the interface specified by <code>xface</code>.
+ 
+ An exception is thrown if any of the classes does not exist, or if it does
+ not implement the named interface.
+ 
+ @param name the property name.
+ @param xface the interface implemented by the classes named by
+        <code>name</code>.
+ @return a <code>List</code> of objects implementing <code>xface</code>.]]>
+      </doc>
+    </method>
+    <method name="setClass"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <param name="theClass" type="java.lang.Class"/>
+      <param name="xface" type="java.lang.Class"/>
+      <doc>
+      <![CDATA[Set the value of the <code>name</code> property to the name of a 
+ <code>theClass</code> implementing the given interface <code>xface</code>.
+ 
+ An exception is thrown if <code>theClass</code> does not implement the 
+ interface <code>xface</code>. 
+ 
+ @param name property name.
+ @param theClass property value.
+ @param xface the interface implemented by the named class.]]>
+      </doc>
+    </method>
+    <method name="getLocalPath" return="org.apache.hadoop.fs.Path"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="dirsProp" type="java.lang.String"/>
+      <param name="path" type="java.lang.String"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Get a local file under a directory named by <i>dirsProp</i> with
+ the given <i>path</i>.  If <i>dirsProp</i> contains multiple directories,
+ then one is chosen based on <i>path</i>'s hash code.  If the selected
+ directory does not exist, an attempt is made to create it.
+ 
+ @param dirsProp directory in which to locate the file.
+ @param path file-path.
+ @return local file under the directory with the given path.]]>
+      </doc>
+    </method>
+    <method name="getFile" return="java.io.File"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="dirsProp" type="java.lang.String"/>
+      <param name="path" type="java.lang.String"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Get a local file name under a directory named in <i>dirsProp</i> with
+ the given <i>path</i>.  If <i>dirsProp</i> contains multiple directories,
+ then one is chosen based on <i>path</i>'s hash code.  If the selected
+ directory does not exist, an attempt is made to create it.
+ 
+ @param dirsProp directory in which to locate the file.
+ @param path file-path.
+ @return local file under the directory with the given path.]]>
+      </doc>
+    </method>
+    <method name="getResource" return="java.net.URL"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <doc>
+      <![CDATA[Get the {@link URL} for the named resource.
+ 
+ @param name resource name.
+ @return the url for the named resource.]]>
+      </doc>
+    </method>
+    <method name="getConfResourceAsInputStream" return="java.io.InputStream"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <doc>
+      <![CDATA[Get an input stream attached to the configuration resource with the
+ given <code>name</code>.
+ 
+ @param name configuration resource name.
+ @return an input stream attached to the resource.]]>
+      </doc>
+    </method>
+    <method name="getConfResourceAsReader" return="java.io.Reader"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <doc>
+      <![CDATA[Get a {@link Reader} attached to the configuration resource with the
+ given <code>name</code>.
+ 
+ @param name configuration resource name.
+ @return a reader attached to the resource.]]>
+      </doc>
+    </method>
+    <method name="getFinalParameters" return="java.util.Set"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the set of parameters marked final.
+
+ @return final parameter set.]]>
+      </doc>
+    </method>
+    <method name="getProps" return="java.util.Properties"
+      abstract="false" native="false" synchronized="true"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </method>
+    <method name="size" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Return the number of keys in the configuration.
+
+ @return number of keys in the configuration.]]>
+      </doc>
+    </method>
+    <method name="clear"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Clears all keys from the configuration.]]>
+      </doc>
+    </method>
+    <method name="iterator" return="java.util.Iterator"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get an {@link Iterator} to go through the list of <code>String</code> 
+ key-value pairs in the configuration.
+ 
+ @return an iterator over the entries.]]>
+      </doc>
+    </method>
+    <method name="getPropsWithPrefix" return="java.util.Map"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="confPrefix" type="java.lang.String"/>
+      <doc>
+      <![CDATA[Constructs a mapping of configuration and includes all properties that
+ start with the specified configuration prefix.  Property names in the
+ mapping are trimmed to remove the configuration prefix.
+
+ @param confPrefix configuration prefix
+ @return mapping of configuration properties with prefix stripped]]>
+      </doc>
+    </method>
+    <method name="writeXml"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="out" type="java.io.OutputStream"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Write out the non-default properties in this configuration to the given
+ {@link OutputStream} using UTF-8 encoding.
+ 
+ @param out the output stream to write to.]]>
+      </doc>
+    </method>
+    <method name="writeXml"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="out" type="java.io.Writer"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="writeXml"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="propertyName" type="java.lang.String"/>
+      <param name="out" type="java.io.Writer"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <exception name="IllegalArgumentException" type="java.lang.IllegalArgumentException"/>
+      <doc>
+      <![CDATA[Write out the non-default properties in this configuration to the
+ given {@link Writer}.
+
+ <li>
+ When property name is not empty and the property exists in the
+ configuration, this method writes the property and its attributes
+ to the {@link Writer}.
+ </li>
+ <p>
+
+ <li>
+ When property name is null or empty, this method writes all the
+ configuration properties and their attributes to the {@link Writer}.
+ </li>
+ <p>
+
+ <li>
+ When property name is not empty but the property doesn't exist in
+ the configuration, this method throws an {@link IllegalArgumentException}.
+ </li>
+ <p>
+ @param out the writer to write to.]]>
+      </doc>
+    </method>
+    <method name="dumpConfiguration"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="config" type="org.apache.hadoop.conf.Configuration"/>
+      <param name="propertyName" type="java.lang.String"/>
+      <param name="out" type="java.io.Writer"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Writes properties and their attributes (final and resource)
+  to the given {@link Writer}.
+
+  <li>
+  When propertyName is not empty, and the property exists
+  in the configuration, the format of the output would be,
+  <pre>
+  {
+    "property": {
+      "key" : "key1",
+      "value" : "value1",
+      "isFinal" : "key1.isFinal",
+      "resource" : "key1.resource"
+    }
+  }
+  </pre>
+  </li>
+
+  <li>
+  When propertyName is null or empty, it behaves same as
+  {@link #dumpConfiguration(Configuration, Writer)}, the
+  output would be,
+  <pre>
+  { "properties" :
+      [ { key : "key1",
+          value : "value1",
+          isFinal : "key1.isFinal",
+          resource : "key1.resource" },
+        { key : "key2",
+          value : "value2",
+          isFinal : "ke2.isFinal",
+          resource : "key2.resource" }
+       ]
+   }
+  </pre>
+  </li>
+
+  <li>
+  When propertyName is not empty, and the property is not
+  found in the configuration, this method will throw an
+  {@link IllegalArgumentException}.
+  </li>
+  <p>
+ @param config the configuration
+ @param propertyName property name
+ @param out the Writer to write to
+ @throws IOException
+ @throws IllegalArgumentException when property name is not
+   empty and the property is not found in configuration]]>
+      </doc>
+    </method>
+    <method name="dumpConfiguration"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="config" type="org.apache.hadoop.conf.Configuration"/>
+      <param name="out" type="java.io.Writer"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Writes out all properties and their attributes (final and resource) to
+  the given {@link Writer}, the format of the output would be,
+
+  <pre>
+  { "properties" :
+      [ { key : "key1",
+          value : "value1",
+          isFinal : "key1.isFinal",
+          resource : "key1.resource" },
+        { key : "key2",
+          value : "value2",
+          isFinal : "ke2.isFinal",
+          resource : "key2.resource" }
+       ]
+   }
+  </pre>
+
+  It does not output the properties of the configuration object which
+  is loaded from an input stream.
+  <p>
+
+ @param config the configuration
+ @param out the Writer to write to
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getClassLoader" return="java.lang.ClassLoader"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the {@link ClassLoader} for this job.
+
+ @return the correct class loader.]]>
+      </doc>
+    </method>
+    <method name="setClassLoader"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="classLoader" type="java.lang.ClassLoader"/>
+      <doc>
+      <![CDATA[Set the class loader that will be used to load the various objects.
+ 
+ @param classLoader the new class loader.]]>
+      </doc>
+    </method>
+    <method name="toString" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="setQuietMode"
+      abstract="false" native="false" synchronized="true"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="quietmode" type="boolean"/>
+      <doc>
+      <![CDATA[Set the quietness-mode. 
+ 
+ In the quiet-mode, error and informational messages might not be logged.
+ 
+ @param quietmode <code>true</code> to set quiet-mode on, <code>false</code>
+              to turn it off.]]>
+      </doc>
+    </method>
+    <method name="main"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="args" type="java.lang.String[]"/>
+      <exception name="Exception" type="java.lang.Exception"/>
+      <doc>
+      <![CDATA[For debugging.  List non-default properties to the terminal and exit.]]>
+      </doc>
+    </method>
+    <method name="readFields"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="in" type="java.io.DataInput"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="write"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="out" type="java.io.DataOutput"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="getValByRegex" return="java.util.Map"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="regex" type="java.lang.String"/>
+      <doc>
+      <![CDATA[get keys matching the the regex 
+ @param regex
+ @return Map<String,String> with matching keys]]>
+      </doc>
+    </method>
+    <method name="dumpDeprecatedKeys"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="hasWarnedDeprecation" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <doc>
+      <![CDATA[Returns whether or not a deprecated name has been warned. If the name is not
+ deprecated then always return false]]>
+      </doc>
+    </method>
+    <doc>
+    <![CDATA[Provides access to configuration parameters.
+
+ <h4 id="Resources">Resources</h4>
+
+ <p>Configurations are specified by resources. A resource contains a set of
+ name/value pairs as XML data. Each resource is named by either a 
+ <code>String</code> or by a {@link Path}. If named by a <code>String</code>, 
+ then the classpath is examined for a file with that name.  If named by a 
+ <code>Path</code>, then the local filesystem is examined directly, without 
+ referring to the classpath.
+
+ <p>Unless explicitly turned off, Hadoop by default specifies two 
+ resources, loaded in-order from the classpath: <ol>
+ <li><tt>
+ <a href="{@docRoot}/../hadoop-project-dist/hadoop-common/core-default.xml">
+ core-default.xml</a></tt>: Read-only defaults for hadoop.</li>
+ <li><tt>core-site.xml</tt>: Site-specific configuration for a given hadoop
+ installation.</li>
+ </ol>
+ Applications may add additional resources, which are loaded
+ subsequent to these resources in the order they are added.
+ 
+ <h4 id="FinalParams">Final Parameters</h4>
+
+ <p>Configuration parameters may be declared <i>final</i>. 
+ Once a resource declares a value final, no subsequently-loaded 
+ resource can alter that value.  
+ For example, one might define a final parameter with:
+ <tt><pre>
+  &lt;property&gt;
+    &lt;name&gt;dfs.hosts.include&lt;/name&gt;
+    &lt;value&gt;/etc/hadoop/conf/hosts.include&lt;/value&gt;
+    <b>&lt;final&gt;true&lt;/final&gt;</b>
+  &lt;/property&gt;</pre></tt>
+
+ Administrators typically define parameters as final in 
+ <tt>core-site.xml</tt> for values that user applications may not alter.
+
+ <h4 id="VariableExpansion">Variable Expansion</h4>
+
+ <p>Value strings are first processed for <i>variable expansion</i>. The
+ available properties are:<ol>
+ <li>Other properties defined in this Configuration; and, if a name is
+ undefined here,</li>
+ <li>Properties in {@link System#getProperties()}.</li>
+ </ol>
+
+ <p>For example, if a configuration resource contains the following property
+ definitions: 
+ <tt><pre>
+  &lt;property&gt;
+    &lt;name&gt;basedir&lt;/name&gt;
+    &lt;value&gt;/user/${<i>user.name</i>}&lt;/value&gt;
+  &lt;/property&gt;
+  
+  &lt;property&gt;
+    &lt;name&gt;tempdir&lt;/name&gt;
+    &lt;value&gt;${<i>basedir</i>}/tmp&lt;/value&gt;
+  &lt;/property&gt;</pre></tt>
+
+ When <tt>conf.get("tempdir")</tt> is called, then <tt>${<i>basedir</i>}</tt>
+ will be resolved to another property in this Configuration, while
+ <tt>${<i>user.name</i>}</tt> would then ordinarily be resolved to the value
+ of the System property with that name.
+ <p>When <tt>conf.get("otherdir")</tt> is called, then <tt>${<i>env.BASE_DIR</i>}</tt>
+ will be resolved to the value of the <tt>${<i>BASE_DIR</i>}</tt> environment variable.
+ It supports <tt>${<i>env.NAME:-default</i>}</tt> and <tt>${<i>env.NAME-default</i>}</tt> notations.
+ The former is resolved to "default" if <tt>${<i>NAME</i>}</tt> environment variable is undefined
+ or its value is empty.
+ The latter behaves the same way only if <tt>${<i>NAME</i>}</tt> is undefined.
+ <p>By default, warnings will be given to any deprecated configuration 
+ parameters and these are suppressible by configuring
+ <tt>log4j.logger.org.apache.hadoop.conf.Configuration.deprecation</tt> in
+ log4j.properties file.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.conf.Configuration -->
+  <!-- start class org.apache.hadoop.conf.Configured -->
+  <class name="Configured" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="org.apache.hadoop.conf.Configurable"/>
+    <constructor name="Configured"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Construct a Configured.]]>
+      </doc>
+    </constructor>
+    <constructor name="Configured" type="org.apache.hadoop.conf.Configuration"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Construct a Configured.]]>
+      </doc>
+    </constructor>
+    <method name="setConf"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+    </method>
+    <method name="getConf" return="org.apache.hadoop.conf.Configuration"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <doc>
+    <![CDATA[Base class for things that may be configured with a {@link Configuration}.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.conf.Configured -->
+  <!-- start class org.apache.hadoop.conf.ReconfigurationTaskStatus -->
+  <class name="ReconfigurationTaskStatus" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="ReconfigurationTaskStatus" type="long, long, java.util.Map"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="hasTask" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Return true if
+   - A reconfiguration task has finished or
+   - an active reconfiguration task is running]]>
+      </doc>
+    </method>
+    <method name="stopped" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Return true if the latest reconfiguration task has finished and there is
+ no another active task running.]]>
+      </doc>
+    </method>
+    <method name="getStartTime" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getEndTime" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getStatus" return="java.util.Map"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="true" visibility="public"
+      deprecated="not deprecated">
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.conf.ReconfigurationTaskStatus -->
+  <doc>
+  <![CDATA[Configuration of system parameters.]]>
+  </doc>
+</package>
+<package name="org.apache.hadoop.crypto">
+</package>
+<package name="org.apache.hadoop.crypto.key">
+  <!-- start class org.apache.hadoop.crypto.key.KeyProvider -->
+  <class name="KeyProvider" extends="java.lang.Object"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="KeyProvider" type="org.apache.hadoop.conf.Configuration"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Constructor.
+ 
+ @param conf configuration for the provider]]>
+      </doc>
+    </constructor>
+    <method name="getConf" return="org.apache.hadoop.conf.Configuration"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Return the provider configuration.
+ 
+ @return the provider configuration]]>
+      </doc>
+    </method>
+    <method name="options" return="org.apache.hadoop.crypto.key.KeyProvider.Options"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <doc>
+      <![CDATA[A helper function to create an options object.
+ @param conf the configuration to use
+ @return a new options object]]>
+      </doc>
+    </method>
+    <method name="isTransient" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Indicates whether this provider represents a store
+ that is intended for transient use - such as the UserProvider
+ is. These providers are generally used to provide access to
+ keying material rather than for long term storage.
+ @return true if transient, false otherwise]]>
+      </doc>
+    </method>
+    <method name="getKeyVersion" return="org.apache.hadoop.crypto.key.KeyProvider.KeyVersion"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="versionName" type="java.lang.String"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Get the key material for a specific version of the key. This method is used
+ when decrypting data.
+ @param versionName the name of a specific version of the key
+ @return the key material
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getKeys" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Get the key names for all keys.
+ @return the list of key names
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getKeysMetadata" return="org.apache.hadoop.crypto.key.KeyProvider.Metadata[]"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="names" type="java.lang.String[]"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Get key metadata in bulk.
+ @param names the names of the keys to get
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getKeyVersions" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Get the key material for all versions of a specific key name.
+ @return the list of key material
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getCurrentKey" return="org.apache.hadoop.crypto.key.KeyProvider.KeyVersion"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Get the current version of the key, which should be used for encrypting new
+ data.
+ @param name the base name of the key
+ @return the version name of the current version of the key or null if the
+    key version doesn't exist
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getMetadata" return="org.apache.hadoop.crypto.key.KeyProvider.Metadata"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Get metadata about the key.
+ @param name the basename of the key
+ @return the key's metadata or null if the key doesn't exist
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="createKey" return="org.apache.hadoop.crypto.key.KeyProvider.KeyVersion"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <param name="material" type="byte[]"/>
+      <param name="options" type="org.apache.hadoop.crypto.key.KeyProvider.Options"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Create a new key. The given key must not already exist.
+ @param name the base name of the key
+ @param material the key material for the first version of the key.
+ @param options the options for the new key.
+ @return the version name of the first version of the key.
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="generateKey" return="byte[]"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="size" type="int"/>
+      <param name="algorithm" type="java.lang.String"/>
+      <exception name="NoSuchAlgorithmException" type="java.security.NoSuchAlgorithmException"/>
+      <doc>
+      <![CDATA[Generates a key material.
+
+ @param size length of the key.
+ @param algorithm algorithm to use for generating the key.
+ @return the generated key.
+ @throws NoSuchAlgorithmException]]>
+      </doc>
+    </method>
+    <method name="createKey" return="org.apache.hadoop.crypto.key.KeyProvider.KeyVersion"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <param name="options" type="org.apache.hadoop.crypto.key.KeyProvider.Options"/>
+      <exception name="NoSuchAlgorithmException" type="java.security.NoSuchAlgorithmException"/>
+      <exception name="IOException" type="java.io

<TRUNCATED>

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


[43/50] [abbrv] hadoop git commit: HADOOP-15133. [JDK9] Ignore com.sun.javadoc.* and com.sun.tools.* in animal-sniffer-maven-plugin to compile with Java 9.

Posted by na...@apache.org.
HADOOP-15133. [JDK9] Ignore com.sun.javadoc.* and com.sun.tools.* in animal-sniffer-maven-plugin to compile with Java 9.


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

Branch: refs/heads/yarn-3409
Commit: d2d8f4aeb3e214d1a96eeaf96bbe1e9301824ccd
Parents: 5ab632b
Author: Akira Ajisaka <aa...@apache.org>
Authored: Thu Dec 21 11:58:34 2017 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Thu Dec 21 11:58:34 2017 +0900

----------------------------------------------------------------------
 hadoop-project/pom.xml | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d2d8f4ae/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index efc8c2d..3c49182 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -1569,6 +1569,8 @@
             <ignore>sun.misc.*</ignore>
             <ignore>sun.net.*</ignore>
             <ignore>sun.nio.ch.*</ignore>
+            <ignore>com.sun.javadoc.*</ignore>
+            <ignore>com.sun.tools.*</ignore>
           </ignores>
         </configuration>
       </plugin>


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


[45/50] [abbrv] hadoop git commit: HADOOP-13282. S3 blob etags to be made visible in S3A status/getFileChecksum() calls. Contributed by Steve Loughran

Posted by na...@apache.org.
HADOOP-13282. S3 blob etags to be made visible in S3A status/getFileChecksum() calls.
Contributed by Steve Loughran


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

Branch: refs/heads/yarn-3409
Commit: c8ff0cc304f07bf793192291e0611b2fb4bcc4e3
Parents: ef450df
Author: Steve Loughran <st...@apache.org>
Authored: Thu Dec 21 14:58:58 2017 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Thu Dec 21 14:58:58 2017 +0000

----------------------------------------------------------------------
 .../apache/hadoop/fs/store/EtagChecksum.java    |  90 +++++++++++++
 .../apache/hadoop/fs/store/package-info.java    |  28 ++++
 .../hadoop/fs/store/TestEtagChecksum.java       |  85 ++++++++++++
 .../org/apache/hadoop/fs/s3a/S3AFileSystem.java |  40 ++++++
 .../hadoop/fs/s3a/ITestS3AMiscOperations.java   | 133 ++++++++++++++++---
 5 files changed, 359 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c8ff0cc3/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/EtagChecksum.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/EtagChecksum.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/EtagChecksum.java
new file mode 100644
index 0000000..cc29f1b
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/EtagChecksum.java
@@ -0,0 +1,90 @@
+/*
+ * 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.store;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+
+import org.apache.hadoop.fs.FileChecksum;
+
+/**
+ * An etag as a checksum.
+ * Consider these suitable for checking if an object has changed, but
+ * not suitable for comparing two different objects for equivalence,
+ * especially between object stores.
+ */
+public class EtagChecksum extends FileChecksum {
+
+  /** The algorithm name: {@value}. */
+  private static final String ETAG = "etag";
+
+  /**
+   * Etag string.
+   */
+  private String eTag = "";
+
+  /**
+   * Create with an empty etag.
+   */
+  public EtagChecksum() {
+  }
+
+  /**
+   * Create with a string etag.
+   * @param eTag etag
+   */
+  public EtagChecksum(String eTag) {
+    this.eTag = eTag;
+  }
+
+  @Override
+  public String getAlgorithmName() {
+    return ETAG;
+  }
+
+  @Override
+  public int getLength() {
+    return eTag.getBytes(StandardCharsets.UTF_8).length;
+  }
+
+  @Override
+  public byte[] getBytes() {
+    return eTag != null
+        ? eTag.getBytes(StandardCharsets.UTF_8)
+        : new byte[0];
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    out.writeUTF(eTag != null ? eTag : "");
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    eTag = in.readUTF();
+  }
+
+  @Override
+  public String toString() {
+    return "etag: \"" + eTag  + '"';
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c8ff0cc3/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/package-info.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/package-info.java
new file mode 100644
index 0000000..ebe1db4
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/package-info.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * This package is for classes to be shared across object stores; for internal
+ * use within the hadoop-* modules only. No stability guarantees.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.fs.store;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c8ff0cc3/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/store/TestEtagChecksum.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/store/TestEtagChecksum.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/store/TestEtagChecksum.java
new file mode 100644
index 0000000..ef9613f
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/store/TestEtagChecksum.java
@@ -0,0 +1,85 @@
+/*
+ * 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.store;
+
+import java.io.IOException;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.io.DataOutputBuffer;
+
+/**
+ * Unit test of etag operations.
+ */
+public class TestEtagChecksum extends Assert {
+
+  private final EtagChecksum empty1 = tag("");
+  private final EtagChecksum empty2 = tag("");
+  private final EtagChecksum valid1 = tag("valid");
+  private final EtagChecksum valid2 = tag("valid");
+
+  @Test
+  public void testEmptyTagsEqual() {
+    assertEquals(empty1, empty2);
+  }
+
+  @Test
+  public void testEmptyTagRoundTrip() throws Throwable {
+    assertEquals(empty1, roundTrip(empty1));
+  }
+
+  @Test
+  public void testValidTagsEqual() {
+    assertEquals(valid1, valid2);
+  }
+
+  @Test
+  public void testValidTagRoundTrip() throws Throwable {
+    assertEquals(valid1, roundTrip(valid1));
+  }
+
+  @Test
+  public void testValidAndEmptyTagsDontMatch() {
+    assertNotEquals(valid1, empty1);
+    assertNotEquals(valid1, tag("other valid one"));
+  }
+
+  @Test
+  public void testDifferentTagsDontMatch() {
+    assertNotEquals(valid1, tag("other valid one"));
+  }
+
+  private EtagChecksum tag(String t) {
+    return new EtagChecksum(t);
+  }
+
+  private EtagChecksum roundTrip(EtagChecksum tag) throws IOException {
+    try (DataOutputBuffer dob = new DataOutputBuffer();
+         DataInputBuffer dib = new DataInputBuffer()) {
+      tag.write(dob);
+      dib.reset(dob.getData(), dob.getLength());
+      EtagChecksum t2 = new EtagChecksum();
+      t2.readFields(dib);
+      return t2;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c8ff0cc3/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
index f461c9e..a8147ed 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
@@ -112,6 +112,7 @@ import org.apache.hadoop.fs.s3a.s3guard.PathMetadata;
 import org.apache.hadoop.fs.s3a.s3guard.S3Guard;
 import org.apache.hadoop.fs.s3native.S3xLoginHelper;
 import org.apache.hadoop.io.retry.RetryPolicies;
+import org.apache.hadoop.fs.store.EtagChecksum;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.BlockingThreadPoolExecutorService;
 import org.apache.hadoop.util.Progressable;
@@ -539,6 +540,14 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
   }
 
   /**
+   * Get the encryption algorithm of this endpoint.
+   * @return the encryption algorithm.
+   */
+  public S3AEncryptionMethods getServerSideEncryptionAlgorithm() {
+    return serverSideEncryptionAlgorithm;
+  }
+
+  /**
    * Demand create the directory allocator, then create a temporary file.
    * {@link LocalDirAllocator#createTmpFileForWrite(String, long, Configuration)}.
    *  @param pathStr prefix for the temporary file
@@ -1069,6 +1078,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
    * @throws IOException IO and object access problems.
    */
   @VisibleForTesting
+  @Retries.RetryRaw
   public ObjectMetadata getObjectMetadata(Path path) throws IOException {
     return getObjectMetadata(pathToKey(path));
   }
@@ -2935,6 +2945,36 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
   }
 
   /**
+   * Get the etag of a object at the path via HEAD request and return it
+   * as a checksum object. This has the whatever guarantees about equivalence
+   * the S3 implementation offers.
+   * <ol>
+   *   <li>If a tag has not changed, consider the object unchanged.</li>
+   *   <li>Two tags being different does not imply the data is different.</li>
+   * </ol>
+   * Different S3 implementations may offer different guarantees.
+   * @param f The file path
+   * @param length The length of the file range for checksum calculation
+   * @return The EtagChecksum or null if checksums are not supported.
+   * @throws IOException IO failure
+   * @see <a href="http://docs.aws.amazon.com/AmazonS3/latest/API/RESTCommonResponseHeaders.html">Common Response Headers</a>
+   */
+
+  public EtagChecksum getFileChecksum(Path f, final long length)
+      throws IOException {
+    Preconditions.checkArgument(length >= 0);
+    Path path = qualify(f);
+    LOG.debug("getFileChecksum({})", path);
+    return once("getFileChecksum", path.toString(),
+        () -> {
+          // this always does a full HEAD to the object
+          ObjectMetadata headers = getObjectMetadata(path);
+          String eTag = headers.getETag();
+          return eTag != null ? new EtagChecksum(eTag) : null;
+        });
+  }
+
+  /**
    * {@inheritDoc}.
    *
    * This implementation is optimized for S3, which can do a bulk listing

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c8ff0cc3/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java
index 869d64c..ddf2529 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java
@@ -18,21 +18,24 @@
 
 package org.apache.hadoop.fs.s3a;
 
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileAlreadyExistsException;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.contract.ContractTestUtils;
-import org.apache.hadoop.test.LambdaTestUtils;
+import java.io.ByteArrayInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
 
 import com.amazonaws.services.s3.model.ObjectMetadata;
 import com.amazonaws.services.s3.model.PutObjectRequest;
-import com.amazonaws.services.s3.model.PutObjectResult;
+import org.junit.Assume;
 import org.junit.Test;
 
-import java.io.ByteArrayInputStream;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.util.concurrent.Callable;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.store.EtagChecksum;
+import org.apache.hadoop.test.LambdaTestUtils;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.touch;
 
 /**
  * Tests of the S3A FileSystem which don't have a specific home and can share
@@ -40,6 +43,8 @@ import java.util.concurrent.Callable;
  */
 public class ITestS3AMiscOperations extends AbstractS3ATestBase {
 
+  private static final byte[] HELLO = "hello".getBytes(StandardCharsets.UTF_8);
+
   @Test
   public void testCreateNonRecursiveSuccess() throws IOException {
     Path shouldWork = path("nonrecursivenode");
@@ -58,7 +63,7 @@ public class ITestS3AMiscOperations extends AbstractS3ATestBase {
   @Test(expected = FileAlreadyExistsException.class)
   public void testCreateNonRecursiveParentIsFile() throws IOException {
     Path parent = path("/file.txt");
-    ContractTestUtils.touch(getFileSystem(), parent);
+    touch(getFileSystem(), parent);
     createNonRecursive(new Path(parent, "fail"));
   }
 
@@ -73,12 +78,7 @@ public class ITestS3AMiscOperations extends AbstractS3ATestBase {
         new ByteArrayInputStream("PUT".getBytes()),
         metadata);
     LambdaTestUtils.intercept(IllegalStateException.class,
-        new Callable<PutObjectResult>() {
-          @Override
-          public PutObjectResult call() throws Exception {
-            return fs.putObjectDirect(put);
-          }
-        });
+        () -> fs.putObjectDirect(put));
     assertPathDoesNotExist("put object was created", path);
   }
 
@@ -87,4 +87,103 @@ public class ITestS3AMiscOperations extends AbstractS3ATestBase {
         (short) 3, (short) 4096,
         null);
   }
+
+  /**
+   * Touch a path, return the full path.
+   * @param name relative name
+   * @return the path
+   * @throws IOException IO failure
+   */
+  Path touchFile(String name) throws IOException {
+    Path path = path(name);
+    touch(getFileSystem(), path);
+    return path;
+  }
+
+  /**
+   * Create a file with the data, return the path.
+   * @param name relative name
+   * @param data data to write
+   * @return the path
+   * @throws IOException IO failure
+   */
+  Path mkFile(String name, byte[] data) throws IOException {
+    final Path f = path(name);
+    createFile(getFileSystem(), f, true, data);
+    return f;
+  }
+
+  /**
+   * The assumption here is that 0-byte files uploaded in a single PUT
+   * always have the same checksum, including stores with encryption.
+   * @throws Throwable on a failure
+   */
+  @Test
+  public void testEmptyFileChecksums() throws Throwable {
+    final S3AFileSystem fs = getFileSystem();
+    Path file1 = touchFile("file1");
+    EtagChecksum checksum1 = fs.getFileChecksum(file1, 0);
+    LOG.info("Checksum for {}: {}", file1, checksum1);
+    assertNotNull("file 1 checksum", checksum1);
+    assertNotEquals("file 1 checksum", 0, checksum1.getLength());
+    assertEquals("checksums", checksum1,
+        fs.getFileChecksum(touchFile("file2"), 0));
+  }
+
+  /**
+   * Verify that different file contents have different
+   * checksums, and that that they aren't the same as the empty file.
+   * @throws Throwable failure
+   */
+  @Test
+  public void testNonEmptyFileChecksums() throws Throwable {
+    final S3AFileSystem fs = getFileSystem();
+    final Path file3 = mkFile("file3", HELLO);
+    final EtagChecksum checksum1 = fs.getFileChecksum(file3, 0);
+    assertNotNull("file 3 checksum", checksum1);
+    final Path file4 = touchFile("file4");
+    final EtagChecksum checksum2 = fs.getFileChecksum(file4, 0);
+    assertNotEquals("checksums", checksum1, checksum2);
+    // overwrite
+    createFile(fs, file4, true,
+        "hello, world".getBytes(StandardCharsets.UTF_8));
+    assertNotEquals(checksum2, fs.getFileChecksum(file4, 0));
+  }
+
+  /**
+   * Verify that on an unencrypted store, the checksum of two non-empty
+   * (single PUT) files is the same if the data is the same.
+   * This will fail if the bucket has S3 default encryption enabled.
+   * @throws Throwable failure
+   */
+  @Test
+  public void testNonEmptyFileChecksumsUnencrypted() throws Throwable {
+    Assume.assumeTrue(encryptionAlgorithm().equals(S3AEncryptionMethods.NONE));
+    final S3AFileSystem fs = getFileSystem();
+    final EtagChecksum checksum1 =
+        fs.getFileChecksum(mkFile("file5", HELLO), 0);
+    assertNotNull("file 3 checksum", checksum1);
+    assertEquals("checksums", checksum1,
+        fs.getFileChecksum(mkFile("file6", HELLO), 0));
+  }
+
+  private S3AEncryptionMethods encryptionAlgorithm() {
+    return getFileSystem().getServerSideEncryptionAlgorithm();
+  }
+
+  @Test
+  public void testNegativeLength() throws Throwable {
+    LambdaTestUtils.intercept(IllegalArgumentException.class,
+        () -> getFileSystem().getFileChecksum(mkFile("negative", HELLO), -1));
+  }
+
+  @Test
+  public void testLengthPastEOF() throws Throwable {
+    final S3AFileSystem fs = getFileSystem();
+    Path f = mkFile("file5", HELLO);
+    assertEquals(
+        fs.getFileChecksum(f, HELLO.length),
+        fs.getFileChecksum(f, HELLO.length * 2));
+  }
+
 }


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


[49/50] [abbrv] hadoop git commit: HADOOP-15086. NativeAzureFileSystem file rename is not atomic. Contributed by Thomas Marquardt

Posted by na...@apache.org.
HADOOP-15086. NativeAzureFileSystem file rename is not atomic.
Contributed by Thomas Marquardt


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

Branch: refs/heads/yarn-3409
Commit: 52babbb4a0e3c89f2025bf6e9a1b51a96e8f8fb0
Parents: 76e664e
Author: Steve Loughran <st...@apache.org>
Authored: Fri Dec 22 11:39:55 2017 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Fri Dec 22 11:39:55 2017 +0000

----------------------------------------------------------------------
 .../fs/azure/AzureNativeFileSystemStore.java    | 16 +++--
 .../hadoop/fs/azure/NativeAzureFileSystem.java  | 25 +++++--
 .../fs/azure/NativeAzureFileSystemHelper.java   | 18 +++++
 .../hadoop/fs/azure/NativeFileSystemStore.java  |  4 ++
 .../fs/azure/SecureStorageInterfaceImpl.java    |  8 ++-
 .../hadoop/fs/azure/StorageInterface.java       |  2 +-
 .../hadoop/fs/azure/StorageInterfaceImpl.java   |  8 ++-
 .../azure/ITestNativeAzureFileSystemLive.java   | 72 ++++++++++++++++++++
 .../hadoop/fs/azure/MockStorageInterface.java   |  9 ++-
 9 files changed, 145 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/52babbb4/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java
index f1031b4..9396a51 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java
@@ -2605,12 +2605,18 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
 
   @Override
   public void rename(String srcKey, String dstKey) throws IOException {
-    rename(srcKey, dstKey, false, null);
+    rename(srcKey, dstKey, false, null, true);
   }
 
   @Override
   public void rename(String srcKey, String dstKey, boolean acquireLease,
-      SelfRenewingLease existingLease) throws IOException {
+                     SelfRenewingLease existingLease) throws IOException {
+    rename(srcKey, dstKey, acquireLease, existingLease, true);
+  }
+
+    @Override
+  public void rename(String srcKey, String dstKey, boolean acquireLease,
+      SelfRenewingLease existingLease, boolean overwriteDestination) throws IOException {
 
     LOG.debug("Moving {} to {}", srcKey, dstKey);
 
@@ -2672,7 +2678,8 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
       // a more intensive exponential retry policy when the cluster is getting
       // throttled.
       try {
-        dstBlob.startCopyFromBlob(srcBlob, null, getInstrumentedContext());
+        dstBlob.startCopyFromBlob(srcBlob, null,
+            getInstrumentedContext(), overwriteDestination);
       } catch (StorageException se) {
         if (se.getHttpStatusCode() == HttpURLConnection.HTTP_UNAVAILABLE) {
           int copyBlobMinBackoff = sessionConfiguration.getInt(
@@ -2695,7 +2702,8 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
           options.setRetryPolicyFactory(new RetryExponentialRetry(
             copyBlobMinBackoff, copyBlobDeltaBackoff, copyBlobMaxBackoff,
             copyBlobMaxRetries));
-          dstBlob.startCopyFromBlob(srcBlob, options, getInstrumentedContext());
+          dstBlob.startCopyFromBlob(srcBlob, options,
+              getInstrumentedContext(), overwriteDestination);
         } else {
           throw se;
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52babbb4/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java
index 85a46ea..3d44b20 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java
@@ -3269,16 +3269,27 @@ public class NativeAzureFileSystem extends FileSystem {
     } else if (!srcMetadata.isDir()) {
       LOG.debug("Source {} found as a file, renaming.", src);
       try {
-        store.rename(srcKey, dstKey);
+        // HADOOP-15086 - file rename must ensure that the destination does
+        // not exist.  The fix is targeted to this call only to avoid
+        // regressions.  Other call sites are attempting to rename temporary
+        // files, redo a failed rename operation, or rename a directory
+        // recursively; for these cases the destination may exist.
+        store.rename(srcKey, dstKey, false, null,
+            false);
       } catch(IOException ex) {
-
         Throwable innerException = NativeAzureFileSystemHelper.checkForAzureStorageException(ex);
 
-        if (innerException instanceof StorageException
-            && NativeAzureFileSystemHelper.isFileNotFoundException((StorageException) innerException)) {
-
-          LOG.debug("BlobNotFoundException encountered. Failing rename", src);
-          return false;
+        if (innerException instanceof StorageException) {
+          if (NativeAzureFileSystemHelper.isFileNotFoundException(
+              (StorageException) innerException)) {
+            LOG.debug("BlobNotFoundException encountered. Failing rename", src);
+            return false;
+          }
+          if (NativeAzureFileSystemHelper.isBlobAlreadyExistsConflict(
+              (StorageException) innerException)) {
+            LOG.debug("Destination BlobAlreadyExists. Failing rename", src);
+            return false;
+          }
         }
 
         throw ex;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52babbb4/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystemHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystemHelper.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystemHelper.java
index 57af1f8..754f343 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystemHelper.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystemHelper.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.fs.azure;
 
 import java.io.EOFException;
 import java.io.IOException;
+import java.net.HttpURLConnection;
 import java.util.Map;
 
 import com.google.common.base.Preconditions;
@@ -96,6 +97,23 @@ final class NativeAzureFileSystemHelper {
   }
 
   /*
+   * Determines if a conditional request failed because the blob already
+   * exists.
+   *
+   * @param e - the storage exception thrown by the failed operation.
+   *
+   * @return true if a conditional request failed because the blob already
+   * exists; otherwise, returns false.
+   */
+  static boolean isBlobAlreadyExistsConflict(StorageException e) {
+    if (e.getHttpStatusCode() == HttpURLConnection.HTTP_CONFLICT
+        && StorageErrorCodeStrings.BLOB_ALREADY_EXISTS.equals(e.getErrorCode())) {
+      return true;
+    }
+    return false;
+  }
+
+  /*
    * Helper method that logs stack traces from all live threads.
    */
   public static void logAllLiveStackTraces() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52babbb4/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeFileSystemStore.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeFileSystemStore.java
index 57a729d..b67ab1b 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeFileSystemStore.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeFileSystemStore.java
@@ -91,6 +91,10 @@ interface NativeFileSystemStore {
   void rename(String srcKey, String dstKey, boolean acquireLease, SelfRenewingLease existingLease)
       throws IOException;
 
+  void rename(String srcKey, String dstKey, boolean acquireLease,
+              SelfRenewingLease existingLease, boolean overwriteDestination)
+      throws IOException;
+
   /**
    * Delete all keys with the given prefix. Used for testing.
    *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52babbb4/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SecureStorageInterfaceImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SecureStorageInterfaceImpl.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SecureStorageInterfaceImpl.java
index 7c2722e..0f54249 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SecureStorageInterfaceImpl.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/SecureStorageInterfaceImpl.java
@@ -503,10 +503,14 @@ public class SecureStorageInterfaceImpl extends StorageInterface {
 
     @Override
     public void startCopyFromBlob(CloudBlobWrapper sourceBlob, BlobRequestOptions options,
-        OperationContext opContext)
+        OperationContext opContext, boolean overwriteDestination)
             throws StorageException, URISyntaxException {
+      AccessCondition dstAccessCondition =
+          overwriteDestination
+              ? null
+              : AccessCondition.generateIfNotExistsCondition();
       getBlob().startCopy(sourceBlob.getBlob().getQualifiedUri(),
-          null, null, options, opContext);
+          null, dstAccessCondition, options, opContext);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52babbb4/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterface.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterface.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterface.java
index e03d731..dbb3849 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterface.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterface.java
@@ -406,7 +406,7 @@ abstract class StorageInterface {
      *
      */
     public abstract void startCopyFromBlob(CloudBlobWrapper sourceBlob,
-        BlobRequestOptions options, OperationContext opContext)
+        BlobRequestOptions options, OperationContext opContext, boolean overwriteDestination)
         throws StorageException, URISyntaxException;
     
     /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52babbb4/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterfaceImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterfaceImpl.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterfaceImpl.java
index 41a4dbb..e600f9e 100644
--- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterfaceImpl.java
+++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/StorageInterfaceImpl.java
@@ -425,10 +425,14 @@ class StorageInterfaceImpl extends StorageInterface {
 
     @Override
     public void startCopyFromBlob(CloudBlobWrapper sourceBlob, BlobRequestOptions options,
-        OperationContext opContext)
+        OperationContext opContext, boolean overwriteDestination)
             throws StorageException, URISyntaxException {
+      AccessCondition dstAccessCondition =
+          overwriteDestination
+              ? null
+              : AccessCondition.generateIfNotExistsCondition();
       getBlob().startCopy(sourceBlob.getBlob().getQualifiedUri(),
-          null, null, options, opContext);
+          null, dstAccessCondition, options, opContext);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52babbb4/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemLive.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemLive.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemLive.java
index f969968..9033674 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemLive.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemLive.java
@@ -18,8 +18,16 @@
 
 package org.apache.hadoop.fs.azure;
 
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
 import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
 
+import org.apache.hadoop.fs.contract.ContractTestUtils;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
@@ -40,6 +48,70 @@ public class ITestNativeAzureFileSystemLive extends
     return AzureBlobStorageTestAccount.create();
   }
 
+  /**
+   * Tests the rename file operation to ensure that when there are multiple
+   * attempts to rename a file to the same destination, only one rename
+   * operation is successful (HADOOP-15086).
+   */
+  @Test
+  public void testMultipleRenameFileOperationsToSameDestination()
+      throws IOException, InterruptedException {
+    final CountDownLatch latch = new CountDownLatch(1);
+    final AtomicInteger successfulRenameCount = new AtomicInteger(0);
+    final AtomicReference<IOException> unexpectedError = new AtomicReference<IOException>();
+    final Path dest = path("dest");
+
+    // Run 10 threads to rename multiple files to the same target path
+    List<Thread> threads = new ArrayList<>();
+
+    for (int i = 0; i < 10; i++) {
+      final int threadNumber = i;
+      Path src = path("test" + threadNumber);
+      threads.add(new Thread(() -> {
+        try {
+          latch.await(Long.MAX_VALUE, TimeUnit.SECONDS);
+        } catch (InterruptedException e) {
+        }
+        try {
+          try (OutputStream output = fs.create(src)) {
+            output.write(("Source file number " + threadNumber).getBytes());
+          }
+
+          if (fs.rename(src, dest)) {
+            LOG.info("rename succeeded for thread " + threadNumber);
+            successfulRenameCount.incrementAndGet();
+          }
+        } catch (IOException e) {
+          unexpectedError.compareAndSet(null, e);
+          ContractTestUtils.fail("Exception unexpected", e);
+        }
+      }));
+    }
+
+    // Start each thread
+    threads.forEach(t -> t.start());
+
+    // Wait for threads to start and wait on latch
+    Thread.sleep(2000);
+
+    // Now start to rename
+    latch.countDown();
+
+    // Wait for all threads to complete
+    threads.forEach(t -> {
+      try {
+        t.join();
+      } catch (InterruptedException e) {
+      }
+    });
+
+    if (unexpectedError.get() != null) {
+      throw unexpectedError.get();
+    }
+    assertEquals(1, successfulRenameCount.get());
+    LOG.info("Success, only one rename operation succeeded!");
+  }
+
   @Test
   public void testLazyRenamePendingCanOverwriteExistingFile()
     throws Exception {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/52babbb4/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/MockStorageInterface.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/MockStorageInterface.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/MockStorageInterface.java
index e0ae7b4..d5f6437 100644
--- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/MockStorageInterface.java
+++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/MockStorageInterface.java
@@ -425,7 +425,14 @@ public class MockStorageInterface extends StorageInterface {
 
     @Override
     public void startCopyFromBlob(CloudBlobWrapper sourceBlob, BlobRequestOptions options,
-        OperationContext opContext) throws StorageException, URISyntaxException {
+        OperationContext opContext, boolean overwriteDestination) throws StorageException, URISyntaxException {
+      if (!overwriteDestination && backingStore.exists(convertUriToDecodedString(uri))) {
+        throw new StorageException("BlobAlreadyExists",
+            "The blob already exists.",
+            HttpURLConnection.HTTP_CONFLICT,
+            null,
+            null);
+      }
       backingStore.copy(convertUriToDecodedString(sourceBlob.getUri()), convertUriToDecodedString(uri));
       //TODO: set the backingStore.properties.CopyState and
       //      update azureNativeFileSystemStore.waitForCopyToComplete


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


[20/50] [abbrv] hadoop git commit: HADOOP-15109. TestDFSIO -read -random doesn't work on file sized 4GB. Contributed by Ajay Kumar.

Posted by na...@apache.org.
HADOOP-15109. TestDFSIO -read -random doesn't work on file sized 4GB. Contributed by Ajay Kumar.


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

Branch: refs/heads/yarn-3409
Commit: c7a4dda3c5571e64c216810f8eb1a824c9b8f6f8
Parents: 811fabd
Author: Chen Liang <cl...@apache.org>
Authored: Mon Dec 18 13:25:47 2017 -0800
Committer: Chen Liang <cl...@apache.org>
Committed: Mon Dec 18 13:25:47 2017 -0800

----------------------------------------------------------------------
 .../src/test/java/org/apache/hadoop/fs/TestDFSIO.java     | 10 +++++-----
 1 file changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c7a4dda3/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/TestDFSIO.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/TestDFSIO.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/TestDFSIO.java
index 68befea..10709be 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/TestDFSIO.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/fs/TestDFSIO.java
@@ -31,8 +31,8 @@ import java.io.PrintStream;
 import java.text.DecimalFormat;
 import java.util.Collection;
 import java.util.Date;
-import java.util.Random;
 import java.util.StringTokenizer;
+import java.util.concurrent.ThreadLocalRandom;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
@@ -582,7 +582,7 @@ public class TestDFSIO implements Tool {
    * 3) Skip-read skips skipSize bytes after every read          : skipSize > 0
    */
   public static class RandomReadMapper extends IOStatMapper {
-    private Random rnd;
+    private ThreadLocalRandom rnd;
     private long fileSize;
     private long skipSize;
 
@@ -593,7 +593,7 @@ public class TestDFSIO implements Tool {
     }
 
     public RandomReadMapper() { 
-      rnd = new Random();
+      rnd = ThreadLocalRandom.current();
     }
 
     @Override // IOMapperBase
@@ -635,8 +635,8 @@ public class TestDFSIO implements Tool {
      * @return
      */
     private long nextOffset(long current) {
-      if(skipSize == 0)
-        return rnd.nextInt((int)(fileSize));
+      if (skipSize == 0)
+        return rnd.nextLong(fileSize);
       if(skipSize > 0)
         return (current < 0) ? 0 : (current + bufferSize + skipSize);
       // skipSize < 0


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


[35/50] [abbrv] hadoop git commit: YARN-7466. addendum patch for failing unit test. (Contributed by Chandni Singh)

Posted by na...@apache.org.
YARN-7466.  addendum patch for failing unit test.  (Contributed by Chandni Singh)


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

Branch: refs/heads/yarn-3409
Commit: 94a2ac6b719913aa698b66bf40b7ebbe6fa606da
Parents: 989c751
Author: Eric Yang <ey...@apache.org>
Authored: Tue Dec 19 18:42:27 2017 -0500
Committer: Eric Yang <ey...@apache.org>
Committed: Tue Dec 19 18:42:27 2017 -0500

----------------------------------------------------------------------
 .../java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/94a2ac6b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java
index a698ecf..12dfe18 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java
@@ -177,7 +177,7 @@ public class MockAM {
       List<ContainerId> releases, String labelExpression) throws Exception {
     List<ResourceRequest> reqs =
         createReq(new String[] { host }, memory, priority, numContainers,
-            labelExpression, 0L);
+            labelExpression, -1);
     return allocate(reqs, releases);
   }
   


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


[06/50] [abbrv] hadoop git commit: HDFS-11640. [READ] Datanodes should use a unique identifier when reading from external stores

Posted by na...@apache.org.
HDFS-11640. [READ] Datanodes should use a unique identifier when reading from external stores


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

Branch: refs/heads/yarn-3409
Commit: 4531588a94dcd2b4141b12828cb60ca3b953a58c
Parents: fb996a3
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Wed Dec 6 09:39:56 2017 -0800
Committer: Chris Douglas <cd...@apache.org>
Committed: Fri Dec 15 17:51:41 2017 -0800

----------------------------------------------------------------------
 .../hadoop/hdfs/server/common/FileRegion.java   |  7 ++-
 .../impl/TextFileRegionAliasMap.java            | 16 ++++--
 .../datanode/FinalizedProvidedReplica.java      | 20 ++++---
 .../hdfs/server/datanode/ProvidedReplica.java   | 34 ++++++++++--
 .../hdfs/server/datanode/ReplicaBuilder.java    | 12 ++++-
 .../fsdataset/impl/ProvidedVolumeImpl.java      |  9 ++++
 .../datanode/TestProvidedReplicaImpl.java       |  2 +-
 .../fsdataset/impl/TestProvidedImpl.java        | 57 ++++++++++++++++++++
 .../hadoop/hdfs/server/namenode/FSTreeWalk.java |  6 +--
 .../hdfs/server/namenode/ImageWriter.java       |  2 +-
 .../hadoop/hdfs/server/namenode/TreePath.java   | 40 ++++++++++----
 .../hdfs/server/namenode/RandomTreeWalk.java    |  6 +--
 12 files changed, 174 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4531588a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/FileRegion.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/FileRegion.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/FileRegion.java
index e6f0d0a..b605234 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/FileRegion.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/FileRegion.java
@@ -37,8 +37,13 @@ public class FileRegion implements BlockAlias {
 
   public FileRegion(long blockId, Path path, long offset,
       long length, long genStamp) {
+    this(blockId, path, offset, length, genStamp, new byte[0]);
+  }
+
+  public FileRegion(long blockId, Path path, long offset,
+                    long length, long genStamp, byte[] nonce) {
     this(new Block(blockId, length, genStamp),
-        new ProvidedStorageLocation(path, offset, length, new byte[0]));
+            new ProvidedStorageLocation(path, offset, length, nonce));
   }
 
   public FileRegion(long blockId, Path path, long offset, long length) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4531588a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TextFileRegionAliasMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TextFileRegionAliasMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TextFileRegionAliasMap.java
index 878a208..150371d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TextFileRegionAliasMap.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TextFileRegionAliasMap.java
@@ -26,6 +26,7 @@ import java.io.InputStream;
 import java.io.InputStreamReader;
 import java.io.OutputStream;
 import java.io.OutputStreamWriter;
+import java.nio.charset.Charset;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.Map;
@@ -353,11 +354,16 @@ public class TextFileRegionAliasMap
         return null;
       }
       String[] f = line.split(delim);
-      if (f.length != 5) {
+      if (f.length != 5 && f.length != 6) {
         throw new IOException("Invalid line: " + line);
       }
+      byte[] nonce = new byte[0];
+      if (f.length == 6) {
+        nonce = f[5].getBytes(Charset.forName("UTF-8"));
+      }
       return new FileRegion(Long.parseLong(f[0]), new Path(f[1]),
-          Long.parseLong(f[2]), Long.parseLong(f[3]), Long.parseLong(f[4]));
+          Long.parseLong(f[2]), Long.parseLong(f[3]), Long.parseLong(f[4]),
+          nonce);
     }
 
     public InputStream createStream() throws IOException {
@@ -442,7 +448,11 @@ public class TextFileRegionAliasMap
       out.append(psl.getPath().toString()).append(delim);
       out.append(Long.toString(psl.getOffset())).append(delim);
       out.append(Long.toString(psl.getLength())).append(delim);
-      out.append(Long.toString(block.getGenerationStamp())).append(delim);
+      out.append(Long.toString(block.getGenerationStamp()));
+      if (psl.getNonce().length > 0) {
+        out.append(delim)
+            .append(new String(psl.getNonce(), Charset.forName("UTF-8")));
+      }
       out.append("\n");
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4531588a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FinalizedProvidedReplica.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FinalizedProvidedReplica.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FinalizedProvidedReplica.java
index 43fd234..3818de2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FinalizedProvidedReplica.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FinalizedProvidedReplica.java
@@ -18,12 +18,15 @@
 package org.apache.hadoop.hdfs.server.datanode;
 
 import java.net.URI;
+import java.nio.ByteBuffer;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathHandle;
+import org.apache.hadoop.fs.RawPathHandle;
 import org.apache.hadoop.hdfs.server.common.FileRegion;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
@@ -36,11 +39,11 @@ import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
 @InterfaceStability.Unstable
 public class FinalizedProvidedReplica extends ProvidedReplica {
 
-  public FinalizedProvidedReplica(long blockId, URI fileURI,
-      long fileOffset, long blockLen, long genStamp,
-      FsVolumeSpi volume, Configuration conf, FileSystem remoteFS) {
-    super(blockId, fileURI, fileOffset, blockLen, genStamp, volume, conf,
-        remoteFS);
+  public FinalizedProvidedReplica(long blockId, URI fileURI, long fileOffset,
+      long blockLen, long genStamp, PathHandle pathHandle, FsVolumeSpi volume,
+      Configuration conf, FileSystem remoteFS) {
+    super(blockId, fileURI, fileOffset, blockLen, genStamp, pathHandle, volume,
+        conf, remoteFS);
   }
 
   public FinalizedProvidedReplica(FileRegion fileRegion, FsVolumeSpi volume,
@@ -50,14 +53,17 @@ public class FinalizedProvidedReplica extends ProvidedReplica {
         fileRegion.getProvidedStorageLocation().getOffset(),
         fileRegion.getBlock().getNumBytes(),
         fileRegion.getBlock().getGenerationStamp(),
+        new RawPathHandle(ByteBuffer
+            .wrap(fileRegion.getProvidedStorageLocation().getNonce())),
         volume, conf, remoteFS);
   }
 
   public FinalizedProvidedReplica(long blockId, Path pathPrefix,
       String pathSuffix, long fileOffset, long blockLen, long genStamp,
-      FsVolumeSpi volume, Configuration conf, FileSystem remoteFS) {
+      PathHandle pathHandle, FsVolumeSpi volume, Configuration conf,
+      FileSystem remoteFS) {
     super(blockId, pathPrefix, pathSuffix, fileOffset, blockLen,
-        genStamp, volume, conf, remoteFS);
+        genStamp, pathHandle, volume, conf, remoteFS);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4531588a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ProvidedReplica.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ProvidedReplica.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ProvidedReplica.java
index 3d20b00..5e02d4f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ProvidedReplica.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ProvidedReplica.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathHandle;
 import org.apache.hadoop.hdfs.server.common.FileRegion;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi.ScanInfo;
@@ -41,6 +42,9 @@ import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY;
+
 /**
  * This abstract class is used as a base class for provided replicas.
  */
@@ -60,6 +64,7 @@ public abstract class ProvidedReplica extends ReplicaInfo {
   private String pathSuffix;
   private long fileOffset;
   private Configuration conf;
+  private PathHandle pathHandle;
   private FileSystem remoteFS;
 
   /**
@@ -75,12 +80,13 @@ public abstract class ProvidedReplica extends ReplicaInfo {
    * @param remoteFS reference to the remote filesystem to use for this replica.
    */
   public ProvidedReplica(long blockId, URI fileURI, long fileOffset,
-      long blockLen, long genStamp, FsVolumeSpi volume, Configuration conf,
-      FileSystem remoteFS) {
+      long blockLen, long genStamp, PathHandle pathHandle, FsVolumeSpi volume,
+      Configuration conf, FileSystem remoteFS) {
     super(volume, blockId, blockLen, genStamp);
     this.fileURI = fileURI;
     this.fileOffset = fileOffset;
     this.conf = conf;
+    this.pathHandle = pathHandle;
     if (remoteFS != null) {
       this.remoteFS = remoteFS;
     } else {
@@ -114,14 +120,15 @@ public abstract class ProvidedReplica extends ReplicaInfo {
    * @param remoteFS reference to the remote filesystem to use for this replica.
    */
   public ProvidedReplica(long blockId, Path pathPrefix, String pathSuffix,
-      long fileOffset, long blockLen, long genStamp, FsVolumeSpi volume,
-      Configuration conf, FileSystem remoteFS) {
+      long fileOffset, long blockLen, long genStamp, PathHandle pathHandle,
+      FsVolumeSpi volume, Configuration conf, FileSystem remoteFS) {
     super(volume, blockId, blockLen, genStamp);
     this.fileURI = null;
     this.pathPrefix = pathPrefix;
     this.pathSuffix = pathSuffix;
     this.fileOffset = fileOffset;
     this.conf = conf;
+    this.pathHandle = pathHandle;
     if (remoteFS != null) {
       this.remoteFS = remoteFS;
     } else {
@@ -142,6 +149,7 @@ public abstract class ProvidedReplica extends ReplicaInfo {
     this.fileOffset = r.fileOffset;
     this.conf = r.conf;
     this.remoteFS = r.remoteFS;
+    this.pathHandle = r.pathHandle;
     this.pathPrefix = r.pathPrefix;
     this.pathSuffix = r.pathSuffix;
   }
@@ -174,7 +182,18 @@ public abstract class ProvidedReplica extends ReplicaInfo {
   @Override
   public InputStream getDataInputStream(long seekOffset) throws IOException {
     if (remoteFS != null) {
-      FSDataInputStream ins = remoteFS.open(new Path(getRemoteURI()));
+      FSDataInputStream ins;
+      try {
+        if (pathHandle != null) {
+          ins = remoteFS.open(pathHandle, conf.getInt(IO_FILE_BUFFER_SIZE_KEY,
+              IO_FILE_BUFFER_SIZE_DEFAULT));
+        } else {
+          ins = remoteFS.open(new Path(getRemoteURI()));
+        }
+      } catch (UnsupportedOperationException e) {
+        throw new IOException("PathHandle specified, but unsuported", e);
+      }
+
       ins.seek(fileOffset + seekOffset);
       return new BoundedInputStream(
           new FSDataInputStream(ins), getBlockDataLength());
@@ -324,4 +343,9 @@ public abstract class ProvidedReplica extends ReplicaInfo {
     throw new UnsupportedOperationException(
         "ProvidedReplica does not yet support copy data");
   }
+
+  @VisibleForTesting
+  public void setPathHandle(PathHandle pathHandle) {
+    this.pathHandle = pathHandle;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4531588a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaBuilder.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaBuilder.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaBuilder.java
index 8748918..2c55e73 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaBuilder.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaBuilder.java
@@ -22,6 +22,7 @@ import java.net.URI;
 
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathHandle;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.StorageType;
@@ -53,6 +54,7 @@ public class ReplicaBuilder {
   private Configuration conf;
   private FileRegion fileRegion;
   private FileSystem remoteFS;
+  private PathHandle pathHandle;
   private String pathSuffix;
   private Path pathPrefix;
 
@@ -66,6 +68,7 @@ public class ReplicaBuilder {
     fromReplica = null;
     uri = null;
     this.state = state;
+    pathHandle = null;
   }
 
   public ReplicaBuilder setState(ReplicaState state) {
@@ -170,6 +173,11 @@ public class ReplicaBuilder {
     return this;
   }
 
+  public ReplicaBuilder setPathHandle(PathHandle pathHandle) {
+    this.pathHandle = pathHandle;
+    return this;
+  }
+
   public LocalReplicaInPipeline buildLocalReplicaInPipeline()
       throws IllegalArgumentException {
     LocalReplicaInPipeline info = null;
@@ -309,10 +317,10 @@ public class ReplicaBuilder {
     if (fileRegion == null) {
       if (uri != null) {
         info = new FinalizedProvidedReplica(blockId, uri, offset,
-            length, genStamp, volume, conf, remoteFS);
+            length, genStamp, pathHandle, volume, conf, remoteFS);
       } else {
         info = new FinalizedProvidedReplica(blockId, pathPrefix, pathSuffix,
-            offset, length, genStamp, volume, conf, remoteFS);
+            offset, length, genStamp, pathHandle, volume, conf, remoteFS);
       }
     } else {
       info = new FinalizedProvidedReplica(fileRegion, volume, conf, remoteFS);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4531588a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeImpl.java
index bab788b..f65fbbc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeImpl.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
 import java.io.File;
 import java.io.IOException;
 import java.net.URI;
+import java.nio.ByteBuffer;
 import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.Map;
@@ -32,6 +33,8 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathHandle;
+import org.apache.hadoop.fs.RawPathHandle;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.Block;
@@ -180,6 +183,11 @@ class ProvidedVolumeImpl extends FsVolumeImpl {
             region.getProvidedStorageLocation().getPath().toUri())) {
           String blockSuffix = getSuffix(blockPrefixPath,
               new Path(region.getProvidedStorageLocation().getPath().toUri()));
+          PathHandle pathHandle = null;
+          if (region.getProvidedStorageLocation().getNonce().length > 0) {
+            pathHandle = new RawPathHandle(ByteBuffer
+                .wrap(region.getProvidedStorageLocation().getNonce()));
+          }
           ReplicaInfo newReplica = new ReplicaBuilder(ReplicaState.FINALIZED)
               .setBlockId(region.getBlock().getBlockId())
               .setPathPrefix(blockPrefixPath)
@@ -187,6 +195,7 @@ class ProvidedVolumeImpl extends FsVolumeImpl {
               .setOffset(region.getProvidedStorageLocation().getOffset())
               .setLength(region.getBlock().getNumBytes())
               .setGenerationStamp(region.getBlock().getGenerationStamp())
+              .setPathHandle(pathHandle)
               .setFsVolume(providedVolume)
               .setConf(conf)
               .setRemoteFS(remoteFS)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4531588a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestProvidedReplicaImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestProvidedReplicaImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestProvidedReplicaImpl.java
index 967e94d..210be6e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestProvidedReplicaImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestProvidedReplicaImpl.java
@@ -87,7 +87,7 @@ public class TestProvidedReplicaImpl {
           FILE_LEN >= (i+1)*BLK_LEN ? BLK_LEN : FILE_LEN - i*BLK_LEN;
       replicas.add(
           new FinalizedProvidedReplica(i, providedFile.toURI(), i*BLK_LEN,
-          currentReplicaLength, 0, null, conf, null));
+          currentReplicaLength, 0, null, null, conf, null));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4531588a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestProvidedImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestProvidedImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestProvidedImpl.java
index 1a89f76..e057022 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestProvidedImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestProvidedImpl.java
@@ -18,8 +18,10 @@
 package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
 
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_KEY;
+import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -44,14 +46,20 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Optional;
+import java.util.Random;
 import java.util.Set;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystemTestHelper;
+import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathHandle;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@@ -63,6 +71,7 @@ import org.apache.hadoop.hdfs.server.datanode.DNConf;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataStorage;
 import org.apache.hadoop.hdfs.server.datanode.DirectoryScanner;
+import org.apache.hadoop.hdfs.server.datanode.FinalizedProvidedReplica;
 import org.apache.hadoop.hdfs.server.datanode.ProvidedReplica;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
 import org.apache.hadoop.hdfs.server.datanode.ShortCircuitRegistry;
@@ -71,6 +80,7 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi.BlockIterator;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi.FsVolumeReferences;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.AutoCloseableLock;
 import org.apache.hadoop.util.StringUtils;
 import org.junit.Before;
@@ -619,4 +629,51 @@ public class TestProvidedImpl {
     assertEquals(0, report.get(BLOCK_POOL_IDS[CHOSEN_BP_ID]).length);
   }
 
+  /**
+   * Tests that a ProvidedReplica supports path handles.
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testProvidedReplicaWithPathHandle() throws Exception {
+
+    Configuration conf = new Configuration();
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
+    cluster.waitActive();
+
+    DistributedFileSystem fs = cluster.getFileSystem();
+
+    // generate random data
+    int chunkSize = 512;
+    Random r = new Random(12345L);
+    byte[] data = new byte[chunkSize];
+    r.nextBytes(data);
+
+    Path file = new Path("/testfile");
+    try (FSDataOutputStream fout = fs.create(file)) {
+      fout.write(data);
+    }
+
+    PathHandle pathHandle = fs.getPathHandle(fs.getFileStatus(file),
+        Options.HandleOpt.changed(true), Options.HandleOpt.moved(true));
+    FinalizedProvidedReplica replica = new FinalizedProvidedReplica(0,
+        file.toUri(), 0, chunkSize, 0, pathHandle, null, conf, fs);
+    byte[] content = new byte[chunkSize];
+    IOUtils.readFully(replica.getDataInputStream(0), content, 0, chunkSize);
+    assertArrayEquals(data, content);
+
+    fs.rename(file, new Path("/testfile.1"));
+    // read should continue succeeding after the rename operation
+    IOUtils.readFully(replica.getDataInputStream(0), content, 0, chunkSize);
+    assertArrayEquals(data, content);
+
+    replica.setPathHandle(null);
+    try {
+      // expected to fail as URI of the provided replica is no longer valid.
+      replica.getDataInputStream(0);
+      fail("Expected an exception");
+    } catch (IOException e) {
+      LOG.info("Expected exception " + e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4531588a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSTreeWalk.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSTreeWalk.java b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSTreeWalk.java
index 7d66c64..2d86503 100644
--- a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSTreeWalk.java
+++ b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSTreeWalk.java
@@ -55,7 +55,7 @@ public class FSTreeWalk extends TreeWalk {
     try {
       ArrayList<TreePath> ret = new ArrayList<>();
       for (FileStatus s : fs.listStatus(path.getFileStatus().getPath())) {
-        ret.add(new TreePath(s, id, i));
+        ret.add(new TreePath(s, id, i, fs));
       }
       return ret;
     } catch (FileNotFoundException e) {
@@ -72,13 +72,13 @@ public class FSTreeWalk extends TreeWalk {
 
     FSTreeIterator(TreePath p) {
       getPendingQueue().addFirst(
-          new TreePath(p.getFileStatus(), p.getParentId(), this));
+          new TreePath(p.getFileStatus(), p.getParentId(), this, fs));
     }
 
     FSTreeIterator(Path p) throws IOException {
       try {
         FileStatus s = fs.getFileStatus(root);
-        getPendingQueue().addFirst(new TreePath(s, -1L, this));
+        getPendingQueue().addFirst(new TreePath(s, -1L, this, fs));
       } catch (FileNotFoundException e) {
         if (p.equals(root)) {
           throw e;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4531588a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageWriter.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageWriter.java b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageWriter.java
index 282429a..0abc7a7 100644
--- a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageWriter.java
+++ b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageWriter.java
@@ -208,7 +208,7 @@ public class ImageWriter implements Closeable {
     long id = curInode.getAndIncrement();
     e.accept(id);
     assert e.getId() < curInode.get();
-    INode n = e.toINode(ugis, blockIds, blocks, blockPoolID);
+    INode n = e.toINode(ugis, blockIds, blocks);
     writeInode(n);
 
     if (e.getParentId() > 0) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4531588a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/TreePath.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/TreePath.java b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/TreePath.java
index aca1220..dde351f 100644
--- a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/TreePath.java
+++ b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/TreePath.java
@@ -24,6 +24,9 @@ import com.google.protobuf.ByteString;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Options;
+import org.apache.hadoop.fs.PathHandle;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
 import org.apache.hadoop.hdfs.server.common.FileRegion;
@@ -31,6 +34,8 @@ import org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.INode;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.INodeDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.INodeFile;
+
+import static org.apache.hadoop.hdfs.DFSUtil.LOG;
 import static org.apache.hadoop.hdfs.DFSUtil.string2Bytes;
 import static org.apache.hadoop.hdfs.server.namenode.DirectoryWithQuotaFeature.DEFAULT_NAMESPACE_QUOTA;
 import static org.apache.hadoop.hdfs.server.namenode.DirectoryWithQuotaFeature.DEFAULT_STORAGE_SPACE_QUOTA;
@@ -46,11 +51,14 @@ public class TreePath {
   private final long parentId;
   private final FileStatus stat;
   private final TreeWalk.TreeIterator i;
+  private final FileSystem fs;
 
-  protected TreePath(FileStatus stat, long parentId, TreeWalk.TreeIterator i) {
+  protected TreePath(FileStatus stat, long parentId, TreeWalk.TreeIterator i,
+      FileSystem fs) {
     this.i = i;
     this.stat = stat;
     this.parentId = parentId;
+    this.fs = fs;
   }
 
   public FileStatus getFileStatus() {
@@ -74,10 +82,9 @@ public class TreePath {
   }
 
   public INode toINode(UGIResolver ugi, BlockResolver blk,
-                       BlockAliasMap.Writer<FileRegion> out, String blockPoolID)
-          throws IOException {
+      BlockAliasMap.Writer<FileRegion> out) throws IOException {
     if (stat.isFile()) {
-      return toFile(ugi, blk, out, blockPoolID);
+      return toFile(ugi, blk, out);
     } else if (stat.isDirectory()) {
       return toDirectory(ugi);
     } else if (stat.isSymlink()) {
@@ -103,16 +110,16 @@ public class TreePath {
     return (int)(pId ^ (pId >>> 32));
   }
 
-  void writeBlock(long blockId, long offset, long length,
-      long genStamp, String blockPoolID,
-      BlockAliasMap.Writer<FileRegion> out) throws IOException {
+  void writeBlock(long blockId, long offset, long length, long genStamp,
+      PathHandle pathHandle, BlockAliasMap.Writer<FileRegion> out)
+      throws IOException {
     FileStatus s = getFileStatus();
-    out.store(new FileRegion(blockId, s.getPath(), offset, length, genStamp));
+    out.store(new FileRegion(blockId, s.getPath(), offset, length, genStamp,
+        (pathHandle != null ? pathHandle.toByteArray() : new byte[0])));
   }
 
   INode toFile(UGIResolver ugi, BlockResolver blk,
-               BlockAliasMap.Writer<FileRegion> out, String blockPoolID)
-          throws IOException {
+      BlockAliasMap.Writer<FileRegion> out) throws IOException {
     final FileStatus s = getFileStatus();
     // TODO should this store resolver's user/group?
     ugi.addUser(s.getOwner());
@@ -124,12 +131,23 @@ public class TreePath {
         .setPreferredBlockSize(blk.preferredBlockSize(s))
         .setPermission(ugi.resolve(s))
         .setStoragePolicyID(HdfsConstants.PROVIDED_STORAGE_POLICY_ID);
+
+    // pathhandle allows match as long as the file matches exactly.
+    PathHandle pathHandle = null;
+    if (fs != null) {
+      try {
+        pathHandle = fs.getPathHandle(s, Options.HandleOpt.exact());
+      } catch (UnsupportedOperationException e) {
+        LOG.warn(
+            "Exact path handle not supported by filesystem " + fs.toString());
+      }
+    }
     //TODO: storage policy should be configurable per path; use BlockResolver
     long off = 0L;
     for (BlockProto block : blk.resolve(s)) {
       b.addBlocks(block);
       writeBlock(block.getBlockId(), off, block.getNumBytes(),
-          block.getGenStamp(), blockPoolID, out);
+          block.getGenStamp(), pathHandle, out);
       off += block.getNumBytes();
     }
     INode.Builder ib = INode.newBuilder()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4531588a/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/RandomTreeWalk.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/RandomTreeWalk.java b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/RandomTreeWalk.java
index 27152fd..c9d109a 100644
--- a/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/RandomTreeWalk.java
+++ b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/RandomTreeWalk.java
@@ -97,7 +97,7 @@ public class RandomTreeWalk extends TreeWalk {
     int nChildren = r.nextInt(children);
     ArrayList<TreePath> ret = new ArrayList<TreePath>();
     for (int i = 0; i < nChildren; ++i) {
-      ret.add(new TreePath(genFileStatus(p, r), p.getId(), walk));
+      ret.add(new TreePath(genFileStatus(p, r), p.getId(), walk, null));
     }
     return ret;
   }
@@ -165,12 +165,12 @@ public class RandomTreeWalk extends TreeWalk {
     RandomTreeIterator(long seed) {
       Random r = new Random(seed);
       FileStatus iroot = genFileStatus(null, r);
-      getPendingQueue().addFirst(new TreePath(iroot, -1, this));
+      getPendingQueue().addFirst(new TreePath(iroot, -1, this, null));
     }
 
     RandomTreeIterator(TreePath p) {
       getPendingQueue().addFirst(
-          new TreePath(p.getFileStatus(), p.getParentId(), this));
+          new TreePath(p.getFileStatus(), p.getParentId(), this, null));
     }
 
     @Override


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


[42/50] [abbrv] hadoop git commit: HDFS-12949. Fix findbugs warning in ImageWriter.java.

Posted by na...@apache.org.
HDFS-12949. Fix findbugs warning in ImageWriter.java.


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

Branch: refs/heads/yarn-3409
Commit: 5ab632baf52f0ecc737845051b382f68bf1385bb
Parents: 382215c
Author: Akira Ajisaka <aa...@apache.org>
Authored: Thu Dec 21 10:04:34 2017 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Thu Dec 21 10:04:34 2017 +0900

----------------------------------------------------------------------
 hadoop-tools/hadoop-fs2img/pom.xml | 10 ++++++++++
 1 file changed, 10 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5ab632ba/hadoop-tools/hadoop-fs2img/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/pom.xml b/hadoop-tools/hadoop-fs2img/pom.xml
index 2e3e66a..1ae17dc 100644
--- a/hadoop-tools/hadoop-fs2img/pom.xml
+++ b/hadoop-tools/hadoop-fs2img/pom.xml
@@ -87,6 +87,16 @@
          </archive>
         </configuration>
        </plugin>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>findbugs-maven-plugin</artifactId>
+        <configuration>
+          <findbugsXmlOutput>true</findbugsXmlOutput>
+          <xmlOutput>true</xmlOutput>
+          <excludeFilterFile>${basedir}/dev-support/findbugs-exclude.xml</excludeFilterFile>
+          <effort>Max</effort>
+        </configuration>
+      </plugin>
     </plugins>
   </build>
 


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


[18/50] [abbrv] hadoop git commit: HADOOP-13974. S3Guard CLI to support list/purge of pending multipart commits. Contributed by Aaron Fabbri

Posted by na...@apache.org.
HADOOP-13974. S3Guard CLI to support list/purge of pending multipart commits.
Contributed by Aaron Fabbri


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

Branch: refs/heads/yarn-3409
Commit: 35ad9b1dd279b769381ea1625d9bf776c309c5cb
Parents: 94576b1
Author: Steve Loughran <st...@apache.org>
Authored: Mon Dec 18 21:18:52 2017 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Mon Dec 18 21:19:06 2017 +0000

----------------------------------------------------------------------
 .../java/org/apache/hadoop/security/KDiag.java  |  30 +-
 .../java/org/apache/hadoop/fs/s3a/Invoker.java  |   7 +-
 .../apache/hadoop/fs/s3a/MultipartUtils.java    | 214 ++++++++++++++
 .../org/apache/hadoop/fs/s3a/S3AFileSystem.java |  30 +-
 .../java/org/apache/hadoop/fs/s3a/S3AUtils.java |   3 +-
 .../hadoop/fs/s3a/WriteOperationHelper.java     |   5 +-
 .../hadoop/fs/s3a/commit/CommitOperations.java  |   2 +-
 .../fs/s3a/commit/MagicCommitIntegration.java   |   2 +-
 .../hadoop/fs/s3a/s3guard/S3GuardTool.java      | 287 +++++++++++++++++--
 .../src/site/markdown/tools/hadoop-aws/index.md |   7 +-
 .../site/markdown/tools/hadoop-aws/s3guard.md   |  35 ++-
 .../hadoop/fs/s3a/ITestS3AMultipartUtils.java   | 126 ++++++++
 .../apache/hadoop/fs/s3a/MockS3AFileSystem.java |   7 +
 .../hadoop/fs/s3a/MultipartTestUtils.java       | 184 ++++++++++++
 .../org/apache/hadoop/fs/s3a/S3ATestUtils.java  |  21 +-
 .../fs/s3a/commit/AbstractCommitITest.java      |   3 +-
 .../commit/magic/ITestS3AHugeMagicCommits.java  |   2 +-
 .../fs/s3a/s3guard/ITestS3GuardToolLocal.java   | 187 ++++++++++++
 18 files changed, 1082 insertions(+), 70 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/35ad9b1d/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/KDiag.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/KDiag.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/KDiag.java
index c8d0b33..b4e535c 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/KDiag.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/KDiag.java
@@ -81,6 +81,11 @@ public class KDiag extends Configured implements Tool, Closeable {
    * variable. This is what kinit will use by default: {@value}
    */
   public static final String KRB5_CCNAME = "KRB5CCNAME";
+  /**
+   * Location of main kerberos configuration file as passed down via an
+   * environment variable.
+   */
+  public static final String KRB5_CONFIG = "KRB5_CONFIG";
   public static final String JAVA_SECURITY_KRB5_CONF
     = "java.security.krb5.conf";
   public static final String JAVA_SECURITY_KRB5_REALM
@@ -321,14 +326,15 @@ public class KDiag extends Configured implements Tool, Closeable {
 
     title("Environment Variables");
     for (String env : new String[]{
-      HADOOP_JAAS_DEBUG,
-      KRB5_CCNAME,
-      HADOOP_USER_NAME,
-      HADOOP_PROXY_USER,
-      HADOOP_TOKEN_FILE_LOCATION,
-      "HADOOP_SECURE_LOG",
-      "HADOOP_OPTS",
-      "HADOOP_CLIENT_OPTS",
+        HADOOP_JAAS_DEBUG,
+        KRB5_CCNAME,
+        KRB5_CONFIG,
+        HADOOP_USER_NAME,
+        HADOOP_PROXY_USER,
+        HADOOP_TOKEN_FILE_LOCATION,
+        "HADOOP_SECURE_LOG",
+        "HADOOP_OPTS",
+        "HADOOP_CLIENT_OPTS",
     }) {
       printEnv(env);
     }
@@ -562,14 +568,14 @@ public class KDiag extends Configured implements Tool, Closeable {
         krbPath = jvmKrbPath;
       }
 
-      String krb5name = System.getenv(KRB5_CCNAME);
+      String krb5name = System.getenv(KRB5_CONFIG);
       if (krb5name != null) {
         println("Setting kerberos path from environment variable %s: \"%s\"",
-          KRB5_CCNAME, krb5name);
+            KRB5_CONFIG, krb5name);
         krbPath = krb5name;
         if (jvmKrbPath != null) {
           println("Warning - both %s and %s were set - %s takes priority",
-            JAVA_SECURITY_KRB5_CONF, KRB5_CCNAME, KRB5_CCNAME);
+              JAVA_SECURITY_KRB5_CONF, KRB5_CONFIG, KRB5_CONFIG);
         }
       }
 
@@ -919,7 +925,7 @@ public class KDiag extends Configured implements Tool, Closeable {
   private void dump(File file) throws IOException {
     try (FileInputStream in = new FileInputStream(file)) {
       for (String line : IOUtils.readLines(in)) {
-        println(line);
+        println("%s", line);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35ad9b1d/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Invoker.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Invoker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Invoker.java
index 107a247..875948e 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Invoker.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Invoker.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.fs.s3a;
 import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.util.Optional;
+import javax.annotation.Nullable;
 
 import com.amazonaws.AmazonClientException;
 import com.amazonaws.SdkBaseException;
@@ -222,7 +223,7 @@ public class Invoker {
    */
   @Retries.RetryTranslated
   public <T> T retry(String action,
-      String path,
+      @Nullable String path,
       boolean idempotent,
       Operation<T> operation)
       throws IOException {
@@ -247,7 +248,7 @@ public class Invoker {
   @Retries.RetryTranslated
   public <T> T retry(
       String action,
-      String path,
+      @Nullable String path,
       boolean idempotent,
       Retried retrying,
       Operation<T> operation)
@@ -413,7 +414,7 @@ public class Invoker {
    * @param path path (may be null or empty)
    * @return string for logs
    */
-  private static String toDescription(String action, String path) {
+  private static String toDescription(String action, @Nullable String path) {
     return action +
         (StringUtils.isNotEmpty(path) ? (" on " + path) : "");
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35ad9b1d/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/MultipartUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/MultipartUtils.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/MultipartUtils.java
new file mode 100644
index 0000000..6eb490f
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/MultipartUtils.java
@@ -0,0 +1,214 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a;
+
+import java.io.IOException;
+import java.util.ListIterator;
+import java.util.NoSuchElementException;
+import javax.annotation.Nullable;
+
+import com.amazonaws.services.s3.AmazonS3;
+import com.amazonaws.services.s3.model.ListMultipartUploadsRequest;
+import com.amazonaws.services.s3.model.MultipartUpload;
+import com.amazonaws.services.s3.model.MultipartUploadListing;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.RemoteIterator;
+
+
+/**
+ * MultipartUtils upload-specific functions for use by S3AFileSystem and Hadoop
+ * CLI.
+ */
+public final class MultipartUtils {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(MultipartUtils.class);
+
+  /** Not instantiated. */
+  private MultipartUtils() { }
+
+  /**
+   * List outstanding multipart uploads.
+   * Package private: S3AFileSystem and tests are the users of this.
+   * @param s3 AmazonS3 client to use.
+   * @param bucketName name of S3 bucket to use.
+   * @param maxKeys maximum batch size to request at a time from S3.
+   * @param prefix optional key prefix to narrow search.  If null then whole
+   *               bucket will be searched.
+   * @return an iterator of matching uploads
+   */
+  static MultipartUtils.UploadIterator listMultipartUploads(AmazonS3 s3,
+      Invoker invoker, String bucketName, int maxKeys, @Nullable String prefix)
+      throws IOException {
+    return new MultipartUtils.UploadIterator(s3, invoker, bucketName, maxKeys,
+        prefix);
+  }
+
+  /**
+   * Simple RemoteIterator wrapper for AWS `listMultipartUpload` API.
+   * Iterates over batches of multipart upload metadata listings.
+   */
+  static class ListingIterator implements
+      RemoteIterator<MultipartUploadListing> {
+
+    private final String bucketName;
+    private final String prefix;
+    private final int maxKeys;
+    private final AmazonS3 s3;
+    private final Invoker invoker;
+
+    /**
+     * Most recent listing results.
+     */
+    private MultipartUploadListing listing;
+
+    /**
+     * Indicator that this is the first listing.
+     */
+    private boolean firstListing = true;
+
+    private int listCount = 1;
+
+    ListingIterator(AmazonS3 s3, Invoker invoker, String bucketName,
+        int maxKeys, @Nullable String prefix) throws IOException {
+      this.s3 = s3;
+      this.bucketName = bucketName;
+      this.maxKeys = maxKeys;
+      this.prefix = prefix;
+      this.invoker = invoker;
+
+      requestNextBatch();
+    }
+
+    /**
+     * Iterator has data if it is either is the initial iteration, or
+     * the last listing obtained was incomplete.
+     * @throws IOException not thrown by this implementation.
+     */
+    @Override
+    public boolean hasNext() throws IOException {
+      if (listing == null) {
+        // shouldn't happen, but don't trust AWS SDK
+        return false;
+      } else {
+        return firstListing || listing.isTruncated();
+      }
+    }
+
+    /**
+     * Get next listing. First call, this returns initial set (possibly
+     * empty) obtained from S3. Subsequent calls my block on I/O or fail.
+     * @return next upload listing.
+     * @throws IOException if S3 operation fails.
+     * @throws NoSuchElementException if there are no more uploads.
+     */
+    @Override
+    @Retries.RetryTranslated
+    public MultipartUploadListing next() throws IOException {
+      if (firstListing) {
+        firstListing = false;
+      } else {
+        if (listing == null || !listing.isTruncated()) {
+          // nothing more to request: fail.
+          throw new NoSuchElementException("No more uploads under " + prefix);
+        }
+        // need to request a new set of objects.
+        requestNextBatch();
+      }
+      return listing;
+    }
+
+    @Override
+    public String toString() {
+      return "Upload iterator: prefix " + prefix + "; list count " +
+          listCount + "; isTruncated=" + listing.isTruncated();
+    }
+
+    @Retries.RetryTranslated
+    private void requestNextBatch() throws IOException {
+      ListMultipartUploadsRequest req =
+          new ListMultipartUploadsRequest(bucketName);
+      if (prefix != null) {
+        req.setPrefix(prefix);
+      }
+      if (!firstListing) {
+        req.setKeyMarker(listing.getNextKeyMarker());
+        req.setUploadIdMarker(listing.getNextUploadIdMarker());
+      }
+      req.setMaxUploads(listCount);
+
+      LOG.debug("[{}], Requesting next {} uploads prefix {}, " +
+          "next key {}, next upload id {}", listCount, maxKeys, prefix,
+          req.getKeyMarker(), req.getUploadIdMarker());
+      listCount++;
+
+      listing = invoker.retry("listMultipartUploads", prefix, true,
+          () -> s3.listMultipartUploads(req));
+      LOG.debug("New listing state: {}", this);
+    }
+  }
+
+  /**
+   * Iterator over multipart uploads. Similar to
+   * {@link org.apache.hadoop.fs.s3a.Listing.FileStatusListingIterator}, but
+   * iterates over pending uploads instead of existing objects.
+   */
+  public static class UploadIterator
+      implements RemoteIterator<MultipartUpload> {
+
+    private ListingIterator lister;
+    /** Current listing: the last upload listing we fetched. */
+    private MultipartUploadListing listing;
+    /** Iterator over the current listing. */
+    private ListIterator<MultipartUpload> batchIterator;
+
+    @Retries.RetryTranslated
+    public UploadIterator(AmazonS3 s3, Invoker invoker, String bucketName,
+        int maxKeys, @Nullable String prefix)
+        throws IOException {
+
+      lister = new ListingIterator(s3, invoker, bucketName, maxKeys, prefix);
+      requestNextBatch();
+    }
+
+    @Override
+    public boolean hasNext() throws IOException {
+      return (batchIterator.hasNext() || requestNextBatch());
+    }
+
+    @Override
+    public MultipartUpload next() throws IOException {
+      if (!hasNext())  {
+        throw new NoSuchElementException();
+      }
+      return batchIterator.next();
+    }
+
+    private boolean requestNextBatch() throws IOException {
+      if (lister.hasNext()) {
+        listing = lister.next();
+        batchIterator = listing.getMultipartUploads().listIterator();
+        return batchIterator.hasNext();
+      }
+      return false;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35ad9b1d/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
index e927758..9431f17 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
@@ -43,6 +43,7 @@ import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
+import javax.annotation.Nullable;
 
 import com.amazonaws.AmazonClientException;
 import com.amazonaws.AmazonServiceException;
@@ -194,6 +195,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
   private String blockOutputBuffer;
   private S3ADataBlocks.BlockFactory blockFactory;
   private int blockOutputActiveBlocks;
+  private WriteOperationHelper writeHelper;
   private boolean useListV1;
   private MagicCommitIntegration committerIntegration;
 
@@ -247,6 +249,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
       s3 = ReflectionUtils.newInstance(s3ClientFactoryClass, conf)
           .createS3Client(name);
       invoker = new Invoker(new S3ARetryPolicy(getConf()), onRetry);
+      writeHelper = new WriteOperationHelper(this, getConf());
 
       maxKeys = intOption(conf, MAX_PAGING_KEYS, DEFAULT_MAX_PAGING_KEYS, 1);
       listing = new Listing(this);
@@ -753,13 +756,13 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
             partSize,
             blockFactory,
             instrumentation.newOutputStreamStatistics(statistics),
-            createWriteOperationHelper(),
+            getWriteOperationHelper(),
             putTracker),
         null);
   }
 
   /**
-   * Create a new {@code WriteOperationHelper} instance.
+   * Get a {@code WriteOperationHelper} instance.
    *
    * This class permits other low-level operations against the store.
    * It is unstable and
@@ -768,8 +771,8 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
    * @return a new helper.
    */
   @InterfaceAudience.Private
-  public WriteOperationHelper createWriteOperationHelper() {
-    return new WriteOperationHelper(this);
+  public WriteOperationHelper getWriteOperationHelper() {
+    return writeHelper;
   }
 
   /**
@@ -3079,7 +3082,25 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
   }
 
   /**
+   * List any pending multipart uploads whose keys begin with prefix, using
+   * an iterator that can handle an unlimited number of entries.
+   * See {@link #listMultipartUploads(String)} for a non-iterator version of
+   * this.
+   *
+   * @param prefix optional key prefix to search
+   * @return Iterator over multipart uploads.
+   * @throws IOException on failure
+   */
+  public MultipartUtils.UploadIterator listUploads(@Nullable String prefix)
+      throws IOException {
+    return MultipartUtils.listMultipartUploads(s3, invoker, bucket, maxKeys,
+        prefix);
+  }
+
+  /**
    * Listing all multipart uploads; limited to the first few hundred.
+   * See {@link #listUploads(String)} for an iterator-based version that does
+   * not limit the number of entries returned.
    * Retry policy: retry, translated.
    * @return a listing of multipart uploads.
    * @param prefix prefix to scan for, "" for none
@@ -3166,5 +3187,4 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
       return false;
     }
   }
-
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35ad9b1d/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
index 23ceafa..2457217 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
@@ -51,6 +51,7 @@ import com.google.common.collect.Lists;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import javax.annotation.Nullable;
 import java.io.EOFException;
 import java.io.FileNotFoundException;
 import java.io.IOException;
@@ -149,7 +150,7 @@ public final class S3AUtils {
    * @return an IOE which wraps the caught exception.
    */
   @SuppressWarnings("ThrowableInstanceNeverThrown")
-  public static IOException translateException(String operation,
+  public static IOException translateException(@Nullable String operation,
       String path,
       SdkBaseException exception) {
     String message = String.format("%s%s: %s",

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35ad9b1d/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java
index 477200e..c611b94 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java
@@ -38,6 +38,7 @@ import com.amazonaws.services.s3.model.UploadPartRequest;
 import com.amazonaws.services.s3.model.UploadPartResult;
 import com.amazonaws.services.s3.transfer.model.UploadResult;
 import com.google.common.base.Preconditions;
+import org.apache.hadoop.conf.Configuration;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -83,9 +84,9 @@ public class WriteOperationHelper {
    * @param owner owner FS creating the helper
    *
    */
-  protected WriteOperationHelper(S3AFileSystem owner) {
+  protected WriteOperationHelper(S3AFileSystem owner, Configuration conf) {
     this.owner = owner;
-    this.invoker = new Invoker(new S3ARetryPolicy(owner.getConf()),
+    this.invoker = new Invoker(new S3ARetryPolicy(conf),
         this::operationRetried);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35ad9b1d/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java
index 1338d2e..f6e12f4 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java
@@ -101,7 +101,7 @@ public class CommitOperations {
     Preconditions.checkArgument(fs != null, "null fs");
     this.fs = fs;
     statistics = fs.newCommitterStatistics();
-    writeOperations = fs.createWriteOperationHelper();
+    writeOperations = fs.getWriteOperationHelper();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35ad9b1d/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/MagicCommitIntegration.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/MagicCommitIntegration.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/MagicCommitIntegration.java
index a07b5c9..7f9dadf 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/MagicCommitIntegration.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/MagicCommitIntegration.java
@@ -101,7 +101,7 @@ public class MagicCommitIntegration {
             key,
             destKey,
             pendingsetPath,
-            owner.createWriteOperationHelper());
+            owner.getWriteOperationHelper());
         LOG.debug("Created {}", tracker);
       } else {
         LOG.warn("File being created has a \"magic\" path, but the filesystem"

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35ad9b1d/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java
index ace043b..e764021 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java
@@ -23,14 +23,17 @@ import java.io.IOException;
 import java.io.PrintStream;
 import java.net.URI;
 import java.net.URISyntaxException;
+import java.util.Date;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Locale;
 import java.util.Map;
+import java.util.Scanner;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 
+import com.amazonaws.services.s3.model.MultipartUpload;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import org.slf4j.Logger;
@@ -44,6 +47,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.s3a.MultipartUtils;
 import org.apache.hadoop.fs.s3a.S3AFileStatus;
 import org.apache.hadoop.fs.s3a.S3AFileSystem;
 import org.apache.hadoop.fs.s3a.S3AUtils;
@@ -55,6 +59,7 @@ import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 
 import static org.apache.hadoop.fs.s3a.Constants.*;
+import static org.apache.hadoop.fs.s3a.Invoker.LOG_EVENT;
 import static org.apache.hadoop.service.launcher.LauncherExitCodes.*;
 
 /**
@@ -79,6 +84,7 @@ public abstract class S3GuardTool extends Configured implements Tool {
       "\t" + Destroy.NAME + " - " + Destroy.PURPOSE + "\n" +
       "\t" + Import.NAME + " - " + Import.PURPOSE + "\n" +
       "\t" + BucketInfo.NAME + " - " + BucketInfo.PURPOSE + "\n" +
+      "\t" + Uploads.NAME + " - " + Uploads.PURPOSE + "\n" +
       "\t" + Diff.NAME + " - " + Diff.PURPOSE + "\n" +
       "\t" + Prune.NAME + " - " + Prune.PURPOSE + "\n" +
       "\t" + SetCapacity.NAME + " - " +SetCapacity.PURPOSE + "\n";
@@ -100,10 +106,14 @@ public abstract class S3GuardTool extends Configured implements Tool {
   private final CommandFormat commandFormat;
 
   public static final String META_FLAG = "meta";
+
+  // These are common to prune, upload subcommands.
   public static final String DAYS_FLAG = "days";
   public static final String HOURS_FLAG = "hours";
   public static final String MINUTES_FLAG = "minutes";
   public static final String SECONDS_FLAG = "seconds";
+  public static final String AGE_OPTIONS_USAGE = "[-days <days>] "
+      + "[-hours <hours>] [-minutes <minutes>] [-seconds <seconds>]";
 
   public static final String REGION_FLAG = "region";
   public static final String READ_FLAG = "read";
@@ -177,6 +187,36 @@ public abstract class S3GuardTool extends Configured implements Tool {
         "config, or S3 bucket");
   }
 
+  private long getDeltaComponent(TimeUnit unit, String arg) {
+    String raw = getCommandFormat().getOptValue(arg);
+    if (raw == null || raw.isEmpty()) {
+      return 0;
+    }
+    Long parsed = Long.parseLong(raw);
+    return unit.toMillis(parsed);
+  }
+
+  /**
+   * Convert all age options supplied to total milliseconds of time.
+   * @return Sum of all age options, or zero if none were given.
+   */
+  long ageOptionsToMsec() {
+    long cliDelta = 0;
+    cliDelta += getDeltaComponent(TimeUnit.DAYS, DAYS_FLAG);
+    cliDelta += getDeltaComponent(TimeUnit.HOURS, HOURS_FLAG);
+    cliDelta += getDeltaComponent(TimeUnit.MINUTES, MINUTES_FLAG);
+    cliDelta += getDeltaComponent(TimeUnit.SECONDS, SECONDS_FLAG);
+    return cliDelta;
+  }
+
+  protected void addAgeOptions() {
+    CommandFormat format = getCommandFormat();
+    format.addOptionWithValue(DAYS_FLAG);
+    format.addOptionWithValue(HOURS_FLAG);
+    format.addOptionWithValue(MINUTES_FLAG);
+    format.addOptionWithValue(SECONDS_FLAG);
+  }
+
   /**
    * Parse metadata store from command line option or HDFS configuration.
    *
@@ -867,7 +907,8 @@ public abstract class S3GuardTool extends Configured implements Tool {
         "Common options:\n" +
         "  -" + META_FLAG + " URL - Metadata repository details " +
         "(implementation-specific)\n" +
-        "\n" +
+        "Age options. Any combination of these integer-valued options:\n" +
+        AGE_OPTIONS_USAGE + "\n" +
         "Amazon DynamoDB-specific options:\n" +
         "  -" + REGION_FLAG + " REGION - Service region for connections\n" +
         "\n" +
@@ -877,12 +918,7 @@ public abstract class S3GuardTool extends Configured implements Tool {
 
     Prune(Configuration conf) {
       super(conf);
-
-      CommandFormat format = getCommandFormat();
-      format.addOptionWithValue(DAYS_FLAG);
-      format.addOptionWithValue(HOURS_FLAG);
-      format.addOptionWithValue(MINUTES_FLAG);
-      format.addOptionWithValue(SECONDS_FLAG);
+      addAgeOptions();
     }
 
     @VisibleForTesting
@@ -901,15 +937,6 @@ public abstract class S3GuardTool extends Configured implements Tool {
       return USAGE;
     }
 
-    private long getDeltaComponent(TimeUnit unit, String arg) {
-      String raw = getCommandFormat().getOptValue(arg);
-      if (raw == null || raw.isEmpty()) {
-        return 0;
-      }
-      Long parsed = Long.parseLong(raw);
-      return unit.toMillis(parsed);
-    }
-
     public int run(String[] args, PrintStream out) throws
         InterruptedException, IOException {
       List<String> paths = parseArgs(args);
@@ -924,11 +951,7 @@ public abstract class S3GuardTool extends Configured implements Tool {
       Configuration conf = getConf();
       long confDelta = conf.getLong(S3GUARD_CLI_PRUNE_AGE, 0);
 
-      long cliDelta = 0;
-      cliDelta += getDeltaComponent(TimeUnit.DAYS, "days");
-      cliDelta += getDeltaComponent(TimeUnit.HOURS, "hours");
-      cliDelta += getDeltaComponent(TimeUnit.MINUTES, "minutes");
-      cliDelta += getDeltaComponent(TimeUnit.SECONDS, "seconds");
+      long cliDelta = ageOptionsToMsec();
 
       if (confDelta <= 0 && cliDelta <= 0) {
         errorln("You must specify a positive age for metadata to prune.");
@@ -1080,6 +1103,214 @@ public abstract class S3GuardTool extends Configured implements Tool {
 
   }
 
+  /**
+   * Command to list / abort pending multipart uploads.
+   */
+  static class Uploads extends S3GuardTool {
+    public static final String NAME = "uploads";
+    public static final String ABORT = "abort";
+    public static final String LIST = "list";
+    public static final String EXPECT = "expect";
+    public static final String VERBOSE = "verbose";
+    public static final String FORCE = "force";
+
+    public static final String PURPOSE = "list or abort pending " +
+        "multipart uploads";
+    private static final String USAGE = NAME + " [OPTIONS] " +
+        "s3a://BUCKET[/path]\n"
+        + "\t" + PURPOSE + "\n\n"
+        + "Common options:\n"
+        + " (-" + LIST + " | -" + EXPECT +" <num-uploads> | -" + ABORT
+        + ") [-" + VERBOSE +"] "
+        + "[<age-options>] [-force]\n"
+        + "\t - Under given path, list or delete all uploads," +
+        " or only those \n"
+        + "older than specified by <age-options>\n"
+        + "<age-options> are any combination of the integer-valued options:\n"
+        + "\t" + AGE_OPTIONS_USAGE + "\n"
+        + "-" + EXPECT + " is similar to list, except no output is printed,\n"
+        + "\tbut the exit code will be an error if the provided number\n"
+        + "\tis different that the number of uploads found by the command.\n"
+        + "-" + FORCE + " option prevents the \"Are you sure\" prompt when\n"
+        + "\tusing -" + ABORT;
+
+    /** Constant used for output and parsed by tests. */
+    public static final String TOTAL = "Total";
+
+    /** Runs in one of three modes. */
+    private enum Mode { LIST, EXPECT, ABORT };
+    private Mode mode = null;
+
+    /** For Mode == EXPECT, expected listing size. */
+    private int expectedCount;
+
+    /** List/abort uploads older than this many milliseconds. */
+    private long ageMsec = 0;
+
+    /** Verbose output flag. */
+    private boolean verbose = false;
+
+    /** Whether to delete with out "are you sure" prompt. */
+    private boolean force = false;
+
+    /** Path prefix to use when searching multipart uploads. */
+    private String prefix;
+
+    Uploads(Configuration conf) {
+      super(conf, ABORT, LIST, VERBOSE, FORCE);
+      addAgeOptions();
+      getCommandFormat().addOptionWithValue(EXPECT);
+    }
+
+    @Override
+    String getName() {
+      return NAME;
+    }
+
+    @Override
+    public String getUsage() {
+      return USAGE;
+    }
+
+    public int run(String[] args, PrintStream out)
+        throws InterruptedException, IOException {
+      List<String> paths = parseArgs(args);
+      if (paths.isEmpty()) {
+        errorln(getUsage());
+        throw invalidArgs("No options specified");
+      }
+      processArgs(paths, out);
+      promptBeforeAbort(out);
+      processUploads(out);
+
+      out.flush();
+      return SUCCESS;
+    }
+
+    private void promptBeforeAbort(PrintStream out) throws IOException {
+      if (mode != Mode.ABORT || force) {
+        return;
+      }
+      Scanner scanner = new Scanner(System.in, "UTF-8");
+      out.println("Are you sure you want to delete any pending " +
+          "uploads? (yes/no) >");
+      String response = scanner.nextLine();
+      if (!"yes".equalsIgnoreCase(response)) {
+        throw S3GuardTool.userAborted("User did not answer yes, quitting.");
+      }
+    }
+
+    private void processUploads(PrintStream out) throws IOException {
+      MultipartUtils.UploadIterator uploads;
+      uploads = getFilesystem().listUploads(prefix);
+
+      int count = 0;
+      while (uploads.hasNext()) {
+        MultipartUpload upload = uploads.next();
+        if (!olderThan(upload, ageMsec)) {
+          continue;
+        }
+        count++;
+        if (mode == Mode.ABORT || mode == Mode.LIST || verbose) {
+          println(out, "%s%s %s", mode == Mode.ABORT ? "Deleting: " : "",
+              upload.getKey(), upload.getUploadId());
+        }
+        if (mode == Mode.ABORT) {
+          getFilesystem().getWriteOperationHelper()
+              .abortMultipartUpload(upload.getKey(), upload.getUploadId(),
+                  LOG_EVENT);
+        }
+      }
+      if (mode != Mode.EXPECT || verbose) {
+        println(out, "%s %d uploads %s.", TOTAL, count,
+            mode == Mode.ABORT ? "deleted" : "found");
+      }
+      if (mode == Mode.EXPECT) {
+        if (count != expectedCount) {
+          throw badState("Expected %d uploads, found %d", expectedCount, count);
+        }
+      }
+    }
+
+    /**
+     * Check if upload is at least as old as given age.
+     * @param u upload to check
+     * @param msec age in milliseconds
+     * @return true iff u was created at least age milliseconds ago.
+     */
+    private boolean olderThan(MultipartUpload u, long msec) {
+      Date ageDate = new Date(System.currentTimeMillis() - msec);
+      return ageDate.compareTo(u.getInitiated()) >= 0;
+    }
+
+    private void processArgs(List<String> args, PrintStream out)
+        throws IOException {
+      CommandFormat commands = getCommandFormat();
+      String err = "Can only specify one of -" + LIST + ", " +
+          " -" + ABORT + ", and " + EXPECT;
+
+      // Three mutually-exclusive options
+      if (commands.getOpt(LIST)) {
+        mode = Mode.LIST;
+      }
+      if (commands.getOpt(ABORT)) {
+        if (mode != null) {
+          throw invalidArgs(err);
+        }
+        mode = Mode.ABORT;
+      }
+
+      String expectVal = commands.getOptValue(EXPECT);
+      if (expectVal != null) {
+        if (mode != null) {
+          throw invalidArgs(err);
+        }
+        mode = Mode.EXPECT;
+        expectedCount = Integer.parseInt(expectVal);
+      }
+
+      // Default to list
+      if (mode == null) {
+        vprintln(out, "No mode specified, defaulting to -" + LIST);
+        mode = Mode.LIST;
+      }
+
+      // Other flags
+      if (commands.getOpt(VERBOSE)) {
+        verbose = true;
+      }
+      if (commands.getOpt(FORCE)) {
+        force = true;
+      }
+      ageMsec = ageOptionsToMsec();
+
+      String s3Path = args.get(0);
+      URI uri = S3GuardTool.toUri(s3Path);
+      prefix = uri.getPath();
+      if (prefix.length() > 0) {
+        prefix = prefix.substring(1);
+      }
+      vprintln(out, "Command: %s, age %d msec, path %s (prefix \"%s\")",
+          mode.name(), ageMsec, s3Path, prefix);
+
+      initS3AFileSystem(s3Path);
+    }
+
+    /**
+     * If verbose flag is set, print a formatted string followed by a newline
+     * to the output stream.
+     * @param out destination
+     * @param format format string
+     * @param args optional arguments
+     */
+    private void vprintln(PrintStream out, String format, Object...
+        args) {
+      if (verbose) {
+        out.println(String.format(format, args));
+      }
+    }
+  }
+
   private static S3GuardTool command;
 
   /**
@@ -1183,6 +1414,17 @@ public abstract class S3GuardTool extends Configured implements Tool {
   }
 
   /**
+   * Build the exception to raise on user-aborted action.
+   * @param format string format
+   * @param args optional arguments for the string
+   * @return a new exception to throw
+   */
+  protected static ExitUtil.ExitException userAborted(
+      String format, Object...args) {
+    return new ExitUtil.ExitException(ERROR, String.format(format, args));
+  }
+
+  /**
    * Execute the command with the given arguments.
    *
    * @param conf Hadoop configuration.
@@ -1224,6 +1466,9 @@ public abstract class S3GuardTool extends Configured implements Tool {
     case SetCapacity.NAME:
       command = new SetCapacity(conf);
       break;
+    case Uploads.NAME:
+      command = new Uploads(conf);
+      break;
     default:
       printHelp();
       throw new ExitUtil.ExitException(E_USAGE,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35ad9b1d/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
index a8d2e48..fbcd54a 100644
--- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
+++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
@@ -1490,8 +1490,13 @@ from VMs running on EC2.
 </property>
 ```
 
-### <a name="multipart_purge"></a>Cleaning up after partial Upload Failures: `fs.s3a.multipart.purge`
+### <a name="multipart_purge"></a>Cleaning up after partial Upload Failures
 
+There are two mechanisms for cleaning up after leftover multipart
+uploads:
+- Hadoop s3guard CLI commands for listing and deleting uploads by their
+age. Doumented in the [S3Guard](./s3guard.html) section.
+- The configuration parameter `fs.s3a.multipart.purge`, covered below.
 
 If an large stream writeoperation is interrupted, there may be
 intermediate partitions uploaded to S3 —data which will be billed for.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35ad9b1d/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3guard.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3guard.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3guard.md
index e2cb549..1050f8a 100644
--- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3guard.md
+++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3guard.md
@@ -515,9 +515,42 @@ hadoop s3guard bucket-info -guarded -auth s3a://landsat-pds
 Require the bucket to be using S3Guard in authoritative mode. This will normally
 fail against this specific bucket.
 
+### List or Delete Leftover Multipart Uploads: `s3guard uploads`
 
-### Delete a table: `s3guard destroy`
+Lists or deletes all pending (uncompleted) multipart uploads older than
+given age.
+
+```bash
+hadoop s3guard uploads (-list | -abort | -expect <num-uploads>) [-verbose] \
+    [-days <days>] [-hours <hours>] [-minutes <minutes>] [-seconds <seconds>] \
+    [-force] s3a://bucket/prefix
+```
 
+The command lists or deletes all multipart uploads which are older than
+the given age, and that match the prefix supplied, if any.
+
+For example, to delete all uncompleted multipart uploads older than two
+days in the folder at `s3a://my-bucket/path/to/stuff`, use the following
+command:
+
+```bash
+hadoop s3guard uploads -abort -days 2 s3a://my-bucket/path/to/stuff
+```
+
+We recommend running with `-list` first to confirm the parts shown
+are those that you wish to delete. Note that the command will prompt
+you with a "Are you sure?" prompt unless you specify the `-force`
+option. This is to safeguard against accidental deletion of data, which
+is especially risky without a long age parameter as it can affect
+in-fight uploads.
+
+The `-expect` option is similar to `-list`, except it is silent by
+default, and terminates with a success or failure exit code depending
+on whether or not the supplied number matches the number of uploads
+found that match the given options (path, age).
+
+
+### Delete a table: `s3guard destroy`
 
 Deletes a metadata store. With DynamoDB as the store, this means
 the specific DynamoDB table use to store the metadata.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35ad9b1d/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMultipartUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMultipartUtils.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMultipartUtils.java
new file mode 100644
index 0000000..4746ad5
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMultipartUtils.java
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a;
+
+import com.amazonaws.services.s3.model.MultipartUpload;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Set;
+
+
+/**
+ * Tests for {@link MultipartUtils}.
+ */
+public class ITestS3AMultipartUtils extends AbstractS3ATestBase {
+
+  private static final int UPLOAD_LEN = 1024;
+  private static final String PART_FILENAME_BASE = "pending-part";
+  private static final int LIST_BATCH_SIZE = 2;
+  private static final int NUM_KEYS = 5;
+
+
+  @Override
+  protected Configuration createConfiguration() {
+    Configuration conf = super.createConfiguration();
+    S3ATestUtils.disableFilesystemCaching(conf);
+    // Forces listings to come back in multiple batches to test that part of
+    // the iterators.
+    conf.setInt(Constants.MAX_PAGING_KEYS, LIST_BATCH_SIZE);
+    return conf;
+  }
+
+  /**
+   * Main test case for upload part listing and iterator paging.
+   * @throws Exception on failure.
+   */
+  @Test
+  public void testListMultipartUploads() throws Exception {
+    S3AFileSystem fs = getFileSystem();
+    Set<MultipartTestUtils.IdKey> keySet = new HashSet<>();
+    try {
+      // 1. Create NUM_KEYS pending upload parts
+      for (int i = 0; i < NUM_KEYS; i++) {
+        Path filePath = getPartFilename(i);
+        String key = fs.pathToKey(filePath);
+        describe("creating upload part with key %s", key);
+        // create a multipart upload
+        MultipartTestUtils.IdKey idKey = MultipartTestUtils
+            .createPartUpload(fs, key, UPLOAD_LEN,
+            1);
+        keySet.add(idKey);
+      }
+
+      // 2. Verify all uploads are found listing by prefix
+      describe("Verifying upload list by prefix");
+      MultipartUtils.UploadIterator uploads = fs.listUploads(getPartPrefix(fs));
+      assertUploadsPresent(uploads, keySet);
+
+      // 3. Verify all uploads are found listing without prefix
+      describe("Verifying list all uploads");
+      uploads = fs.listUploads(null);
+      assertUploadsPresent(uploads, keySet);
+
+    } finally {
+      // 4. Delete all uploads we created
+      MultipartTestUtils.cleanupParts(fs, keySet);
+    }
+  }
+
+  /**
+   * Assert that all provided multipart uploads are contained in the upload
+   * iterator's results.
+   * @param list upload iterator
+   * @param ourUploads set up uploads that should be present
+   * @throws IOException on I/O error
+   */
+  private void assertUploadsPresent(MultipartUtils.UploadIterator list,
+      Set<MultipartTestUtils.IdKey> ourUploads) throws IOException {
+
+    // Don't modify passed-in set, use copy.
+    Set<MultipartTestUtils.IdKey> uploads = new HashSet<>(ourUploads);
+    while (list.hasNext()) {
+      MultipartTestUtils.IdKey listing = toIdKey(list.next());
+      if (uploads.contains(listing)) {
+        LOG.debug("Matched: {},{}", listing.getKey(), listing.getUploadId());
+        uploads.remove(listing);
+      } else {
+        LOG.debug("Not our upload {},{}", listing.getKey(),
+            listing.getUploadId());
+      }
+    }
+    assertTrue("Not all our uploads were listed", uploads.isEmpty());
+  }
+
+  private MultipartTestUtils.IdKey toIdKey(MultipartUpload mu) {
+    return new MultipartTestUtils.IdKey(mu.getKey(), mu.getUploadId());
+  }
+
+  private Path getPartFilename(int index) throws IOException {
+    return path(String.format("%s-%d", PART_FILENAME_BASE, index));
+  }
+
+  private String getPartPrefix(S3AFileSystem fs) throws IOException {
+    return fs.pathToKey(path("blah").getParent());
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35ad9b1d/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java
index 55e3e37..4952580 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java
@@ -78,6 +78,7 @@ public class MockS3AFileSystem extends S3AFileSystem {
   private final S3AInstrumentation instrumentation =
       new S3AInstrumentation(FS_URI);
   private Configuration conf;
+  private WriteOperationHelper writeHelper;
 
   public MockS3AFileSystem(S3AFileSystem mock,
       Pair<StagingTestBase.ClientResults, StagingTestBase.ClientErrors> outcome) {
@@ -125,6 +126,12 @@ public class MockS3AFileSystem extends S3AFileSystem {
   public void initialize(URI name, Configuration originalConf)
       throws IOException {
     conf = originalConf;
+    writeHelper = new WriteOperationHelper(this, conf);
+  }
+
+  @Override
+  public WriteOperationHelper getWriteOperationHelper() {
+    return writeHelper;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35ad9b1d/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MultipartTestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MultipartTestUtils.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MultipartTestUtils.java
new file mode 100644
index 0000000..8be3ff7
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MultipartTestUtils.java
@@ -0,0 +1,184 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a;
+
+import com.amazonaws.services.s3.model.MultipartUpload;
+import com.amazonaws.services.s3.model.PartETag;
+import com.amazonaws.services.s3.model.UploadPartRequest;
+import org.apache.hadoop.fs.Path;
+import org.junit.Assert;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
+import static org.apache.hadoop.fs.s3a.Invoker.LOG_EVENT;
+
+/**
+ * Utilities for S3A multipart upload tests.
+ */
+public final class MultipartTestUtils {
+  private static final Logger LOG = LoggerFactory.getLogger(
+      MultipartTestUtils.class);
+
+  /** Not instantiated. */
+  private MultipartTestUtils() { }
+
+  /**
+   * Clean up all provided uploads.
+   * @param keySet set of uploads to abort
+   */
+  static void cleanupParts(S3AFileSystem fs, Set <IdKey> keySet) {
+    boolean anyFailure = false;
+    for (IdKey ik : keySet) {
+      try {
+        LOG.debug("aborting upload id {}", ik.getUploadId());
+        fs.abortMultipartUpload(ik.getKey(), ik.getUploadId());
+      } catch (Exception e) {
+        LOG.error(String.format("Failure aborting upload %s, continuing.",
+            ik.getKey()), e);
+        anyFailure = true;
+      }
+    }
+    Assert.assertFalse("Failure aborting multipart upload(s), see log.",
+        anyFailure);
+  }
+
+  public static IdKey createPartUpload(S3AFileSystem fs, String key, int len,
+      int partNo) throws IOException {
+    WriteOperationHelper writeHelper = fs.getWriteOperationHelper();
+    byte[] data = dataset(len, 'a', 'z');
+    InputStream in = new ByteArrayInputStream(data);
+    String uploadId = writeHelper.initiateMultiPartUpload(key);
+    UploadPartRequest req = writeHelper.newUploadPartRequest(key, uploadId,
+        partNo, len, in, null, 0L);
+    PartETag partEtag = fs.uploadPart(req).getPartETag();
+    LOG.debug("uploaded part etag {}, upid {}", partEtag.getETag(), uploadId);
+    return new IdKey(key, uploadId);
+  }
+
+  /** Delete any uploads under given path (recursive).  Silent on failure. */
+  public static void clearAnyUploads(S3AFileSystem fs, Path path) {
+    try {
+      String key = fs.pathToKey(path);
+      MultipartUtils.UploadIterator uploads = fs.listUploads(key);
+      while (uploads.hasNext()) {
+        MultipartUpload upload = uploads.next();
+        fs.getWriteOperationHelper().abortMultipartUpload(upload.getKey(),
+            upload.getUploadId(), LOG_EVENT);
+        LOG.debug("Cleaning up upload: {} {}", upload.getKey(),
+            truncatedUploadId(upload.getUploadId()));
+      }
+    } catch (IOException ioe) {
+      LOG.info("Ignoring exception: ", ioe);
+    }
+  }
+
+  /** Assert that there are not any upload parts at given path. */
+  public static void assertNoUploadsAt(S3AFileSystem fs, Path path) throws
+      Exception {
+    String key = fs.pathToKey(path);
+    MultipartUtils.UploadIterator uploads = fs.listUploads(key);
+    while (uploads.hasNext()) {
+      MultipartUpload upload = uploads.next();
+      Assert.fail("Found unexpected upload " + upload.getKey() + " " +
+          truncatedUploadId(upload.getUploadId()));
+    }
+  }
+
+  /** Get number of part uploads under given path. */
+  public static int countUploadsAt(S3AFileSystem fs, Path path) throws
+      IOException {
+    String key = fs.pathToKey(path);
+    MultipartUtils.UploadIterator uploads = fs.listUploads(key);
+    int count = 0;
+    while (uploads.hasNext()) {
+      MultipartUpload upload = uploads.next();
+      count++;
+    }
+    return count;
+  }
+
+  /**
+   * Get a list of all pending uploads under a prefix, one which can be printed.
+   * @param prefix prefix to look under
+   * @return possibly empty list
+   * @throws IOException IO failure.
+   */
+  public static List<String> listMultipartUploads(S3AFileSystem fs,
+      String prefix) throws IOException {
+
+    return fs
+        .listMultipartUploads(prefix).stream()
+        .map(upload -> String.format("Upload to %s with ID %s; initiated %s",
+            upload.getKey(),
+            upload.getUploadId(),
+            S3ATestUtils.LISTING_FORMAT.format(upload.getInitiated())))
+        .collect(Collectors.toList());
+  }
+
+
+  private static String truncatedUploadId(String fullId) {
+    return fullId.substring(0, 12) + " ...";
+  }
+
+  /** Struct of object key, upload ID. */
+  static class IdKey {
+    private String key;
+    private String uploadId;
+
+    IdKey(String key, String uploadId) {
+      this.key = key;
+      this.uploadId = uploadId;
+    }
+
+    public String getKey() {
+      return key;
+    }
+
+    public String getUploadId() {
+      return uploadId;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      IdKey key1 = (IdKey) o;
+      return Objects.equals(key, key1.key) &&
+          Objects.equals(uploadId, key1.uploadId);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(key, uploadId);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35ad9b1d/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
index 773c25a..f4e7c68 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
@@ -45,7 +45,6 @@ import java.net.URISyntaxException;
 import java.text.DateFormat;
 import java.text.SimpleDateFormat;
 import java.util.List;
-import java.util.stream.Collectors;
 
 import static org.apache.hadoop.fs.contract.ContractTestUtils.skip;
 import static org.apache.hadoop.fs.s3a.InconsistentAmazonS3Client.*;
@@ -822,28 +821,10 @@ public final class S3ATestUtils {
   /**
    * Date format used for mapping upload initiation time to human string.
    */
-  private static final DateFormat LISTING_FORMAT = new SimpleDateFormat(
+  public static final DateFormat LISTING_FORMAT = new SimpleDateFormat(
       "yyyy-MM-dd HH:mm:ss");
 
   /**
-   * Get a list of all pending uploads under a prefix, one which can be printed.
-   * @param prefix prefix to look under
-   * @return possibly empty list
-   * @throws IOException IO failure.
-   */
-  public static List<String> listMultipartUploads(S3AFileSystem fs,
-      String prefix) throws IOException {
-
-    return fs
-        .listMultipartUploads(prefix).stream()
-        .map(upload -> String.format("Upload to %s with ID %s; initiated %s",
-            upload.getKey(),
-            upload.getUploadId(),
-            LISTING_FORMAT.format(upload.getInitiated())))
-        .collect(Collectors.toList());
-  }
-
-  /**
    * Skip a test if the FS isn't marked as supporting magic commits.
    * @param fs filesystem
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35ad9b1d/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractCommitITest.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractCommitITest.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractCommitITest.java
index 267d4df..04676db 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractCommitITest.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractCommitITest.java
@@ -47,6 +47,7 @@ import org.apache.hadoop.mapreduce.v2.api.records.JobId;
 import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils;
 
 import static org.apache.hadoop.fs.s3a.Constants.*;
+import static org.apache.hadoop.fs.s3a.MultipartTestUtils.listMultipartUploads;
 import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
 import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*;
 
@@ -247,7 +248,7 @@ public abstract class AbstractCommitITest extends AbstractS3ATestBase {
     S3AFileSystem fs = getFileSystem();
     if (fs != null && path != null) {
       String key = fs.pathToKey(path);
-      WriteOperationHelper writeOps = fs.createWriteOperationHelper();
+      WriteOperationHelper writeOps = fs.getWriteOperationHelper();
       int count = writeOps.abortMultipartUploadsUnderPath(key);
       if (count > 0) {
         log().info("Multipart uploads deleted: {}", count);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35ad9b1d/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestS3AHugeMagicCommits.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestS3AHugeMagicCommits.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestS3AHugeMagicCommits.java
index e3a295b..0722959 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestS3AHugeMagicCommits.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestS3AHugeMagicCommits.java
@@ -40,8 +40,8 @@ import org.apache.hadoop.fs.s3a.commit.files.PendingSet;
 import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit;
 import org.apache.hadoop.fs.s3a.scale.AbstractSTestS3AHugeFiles;
 
+import static org.apache.hadoop.fs.s3a.MultipartTestUtils.listMultipartUploads;
 import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*;
-import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
 
 
 /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/35ad9b1d/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolLocal.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolLocal.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolLocal.java
index 43cbe93..2178f47 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolLocal.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardToolLocal.java
@@ -24,10 +24,15 @@ import java.io.ByteArrayOutputStream;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InputStreamReader;
+import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Set;
 import java.util.concurrent.Callable;
 
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.apache.hadoop.util.StringUtils;
 import org.junit.Test;
 
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -35,15 +40,20 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.s3a.S3AFileSystem;
 import org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.Diff;
 
+import static org.apache.hadoop.fs.s3a.MultipartTestUtils.*;
 import static org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.*;
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 
 /**
  * Test S3Guard related CLI commands against a LocalMetadataStore.
+ * Also responsible for testing the non s3guard-specific commands that, for
+ * now, live under the s3guard CLI command.
  */
 public class ITestS3GuardToolLocal extends AbstractS3GuardToolTestBase {
 
   private static final String LOCAL_METADATA = "local://metadata";
+  private static final String[] ABORT_FORCE_OPTIONS = new String[] {"-abort",
+      "-force", "-verbose"};
 
   @Override
   protected MetadataStore newMetadataStore() {
@@ -261,5 +271,182 @@ public class ITestS3GuardToolLocal extends AbstractS3GuardToolTestBase {
     LOG.info("Exec output=\n{}", output);
   }
 
+  private final static String UPLOAD_PREFIX = "test-upload-prefix";
+  private final static String UPLOAD_NAME = "test-upload";
 
+  @Test
+  public void testUploads() throws Throwable {
+    S3AFileSystem fs = getFileSystem();
+    Path path = path(UPLOAD_PREFIX + "/" + UPLOAD_NAME);
+
+    describe("Cleaning up any leftover uploads from previous runs.");
+    // 1. Make sure key doesn't already exist
+    clearAnyUploads(fs, path);
+
+    // 2. Confirm no uploads are listed via API
+    assertNoUploadsAt(fs, path.getParent());
+
+    // 3. Confirm no uploads are listed via CLI
+    describe("Confirming CLI lists nothing.");
+    assertNumUploads(path, 0);
+
+    // 4. Create a upload part
+    describe("Uploading single part.");
+    createPartUpload(fs, fs.pathToKey(path), 128, 1);
+
+    try {
+      // 5. Confirm it exists via API..
+      LambdaTestUtils.eventually(5000, /* 5 seconds until failure */
+          1000, /* one second retry interval */
+          () -> {
+            assertEquals("Should be one upload", 1, countUploadsAt(fs, path));
+          });
+
+      // 6. Confirm part exists via CLI, direct path and parent path
+      describe("Confirming CLI lists one part");
+      LambdaTestUtils.eventually(5000, 1000,
+          () -> assertNumUploads(path, 1));
+      LambdaTestUtils.eventually(5000, 1000,
+          () -> assertNumUploads(path.getParent(), 1));
+
+      // 7. Use CLI to delete part, assert it worked
+      describe("Deleting part via CLI");
+      assertNumDeleted(fs, path, 1);
+
+      // 8. Confirm deletion via API
+      describe("Confirming deletion via API");
+      assertEquals("Should be no uploads", 0, countUploadsAt(fs, path));
+
+      // 9. Confirm no uploads are listed via CLI
+      describe("Confirming CLI lists nothing.");
+      assertNumUploads(path, 0);
+
+    } catch (Throwable t) {
+      // Clean up on intermediate failure
+      clearAnyUploads(fs, path);
+      throw t;
+    }
+  }
+
+  @Test
+  public void testUploadListByAge() throws Throwable {
+    S3AFileSystem fs = getFileSystem();
+    Path path = path(UPLOAD_PREFIX + "/" + UPLOAD_NAME);
+
+    describe("Cleaning up any leftover uploads from previous runs.");
+    // 1. Make sure key doesn't already exist
+    clearAnyUploads(fs, path);
+
+    // 2. Create a upload part
+    describe("Uploading single part.");
+    createPartUpload(fs, fs.pathToKey(path), 128, 1);
+
+    try {
+      // 3. Confirm it exists via API.. may want to wrap with
+      // LambdaTestUtils.eventually() ?
+      LambdaTestUtils.eventually(5000, 1000,
+          () -> {
+            assertEquals("Should be one upload", 1, countUploadsAt(fs, path));
+          });
+
+      // 4. Confirm part does appear in listing with long age filter
+      describe("Confirming CLI older age doesn't list");
+      assertNumUploadsAge(path, 0, 600);
+
+      // 5. Confirm part does not get deleted with long age filter
+      describe("Confirming CLI older age doesn't delete");
+      uploadCommandAssertCount(fs, ABORT_FORCE_OPTIONS, path, 0,
+          600);
+
+      // 6. Wait a second and then assert the part is in listing of things at
+      // least a second old
+      describe("Sleeping 1 second then confirming upload still there");
+      Thread.sleep(1000);
+      LambdaTestUtils.eventually(5000, 1000,
+          () -> assertNumUploadsAge(path, 1, 1));
+
+      // 7. Assert deletion works when age filter matches
+      describe("Doing aged deletion");
+      uploadCommandAssertCount(fs, ABORT_FORCE_OPTIONS, path, 1, 1);
+      describe("Confirming age deletion happened");
+      assertEquals("Should be no uploads", 0, countUploadsAt(fs, path));
+    } catch (Throwable t) {
+      // Clean up on intermediate failure
+      clearAnyUploads(fs, path);
+      throw t;
+    }
+  }
+
+  @Test
+  public void testUploadNegativeExpect() throws Throwable {
+    runToFailure(E_BAD_STATE, Uploads.NAME, "-expect", "1",
+        path("/we/are/almost/postive/this/doesnt/exist/fhfsadfoijew")
+            .toString());
+  }
+
+  private void assertNumUploads(Path path, int numUploads) throws Exception {
+    assertNumUploadsAge(path, numUploads, 0);
+  }
+
+  private void assertNumUploadsAge(Path path, int numUploads, int ageSeconds)
+      throws Exception {
+    if (ageSeconds > 0) {
+      run(Uploads.NAME, "-expect", String.valueOf(numUploads), "-seconds",
+          String.valueOf(ageSeconds), path.toString());
+    } else {
+      run(Uploads.NAME, "-expect", String.valueOf(numUploads), path.toString());
+    }
+  }
+
+  private void assertNumDeleted(S3AFileSystem fs, Path path, int numDeleted)
+      throws Exception {
+    uploadCommandAssertCount(fs, ABORT_FORCE_OPTIONS, path,
+        numDeleted, 0);
+  }
+
+  /**
+   * Run uploads cli command and assert the reported count (listed or
+   * deleted) matches.
+   * @param fs  S3AFileSystem
+   * @param options main command options
+   * @param path path of part(s)
+   * @param numUploads expected number of listed/deleted parts
+   * @param ageSeconds optional seconds of age to specify to CLI, or zero to
+   *                   search all parts
+   * @throws Exception on failure
+   */
+  private void uploadCommandAssertCount(S3AFileSystem fs, String[] options,
+      Path path, int numUploads, int ageSeconds)
+      throws Exception {
+    List<String> allOptions = new ArrayList<>();
+    List<String> output = new ArrayList<>();
+    S3GuardTool.Uploads cmd = new S3GuardTool.Uploads(fs.getConf());
+    ByteArrayOutputStream buf = new ByteArrayOutputStream();
+    allOptions.add(cmd.getName());
+    allOptions.addAll(Arrays.asList(options));
+    if (ageSeconds > 0) {
+      allOptions.add("-" + Uploads.SECONDS_FLAG);
+      allOptions.add(String.valueOf(ageSeconds));
+    }
+    allOptions.add(path.toString());
+    exec(cmd, buf, allOptions.toArray(new String[0]));
+
+    try (BufferedReader reader = new BufferedReader(
+        new InputStreamReader(new ByteArrayInputStream(buf.toByteArray())))) {
+      String line;
+      while ((line = reader.readLine()) != null) {
+        String[] fields = line.split("\\s");
+        if (fields.length == 4 && fields[0].equals(Uploads.TOTAL)) {
+          int parsedUploads = Integer.valueOf(fields[1]);
+          LOG.debug("Matched CLI output: {} {} {} {}", fields);
+          assertEquals("Unexpected number of uploads", numUploads,
+              parsedUploads);
+          return;
+        }
+        LOG.debug("Not matched: {}", line);
+        output.add(line);
+      }
+    }
+    fail("Command output did not match: \n" + StringUtils.join("\n", output));
+  }
 }


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


[28/50] [abbrv] hadoop git commit: Add 2.8.3 release jdiff files.

Posted by na...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/a7f8caf5/hadoop-hdfs-project/hadoop-hdfs/dev-support/jdiff/Apache_Hadoop_HDFS_2.8.3.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/dev-support/jdiff/Apache_Hadoop_HDFS_2.8.3.xml b/hadoop-hdfs-project/hadoop-hdfs/dev-support/jdiff/Apache_Hadoop_HDFS_2.8.3.xml
new file mode 100644
index 0000000..331dd1e
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/dev-support/jdiff/Apache_Hadoop_HDFS_2.8.3.xml
@@ -0,0 +1,312 @@
+<?xml version="1.0" encoding="iso-8859-1" standalone="no"?>
+<!-- Generated by the JDiff Javadoc doclet -->
+<!-- (http://www.jdiff.org) -->
+<!-- on Tue Dec 05 05:19:32 UTC 2017 -->
+
+<api
+  xmlns:xsi='http://www.w3.org/2001/XMLSchema-instance'
+  xsi:noNamespaceSchemaLocation='api.xsd'
+  name="Apache Hadoop HDFS 2.8.3"
+  jdversion="1.0.9">
+
+<!--  Command line arguments =  -doclet org.apache.hadoop.classification.tools.IncludePublicAnnotationsJDiffDoclet -docletpath /build/source/hadoop-hdfs-project/hadoop-hdfs/target/hadoop-annotations.jar:/build/source/hadoop-hdfs-project/hadoop-hdfs/target/jdiff.jar -verbose -classpath /build/source/hadoop-hdfs-project/hadoop-hdfs/target/classes:/build/source/hadoop-common-project/hadoop-annotations/target/hadoop-annotations-2.8.3.jar:/usr/lib/jvm/java-7-openjdk-amd64/lib/tools.jar:/build/source/hadoop-common-project/hadoop-auth/target/hadoop-auth-2.8.3.jar:/maven/org/slf4j/slf4j-api/1.7.10/slf4j-api-1.7.10.jar:/maven/org/apache/httpcomponents/httpclient/4.5.2/httpclient-4.5.2.jar:/maven/org/apache/httpcomponents/httpcore/4.4.4/httpcore-4.4.4.jar:/maven/com/nimbusds/nimbus-jose-jwt/3.9/nimbus-jose-jwt-3.9.jar:/maven/net/jcip/jcip-annotations/1.0/jcip-annotations-1.0.jar:/maven/net/minidev/json-smart/1.1.1/json-smart-1.1.1.jar:/maven/org/apache/directory/server/apacheds-kerberos-codec
 /2.0.0-M15/apacheds-kerberos-codec-2.0.0-M15.jar:/maven/org/apache/directory/server/apacheds-i18n/2.0.0-M15/apacheds-i18n-2.0.0-M15.jar:/maven/org/apache/directory/api/api-asn1-api/1.0.0-M20/api-asn1-api-1.0.0-M20.jar:/maven/org/apache/directory/api/api-util/1.0.0-M20/api-util-1.0.0-M20.jar:/maven/org/apache/zookeeper/zookeeper/3.4.6/zookeeper-3.4.6.jar:/maven/jline/jline/0.9.94/jline-0.9.94.jar:/maven/org/apache/curator/curator-framework/2.7.1/curator-framework-2.7.1.jar:/build/source/hadoop-common-project/hadoop-common/target/hadoop-common-2.8.3.jar:/maven/org/apache/commons/commons-math3/3.1.1/commons-math3-3.1.1.jar:/maven/commons-net/commons-net/3.1/commons-net-3.1.jar:/maven/commons-collections/commons-collections/3.2.2/commons-collections-3.2.2.jar:/maven/org/mortbay/jetty/jetty-sslengine/6.1.26/jetty-sslengine-6.1.26.jar:/maven/javax/servlet/jsp/jsp-api/2.1/jsp-api-2.1.jar:/maven/com/sun/jersey/jersey-json/1.9/jersey-json-1.9.jar:/maven/org/codehaus/jettison/jettison/1.1/jet
 tison-1.1.jar:/maven/com/sun/xml/bind/jaxb-impl/2.2.3-1/jaxb-impl-2.2.3-1.jar:/maven/javax/xml/bind/jaxb-api/2.2.2/jaxb-api-2.2.2.jar:/maven/javax/xml/stream/stax-api/1.0-2/stax-api-1.0-2.jar:/maven/javax/activation/activation/1.1/activation-1.1.jar:/maven/org/codehaus/jackson/jackson-jaxrs/1.9.13/jackson-jaxrs-1.9.13.jar:/maven/org/codehaus/jackson/jackson-xc/1.9.13/jackson-xc-1.9.13.jar:/maven/net/java/dev/jets3t/jets3t/0.9.0/jets3t-0.9.0.jar:/maven/com/jamesmurty/utils/java-xmlbuilder/0.4/java-xmlbuilder-0.4.jar:/maven/commons-configuration/commons-configuration/1.6/commons-configuration-1.6.jar:/maven/commons-digester/commons-digester/1.8/commons-digester-1.8.jar:/maven/commons-beanutils/commons-beanutils/1.7.0/commons-beanutils-1.7.0.jar:/maven/commons-beanutils/commons-beanutils-core/1.8.0/commons-beanutils-core-1.8.0.jar:/maven/org/apache/avro/avro/1.7.4/avro-1.7.4.jar:/maven/com/thoughtworks/paranamer/paranamer/2.3/paranamer-2.3.jar:/maven/org/xerial/snappy/snappy-java/1.0.4
 .1/snappy-java-1.0.4.1.jar:/maven/com/google/code/gson/gson/2.2.4/gson-2.2.4.jar:/maven/com/jcraft/jsch/0.1.54/jsch-0.1.54.jar:/maven/org/apache/curator/curator-client/2.7.1/curator-client-2.7.1.jar:/maven/org/apache/curator/curator-recipes/2.7.1/curator-recipes-2.7.1.jar:/maven/com/google/code/findbugs/jsr305/3.0.0/jsr305-3.0.0.jar:/maven/org/apache/commons/commons-compress/1.4.1/commons-compress-1.4.1.jar:/maven/org/tukaani/xz/1.0/xz-1.0.jar:/build/source/hadoop-hdfs-project/hadoop-hdfs-client/target/hadoop-hdfs-client-2.8.3.jar:/maven/com/squareup/okhttp/okhttp/2.4.0/okhttp-2.4.0.jar:/maven/com/squareup/okio/okio/1.4.0/okio-1.4.0.jar:/maven/com/google/guava/guava/11.0.2/guava-11.0.2.jar:/maven/org/mortbay/jetty/jetty/6.1.26/jetty-6.1.26.jar:/maven/org/mortbay/jetty/jetty-util/6.1.26/jetty-util-6.1.26.jar:/maven/com/sun/jersey/jersey-core/1.9/jersey-core-1.9.jar:/maven/com/sun/jersey/jersey-server/1.9/jersey-server-1.9.jar:/maven/asm/asm/3.2/asm-3.2.jar:/maven/commons-cli/commons-
 cli/1.2/commons-cli-1.2.jar:/maven/commons-codec/commons-codec/1.4/commons-codec-1.4.jar:/maven/commons-io/commons-io/2.4/commons-io-2.4.jar:/maven/commons-lang/commons-lang/2.6/commons-lang-2.6.jar:/maven/commons-logging/commons-logging/1.1.3/commons-logging-1.1.3.jar:/maven/commons-daemon/commons-daemon/1.0.13/commons-daemon-1.0.13.jar:/maven/log4j/log4j/1.2.17/log4j-1.2.17.jar:/maven/com/google/protobuf/protobuf-java/2.5.0/protobuf-java-2.5.0.jar:/maven/javax/servlet/servlet-api/2.5/servlet-api-2.5.jar:/maven/org/apache/directory/api/api-i18n/1.0.0-M20/api-i18n-1.0.0-M20.jar:/maven/org/apache/directory/api/api-ldap-model/1.0.0-M20/api-ldap-model-1.0.0-M20.jar:/maven/org/apache/mina/mina-core/2.0.0-M5/mina-core-2.0.0-M5.jar:/maven/net/sf/ehcache/ehcache-core/2.4.4/ehcache-core-2.4.4.jar:/maven/antlr/antlr/2.7.7/antlr-2.7.7.jar:/maven/org/apache/directory/api/api-asn1-ber/1.0.0-M20/api-asn1-ber-1.0.0-M20.jar:/maven/org/slf4j/slf4j-log4j12/1.7.10/slf4j-log4j12-1.7.10.jar:/maven/org/
 codehaus/jackson/jackson-core-asl/1.9.13/jackson-core-asl-1.9.13.jar:/maven/org/codehaus/jackson/jackson-mapper-asl/1.9.13/jackson-mapper-asl-1.9.13.jar:/maven/xmlenc/xmlenc/0.52/xmlenc-0.52.jar:/maven/io/netty/netty/3.6.2.Final/netty-3.6.2.Final.jar:/maven/io/netty/netty-all/4.0.23.Final/netty-all-4.0.23.Final.jar:/maven/xerces/xercesImpl/2.9.1/xercesImpl-2.9.1.jar:/maven/xml-apis/xml-apis/1.3.04/xml-apis-1.3.04.jar:/maven/org/apache/htrace/htrace-core4/4.0.1-incubating/htrace-core4-4.0.1-incubating.jar:/maven/org/fusesource/leveldbjni/leveldbjni-all/1.8/leveldbjni-all-1.8.jar -sourcepath /build/source/hadoop-hdfs-project/hadoop-hdfs/src/main/java -apidir /build/source/hadoop-hdfs-project/hadoop-hdfs/target/site/jdiff/xml -apiname Apache Hadoop HDFS 2.8.3 -->
+<package name="org.apache.hadoop.hdfs">
+  <doc>
+  <![CDATA[<p>A distributed implementation of {@link
+org.apache.hadoop.fs.FileSystem}.  This is loosely modelled after
+Google's <a href="http://research.google.com/archive/gfs.html">GFS</a>.</p>
+
+<p>The most important difference is that unlike GFS, Hadoop DFS files 
+have strictly one writer at any one time.  Bytes are always appended 
+to the end of the writer's stream.  There is no notion of "record appends"
+or "mutations" that are then checked or reordered.  Writers simply emit 
+a byte stream.  That byte stream is guaranteed to be stored in the 
+order written.</p>]]>
+  </doc>
+</package>
+<package name="org.apache.hadoop.hdfs.net">
+</package>
+<package name="org.apache.hadoop.hdfs.protocol">
+</package>
+<package name="org.apache.hadoop.hdfs.protocol.datatransfer">
+</package>
+<package name="org.apache.hadoop.hdfs.protocol.datatransfer.sasl">
+</package>
+<package name="org.apache.hadoop.hdfs.protocolPB">
+</package>
+<package name="org.apache.hadoop.hdfs.qjournal.client">
+</package>
+<package name="org.apache.hadoop.hdfs.qjournal.protocol">
+</package>
+<package name="org.apache.hadoop.hdfs.qjournal.protocolPB">
+</package>
+<package name="org.apache.hadoop.hdfs.qjournal.server">
+  <!-- start interface org.apache.hadoop.hdfs.qjournal.server.JournalNodeMXBean -->
+  <interface name="JournalNodeMXBean"    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <method name="getJournalsStatus" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get status information (e.g., whether formatted) of JournalNode's journals.
+ 
+ @return A string presenting status for each journal]]>
+      </doc>
+    </method>
+    <doc>
+    <![CDATA[This is the JMX management interface for JournalNode information]]>
+    </doc>
+  </interface>
+  <!-- end interface org.apache.hadoop.hdfs.qjournal.server.JournalNodeMXBean -->
+</package>
+<package name="org.apache.hadoop.hdfs.security.token.block">
+</package>
+<package name="org.apache.hadoop.hdfs.security.token.delegation">
+</package>
+<package name="org.apache.hadoop.hdfs.server.balancer">
+</package>
+<package name="org.apache.hadoop.hdfs.server.blockmanagement">
+</package>
+<package name="org.apache.hadoop.hdfs.server.common">
+</package>
+<package name="org.apache.hadoop.hdfs.server.datanode">
+</package>
+<package name="org.apache.hadoop.hdfs.server.datanode.fsdataset">
+</package>
+<package name="org.apache.hadoop.hdfs.server.datanode.fsdataset.impl">
+</package>
+<package name="org.apache.hadoop.hdfs.server.datanode.metrics">
+</package>
+<package name="org.apache.hadoop.hdfs.server.datanode.web">
+</package>
+<package name="org.apache.hadoop.hdfs.server.datanode.web.webhdfs">
+</package>
+<package name="org.apache.hadoop.hdfs.server.mover">
+</package>
+<package name="org.apache.hadoop.hdfs.server.namenode">
+  <!-- start interface org.apache.hadoop.hdfs.server.namenode.AuditLogger -->
+  <interface name="AuditLogger"    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <method name="initialize"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <doc>
+      <![CDATA[Called during initialization of the logger.
+
+ @param conf The configuration object.]]>
+      </doc>
+    </method>
+    <method name="logAuditEvent"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="succeeded" type="boolean"/>
+      <param name="userName" type="java.lang.String"/>
+      <param name="addr" type="java.net.InetAddress"/>
+      <param name="cmd" type="java.lang.String"/>
+      <param name="src" type="java.lang.String"/>
+      <param name="dst" type="java.lang.String"/>
+      <param name="stat" type="org.apache.hadoop.fs.FileStatus"/>
+      <doc>
+      <![CDATA[Called to log an audit event.
+ <p>
+ This method must return as quickly as possible, since it's called
+ in a critical section of the NameNode's operation.
+
+ @param succeeded Whether authorization succeeded.
+ @param userName Name of the user executing the request.
+ @param addr Remote address of the request.
+ @param cmd The requested command.
+ @param src Path of affected source file.
+ @param dst Path of affected destination file (if any).
+ @param stat File information for operations that change the file's
+             metadata (permissions, owner, times, etc).]]>
+      </doc>
+    </method>
+    <doc>
+    <![CDATA[Interface defining an audit logger.]]>
+    </doc>
+  </interface>
+  <!-- end interface org.apache.hadoop.hdfs.server.namenode.AuditLogger -->
+  <!-- start class org.apache.hadoop.hdfs.server.namenode.HdfsAuditLogger -->
+  <class name="HdfsAuditLogger" extends="java.lang.Object"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="org.apache.hadoop.hdfs.server.namenode.AuditLogger"/>
+    <constructor name="HdfsAuditLogger"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="logAuditEvent"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="succeeded" type="boolean"/>
+      <param name="userName" type="java.lang.String"/>
+      <param name="addr" type="java.net.InetAddress"/>
+      <param name="cmd" type="java.lang.String"/>
+      <param name="src" type="java.lang.String"/>
+      <param name="dst" type="java.lang.String"/>
+      <param name="status" type="org.apache.hadoop.fs.FileStatus"/>
+    </method>
+    <method name="logAuditEvent"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="succeeded" type="boolean"/>
+      <param name="userName" type="java.lang.String"/>
+      <param name="addr" type="java.net.InetAddress"/>
+      <param name="cmd" type="java.lang.String"/>
+      <param name="src" type="java.lang.String"/>
+      <param name="dst" type="java.lang.String"/>
+      <param name="stat" type="org.apache.hadoop.fs.FileStatus"/>
+      <param name="callerContext" type="org.apache.hadoop.ipc.CallerContext"/>
+      <param name="ugi" type="org.apache.hadoop.security.UserGroupInformation"/>
+      <param name="dtSecretManager" type="org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager"/>
+      <doc>
+      <![CDATA[Same as
+ {@link #logAuditEvent(boolean, String, InetAddress, String, String, String,
+ FileStatus)} with additional parameters related to logging delegation token
+ tracking IDs.
+ 
+ @param succeeded Whether authorization succeeded.
+ @param userName Name of the user executing the request.
+ @param addr Remote address of the request.
+ @param cmd The requested command.
+ @param src Path of affected source file.
+ @param dst Path of affected destination file (if any).
+ @param stat File information for operations that change the file's metadata
+          (permissions, owner, times, etc).
+ @param callerContext Context information of the caller
+ @param ugi UserGroupInformation of the current user, or null if not logging
+          token tracking information
+ @param dtSecretManager The token secret manager, or null if not logging
+          token tracking information]]>
+      </doc>
+    </method>
+    <method name="logAuditEvent"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="succeeded" type="boolean"/>
+      <param name="userName" type="java.lang.String"/>
+      <param name="addr" type="java.net.InetAddress"/>
+      <param name="cmd" type="java.lang.String"/>
+      <param name="src" type="java.lang.String"/>
+      <param name="dst" type="java.lang.String"/>
+      <param name="stat" type="org.apache.hadoop.fs.FileStatus"/>
+      <param name="ugi" type="org.apache.hadoop.security.UserGroupInformation"/>
+      <param name="dtSecretManager" type="org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager"/>
+      <doc>
+      <![CDATA[Same as
+ {@link #logAuditEvent(boolean, String, InetAddress, String, String,
+ String, FileStatus, CallerContext, UserGroupInformation,
+ DelegationTokenSecretManager)} without {@link CallerContext} information.]]>
+      </doc>
+    </method>
+    <doc>
+    <![CDATA[Extension of {@link AuditLogger}.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.hdfs.server.namenode.HdfsAuditLogger -->
+  <!-- start class org.apache.hadoop.hdfs.server.namenode.INodeAttributeProvider -->
+  <class name="INodeAttributeProvider" extends="java.lang.Object"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="INodeAttributeProvider"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="start"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Initialize the provider. This method is called at NameNode startup
+ time.]]>
+      </doc>
+    </method>
+    <method name="stop"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Shutdown the provider. This method is called at NameNode shutdown time.]]>
+      </doc>
+    </method>
+    <method name="getAttributes" return="org.apache.hadoop.hdfs.server.namenode.INodeAttributes"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="fullPath" type="java.lang.String"/>
+      <param name="inode" type="org.apache.hadoop.hdfs.server.namenode.INodeAttributes"/>
+    </method>
+    <method name="getAttributes" return="org.apache.hadoop.hdfs.server.namenode.INodeAttributes"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="pathElements" type="java.lang.String[]"/>
+      <param name="inode" type="org.apache.hadoop.hdfs.server.namenode.INodeAttributes"/>
+    </method>
+    <method name="getAttributes" return="org.apache.hadoop.hdfs.server.namenode.INodeAttributes"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="components" type="byte[][]"/>
+      <param name="inode" type="org.apache.hadoop.hdfs.server.namenode.INodeAttributes"/>
+    </method>
+    <method name="getExternalAccessControlEnforcer" return="org.apache.hadoop.hdfs.server.namenode.INodeAttributeProvider.AccessControlEnforcer"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="defaultEnforcer" type="org.apache.hadoop.hdfs.server.namenode.INodeAttributeProvider.AccessControlEnforcer"/>
+      <doc>
+      <![CDATA[Can be over-ridden by implementations to provide a custom Access Control
+ Enforcer that can provide an alternate implementation of the
+ default permission checking logic.
+ @param defaultEnforcer The Default AccessControlEnforcer
+ @return The AccessControlEnforcer to use]]>
+      </doc>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.hdfs.server.namenode.INodeAttributeProvider -->
+</package>
+<package name="org.apache.hadoop.hdfs.server.namenode.ha">
+</package>
+<package name="org.apache.hadoop.hdfs.server.namenode.metrics">
+</package>
+<package name="org.apache.hadoop.hdfs.server.namenode.snapshot">
+</package>
+<package name="org.apache.hadoop.hdfs.server.namenode.top">
+</package>
+<package name="org.apache.hadoop.hdfs.server.namenode.top.metrics">
+</package>
+<package name="org.apache.hadoop.hdfs.server.namenode.top.window">
+</package>
+<package name="org.apache.hadoop.hdfs.server.namenode.web.resources">
+</package>
+<package name="org.apache.hadoop.hdfs.server.protocol">
+</package>
+<package name="org.apache.hadoop.hdfs.tools">
+</package>
+<package name="org.apache.hadoop.hdfs.tools.offlineEditsViewer">
+</package>
+<package name="org.apache.hadoop.hdfs.tools.offlineImageViewer">
+</package>
+<package name="org.apache.hadoop.hdfs.tools.snapshot">
+</package>
+<package name="org.apache.hadoop.hdfs.util">
+</package>
+<package name="org.apache.hadoop.hdfs.web">
+</package>
+<package name="org.apache.hadoop.hdfs.web.resources">
+</package>
+
+</api>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a7f8caf5/hadoop-mapreduce-project/dev-support/jdiff/Apache_Hadoop_MapReduce_Common_2.8.3.xml
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/dev-support/jdiff/Apache_Hadoop_MapReduce_Common_2.8.3.xml b/hadoop-mapreduce-project/dev-support/jdiff/Apache_Hadoop_MapReduce_Common_2.8.3.xml
new file mode 100644
index 0000000..b3d52bf
--- /dev/null
+++ b/hadoop-mapreduce-project/dev-support/jdiff/Apache_Hadoop_MapReduce_Common_2.8.3.xml
@@ -0,0 +1,113 @@
+<?xml version="1.0" encoding="iso-8859-1" standalone="no"?>
+<!-- Generated by the JDiff Javadoc doclet -->
+<!-- (http://www.jdiff.org) -->
+<!-- on Tue Dec 05 05:51:53 UTC 2017 -->
+
+<api
+  xmlns:xsi='http://www.w3.org/2001/XMLSchema-instance'
+  xsi:noNamespaceSchemaLocation='api.xsd'
+  name="Apache Hadoop MapReduce Common 2.8.3"
+  jdversion="1.0.9">
+
+<!--  Command line arguments =  -doclet org.apache.hadoop.classification.tools.IncludePublicAnnotationsJDiffDoclet -docletpath /build/source/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/target/hadoop-annotations.jar:/build/source/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/target/jdiff.jar -verbose -classpath /build/source/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/target/classes:/build/source/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/target/hadoop-yarn-common-2.8.3.jar:/build/source/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/target/hadoop-yarn-api-2.8.3.jar:/maven/javax/xml/bind/jaxb-api/2.2.2/jaxb-api-2.2.2.jar:/maven/javax/xml/stream/stax-api/1.0-2/stax-api-1.0-2.jar:/maven/javax/activation/activation/1.1/activation-1.1.jar:/maven/org/apache/commons/commons-compress/1.4.1/commons-compress-1.4.1.jar:/maven/org/tukaani/xz/1.0/xz-1.0.jar:/maven/javax/servlet/serv
 let-api/2.5/servlet-api-2.5.jar:/maven/org/mortbay/jetty/jetty-util/6.1.26/jetty-util-6.1.26.jar:/maven/com/sun/jersey/jersey-core/1.9/jersey-core-1.9.jar:/maven/com/sun/jersey/jersey-client/1.9/jersey-client-1.9.jar:/maven/org/codehaus/jackson/jackson-core-asl/1.9.13/jackson-core-asl-1.9.13.jar:/maven/org/codehaus/jackson/jackson-mapper-asl/1.9.13/jackson-mapper-asl-1.9.13.jar:/maven/org/codehaus/jackson/jackson-jaxrs/1.9.13/jackson-jaxrs-1.9.13.jar:/maven/org/codehaus/jackson/jackson-xc/1.9.13/jackson-xc-1.9.13.jar:/maven/commons-io/commons-io/2.4/commons-io-2.4.jar:/maven/com/google/inject/guice/3.0/guice-3.0.jar:/maven/javax/inject/javax.inject/1/javax.inject-1.jar:/maven/aopalliance/aopalliance/1.0/aopalliance-1.0.jar:/maven/com/sun/jersey/jersey-server/1.9/jersey-server-1.9.jar:/maven/asm/asm/3.2/asm-3.2.jar:/maven/com/sun/jersey/jersey-json/1.9/jersey-json-1.9.jar:/maven/org/codehaus/jettison/jettison/1.1/jettison-1.1.jar:/maven/com/sun/xml/bind/jaxb-impl/2.2.3-1/jaxb-impl-2.
 2.3-1.jar:/maven/com/sun/jersey/contribs/jersey-guice/1.9/jersey-guice-1.9.jar:/maven/log4j/log4j/1.2.17/log4j-1.2.17.jar:/build/source/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/target/hadoop-yarn-client-2.8.3.jar:/build/source/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/target/hadoop-mapreduce-client-core-2.8.3.jar:/build/source/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/target/hadoop-yarn-server-common-2.8.3.jar:/maven/org/apache/zookeeper/zookeeper/3.4.6/zookeeper-3.4.6.jar:/maven/org/fusesource/leveldbjni/leveldbjni-all/1.8/leveldbjni-all-1.8.jar:/maven/com/google/protobuf/protobuf-java/2.5.0/protobuf-java-2.5.0.jar:/maven/org/apache/avro/avro/1.7.4/avro-1.7.4.jar:/maven/com/thoughtworks/paranamer/paranamer/2.3/paranamer-2.3.jar:/maven/org/xerial/snappy/snappy-java/1.0.4.1/snappy-java-1.0.4.1.jar:/build/source/hadoop-common-project/hadoop-common/target/hadoop-common-2.8.3.jar:/maven/org/apache/commons/c
 ommons-math3/3.1.1/commons-math3-3.1.1.jar:/maven/xmlenc/xmlenc/0.52/xmlenc-0.52.jar:/maven/org/apache/httpcomponents/httpclient/4.5.2/httpclient-4.5.2.jar:/maven/org/apache/httpcomponents/httpcore/4.4.4/httpcore-4.4.4.jar:/maven/commons-net/commons-net/3.1/commons-net-3.1.jar:/maven/org/mortbay/jetty/jetty/6.1.26/jetty-6.1.26.jar:/maven/org/mortbay/jetty/jetty-sslengine/6.1.26/jetty-sslengine-6.1.26.jar:/maven/javax/servlet/jsp/jsp-api/2.1/jsp-api-2.1.jar:/maven/net/java/dev/jets3t/jets3t/0.9.0/jets3t-0.9.0.jar:/maven/com/jamesmurty/utils/java-xmlbuilder/0.4/java-xmlbuilder-0.4.jar:/maven/commons-configuration/commons-configuration/1.6/commons-configuration-1.6.jar:/maven/commons-digester/commons-digester/1.8/commons-digester-1.8.jar:/maven/commons-beanutils/commons-beanutils/1.7.0/commons-beanutils-1.7.0.jar:/maven/commons-beanutils/commons-beanutils-core/1.8.0/commons-beanutils-core-1.8.0.jar:/maven/com/google/code/gson/gson/2.2.4/gson-2.2.4.jar:/build/source/hadoop-common-projec
 t/hadoop-auth/target/hadoop-auth-2.8.3.jar:/maven/com/nimbusds/nimbus-jose-jwt/3.9/nimbus-jose-jwt-3.9.jar:/maven/net/jcip/jcip-annotations/1.0/jcip-annotations-1.0.jar:/maven/net/minidev/json-smart/1.1.1/json-smart-1.1.1.jar:/maven/org/apache/directory/server/apacheds-kerberos-codec/2.0.0-M15/apacheds-kerberos-codec-2.0.0-M15.jar:/maven/org/apache/directory/server/apacheds-i18n/2.0.0-M15/apacheds-i18n-2.0.0-M15.jar:/maven/org/apache/directory/api/api-asn1-api/1.0.0-M20/api-asn1-api-1.0.0-M20.jar:/maven/org/apache/directory/api/api-util/1.0.0-M20/api-util-1.0.0-M20.jar:/maven/org/apache/curator/curator-framework/2.7.1/curator-framework-2.7.1.jar:/maven/com/jcraft/jsch/0.1.54/jsch-0.1.54.jar:/maven/org/apache/curator/curator-client/2.7.1/curator-client-2.7.1.jar:/maven/org/apache/curator/curator-recipes/2.7.1/curator-recipes-2.7.1.jar:/maven/com/google/code/findbugs/jsr305/3.0.0/jsr305-3.0.0.jar:/maven/org/apache/htrace/htrace-core4/4.0.1-incubating/htrace-core4-4.0.1-incubating.jar:
 /maven/org/slf4j/slf4j-api/1.7.10/slf4j-api-1.7.10.jar:/maven/org/slf4j/slf4j-log4j12/1.7.10/slf4j-log4j12-1.7.10.jar:/build/source/hadoop-common-project/hadoop-annotations/target/hadoop-annotations-2.8.3.jar:/usr/lib/jvm/java-7-openjdk-amd64/lib/tools.jar:/maven/com/google/inject/extensions/guice-servlet/3.0/guice-servlet-3.0.jar:/maven/io/netty/netty/3.6.2.Final/netty-3.6.2.Final.jar:/maven/commons-logging/commons-logging/1.1.3/commons-logging-1.1.3.jar:/maven/com/google/guava/guava/11.0.2/guava-11.0.2.jar:/maven/commons-codec/commons-codec/1.4/commons-codec-1.4.jar:/maven/commons-cli/commons-cli/1.2/commons-cli-1.2.jar:/maven/commons-lang/commons-lang/2.6/commons-lang-2.6.jar:/maven/commons-collections/commons-collections/3.2.2/commons-collections-3.2.2.jar -sourcepath /build/source/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java -apidir /build/source/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/targ
 et/site/jdiff/xml -apiname Apache Hadoop MapReduce Common 2.8.3 -->
+<package name="org.apache.hadoop.mapred">
+</package>
+<package name="org.apache.hadoop.mapreduce">
+</package>
+<package name="org.apache.hadoop.mapreduce.v2.api.protocolrecords">
+  <!-- start interface org.apache.hadoop.mapreduce.v2.api.protocolrecords.CancelDelegationTokenRequest -->
+  <interface name="CancelDelegationTokenRequest"    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <method name="getDelegationToken" return="org.apache.hadoop.yarn.api.records.Token"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="setDelegationToken"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="dToken" type="org.apache.hadoop.yarn.api.records.Token"/>
+    </method>
+    <doc>
+    <![CDATA[The request issued by the client to the {@code ResourceManager} to cancel a
+ delegation token.]]>
+    </doc>
+  </interface>
+  <!-- end interface org.apache.hadoop.mapreduce.v2.api.protocolrecords.CancelDelegationTokenRequest -->
+  <!-- start interface org.apache.hadoop.mapreduce.v2.api.protocolrecords.CancelDelegationTokenResponse -->
+  <interface name="CancelDelegationTokenResponse"    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <doc>
+    <![CDATA[The response from the {@code ResourceManager} to a cancelDelegationToken
+ request.]]>
+    </doc>
+  </interface>
+  <!-- end interface org.apache.hadoop.mapreduce.v2.api.protocolrecords.CancelDelegationTokenResponse -->
+  <!-- start interface org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetDelegationTokenRequest -->
+  <interface name="GetDelegationTokenRequest"    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <method name="getRenewer" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="setRenewer"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="renewer" type="java.lang.String"/>
+    </method>
+  </interface>
+  <!-- end interface org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetDelegationTokenRequest -->
+  <!-- start interface org.apache.hadoop.mapreduce.v2.api.protocolrecords.RenewDelegationTokenRequest -->
+  <interface name="RenewDelegationTokenRequest"    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <method name="getDelegationToken" return="org.apache.hadoop.yarn.api.records.Token"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="setDelegationToken"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="dToken" type="org.apache.hadoop.yarn.api.records.Token"/>
+    </method>
+    <doc>
+    <![CDATA[The request issued by the client to renew a delegation token from
+ the {@code ResourceManager}.]]>
+    </doc>
+  </interface>
+  <!-- end interface org.apache.hadoop.mapreduce.v2.api.protocolrecords.RenewDelegationTokenRequest -->
+  <!-- start interface org.apache.hadoop.mapreduce.v2.api.protocolrecords.RenewDelegationTokenResponse -->
+  <interface name="RenewDelegationTokenResponse"    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <method name="getNextExpirationTime" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="setNextExpirationTime"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="expTime" type="long"/>
+    </method>
+    <doc>
+    <![CDATA[The response to a renewDelegationToken call to the {@code ResourceManager}.]]>
+    </doc>
+  </interface>
+  <!-- end interface org.apache.hadoop.mapreduce.v2.api.protocolrecords.RenewDelegationTokenResponse -->
+</package>
+<package name="org.apache.hadoop.mapreduce.v2.security">
+</package>
+<package name="org.apache.hadoop.yarn.proto">
+</package>
+
+</api>


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


[11/50] [abbrv] hadoop git commit: HDFS-12903. [READ] Fix closing streams in ImageWriter. Contributed by Virajith Jalaparti

Posted by na...@apache.org.
HDFS-12903. [READ] Fix closing streams in ImageWriter. Contributed by Virajith Jalaparti


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

Branch: refs/heads/yarn-3409
Commit: 4b3a785914d890c47745e57d12a5a9abd084ffc1
Parents: e515103
Author: Chris Douglas <cd...@apache.org>
Authored: Fri Dec 15 17:41:46 2017 -0800
Committer: Chris Douglas <cd...@apache.org>
Committed: Fri Dec 15 17:51:42 2017 -0800

----------------------------------------------------------------------
 .../dev-support/findbugs-exclude.xml            | 28 ++++++++++++++++++++
 1 file changed, 28 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4b3a7859/hadoop-tools/hadoop-fs2img/dev-support/findbugs-exclude.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/dev-support/findbugs-exclude.xml b/hadoop-tools/hadoop-fs2img/dev-support/findbugs-exclude.xml
new file mode 100644
index 0000000..b60767f
--- /dev/null
+++ b/hadoop-tools/hadoop-fs2img/dev-support/findbugs-exclude.xml
@@ -0,0 +1,28 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+   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.
+-->
+<FindBugsFilter>
+
+  <!-- Spotbugs 3.1.x not detecting that IOUtils::cleanupWithLogger
+       closes streams safely -->
+  <Match>
+    <Class name="org.apache.hadoop.hdfs.server.namenode.ImageWriter" />
+    <Method name="&lt;init&gt;" />
+    <Bug pattern="OBL_UNSATISFIED_OBLIGATION_EXCEPTION_EDGE" />
+  </Match>
+
+</FindBugsFilter>


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


[41/50] [abbrv] hadoop git commit: YARN-7577. Unit Fail: TestAMRestart#testPreemptedAMRestartOnRMRestart (miklos.szegedi@cloudera.com via rkanter)

Posted by na...@apache.org.
YARN-7577. Unit Fail: TestAMRestart#testPreemptedAMRestartOnRMRestart (miklos.szegedi@cloudera.com via rkanter)


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

Branch: refs/heads/yarn-3409
Commit: 382215c72b93d6a97d813f407cf6496a7c3f2a4a
Parents: 1ba491ff
Author: Robert Kanter <rk...@apache.org>
Authored: Wed Dec 20 13:39:00 2017 -0800
Committer: Robert Kanter <rk...@apache.org>
Committed: Wed Dec 20 13:39:00 2017 -0800

----------------------------------------------------------------------
 .../applicationsmanager/TestAMRestart.java      | 131 +++++++++++--------
 1 file changed, 73 insertions(+), 58 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/382215c7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java
index 3d523aa..4add186 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.applicationsmanager;
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
@@ -45,6 +46,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.MockAM;
 import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
 import org.apache.hadoop.yarn.server.resourcemanager.MockMemoryRMStateStore;
+import org.apache.hadoop.yarn.server.resourcemanager.ParameterizedSchedulerTestBase;
 import org.apache.hadoop.yarn.server.resourcemanager.TestRMRestart;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.MemoryRMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.ApplicationStateData;
@@ -63,14 +65,20 @@ import org.apache.hadoop.yarn.util.Records;
 import org.junit.Assert;
 import org.junit.Test;
 
-public class TestAMRestart {
+/**
+ * Test AM restart functions.
+ */
+public class TestAMRestart extends ParameterizedSchedulerTestBase {
+
+  public TestAMRestart(SchedulerType type) throws IOException {
+    super(type);
+  }
 
   @Test(timeout = 30000)
   public void testAMRestartWithExistingContainers() throws Exception {
-    YarnConfiguration conf = new YarnConfiguration();
-    conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
+    getConf().setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
 
-    MockRM rm1 = new MockRM(conf);
+    MockRM rm1 = new MockRM(getConf());
     rm1.start();
     RMApp app1 =
         rm1.submitApp(200, "name", "user",
@@ -266,15 +274,14 @@ public class TestAMRestart {
 
   @Test(timeout = 30000)
   public void testNMTokensRebindOnAMRestart() throws Exception {
-    YarnConfiguration conf = new YarnConfiguration();
-    conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 3);
+    getConf().setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 3);
     // To prevent test from blacklisting nm1 for AM, we sit threshold to half
     // of 2 nodes which is 1
-    conf.setFloat(
+    getConf().setFloat(
         YarnConfiguration.AM_SCHEDULING_NODE_BLACKLISTING_DISABLE_THRESHOLD,
         0.5f);
 
-    MockRM rm1 = new MockRM(conf);
+    MockRM rm1 = new MockRM(getConf());
     rm1.start();
     RMApp app1 =
         rm1.submitApp(200, "myname", "myuser",
@@ -378,11 +385,11 @@ public class TestAMRestart {
   // should not be counted towards AM max retry count.
   @Test(timeout = 100000)
   public void testShouldNotCountFailureToMaxAttemptRetry() throws Exception {
-    YarnConfiguration conf = new YarnConfiguration();
-    conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
-    conf.setBoolean(YarnConfiguration.RECOVERY_ENABLED, true);
-    conf.set(YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName());
-    MockRM rm1 = new MockRM(conf);
+    getConf().setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
+    getConf().setBoolean(YarnConfiguration.RECOVERY_ENABLED, true);
+    getConf().set(
+        YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName());
+    MockRM rm1 = new MockRM(getConf());
     rm1.start();
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 8000, rm1.getResourceTrackerService());
@@ -503,11 +510,11 @@ public class TestAMRestart {
 
   @Test(timeout = 100000)
   public void testMaxAttemptOneMeansOne() throws Exception {
-    YarnConfiguration conf = new YarnConfiguration();
-    conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1);
-    conf.setBoolean(YarnConfiguration.RECOVERY_ENABLED, true);
-    conf.set(YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName());
-    MockRM rm1 = new MockRM(conf);
+    getConf().setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1);
+    getConf().setBoolean(YarnConfiguration.RECOVERY_ENABLED, true);
+    getConf().set(
+        YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName());
+    MockRM rm1 = new MockRM(getConf());
     rm1.start();
     MockNM nm1 =
         new MockNM("127.0.0.1:1234", 8000, rm1.getResourceTrackerService());
@@ -537,14 +544,15 @@ public class TestAMRestart {
   // re-launch the AM.
   @Test(timeout = 60000)
   public void testPreemptedAMRestartOnRMRestart() throws Exception {
-    YarnConfiguration conf = new YarnConfiguration();
-    conf.setBoolean(YarnConfiguration.RECOVERY_ENABLED, true);
-    conf.setBoolean(YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_ENABLED, false);
+    getConf().setBoolean(YarnConfiguration.RECOVERY_ENABLED, true);
+    getConf().setBoolean(
+        YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_ENABLED, false);
 
-    conf.set(YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName());
-    conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
+    getConf().set(
+        YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName());
+    getConf().setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
 
-    MockRM rm1 = new MockRM(conf);
+    MockRM rm1 = new MockRM(getConf());
     MemoryRMStateStore memStore = (MemoryRMStateStore) rm1.getRMStateStore();
     rm1.start();
     MockNM nm1 =
@@ -584,12 +592,19 @@ public class TestAMRestart {
     ApplicationStateData appState =
         memStore.getState().getApplicationState().get(app1.getApplicationId());
     Assert.assertEquals(2, appState.getAttemptCount());
-    // attempt stored has the preempted container exit status.
-    Assert.assertEquals(ContainerExitStatus.PREEMPTED,
-        appState.getAttempt(am2.getApplicationAttemptId())
-            .getAMContainerExitStatus());
+    if (getSchedulerType().equals(SchedulerType.FAIR)) {
+      // attempt stored has the preempted container exit status.
+      Assert.assertEquals(ContainerExitStatus.KILLED_BY_RESOURCEMANAGER,
+          appState.getAttempt(am2.getApplicationAttemptId())
+              .getAMContainerExitStatus());
+    } else {
+      // attempt stored has the preempted container exit status.
+      Assert.assertEquals(ContainerExitStatus.PREEMPTED,
+          appState.getAttempt(am2.getApplicationAttemptId())
+              .getAMContainerExitStatus());
+    }
     // Restart rm.
-    MockRM rm2 = new MockRM(conf, memStore);
+    MockRM rm2 = new MockRM(getConf(), memStore);
     nm1.setResourceTrackerService(rm2.getResourceTrackerService());
     nm1.registerNode();
     rm2.start();
@@ -615,15 +630,16 @@ public class TestAMRestart {
   @Test(timeout = 50000)
   public void testRMRestartOrFailoverNotCountedForAMFailures()
       throws Exception {
-    YarnConfiguration conf = new YarnConfiguration();
-    conf.setBoolean(YarnConfiguration.RECOVERY_ENABLED, true);
-    conf.setBoolean(YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_ENABLED, false);
+    getConf().setBoolean(YarnConfiguration.RECOVERY_ENABLED, true);
+    getConf().setBoolean(
+        YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_ENABLED, false);
 
-    conf.set(YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName());
+    getConf().set(
+        YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName());
     // explicitly set max-am-retry count as 2.
-    conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
+    getConf().setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
 
-    MockRM rm1 = new MockRM(conf);
+    MockRM rm1 = new MockRM(getConf());
     MemoryRMStateStore memStore = (MemoryRMStateStore) rm1.getRMStateStore();
     rm1.start();
     AbstractYarnScheduler scheduler =
@@ -651,7 +667,7 @@ public class TestAMRestart {
     RMAppAttempt attempt2 = app1.getCurrentAppAttempt();
 
     // Restart rm.
-    MockRM rm2 = new MockRM(conf, memStore);
+    MockRM rm2 = new MockRM(getConf(), memStore);
     rm2.start();
     ApplicationStateData appState =
         memStore.getState().getApplicationState().get(app1.getApplicationId());
@@ -688,14 +704,15 @@ public class TestAMRestart {
 
   @Test (timeout = 120000)
   public void testRMAppAttemptFailuresValidityInterval() throws Exception {
-    YarnConfiguration conf = new YarnConfiguration();
-    conf.setBoolean(YarnConfiguration.RECOVERY_ENABLED, true);
-    conf.setBoolean(YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_ENABLED, false);
+    getConf().setBoolean(YarnConfiguration.RECOVERY_ENABLED, true);
+    getConf().setBoolean(
+        YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_ENABLED, false);
 
-    conf.set(YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName());
+    getConf().set(
+        YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName());
     // explicitly set max-am-retry count as 2.
-    conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
-    MockRM rm1 = new MockRM(conf);
+    getConf().setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
+    MockRM rm1 = new MockRM(getConf());
     rm1.start();
 
     MockMemoryRMStateStore memStore =
@@ -765,7 +782,7 @@ public class TestAMRestart {
 
     // Restart rm.
     @SuppressWarnings("resource")
-    MockRM rm2 = new MockRM(conf, memStore);
+    MockRM rm2 = new MockRM(getConf(), memStore);
     rm2.start();
 
     MockMemoryRMStateStore memStore1 =
@@ -834,12 +851,11 @@ public class TestAMRestart {
     return false;
   }
 
-  @Test(timeout = 30000)
+  @Test(timeout = 40000)
   public void testAMRestartNotLostContainerCompleteMsg() throws Exception {
-    YarnConfiguration conf = new YarnConfiguration();
-    conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
+    getConf().setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
 
-    MockRM rm1 = new MockRM(conf);
+    MockRM rm1 = new MockRM(getConf());
     rm1.start();
     RMApp app1 =
         rm1.submitApp(200, "name", "user",
@@ -934,11 +950,10 @@ public class TestAMRestart {
   @Test (timeout = 20000)
   public void testAMRestartNotLostContainerAfterAttemptFailuresValidityInterval()
       throws Exception {
-    YarnConfiguration conf = new YarnConfiguration();
     // explicitly set max-am-retry count as 2.
-    conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
+    getConf().setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
 
-    MockRM rm1 = new MockRM(conf);
+    MockRM rm1 = new MockRM(getConf());
     rm1.start();
     MockNM nm1 =
             new MockNM("127.0.0.1:1234", 8000, rm1.getResourceTrackerService());
@@ -1019,16 +1034,16 @@ public class TestAMRestart {
   @Test(timeout = 200000)
   public void testContainersFromPreviousAttemptsWithRMRestart()
       throws Exception {
-    YarnConfiguration conf = new YarnConfiguration();
-    conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
-    conf.setBoolean(YarnConfiguration.RECOVERY_ENABLED, true);
-    conf.setBoolean(
+    getConf().setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
+    getConf().setBoolean(YarnConfiguration.RECOVERY_ENABLED, true);
+    getConf().setBoolean(
         YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_ENABLED, true);
-    conf.setLong(
+    getConf().setLong(
         YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_SCHEDULING_WAIT_MS, 0);
-    conf.set(YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName());
+    getConf()
+        .set(YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName());
 
-    MockRM rm1 = new MockRM(conf);
+    MockRM rm1 = new MockRM(getConf());
     MemoryRMStateStore memStore = (MemoryRMStateStore) rm1.getRMStateStore();
     rm1.start();
     YarnScheduler scheduler = rm1.getResourceScheduler();
@@ -1071,7 +1086,7 @@ public class TestAMRestart {
         (AbstractYarnScheduler)scheduler, am1.getApplicationAttemptId());
 
     // restart rm
-    MockRM rm2 = new MockRM(conf, memStore);
+    MockRM rm2 = new MockRM(getConf(), memStore);
     rm2.start();
     nm1.setResourceTrackerService(rm2.getResourceTrackerService());
     NMContainerStatus container2Status =


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


[02/50] [abbrv] hadoop git commit: HDFS-12712. [9806] Code style cleanup

Posted by na...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/8239e3af/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java
deleted file mode 100644
index 1023616..0000000
--- a/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java
+++ /dev/null
@@ -1,934 +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.hdfs.server.namenode;
-
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.OutputStreamWriter;
-import java.io.Writer;
-import java.net.InetSocketAddress;
-import java.nio.ByteBuffer;
-import java.nio.channels.Channels;
-import java.nio.channels.ReadableByteChannel;
-import java.nio.file.Files;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.Random;
-import java.util.Set;
-
-import org.apache.commons.io.FileUtils;
-import org.apache.hadoop.fs.BlockLocation;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.FileUtil;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.StorageType;
-import org.apache.hadoop.hdfs.DFSClient;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.DFSTestUtil;
-import org.apache.hadoop.hdfs.DistributedFileSystem;
-import org.apache.hadoop.hdfs.HdfsConfiguration;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
-import org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMap;
-import org.apache.hadoop.hdfs.server.aliasmap.InMemoryLevelDBAliasMapServer;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStatistics;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
-import org.apache.hadoop.hdfs.server.blockmanagement.ProvidedStorageMap;
-import org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap;
-import org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.InMemoryLevelDBAliasMapClient;
-import org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.TextFileRegionAliasMap;
-import org.apache.hadoop.hdfs.server.datanode.DataNode;
-
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_REPLICATOR_CLASSNAME_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY;
-
-import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
-import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsVolumeImpl;
-import org.apache.hadoop.hdfs.server.protocol.StorageReport;
-import org.apache.hadoop.net.NodeBase;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.TextFileRegionAliasMap.fileNameFromBlockPoolID;
-import static org.apache.hadoop.net.NodeBase.PATH_SEPARATOR_STR;
-import static org.junit.Assert.*;
-
-public class TestNameNodeProvidedImplementation {
-
-  @Rule public TestName name = new TestName();
-  public static final Logger LOG =
-      LoggerFactory.getLogger(TestNameNodeProvidedImplementation.class);
-
-  final Random r = new Random();
-  final File fBASE = new File(MiniDFSCluster.getBaseDirectory());
-  final Path BASE = new Path(fBASE.toURI().toString());
-  final Path NAMEPATH = new Path(BASE, "providedDir");
-  final Path NNDIRPATH = new Path(BASE, "nnDir");
-  final String SINGLEUSER = "usr1";
-  final String SINGLEGROUP = "grp1";
-  private final int numFiles = 10;
-  private final String filePrefix = "file";
-  private final String fileSuffix = ".dat";
-  private final int baseFileLen = 1024;
-  private long providedDataSize = 0;
-  private final String bpid = "BP-1234-10.1.1.1-1224";
-
-  Configuration conf;
-  MiniDFSCluster cluster;
-
-  @Before
-  public void setSeed() throws Exception {
-    if (fBASE.exists() && !FileUtil.fullyDelete(fBASE)) {
-      throw new IOException("Could not fully delete " + fBASE);
-    }
-    long seed = r.nextLong();
-    r.setSeed(seed);
-    System.out.println(name.getMethodName() + " seed: " + seed);
-    conf = new HdfsConfiguration();
-    conf.set(SingleUGIResolver.USER, SINGLEUSER);
-    conf.set(SingleUGIResolver.GROUP, SINGLEGROUP);
-
-    conf.set(DFSConfigKeys.DFS_PROVIDER_STORAGEUUID,
-        DFSConfigKeys.DFS_PROVIDER_STORAGEUUID_DEFAULT);
-    conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_PROVIDED_ENABLED, true);
-
-    conf.setClass(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_CLASS,
-        TextFileRegionAliasMap.class, BlockAliasMap.class);
-    conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_WRITE_DIR,
-        NNDIRPATH.toString());
-    conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_READ_FILE,
-        new Path(NNDIRPATH, fileNameFromBlockPoolID(bpid)).toString());
-    conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_DELIMITER, ",");
-
-    conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR_PROVIDED,
-        new File(NAMEPATH.toUri()).toString());
-    File imageDir = new File(NAMEPATH.toUri());
-    if (!imageDir.exists()) {
-      LOG.info("Creating directory: " + imageDir);
-      imageDir.mkdirs();
-    }
-
-    File nnDir = new File(NNDIRPATH.toUri());
-    if (!nnDir.exists()) {
-      nnDir.mkdirs();
-    }
-
-    // create 10 random files under BASE
-    for (int i=0; i < numFiles; i++) {
-      File newFile = new File(
-          new Path(NAMEPATH, filePrefix + i + fileSuffix).toUri());
-      if(!newFile.exists()) {
-        try {
-          LOG.info("Creating " + newFile.toString());
-          newFile.createNewFile();
-          Writer writer = new OutputStreamWriter(
-              new FileOutputStream(newFile.getAbsolutePath()), "utf-8");
-          for(int j=0; j < baseFileLen*i; j++) {
-            writer.write("0");
-          }
-          writer.flush();
-          writer.close();
-          providedDataSize += newFile.length();
-        } catch (IOException e) {
-          e.printStackTrace();
-        }
-      }
-    }
-  }
-
-  @After
-  public void shutdown() throws Exception {
-    try {
-      if (cluster != null) {
-        cluster.shutdown(true, true);
-      }
-    } finally {
-      cluster = null;
-    }
-  }
-
-  void createImage(TreeWalk t, Path out,
-      Class<? extends BlockResolver> blockIdsClass) throws Exception {
-    createImage(t, out, blockIdsClass, "", TextFileRegionAliasMap.class);
-  }
-
-  void createImage(TreeWalk t, Path out,
-      Class<? extends BlockResolver> blockIdsClass, String clusterID,
-      Class<? extends BlockAliasMap> aliasMapClass) throws Exception {
-    ImageWriter.Options opts = ImageWriter.defaults();
-    opts.setConf(conf);
-    opts.output(out.toString())
-        .blocks(aliasMapClass)
-        .blockIds(blockIdsClass)
-        .clusterID(clusterID)
-        .blockPoolID(bpid);
-    try (ImageWriter w = new ImageWriter(opts)) {
-      for (TreePath e : t) {
-        w.accept(e);
-      }
-    }
-  }
-  void startCluster(Path nspath, int numDatanodes,
-      StorageType[] storageTypes,
-      StorageType[][] storageTypesPerDatanode,
-      boolean doFormat) throws IOException {
-    startCluster(nspath, numDatanodes, storageTypes, storageTypesPerDatanode,
-        doFormat, null);
-  }
-
-  void startCluster(Path nspath, int numDatanodes,
-      StorageType[] storageTypes,
-      StorageType[][] storageTypesPerDatanode,
-      boolean doFormat, String[] racks) throws IOException {
-    conf.set(DFS_NAMENODE_NAME_DIR_KEY, nspath.toString());
-
-    if (storageTypesPerDatanode != null) {
-      cluster = new MiniDFSCluster.Builder(conf)
-          .format(doFormat)
-          .manageNameDfsDirs(doFormat)
-          .numDataNodes(numDatanodes)
-          .storageTypes(storageTypesPerDatanode)
-          .racks(racks)
-          .build();
-    } else if (storageTypes != null) {
-      cluster = new MiniDFSCluster.Builder(conf)
-          .format(doFormat)
-          .manageNameDfsDirs(doFormat)
-          .numDataNodes(numDatanodes)
-          .storagesPerDatanode(storageTypes.length)
-          .storageTypes(storageTypes)
-          .racks(racks)
-          .build();
-    } else {
-      cluster = new MiniDFSCluster.Builder(conf)
-          .format(doFormat)
-          .manageNameDfsDirs(doFormat)
-          .numDataNodes(numDatanodes)
-          .racks(racks)
-          .build();
-    }
-    cluster.waitActive();
-  }
-
-  @Test(timeout=20000)
-  public void testLoadImage() throws Exception {
-    final long seed = r.nextLong();
-    LOG.info("NAMEPATH: " + NAMEPATH);
-    createImage(new RandomTreeWalk(seed), NNDIRPATH, FixedBlockResolver.class);
-    startCluster(NNDIRPATH, 0,
-        new StorageType[] {StorageType.PROVIDED, StorageType.DISK}, null,
-        false);
-
-    FileSystem fs = cluster.getFileSystem();
-    for (TreePath e : new RandomTreeWalk(seed)) {
-      FileStatus rs = e.getFileStatus();
-      Path hp = new Path(rs.getPath().toUri().getPath());
-      assertTrue(fs.exists(hp));
-      FileStatus hs = fs.getFileStatus(hp);
-      assertEquals(rs.getPath().toUri().getPath(),
-                   hs.getPath().toUri().getPath());
-      assertEquals(rs.getPermission(), hs.getPermission());
-      assertEquals(rs.getLen(), hs.getLen());
-      assertEquals(SINGLEUSER, hs.getOwner());
-      assertEquals(SINGLEGROUP, hs.getGroup());
-      assertEquals(rs.getAccessTime(), hs.getAccessTime());
-      assertEquals(rs.getModificationTime(), hs.getModificationTime());
-    }
-  }
-
-  @Test(timeout=30000)
-  public void testProvidedReporting() throws Exception {
-    conf.setClass(ImageWriter.Options.UGI_CLASS,
-        SingleUGIResolver.class, UGIResolver.class);
-    createImage(new FSTreeWalk(NAMEPATH, conf), NNDIRPATH,
-        FixedBlockResolver.class);
-    int numDatanodes = 10;
-    startCluster(NNDIRPATH, numDatanodes,
-        new StorageType[] {StorageType.PROVIDED, StorageType.DISK}, null,
-        false);
-    long diskCapacity = 1000;
-    // set the DISK capacity for testing
-    for (DataNode dn: cluster.getDataNodes()) {
-      for (FsVolumeSpi ref : dn.getFSDataset().getFsVolumeReferences()) {
-        if (ref.getStorageType() == StorageType.DISK) {
-          ((FsVolumeImpl) ref).setCapacityForTesting(diskCapacity);
-        }
-      }
-    }
-    // trigger heartbeats to update the capacities
-    cluster.triggerHeartbeats();
-    Thread.sleep(10000);
-    // verify namenode stats
-    FSNamesystem namesystem = cluster.getNameNode().getNamesystem();
-    DatanodeStatistics dnStats = namesystem.getBlockManager()
-        .getDatanodeManager().getDatanodeStatistics();
-
-    // total capacity reported includes only the local volumes and
-    // not the provided capacity
-    assertEquals(diskCapacity * numDatanodes, namesystem.getTotal());
-
-    // total storage used should be equal to the totalProvidedStorage
-    // no capacity should be remaining!
-    assertEquals(providedDataSize, dnStats.getProvidedCapacity());
-    assertEquals(providedDataSize, namesystem.getProvidedCapacityTotal());
-    assertEquals(providedDataSize, dnStats.getStorageTypeStats()
-        .get(StorageType.PROVIDED).getCapacityTotal());
-    assertEquals(providedDataSize, dnStats.getStorageTypeStats()
-        .get(StorageType.PROVIDED).getCapacityUsed());
-
-    // verify datanode stats
-    for (DataNode dn: cluster.getDataNodes()) {
-      for (StorageReport report : dn.getFSDataset()
-          .getStorageReports(namesystem.getBlockPoolId())) {
-        if (report.getStorage().getStorageType() == StorageType.PROVIDED) {
-          assertEquals(providedDataSize, report.getCapacity());
-          assertEquals(providedDataSize, report.getDfsUsed());
-          assertEquals(providedDataSize, report.getBlockPoolUsed());
-          assertEquals(0, report.getNonDfsUsed());
-          assertEquals(0, report.getRemaining());
-        }
-      }
-    }
-
-    DFSClient client = new DFSClient(new InetSocketAddress("localhost",
-            cluster.getNameNodePort()), cluster.getConfiguration(0));
-    BlockManager bm = namesystem.getBlockManager();
-    for (int fileId = 0; fileId < numFiles; fileId++) {
-      String filename = "/" + filePrefix + fileId + fileSuffix;
-      LocatedBlocks locatedBlocks = client.getLocatedBlocks(
-          filename, 0, baseFileLen);
-      for (LocatedBlock locatedBlock : locatedBlocks.getLocatedBlocks()) {
-        BlockInfo blockInfo =
-            bm.getStoredBlock(locatedBlock.getBlock().getLocalBlock());
-        Iterator<DatanodeStorageInfo> storagesItr = blockInfo.getStorageInfos();
-
-        DatanodeStorageInfo info = storagesItr.next();
-        assertEquals(StorageType.PROVIDED, info.getStorageType());
-        DatanodeDescriptor dnDesc = info.getDatanodeDescriptor();
-        // check the locations that are returned by FSCK have the right name
-        assertEquals(ProvidedStorageMap.ProvidedDescriptor.NETWORK_LOCATION
-            + PATH_SEPARATOR_STR + ProvidedStorageMap.ProvidedDescriptor.NAME,
-            NodeBase.getPath(dnDesc));
-        // no DatanodeStorageInfos should remain
-        assertFalse(storagesItr.hasNext());
-      }
-    }
-  }
-
-  @Test(timeout=500000)
-  public void testDefaultReplication() throws Exception {
-    int targetReplication = 2;
-    conf.setInt(FixedBlockMultiReplicaResolver.REPLICATION, targetReplication);
-    createImage(new FSTreeWalk(NAMEPATH, conf), NNDIRPATH,
-        FixedBlockMultiReplicaResolver.class);
-    // make the last Datanode with only DISK
-    startCluster(NNDIRPATH, 3, null,
-        new StorageType[][] {
-            {StorageType.PROVIDED, StorageType.DISK},
-            {StorageType.PROVIDED, StorageType.DISK},
-            {StorageType.DISK}},
-        false);
-    // wait for the replication to finish
-    Thread.sleep(50000);
-
-    FileSystem fs = cluster.getFileSystem();
-    int count = 0;
-    for (TreePath e : new FSTreeWalk(NAMEPATH, conf)) {
-      FileStatus rs = e.getFileStatus();
-      Path hp = removePrefix(NAMEPATH, rs.getPath());
-      LOG.info("hp " + hp.toUri().getPath());
-      //skip HDFS specific files, which may have been created later on.
-      if (hp.toString().contains("in_use.lock")
-          || hp.toString().contains("current")) {
-        continue;
-      }
-      e.accept(count++);
-      assertTrue(fs.exists(hp));
-      FileStatus hs = fs.getFileStatus(hp);
-
-      if (rs.isFile()) {
-        BlockLocation[] bl = fs.getFileBlockLocations(
-            hs.getPath(), 0, hs.getLen());
-        int i = 0;
-        for(; i < bl.length; i++) {
-          int currentRep = bl[i].getHosts().length;
-          assertEquals(targetReplication , currentRep);
-        }
-      }
-    }
-  }
-
-
-  static Path removePrefix(Path base, Path walk) {
-    Path wpath = new Path(walk.toUri().getPath());
-    Path bpath = new Path(base.toUri().getPath());
-    Path ret = new Path("/");
-    while (!(bpath.equals(wpath) || "".equals(wpath.getName()))) {
-      ret = "".equals(ret.getName())
-        ? new Path("/", wpath.getName())
-        : new Path(new Path("/", wpath.getName()),
-                   new Path(ret.toString().substring(1)));
-      wpath = wpath.getParent();
-    }
-    if (!bpath.equals(wpath)) {
-      throw new IllegalArgumentException(base + " not a prefix of " + walk);
-    }
-    return ret;
-  }
-
-  private void verifyFileSystemContents() throws Exception {
-    FileSystem fs = cluster.getFileSystem();
-    int count = 0;
-    // read NN metadata, verify contents match
-    for (TreePath e : new FSTreeWalk(NAMEPATH, conf)) {
-      FileStatus rs = e.getFileStatus();
-      Path hp = removePrefix(NAMEPATH, rs.getPath());
-      LOG.info("hp " + hp.toUri().getPath());
-      //skip HDFS specific files, which may have been created later on.
-      if(hp.toString().contains("in_use.lock")
-          || hp.toString().contains("current")) {
-        continue;
-      }
-      e.accept(count++);
-      assertTrue(fs.exists(hp));
-      FileStatus hs = fs.getFileStatus(hp);
-      assertEquals(hp.toUri().getPath(), hs.getPath().toUri().getPath());
-      assertEquals(rs.getPermission(), hs.getPermission());
-      assertEquals(rs.getOwner(), hs.getOwner());
-      assertEquals(rs.getGroup(), hs.getGroup());
-
-      if (rs.isFile()) {
-        assertEquals(rs.getLen(), hs.getLen());
-        try (ReadableByteChannel i = Channels.newChannel(
-              new FileInputStream(new File(rs.getPath().toUri())))) {
-          try (ReadableByteChannel j = Channels.newChannel(
-                fs.open(hs.getPath()))) {
-            ByteBuffer ib = ByteBuffer.allocate(4096);
-            ByteBuffer jb = ByteBuffer.allocate(4096);
-            while (true) {
-              int il = i.read(ib);
-              int jl = j.read(jb);
-              if (il < 0 || jl < 0) {
-                assertEquals(il, jl);
-                break;
-              }
-              ib.flip();
-              jb.flip();
-              int cmp = Math.min(ib.remaining(), jb.remaining());
-              for (int k = 0; k < cmp; ++k) {
-                assertEquals(ib.get(), jb.get());
-              }
-              ib.compact();
-              jb.compact();
-            }
-
-          }
-        }
-      }
-    }
-  }
-
-  private BlockLocation[] createFile(Path path, short replication,
-      long fileLen, long blockLen) throws IOException {
-    FileSystem fs = cluster.getFileSystem();
-    //create a sample file that is not provided
-    DFSTestUtil.createFile(fs, path, false, (int) blockLen,
-        fileLen, blockLen, replication, 0, true);
-    return fs.getFileBlockLocations(path, 0, fileLen);
-  }
-
-  @Test(timeout=30000)
-  public void testClusterWithEmptyImage() throws IOException {
-    // start a cluster with 2 datanodes without any provided storage
-    startCluster(NNDIRPATH, 2, null,
-        new StorageType[][] {
-            {StorageType.DISK},
-            {StorageType.DISK}},
-        true);
-    assertTrue(cluster.isClusterUp());
-    assertTrue(cluster.isDataNodeUp());
-
-    BlockLocation[] locations = createFile(new Path("/testFile1.dat"),
-        (short) 2, 1024*1024, 1024*1024);
-    assertEquals(1, locations.length);
-    assertEquals(2, locations[0].getHosts().length);
-  }
-
-  private DatanodeInfo[] getAndCheckBlockLocations(DFSClient client,
-      String filename, long fileLen, long expectedBlocks, int expectedLocations)
-      throws IOException {
-    LocatedBlocks locatedBlocks = client.getLocatedBlocks(filename, 0, fileLen);
-    // given the start and length in the above call,
-    // only one LocatedBlock in LocatedBlocks
-    assertEquals(expectedBlocks, locatedBlocks.getLocatedBlocks().size());
-    DatanodeInfo[] locations =
-        locatedBlocks.getLocatedBlocks().get(0).getLocations();
-    assertEquals(expectedLocations, locations.length);
-    checkUniqueness(locations);
-    return locations;
-  }
-
-  /**
-   * verify that the given locations are all unique.
-   * @param locations
-   */
-  private void checkUniqueness(DatanodeInfo[] locations) {
-    Set<String> set = new HashSet<>();
-    for (DatanodeInfo info: locations) {
-      assertFalse("All locations should be unique",
-          set.contains(info.getDatanodeUuid()));
-      set.add(info.getDatanodeUuid());
-    }
-  }
-
-  /**
-   * Tests setting replication of provided files.
-   * @throws Exception
-   */
-  @Test(timeout=50000)
-  public void testSetReplicationForProvidedFiles() throws Exception {
-    createImage(new FSTreeWalk(NAMEPATH, conf), NNDIRPATH,
-        FixedBlockResolver.class);
-    // 10 Datanodes with both DISK and PROVIDED storage
-    startCluster(NNDIRPATH, 10,
-        new StorageType[]{
-            StorageType.PROVIDED, StorageType.DISK},
-        null,
-        false);
-    setAndUnsetReplication("/" + filePrefix + (numFiles - 1) + fileSuffix);
-  }
-
-  private void setAndUnsetReplication(String filename) throws Exception {
-    Path file = new Path(filename);
-    FileSystem fs = cluster.getFileSystem();
-    // set the replication to 4, and test that the file has
-    // the required replication.
-    short newReplication = 4;
-    LOG.info("Setting replication of file {} to {}", filename, newReplication);
-    fs.setReplication(file, newReplication);
-    DFSTestUtil.waitForReplication((DistributedFileSystem) fs,
-        file, newReplication, 10000);
-    DFSClient client = new DFSClient(new InetSocketAddress("localhost",
-        cluster.getNameNodePort()), cluster.getConfiguration(0));
-    getAndCheckBlockLocations(client, filename, baseFileLen, 1, newReplication);
-
-    // set the replication back to 1
-    newReplication = 1;
-    LOG.info("Setting replication of file {} back to {}",
-        filename, newReplication);
-    fs.setReplication(file, newReplication);
-    // defaultReplication number of replicas should be returned
-    int defaultReplication = conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY,
-        DFSConfigKeys.DFS_REPLICATION_DEFAULT);
-    DFSTestUtil.waitForReplication((DistributedFileSystem) fs,
-        file, (short) defaultReplication, 10000);
-    getAndCheckBlockLocations(client, filename, baseFileLen, 1,
-        defaultReplication);
-  }
-
-  @Test(timeout=30000)
-  public void testProvidedDatanodeFailures() throws Exception {
-    createImage(new FSTreeWalk(NAMEPATH, conf), NNDIRPATH,
-            FixedBlockResolver.class);
-    startCluster(NNDIRPATH, 3, null,
-        new StorageType[][] {
-            {StorageType.PROVIDED, StorageType.DISK},
-            {StorageType.PROVIDED, StorageType.DISK},
-            {StorageType.DISK}},
-        false);
-
-    DataNode providedDatanode1 = cluster.getDataNodes().get(0);
-    DataNode providedDatanode2 = cluster.getDataNodes().get(1);
-
-    DFSClient client = new DFSClient(new InetSocketAddress("localhost",
-        cluster.getNameNodePort()), cluster.getConfiguration(0));
-
-    DatanodeStorageInfo providedDNInfo = getProvidedDatanodeStorageInfo();
-
-    if (numFiles >= 1) {
-      String filename = "/" + filePrefix + (numFiles - 1) + fileSuffix;
-      // 2 locations returned as there are 2 PROVIDED datanodes
-      DatanodeInfo[] dnInfos =
-          getAndCheckBlockLocations(client, filename, baseFileLen, 1, 2);
-      //the location should be one of the provided DNs available
-      assertTrue(
-          dnInfos[0].getDatanodeUuid().equals(
-              providedDatanode1.getDatanodeUuid())
-          || dnInfos[0].getDatanodeUuid().equals(
-              providedDatanode2.getDatanodeUuid()));
-
-      //stop the 1st provided datanode
-      MiniDFSCluster.DataNodeProperties providedDNProperties1 =
-          cluster.stopDataNode(0);
-
-      //make NameNode detect that datanode is down
-      BlockManagerTestUtil.noticeDeadDatanode(
-          cluster.getNameNode(),
-          providedDatanode1.getDatanodeId().getXferAddr());
-
-      //should find the block on the 2nd provided datanode
-      dnInfos = getAndCheckBlockLocations(client, filename, baseFileLen, 1, 1);
-      assertEquals(providedDatanode2.getDatanodeUuid(),
-          dnInfos[0].getDatanodeUuid());
-
-      // stop the 2nd provided datanode
-      MiniDFSCluster.DataNodeProperties providedDNProperties2 =
-          cluster.stopDataNode(0);
-      // make NameNode detect that datanode is down
-      BlockManagerTestUtil.noticeDeadDatanode(
-          cluster.getNameNode(),
-          providedDatanode2.getDatanodeId().getXferAddr());
-      getAndCheckBlockLocations(client, filename, baseFileLen, 1, 0);
-
-      // BR count for the provided ProvidedDatanodeStorageInfo should reset to
-      // 0, when all DNs with PROVIDED storage fail.
-      assertEquals(0, providedDNInfo.getBlockReportCount());
-      //restart the provided datanode
-      cluster.restartDataNode(providedDNProperties1, true);
-      cluster.waitActive();
-
-      assertEquals(1, providedDNInfo.getBlockReportCount());
-
-      //should find the block on the 1st provided datanode now
-      dnInfos = getAndCheckBlockLocations(client, filename, baseFileLen, 1, 1);
-      //not comparing UUIDs as the datanode can now have a different one.
-      assertEquals(providedDatanode1.getDatanodeId().getXferAddr(),
-          dnInfos[0].getXferAddr());
-    }
-  }
-
-  @Test(timeout=300000)
-  public void testTransientDeadDatanodes() throws Exception {
-    createImage(new FSTreeWalk(NAMEPATH, conf), NNDIRPATH,
-            FixedBlockResolver.class);
-    // 3 Datanodes, 2 PROVIDED and other DISK
-    startCluster(NNDIRPATH, 3, null,
-        new StorageType[][] {
-            {StorageType.PROVIDED, StorageType.DISK},
-            {StorageType.PROVIDED, StorageType.DISK},
-            {StorageType.DISK}},
-        false);
-
-    DataNode providedDatanode = cluster.getDataNodes().get(0);
-    DatanodeStorageInfo providedDNInfo = getProvidedDatanodeStorageInfo();
-    int initialBRCount = providedDNInfo.getBlockReportCount();
-    for (int i= 0; i < numFiles; i++) {
-      // expect to have 2 locations as we have 2 provided Datanodes.
-      verifyFileLocation(i, 2);
-      // NameNode thinks the datanode is down
-      BlockManagerTestUtil.noticeDeadDatanode(
-          cluster.getNameNode(),
-          providedDatanode.getDatanodeId().getXferAddr());
-      cluster.waitActive();
-      cluster.triggerHeartbeats();
-      Thread.sleep(1000);
-      // the report count should just continue to increase.
-      assertEquals(initialBRCount + i + 1,
-          providedDNInfo.getBlockReportCount());
-      verifyFileLocation(i, 2);
-    }
-  }
-
-  private DatanodeStorageInfo getProvidedDatanodeStorageInfo() {
-    ProvidedStorageMap providedStorageMap =
-        cluster.getNamesystem().getBlockManager().getProvidedStorageMap();
-    return providedStorageMap.getProvidedStorageInfo();
-  }
-
-  @Test(timeout=30000)
-  public void testNamenodeRestart() throws Exception {
-    createImage(new FSTreeWalk(NAMEPATH, conf), NNDIRPATH,
-        FixedBlockResolver.class);
-    // 3 Datanodes, 2 PROVIDED and other DISK
-    startCluster(NNDIRPATH, 3, null,
-        new StorageType[][] {
-            {StorageType.PROVIDED, StorageType.DISK},
-            {StorageType.PROVIDED, StorageType.DISK},
-            {StorageType.DISK}},
-        false);
-
-    verifyFileLocation(numFiles - 1, 2);
-    cluster.restartNameNodes();
-    cluster.waitActive();
-    verifyFileLocation(numFiles - 1, 2);
-  }
-
-  /**
-   * verify that the specified file has a valid provided location.
-   * @param fileIndex the index of the file to verify.
-   * @throws Exception
-   */
-  private void verifyFileLocation(int fileIndex, int replication)
-      throws Exception {
-    DFSClient client = new DFSClient(
-        new InetSocketAddress("localhost", cluster.getNameNodePort()),
-        cluster.getConfiguration(0));
-    if (fileIndex < numFiles && fileIndex >= 0) {
-      String filename = filePrefix + fileIndex + fileSuffix;
-      File file = new File(new Path(NAMEPATH, filename).toUri());
-      long fileLen = file.length();
-      long blockSize = conf.getLong(FixedBlockResolver.BLOCKSIZE,
-          FixedBlockResolver.BLOCKSIZE_DEFAULT);
-      long numLocatedBlocks =
-          fileLen == 0 ? 1 : (long) Math.ceil(fileLen * 1.0 / blockSize);
-      getAndCheckBlockLocations(client, "/" + filename, fileLen,
-          numLocatedBlocks, replication);
-    }
-  }
-
-  @Test(timeout=30000)
-  public void testSetClusterID() throws Exception {
-    String clusterID = "PROVIDED-CLUSTER";
-    createImage(new FSTreeWalk(NAMEPATH, conf), NNDIRPATH,
-        FixedBlockResolver.class, clusterID, TextFileRegionAliasMap.class);
-    // 2 Datanodes, 1 PROVIDED and other DISK
-    startCluster(NNDIRPATH, 2, null,
-        new StorageType[][] {
-            {StorageType.PROVIDED, StorageType.DISK},
-            {StorageType.DISK}},
-        false);
-    NameNode nn = cluster.getNameNode();
-    assertEquals(clusterID, nn.getNamesystem().getClusterId());
-  }
-
-  @Test(timeout=30000)
-  public void testNumberOfProvidedLocations() throws Exception {
-    // set default replication to 4
-    conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, 4);
-    createImage(new FSTreeWalk(NAMEPATH, conf), NNDIRPATH,
-        FixedBlockResolver.class);
-    // start with 4 PROVIDED location
-    startCluster(NNDIRPATH, 4,
-        new StorageType[]{
-            StorageType.PROVIDED, StorageType.DISK},
-        null,
-        false);
-    int expectedLocations = 4;
-    for (int i = 0; i < numFiles; i++) {
-      verifyFileLocation(i, expectedLocations);
-    }
-    // stop 2 datanodes, one after the other and verify number of locations.
-    for (int i = 1; i <= 2; i++) {
-      DataNode dn = cluster.getDataNodes().get(0);
-      cluster.stopDataNode(0);
-      // make NameNode detect that datanode is down
-      BlockManagerTestUtil.noticeDeadDatanode(cluster.getNameNode(),
-          dn.getDatanodeId().getXferAddr());
-
-      expectedLocations = 4 - i;
-      for (int j = 0; j < numFiles; j++) {
-        verifyFileLocation(j, expectedLocations);
-      }
-    }
-  }
-
-  @Test(timeout=30000)
-  public void testNumberOfProvidedLocationsManyBlocks() throws Exception {
-    // increase number of blocks per file to at least 10 blocks per file
-    conf.setLong(FixedBlockResolver.BLOCKSIZE, baseFileLen/10);
-    // set default replication to 4
-    conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, 4);
-    createImage(new FSTreeWalk(NAMEPATH, conf), NNDIRPATH,
-        FixedBlockResolver.class);
-    // start with 4 PROVIDED location
-    startCluster(NNDIRPATH, 4,
-        new StorageType[]{
-            StorageType.PROVIDED, StorageType.DISK},
-        null,
-        false);
-    int expectedLocations = 4;
-    for (int i = 0; i < numFiles; i++) {
-      verifyFileLocation(i, expectedLocations);
-    }
-  }
-
-
-  @Test
-  public void testInMemoryAliasMap() throws Exception {
-    conf.setClass(ImageWriter.Options.UGI_CLASS,
-        FsUGIResolver.class, UGIResolver.class);
-    conf.setClass(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_CLASS,
-        InMemoryLevelDBAliasMapClient.class, BlockAliasMap.class);
-    conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS,
-        "localhost:32445");
-    File tempDirectory =
-        Files.createTempDirectory("in-memory-alias-map").toFile();
-    File leveDBPath = new File(tempDirectory, bpid);
-    leveDBPath.mkdirs();
-    conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_LEVELDB_DIR,
-        tempDirectory.getAbsolutePath());
-    conf.setBoolean(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_ENABLED, true);
-    conf.setInt(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_LOAD_RETRIES, 10);
-    InMemoryLevelDBAliasMapServer levelDBAliasMapServer =
-        new InMemoryLevelDBAliasMapServer(InMemoryAliasMap::init, bpid);
-    levelDBAliasMapServer.setConf(conf);
-    levelDBAliasMapServer.start();
-
-    createImage(new FSTreeWalk(NAMEPATH, conf),
-        NNDIRPATH,
-        FixedBlockResolver.class, "",
-        InMemoryLevelDBAliasMapClient.class);
-    levelDBAliasMapServer.close();
-
-    // start cluster with two datanodes,
-    // each with 1 PROVIDED volume and other DISK volume
-    startCluster(NNDIRPATH, 2,
-        new StorageType[] {StorageType.PROVIDED, StorageType.DISK},
-        null, false);
-    verifyFileSystemContents();
-    FileUtils.deleteDirectory(tempDirectory);
-  }
-
-  private DatanodeDescriptor getDatanodeDescriptor(DatanodeManager dnm,
-      int dnIndex) throws Exception {
-    return dnm.getDatanode(cluster.getDataNodes().get(dnIndex).getDatanodeId());
-  }
-
-  private void startDecommission(FSNamesystem namesystem, DatanodeManager dnm,
-      int dnIndex) throws Exception {
-    namesystem.writeLock();
-    DatanodeDescriptor dnDesc = getDatanodeDescriptor(dnm, dnIndex);
-    dnm.getDatanodeAdminManager().startDecommission(dnDesc);
-    namesystem.writeUnlock();
-  }
-
-  private void startMaintenance(FSNamesystem namesystem, DatanodeManager dnm,
-      int dnIndex) throws Exception {
-    namesystem.writeLock();
-    DatanodeDescriptor dnDesc = getDatanodeDescriptor(dnm, dnIndex);
-    dnm.getDatanodeAdminManager().startMaintenance(dnDesc, Long.MAX_VALUE);
-    namesystem.writeUnlock();
-  }
-
-  private void stopMaintenance(FSNamesystem namesystem, DatanodeManager dnm,
-      int dnIndex) throws Exception {
-    namesystem.writeLock();
-    DatanodeDescriptor dnDesc = getDatanodeDescriptor(dnm, dnIndex);
-    dnm.getDatanodeAdminManager().stopMaintenance(dnDesc);
-    namesystem.writeUnlock();
-  }
-
-  @Test
-  public void testDatanodeLifeCycle() throws Exception {
-    createImage(new FSTreeWalk(NAMEPATH, conf), NNDIRPATH,
-        FixedBlockResolver.class);
-    startCluster(NNDIRPATH, 3,
-        new StorageType[] {StorageType.PROVIDED, StorageType.DISK},
-        null, false);
-
-    int fileIndex = numFiles - 1;
-
-    final BlockManager blockManager = cluster.getNamesystem().getBlockManager();
-    final DatanodeManager dnm = blockManager.getDatanodeManager();
-
-    // to start, all 3 DNs are live in ProvidedDatanodeDescriptor.
-    verifyFileLocation(fileIndex, 3);
-
-    // de-commision first DN; still get 3 replicas.
-    startDecommission(cluster.getNamesystem(), dnm, 0);
-    verifyFileLocation(fileIndex, 3);
-
-    // remains the same even after heartbeats.
-    cluster.triggerHeartbeats();
-    verifyFileLocation(fileIndex, 3);
-
-    // start maintenance for 2nd DN; still get 3 replicas.
-    startMaintenance(cluster.getNamesystem(), dnm, 1);
-    verifyFileLocation(fileIndex, 3);
-
-    DataNode dn1 = cluster.getDataNodes().get(0);
-    DataNode dn2 = cluster.getDataNodes().get(1);
-
-    // stop the 1st DN while being decomissioned.
-    MiniDFSCluster.DataNodeProperties dn1Properties = cluster.stopDataNode(0);
-    BlockManagerTestUtil.noticeDeadDatanode(cluster.getNameNode(),
-        dn1.getDatanodeId().getXferAddr());
-
-    // get 2 locations
-    verifyFileLocation(fileIndex, 2);
-
-    // stop dn2 while in maintenance.
-    MiniDFSCluster.DataNodeProperties dn2Properties = cluster.stopDataNode(1);
-    BlockManagerTestUtil.noticeDeadDatanode(cluster.getNameNode(),
-        dn2.getDatanodeId().getXferAddr());
-
-    // 2 valid locations will be found as blocks on nodes that die during
-    // maintenance are not marked for removal.
-    verifyFileLocation(fileIndex, 2);
-
-    // stop the maintenance; get only 1 replicas
-    stopMaintenance(cluster.getNamesystem(), dnm, 0);
-    verifyFileLocation(fileIndex, 1);
-
-    // restart the stopped DN.
-    cluster.restartDataNode(dn1Properties, true);
-    cluster.waitActive();
-
-    // reports all 3 replicas
-    verifyFileLocation(fileIndex, 2);
-
-    cluster.restartDataNode(dn2Properties, true);
-    cluster.waitActive();
-
-    // reports all 3 replicas
-    verifyFileLocation(fileIndex, 3);
-  }
-
-  @Test
-  public void testProvidedWithHierarchicalTopology() throws Exception {
-    conf.setClass(ImageWriter.Options.UGI_CLASS, FsUGIResolver.class,
-        UGIResolver.class);
-    String packageName = "org.apache.hadoop.hdfs.server.blockmanagement";
-    String[] policies = new String[] {
-        "BlockPlacementPolicyDefault",
-        "BlockPlacementPolicyRackFaultTolerant",
-        "BlockPlacementPolicyWithNodeGroup",
-        "BlockPlacementPolicyWithUpgradeDomain"};
-    createImage(new FSTreeWalk(NAMEPATH, conf), NNDIRPATH,
-        FixedBlockResolver.class);
-    String[] racks =
-        {"/pod0/rack0", "/pod0/rack0", "/pod0/rack1", "/pod0/rack1",
-            "/pod1/rack0", "/pod1/rack0", "/pod1/rack1", "/pod1/rack1" };
-    for (String policy: policies) {
-      LOG.info("Using policy: " + packageName + "." + policy);
-      conf.set(DFS_BLOCK_REPLICATOR_CLASSNAME_KEY, packageName + "." + policy);
-      startCluster(NNDIRPATH, racks.length,
-          new StorageType[]{StorageType.PROVIDED, StorageType.DISK},
-          null, false, racks);
-      verifyFileSystemContents();
-      setAndUnsetReplication("/" + filePrefix + (numFiles - 1) + fileSuffix);
-      cluster.shutdown();
-    }
-  }
-}


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


[26/50] [abbrv] hadoop git commit: Add 2.8.3 release jdiff files.

Posted by na...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/a7f8caf5/hadoop-mapreduce-project/dev-support/jdiff/Apache_Hadoop_MapReduce_JobClient_2.8.3.xml
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/dev-support/jdiff/Apache_Hadoop_MapReduce_JobClient_2.8.3.xml b/hadoop-mapreduce-project/dev-support/jdiff/Apache_Hadoop_MapReduce_JobClient_2.8.3.xml
new file mode 100644
index 0000000..a63b3ac
--- /dev/null
+++ b/hadoop-mapreduce-project/dev-support/jdiff/Apache_Hadoop_MapReduce_JobClient_2.8.3.xml
@@ -0,0 +1,16 @@
+<?xml version="1.0" encoding="iso-8859-1" standalone="no"?>
+<!-- Generated by the JDiff Javadoc doclet -->
+<!-- (http://www.jdiff.org) -->
+<!-- on Tue Dec 05 05:57:09 UTC 2017 -->
+
+<api
+  xmlns:xsi='http://www.w3.org/2001/XMLSchema-instance'
+  xsi:noNamespaceSchemaLocation='api.xsd'
+  name="Apache Hadoop MapReduce JobClient 2.8.3"
+  jdversion="1.0.9">
+
+<!--  Command line arguments =  -doclet org.apache.hadoop.classification.tools.IncludePublicAnnotationsJDiffDoclet -docletpath /build/source/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/target/hadoop-annotations.jar:/build/source/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/target/jdiff.jar -verbose -classpath /build/source/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/target/classes:/build/source/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/target/hadoop-mapreduce-client-common-2.8.3.jar:/build/source/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/target/hadoop-yarn-common-2.8.3.jar:/build/source/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/target/hadoop-yarn-client-2.8.3.jar:/build/source/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/target/hadoop-mapreduce-client-core-2.8.3.jar:/build/source/h
 adoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/target/hadoop-mapreduce-client-shuffle-2.8.3.jar:/maven/org/fusesource/leveldbjni/leveldbjni-all/1.8/leveldbjni-all-1.8.jar:/build/source/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/target/hadoop-yarn-api-2.8.3.jar:/maven/javax/xml/bind/jaxb-api/2.2.2/jaxb-api-2.2.2.jar:/maven/javax/xml/stream/stax-api/1.0-2/stax-api-1.0-2.jar:/maven/javax/activation/activation/1.1/activation-1.1.jar:/maven/org/apache/commons/commons-compress/1.4.1/commons-compress-1.4.1.jar:/maven/org/tukaani/xz/1.0/xz-1.0.jar:/maven/javax/servlet/servlet-api/2.5/servlet-api-2.5.jar:/maven/org/mortbay/jetty/jetty-util/6.1.26/jetty-util-6.1.26.jar:/maven/com/sun/jersey/jersey-core/1.9/jersey-core-1.9.jar:/maven/com/sun/jersey/jersey-client/1.9/jersey-client-1.9.jar:/maven/org/codehaus/jackson/jackson-core-asl/1.9.13/jackson-core-asl-1.9.13.jar:/maven/org/codehaus/jackson/jackson-mapper-asl/1.9.13/jackson-mapper-asl-1.9.13.jar:/maven/
 org/codehaus/jackson/jackson-jaxrs/1.9.13/jackson-jaxrs-1.9.13.jar:/maven/org/codehaus/jackson/jackson-xc/1.9.13/jackson-xc-1.9.13.jar:/maven/commons-io/commons-io/2.4/commons-io-2.4.jar:/maven/com/google/inject/guice/3.0/guice-3.0.jar:/maven/javax/inject/javax.inject/1/javax.inject-1.jar:/maven/aopalliance/aopalliance/1.0/aopalliance-1.0.jar:/maven/com/sun/jersey/jersey-server/1.9/jersey-server-1.9.jar:/maven/asm/asm/3.2/asm-3.2.jar:/maven/com/sun/jersey/jersey-json/1.9/jersey-json-1.9.jar:/maven/com/sun/xml/bind/jaxb-impl/2.2.3-1/jaxb-impl-2.2.3-1.jar:/maven/com/sun/jersey/contribs/jersey-guice/1.9/jersey-guice-1.9.jar:/maven/log4j/log4j/1.2.17/log4j-1.2.17.jar:/maven/org/codehaus/jettison/jettison/1.1/jettison-1.1.jar:/maven/org/apache/curator/curator-client/2.7.1/curator-client-2.7.1.jar:/maven/org/apache/zookeeper/zookeeper/3.4.6/zookeeper-3.4.6.jar:/maven/jline/jline/0.9.94/jline-0.9.94.jar:/maven/org/mortbay/jetty/jetty/6.1.26/jetty-6.1.26.jar:/maven/xmlenc/xmlenc/0.52/xmlenc
 -0.52.jar:/maven/org/apache/htrace/htrace-core4/4.0.1-incubating/htrace-core4-4.0.1-incubating.jar:/maven/com/google/protobuf/protobuf-java/2.5.0/protobuf-java-2.5.0.jar:/maven/org/apache/avro/avro/1.7.4/avro-1.7.4.jar:/maven/com/thoughtworks/paranamer/paranamer/2.3/paranamer-2.3.jar:/maven/org/xerial/snappy/snappy-java/1.0.4.1/snappy-java-1.0.4.1.jar:/build/source/hadoop-common-project/hadoop-common/target/hadoop-common-2.8.3.jar:/maven/org/apache/commons/commons-math3/3.1.1/commons-math3-3.1.1.jar:/maven/org/apache/httpcomponents/httpclient/4.5.2/httpclient-4.5.2.jar:/maven/org/apache/httpcomponents/httpcore/4.4.4/httpcore-4.4.4.jar:/maven/commons-net/commons-net/3.1/commons-net-3.1.jar:/maven/org/mortbay/jetty/jetty-sslengine/6.1.26/jetty-sslengine-6.1.26.jar:/maven/javax/servlet/jsp/jsp-api/2.1/jsp-api-2.1.jar:/maven/net/java/dev/jets3t/jets3t/0.9.0/jets3t-0.9.0.jar:/maven/com/jamesmurty/utils/java-xmlbuilder/0.4/java-xmlbuilder-0.4.jar:/maven/commons-configuration/commons-confi
 guration/1.6/commons-configuration-1.6.jar:/maven/commons-digester/commons-digester/1.8/commons-digester-1.8.jar:/maven/commons-beanutils/commons-beanutils/1.7.0/commons-beanutils-1.7.0.jar:/maven/commons-beanutils/commons-beanutils-core/1.8.0/commons-beanutils-core-1.8.0.jar:/maven/com/google/code/gson/gson/2.2.4/gson-2.2.4.jar:/build/source/hadoop-common-project/hadoop-auth/target/hadoop-auth-2.8.3.jar:/maven/com/nimbusds/nimbus-jose-jwt/3.9/nimbus-jose-jwt-3.9.jar:/maven/net/jcip/jcip-annotations/1.0/jcip-annotations-1.0.jar:/maven/net/minidev/json-smart/1.1.1/json-smart-1.1.1.jar:/maven/org/apache/directory/server/apacheds-kerberos-codec/2.0.0-M15/apacheds-kerberos-codec-2.0.0-M15.jar:/maven/org/apache/directory/server/apacheds-i18n/2.0.0-M15/apacheds-i18n-2.0.0-M15.jar:/maven/org/apache/directory/api/api-asn1-api/1.0.0-M20/api-asn1-api-1.0.0-M20.jar:/maven/org/apache/directory/api/api-util/1.0.0-M20/api-util-1.0.0-M20.jar:/maven/org/apache/curator/curator-framework/2.7.1/curato
 r-framework-2.7.1.jar:/maven/com/jcraft/jsch/0.1.54/jsch-0.1.54.jar:/maven/org/apache/curator/curator-recipes/2.7.1/curator-recipes-2.7.1.jar:/maven/com/google/code/findbugs/jsr305/3.0.0/jsr305-3.0.0.jar:/maven/org/slf4j/slf4j-api/1.7.10/slf4j-api-1.7.10.jar:/maven/org/slf4j/slf4j-log4j12/1.7.10/slf4j-log4j12-1.7.10.jar:/build/source/hadoop-common-project/hadoop-annotations/target/hadoop-annotations-2.8.3.jar:/usr/lib/jvm/java-7-openjdk-amd64/lib/tools.jar:/maven/com/google/inject/extensions/guice-servlet/3.0/guice-servlet-3.0.jar:/maven/io/netty/netty/3.6.2.Final/netty-3.6.2.Final.jar:/maven/commons-logging/commons-logging/1.1.3/commons-logging-1.1.3.jar:/maven/com/google/guava/guava/11.0.2/guava-11.0.2.jar:/maven/commons-codec/commons-codec/1.4/commons-codec-1.4.jar:/maven/commons-cli/commons-cli/1.2/commons-cli-1.2.jar:/maven/commons-lang/commons-lang/2.6/commons-lang-2.6.jar:/maven/commons-collections/commons-collections/3.2.2/commons-collections-3.2.2.jar -sourcepath /build/sou
 rce/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java -apidir /build/source/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/target/site/jdiff/xml -apiname Apache Hadoop MapReduce JobClient 2.8.3 -->
+<package name="org.apache.hadoop.mapred">
+</package>
+
+</api>


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


[03/50] [abbrv] hadoop git commit: HDFS-12712. [9806] Code style cleanup

Posted by na...@apache.org.
HDFS-12712. [9806] Code style cleanup


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

Branch: refs/heads/yarn-3409
Commit: 8239e3afb31d3c4485817d4b8b8b195b554acbe7
Parents: 80c3fec
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Fri Dec 15 10:15:15 2017 -0800
Committer: Chris Douglas <cd...@apache.org>
Committed: Fri Dec 15 17:51:41 2017 -0800

----------------------------------------------------------------------
 .../hadoop/hdfs/protocol/HdfsConstants.java     |   1 -
 .../hadoop/hdfs/protocol/LocatedBlock.java      |  59 +-
 .../hdfs/server/blockmanagement/BlockInfo.java  |   2 +-
 .../server/blockmanagement/BlockManager.java    |   5 +-
 .../server/blockmanagement/DatanodeManager.java |   2 +-
 .../blockmanagement/ProvidedStorageMap.java     |   4 +-
 .../hadoop/hdfs/server/common/Storage.java      |   6 +-
 .../impl/TextFileRegionAliasMap.java            |   2 +-
 .../server/datanode/BlockPoolSliceStorage.java  |   3 +-
 .../hdfs/server/datanode/DataStorage.java       |   4 +-
 .../hdfs/server/datanode/ProvidedReplica.java   |   1 -
 .../hdfs/server/datanode/StorageLocation.java   |  12 +-
 .../datanode/fsdataset/impl/FsDatasetImpl.java  |   6 +-
 .../fsdataset/impl/ProvidedVolumeImpl.java      |  21 +-
 .../hadoop/hdfs/TestBlockStoragePolicy.java     |   8 +-
 .../blockmanagement/TestDatanodeManager.java    |   5 +-
 .../blockmanagement/TestProvidedStorageMap.java |  12 +-
 .../datanode/TestProvidedReplicaImpl.java       |  13 +-
 .../fsdataset/impl/TestProvidedImpl.java        |  64 +-
 hadoop-tools/hadoop-fs2img/pom.xml              |   4 +-
 .../hdfs/server/namenode/FileSystemImage.java   |   3 +-
 .../hdfs/server/namenode/ImageWriter.java       |   7 +-
 .../hdfs/server/namenode/SingleUGIResolver.java |   4 +-
 .../hadoop/hdfs/server/namenode/TreePath.java   |   3 +-
 .../namenode/ITestProvidedImplementation.java   | 927 ++++++++++++++++++
 .../hdfs/server/namenode/RandomTreeWalk.java    |   4 +-
 .../TestNameNodeProvidedImplementation.java     | 934 -------------------
 27 files changed, 1040 insertions(+), 1076 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8239e3af/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
index e9e6103..fd7f9e0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
@@ -47,7 +47,6 @@ public final class HdfsConstants {
   public static final String WARM_STORAGE_POLICY_NAME = "WARM";
   public static final byte COLD_STORAGE_POLICY_ID = 2;
   public static final String COLD_STORAGE_POLICY_NAME = "COLD";
-  // branch HDFS-9806 XXX temporary until HDFS-7076
   public static final byte PROVIDED_STORAGE_POLICY_ID = 1;
   public static final String PROVIDED_STORAGE_POLICY_NAME = "PROVIDED";
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8239e3af/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java
index 5ad0bca..29f1b6d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs.protocol;
 
+import java.io.Serializable;
 import java.util.Arrays;
 import java.util.Comparator;
 import java.util.List;
@@ -40,6 +41,32 @@ import com.google.common.collect.Lists;
 @InterfaceStability.Evolving
 public class LocatedBlock {
 
+  /**
+   * Comparator that ensures that a PROVIDED storage type is greater than any
+   * other storage type. Any other storage types are considered equal.
+   */
+  private static class ProvidedLastComparator
+      implements Comparator<DatanodeInfoWithStorage>, Serializable {
+
+    private static final long serialVersionUID = 6441720011443190984L;
+
+    @Override
+    public int compare(DatanodeInfoWithStorage dns1,
+        DatanodeInfoWithStorage dns2) {
+      if (StorageType.PROVIDED.equals(dns1.getStorageType())
+          && !StorageType.PROVIDED.equals(dns2.getStorageType())) {
+        return 1;
+      }
+      if (!StorageType.PROVIDED.equals(dns1.getStorageType())
+          && StorageType.PROVIDED.equals(dns2.getStorageType())) {
+        return -1;
+      }
+      // Storage types of dns1 and dns2 are now both provided or not provided;
+      // thus, are essentially equal for the purpose of this comparator.
+      return 0;
+    }
+  }
+
   private final ExtendedBlock b;
   private long offset;  // offset of the first byte of the block in the file
   private final DatanodeInfoWithStorage[] locs;
@@ -52,6 +79,10 @@ public class LocatedBlock {
   // their locations are not part of this object
   private boolean corrupt;
   private Token<BlockTokenIdentifier> blockToken = new Token<>();
+
+  // use one instance of the Provided comparator as it uses no state.
+  private static ProvidedLastComparator providedLastComparator =
+      new ProvidedLastComparator();
   /**
    * List of cached datanode locations
    */
@@ -157,29 +188,6 @@ public class LocatedBlock {
   }
 
   /**
-   * Comparator that ensures that a PROVIDED storage type is greater than
-   * any other storage type. Any other storage types are considered equal.
-   */
-  private class ProvidedLastComparator
-    implements Comparator<DatanodeInfoWithStorage> {
-    @Override
-    public int compare(DatanodeInfoWithStorage dns1,
-        DatanodeInfoWithStorage dns2) {
-      if (StorageType.PROVIDED.equals(dns1.getStorageType())
-          && !StorageType.PROVIDED.equals(dns2.getStorageType())) {
-        return 1;
-      }
-      if (!StorageType.PROVIDED.equals(dns1.getStorageType())
-          && StorageType.PROVIDED.equals(dns2.getStorageType())) {
-        return -1;
-      }
-      // Storage types of dns1 and dns2 are now both provided or not provided;
-      // thus, are essentially equal for the purpose of this comparator.
-      return 0;
-    }
-  }
-
-  /**
    * Moves all locations that have {@link StorageType}
    * {@code PROVIDED} to the end of the locations array without
    * changing the relative ordering of the remaining locations
@@ -196,9 +204,8 @@ public class LocatedBlock {
     }
     // as this is a stable sort, for elements that are equal,
     // the current order of the elements is maintained
-    Arrays.sort(locs, 0,
-        (activeLen < locs.length) ? activeLen : locs.length,
-        new ProvidedLastComparator());
+    Arrays.sort(locs, 0, (activeLen < locs.length) ? activeLen : locs.length,
+        providedLastComparator);
   }
 
   public long getStartOffset() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8239e3af/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
index 8f59df6..111ade1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
@@ -192,7 +192,7 @@ public abstract class BlockInfo extends Block
       DatanodeStorageInfo cur = getStorageInfo(idx);
       if(cur != null) {
         if (cur.getStorageType() == StorageType.PROVIDED) {
-          //if block resides on provided storage, only match the storage ids
+          // if block resides on provided storage, only match the storage ids
           if (dn.getStorageInfo(cur.getStorageID()) != null) {
             // do not return here as we have to check the other
             // DatanodeStorageInfos for this block which could be local

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8239e3af/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 c1cd4db..59e06c6 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
@@ -1240,7 +1240,6 @@ public class BlockManager implements BlockStatsMXBean {
         final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations();
         final ExtendedBlock eb = new ExtendedBlock(getBlockPoolId(),
             blk);
-        //TODO use locatedBlocks builder??
         return newLocatedStripedBlock(eb, storages, uc.getBlockIndices(), pos,
             false);
       } else {
@@ -2497,8 +2496,8 @@ public class BlockManager implements BlockStatsMXBean {
 
       // To minimize startup time, we discard any second (or later) block reports
       // that we receive while still in startup phase.
-      // !#! Register DN with provided storage, not with storage owned by DN
-      // !#! DN should still have a ref to the DNStorageInfo
+      // Register DN with provided storage, not with storage owned by DN
+      // DN should still have a ref to the DNStorageInfo.
       DatanodeStorageInfo storageInfo =
           providedStorageMap.getStorage(node, storage);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8239e3af/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
index a7e31a2..e6cd513 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
@@ -532,7 +532,7 @@ public class DatanodeManager {
     } else {
       networktopology.sortByDistance(client, lb.getLocations(), activeLen);
     }
-    //move PROVIDED storage to the end to prefer local replicas.
+    // move PROVIDED storage to the end to prefer local replicas.
     lb.moveProvidedToEnd(activeLen);
     // must update cache since we modified locations array
     lb.updateCachedStorageInfo();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8239e3af/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ProvidedStorageMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ProvidedStorageMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ProvidedStorageMap.java
index 08d1434..6303775 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ProvidedStorageMap.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ProvidedStorageMap.java
@@ -294,6 +294,7 @@ public class ProvidedStorageMap {
 
     @Override
     LocatedBlocks build(DatanodeDescriptor client) {
+      // TODO choose provided locations close to the client.
       return new LocatedBlocks(
           flen, isUC, blocks, last, lastComplete, feInfo, ecPolicy);
     }
@@ -333,7 +334,6 @@ public class ProvidedStorageMap {
         DatanodeDescriptor dn, DatanodeStorage s) {
       dns.put(dn.getDatanodeUuid(), dn);
       dnR.add(dn);
-      // TODO: maintain separate RPC ident per dn
       return storageMap.get(s.getStorageID());
     }
 
@@ -522,7 +522,7 @@ public class ProvidedStorageMap {
 
     @Override
     public int getNumberOfBlocks() {
-      // VERIFY: only printed for debugging
+      // is ignored for ProvidedBlockList.
       return -1;
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8239e3af/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java
index 9ad61d7..5409427 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java
@@ -422,7 +422,7 @@ public abstract class Storage extends StorageInfo {
     public void clearDirectory() throws IOException {
       File curDir = this.getCurrentDir();
       if (curDir == null) {
-        //if the directory is null, there is nothing to do.
+        // if the directory is null, there is nothing to do.
         return;
       }
       if (curDir.exists()) {
@@ -638,7 +638,7 @@ public abstract class Storage extends StorageInfo {
 
       if (location != null &&
           location.getStorageType() == StorageType.PROVIDED) {
-        //currently we assume that PROVIDED storages are always NORMAL
+        // currently we assume that PROVIDED storages are always NORMAL
         return StorageState.NORMAL;
       }
 
@@ -764,7 +764,7 @@ public abstract class Storage extends StorageInfo {
     public void doRecover(StorageState curState) throws IOException {
       File curDir = getCurrentDir();
       if (curDir == null || root == null) {
-        //at this point, we do not support recovery on PROVIDED storages
+        // at this point, we do not support recovery on PROVIDED storages
         return;
       }
       String rootPath = root.getCanonicalPath();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8239e3af/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TextFileRegionAliasMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TextFileRegionAliasMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TextFileRegionAliasMap.java
index 150371d..abe92e3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TextFileRegionAliasMap.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TextFileRegionAliasMap.java
@@ -471,7 +471,7 @@ public class TextFileRegionAliasMap
 
   @Override
   public void close() throws IOException {
-    //nothing to do;
+    // nothing to do;
   }
 
   @VisibleForTesting

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8239e3af/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceStorage.java
index 012d1f5..ac5c3ae 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceStorage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceStorage.java
@@ -443,7 +443,7 @@ public class BlockPoolSliceStorage extends Storage {
         LayoutVersion.Feature.FEDERATION, layoutVersion)) {
       return;
     }
-    //no upgrades for storage directories that are PROVIDED
+    // no upgrades for storage directories that are PROVIDED
     if (bpSd.getRoot() == null) {
       return;
     }
@@ -640,7 +640,6 @@ public class BlockPoolSliceStorage extends Storage {
    * that holds the snapshot.
    */
   void doFinalize(File dnCurDir) throws IOException {
-    LOG.info("doFinalize: " + dnCurDir);
     if (dnCurDir == null) {
       return; //we do nothing if the directory is null
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8239e3af/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
index a1bde31..fc1dad1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
@@ -149,8 +149,8 @@ public class DataStorage extends Storage {
     final String oldStorageID = sd.getStorageUuid();
     if (sd.getStorageLocation() != null &&
         sd.getStorageLocation().getStorageType() == StorageType.PROVIDED) {
-      // We only support one provided storage per datanode for now.
-      // TODO support multiple provided storage ids per datanode.
+      // Only one provided storage id is supported.
+      // TODO support multiple provided storage ids
       sd.setStorageUuid(conf.get(DFSConfigKeys.DFS_PROVIDER_STORAGEUUID,
           DFSConfigKeys.DFS_PROVIDER_STORAGEUUID_DEFAULT));
       return false;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8239e3af/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ProvidedReplica.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ProvidedReplica.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ProvidedReplica.java
index 5e02d4f..bd23021 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ProvidedReplica.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ProvidedReplica.java
@@ -310,7 +310,6 @@ public abstract class ProvidedReplica extends ReplicaInfo {
 
   @Override
   public int compareWith(ScanInfo info) {
-    //local scanning cannot find any provided blocks.
     if (info.getFileRegion().equals(
         new FileRegion(this.getBlockId(), new Path(getRemoteURI()),
             fileOffset, this.getNumBytes(), this.getGenerationStamp()))) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8239e3af/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StorageLocation.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StorageLocation.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StorageLocation.java
index d72448d..8ad51de 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StorageLocation.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StorageLocation.java
@@ -108,10 +108,10 @@ public class StorageLocation
     }
     if (sd.getStorageLocation().getStorageType() == StorageType.PROVIDED ||
         storageType == StorageType.PROVIDED) {
-      //only one of these is PROVIDED; so it cannot be a match!
+      // only one PROVIDED storage directory can exist; so this cannot match!
       return false;
     }
-    //both storage directories are local
+    // both storage directories are local
     return this.getBpURI(bpid, Storage.STORAGE_DIR_CURRENT).normalize()
         .equals(sd.getRoot().toURI().normalize());
   }
@@ -212,7 +212,9 @@ public class StorageLocation
       conf = new HdfsConfiguration();
     }
     if (storageType == StorageType.PROVIDED) {
-      //skip creation if the storage type is PROVIDED
+      // skip creation if the storage type is PROVIDED
+      Storage.LOG.info("Skipping creating directory for block pool "
+          + blockPoolID + " for PROVIDED storage location " + this);
       return;
     }
 
@@ -231,8 +233,8 @@ public class StorageLocation
 
   @Override  // Checkable
   public VolumeCheckResult check(CheckContext context) throws IOException {
-    //we assume provided storage locations are always healthy,
-    //and check only for local storages.
+    // assume provided storage locations are always healthy,
+    // and check only for local storages.
     if (storageType != StorageType.PROVIDED) {
       DiskChecker.checkDir(
           context.localFileSystem,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8239e3af/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
index fd06a56..d9071dd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
@@ -1760,7 +1760,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
 
       Set<String> missingVolumesReported = new HashSet<>();
       for (ReplicaInfo b : volumeMap.replicas(bpid)) {
-        //skip blocks in PROVIDED storage
+        // skip PROVIDED replicas.
         if (b.getVolume().getStorageType() == StorageType.PROVIDED) {
           continue;
         }
@@ -2281,7 +2281,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
 
       if (vol.getStorageType() == StorageType.PROVIDED) {
         if (memBlockInfo == null) {
-          //replica exists on provided store but not in memory
+          // replica exists on provided store but not in memory
           ReplicaInfo diskBlockInfo =
               new ReplicaBuilder(ReplicaState.FINALIZED)
               .setFileRegion(scanInfo.getFileRegion())
@@ -2292,7 +2292,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
           volumeMap.add(bpid, diskBlockInfo);
           LOG.warn("Added missing block to memory " + diskBlockInfo);
         } else {
-          //replica exists in memory but not in the provided store
+          // replica exists in memory but not in the provided store
           volumeMap.remove(bpid, blockId);
           LOG.warn("Deleting missing provided block " + memBlockInfo);
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8239e3af/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeImpl.java
index 59ec100..ec1a8fd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeImpl.java
@@ -224,7 +224,7 @@ class ProvidedVolumeImpl extends FsVolumeImpl {
     }
 
     public void shutdown(BlockListAsLongs blocksListsAsLongs) {
-      //nothing to do!
+      // nothing to do!
     }
 
     public void compileReport(LinkedList<ScanInfo> report,
@@ -264,7 +264,7 @@ class ProvidedVolumeImpl extends FsVolumeImpl {
       new ConcurrentHashMap<String, ProvidedBlockPoolSlice>();
 
   private ProvidedVolumeDF df;
-  //the remote FileSystem to which this ProvidedVolume points to.
+  // the remote FileSystem to which this ProvidedVolume points to.
   private FileSystem remoteFS;
 
   ProvidedVolumeImpl(FsDatasetImpl dataset, String storageID,
@@ -395,9 +395,9 @@ class ProvidedVolumeImpl extends FsVolumeImpl {
     @JsonProperty
     private boolean atEnd;
 
-    //The id of the last block read when the state of the iterator is saved.
-    //This implementation assumes that provided blocks are returned
-    //in sorted order of the block ids.
+    // The id of the last block read when the state of the iterator is saved.
+    // This implementation assumes that provided blocks are returned
+    // in sorted order of the block ids.
     @JsonProperty
     private long lastBlockId;
   }
@@ -421,7 +421,7 @@ class ProvidedVolumeImpl extends FsVolumeImpl {
 
     @Override
     public void close() throws IOException {
-      //No action needed
+      blockAliasMap.close();
     }
 
     @Override
@@ -467,14 +467,14 @@ class ProvidedVolumeImpl extends FsVolumeImpl {
 
     @Override
     public void save() throws IOException {
-      //We do not persist the state of this iterator anywhere, locally.
-      //We just re-scan provided volumes as necessary.
+      // We do not persist the state of this iterator locally.
+      // We just re-scan provided volumes as necessary.
       state.lastSavedMs = Time.now();
     }
 
     @Override
     public void setMaxStalenessMs(long maxStalenessMs) {
-      //do not use max staleness
+      // do not use max staleness
     }
 
     @Override
@@ -493,7 +493,7 @@ class ProvidedVolumeImpl extends FsVolumeImpl {
     }
 
     public void load() throws IOException {
-      //on load, we just rewind the iterator for provided volumes.
+      // on load, we just rewind the iterator for provided volumes.
       rewind();
       LOG.trace("load({}, {}): loaded iterator {}: {}", getStorageID(),
           bpid, name, WRITER.writeValueAsString(state));
@@ -615,7 +615,6 @@ class ProvidedVolumeImpl extends FsVolumeImpl {
       LinkedList<ScanInfo> report, ReportCompiler reportCompiler)
       throws InterruptedException, IOException {
     LOG.info("Compiling report for volume: " + this + " bpid " + bpid);
-    //get the report from the appropriate block pool.
     if(bpSlices.containsKey(bpid)) {
       bpSlices.get(bpid).compileReport(report, reportCompiler);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8239e3af/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
index 55a7b3e..76eb824 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
@@ -144,9 +144,11 @@ public class TestBlockStoragePolicy {
     expectedPolicyStrings.put(ALLSSD, "BlockStoragePolicy{ALL_SSD:" + ALLSSD +
         ", storageTypes=[SSD], creationFallbacks=[DISK], " +
         "replicationFallbacks=[DISK]}");
-    expectedPolicyStrings.put(PROVIDED, "BlockStoragePolicy{PROVIDED:" + PROVIDED +
-        ", storageTypes=[PROVIDED, DISK], creationFallbacks=[PROVIDED, DISK], " +
-        "replicationFallbacks=[PROVIDED, DISK]}");
+    expectedPolicyStrings.put(PROVIDED,
+        "BlockStoragePolicy{PROVIDED:" + PROVIDED
+            + ", storageTypes=[PROVIDED, DISK], "
+            + "creationFallbacks=[PROVIDED, DISK], "
+            + "replicationFallbacks=[PROVIDED, DISK]}");
 
     for(byte i = 1; i < 16; i++) {
       final BlockStoragePolicy policy = POLICY_SUITE.getPolicy(i); 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8239e3af/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java
index 81405eb..96841cf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java
@@ -325,11 +325,12 @@ public class TestDatanodeManager {
    */
   @Test
   public void testBadScript() throws IOException, URISyntaxException {
-    HelperFunction("/"+ Shell.appendScriptExtension("topology-broken-script"), 0);
+    HelperFunction("/" + Shell.appendScriptExtension("topology-broken-script"),
+        0);
   }
 
   /**
-   * Test with different sorting functions but include datanodes
+   * Test with different sorting functions but include datanodes.
    * with provided storage
    * @throws IOException
    * @throws URISyntaxException

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8239e3af/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestProvidedStorageMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestProvidedStorageMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestProvidedStorageMap.java
index b419c38..c7f8379 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestProvidedStorageMap.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestProvidedStorageMap.java
@@ -73,13 +73,13 @@ public class TestProvidedStorageMap {
         nameSystemLock, bm, conf);
     DatanodeStorageInfo providedMapStorage =
         providedMap.getProvidedStorageInfo();
-    //the provided storage cannot be null
+    // the provided storage cannot be null
     assertNotNull(providedMapStorage);
 
-    //create a datanode
+    // create a datanode
     DatanodeDescriptor dn1 = createDatanodeDescriptor(5000);
 
-    //associate two storages to the datanode
+    // associate two storages to the datanode
     DatanodeStorage dn1ProvidedStorage = new DatanodeStorage(
         providedStorageID,
         DatanodeStorage.State.NORMAL,
@@ -96,15 +96,15 @@ public class TestProvidedStorageMap {
         dns1Provided == providedMapStorage);
     assertTrue("Disk storage has not yet been registered with block manager",
         dns1Disk == null);
-    //add the disk storage to the datanode.
+    // add the disk storage to the datanode.
     DatanodeStorageInfo dnsDisk = new DatanodeStorageInfo(dn1, dn1DiskStorage);
     dn1.injectStorage(dnsDisk);
     assertTrue("Disk storage must match the injected storage info",
         dnsDisk == providedMap.getStorage(dn1, dn1DiskStorage));
 
-    //create a 2nd datanode
+    // create a 2nd datanode
     DatanodeDescriptor dn2 = createDatanodeDescriptor(5010);
-    //associate a provided storage with the datanode
+    // associate a provided storage with the datanode
     DatanodeStorage dn2ProvidedStorage = new DatanodeStorage(
         providedStorageID,
         DatanodeStorage.State.NORMAL,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8239e3af/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestProvidedReplicaImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestProvidedReplicaImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestProvidedReplicaImpl.java
index 210be6e..a7e8b1e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestProvidedReplicaImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestProvidedReplicaImpl.java
@@ -50,9 +50,9 @@ public class TestProvidedReplicaImpl {
   private static final String BASE_DIR =
       new FileSystemTestHelper().getTestRootDir();
   private static final String FILE_NAME = "provided-test";
-  //length of the file that is associated with the provided blocks.
+  // length of the file that is associated with the provided blocks.
   private static final long FILE_LEN = 128 * 1024 * 10L + 64 * 1024;
-  //length of each provided block.
+  // length of each provided block.
   private static final long BLK_LEN = 128 * 1024L;
 
   private static List<ProvidedReplica> replicas;
@@ -63,7 +63,6 @@ public class TestProvidedReplicaImpl {
     if(!newFile.exists()) {
       newFile.createNewFile();
       OutputStream writer = new FileOutputStream(newFile.getAbsolutePath());
-      //FILE_LEN is length in bytes.
       byte[] bytes = new byte[1];
       bytes[0] = (byte) 0;
       for(int i=0; i< FILE_LEN; i++) {
@@ -106,7 +105,7 @@ public class TestProvidedReplicaImpl {
    * @param dataLength length
    * @throws IOException
    */
-  private void verifyReplicaContents(File file,
+  public static void verifyReplicaContents(File file,
       InputStream ins, long fileOffset, long dataLength)
           throws IOException {
 
@@ -142,9 +141,9 @@ public class TestProvidedReplicaImpl {
   public void testProvidedReplicaRead() throws IOException {
 
     File providedFile = new File(BASE_DIR, FILE_NAME);
-    for(int i=0; i < replicas.size(); i++) {
+    for (int i = 0; i < replicas.size(); i++) {
       ProvidedReplica replica = replicas.get(i);
-      //block data should exist!
+      // block data should exist!
       assertTrue(replica.blockDataExists());
       assertEquals(providedFile.toURI(), replica.getBlockURI());
       verifyReplicaContents(providedFile, replica.getDataInputStream(0),
@@ -153,7 +152,7 @@ public class TestProvidedReplicaImpl {
     LOG.info("All replica contents verified");
 
     providedFile.delete();
-    //the block data should no longer be found!
+    // the block data should no longer be found!
     for(int i=0; i < replicas.size(); i++) {
       ProvidedReplica replica = replicas.get(i);
       assertTrue(!replica.blockDataExists());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8239e3af/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestProvidedImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestProvidedImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestProvidedImpl.java
index e057022..422acc3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestProvidedImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestProvidedImpl.java
@@ -26,8 +26,6 @@ import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
 import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
@@ -35,9 +33,6 @@ import java.io.OutputStreamWriter;
 import java.io.Writer;
 import java.net.URI;
 import java.net.URISyntaxException;
-import java.nio.ByteBuffer;
-import java.nio.channels.Channels;
-import java.nio.channels.ReadableByteChannel;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -76,6 +71,7 @@ import org.apache.hadoop.hdfs.server.datanode.ProvidedReplica;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
 import org.apache.hadoop.hdfs.server.datanode.ShortCircuitRegistry;
 import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
+import org.apache.hadoop.hdfs.server.datanode.TestProvidedReplicaImpl;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi.BlockIterator;
@@ -97,7 +93,7 @@ public class TestProvidedImpl {
   private static final String BASE_DIR =
       new FileSystemTestHelper().getTestRootDir();
   private static final int NUM_LOCAL_INIT_VOLUMES = 1;
-  //only support one provided volume for now.
+  // only support one provided volume for now.
   private static final int NUM_PROVIDED_INIT_VOLUMES = 1;
   private static final String[] BLOCK_POOL_IDS = {"bpid-0", "bpid-1"};
   private static final int NUM_PROVIDED_BLKS = 10;
@@ -168,7 +164,7 @@ public class TestProvidedImpl {
 
     @Override
     public void remove() {
-      //do nothing.
+      // do nothing.
     }
 
     public void resetMinBlockId(int minId) {
@@ -314,33 +310,6 @@ public class TestProvidedImpl {
     }
   }
 
-  private void compareBlkFile(InputStream ins, String filepath)
-      throws FileNotFoundException, IOException {
-    try (ReadableByteChannel i = Channels.newChannel(
-        new FileInputStream(new File(filepath)))) {
-      try (ReadableByteChannel j = Channels.newChannel(ins)) {
-        ByteBuffer ib = ByteBuffer.allocate(4096);
-        ByteBuffer jb = ByteBuffer.allocate(4096);
-        while (true) {
-          int il = i.read(ib);
-          int jl = j.read(jb);
-          if (il < 0 || jl < 0) {
-            assertEquals(il, jl);
-            break;
-          }
-          ib.flip();
-          jb.flip();
-          int cmp = Math.min(ib.remaining(), jb.remaining());
-          for (int k = 0; k < cmp; ++k) {
-            assertEquals(ib.get(), jb.get());
-          }
-          ib.compact();
-          jb.compact();
-        }
-      }
-    }
-  }
-
   @Before
   public void setUp() throws IOException {
     datanode = mock(DataNode.class);
@@ -392,7 +361,7 @@ public class TestProvidedImpl {
     assertEquals(0, dataset.getNumFailedVolumes());
 
     for (int i = 0; i < providedVolumes.size(); i++) {
-      //check basic information about provided volume
+      // check basic information about provided volume
       assertEquals(DFSConfigKeys.DFS_PROVIDER_STORAGEUUID_DEFAULT,
           providedVolumes.get(i).getStorageID());
       assertEquals(StorageType.PROVIDED,
@@ -400,7 +369,7 @@ public class TestProvidedImpl {
 
       long space = providedVolumes.get(i).getBlockPoolUsed(
               BLOCK_POOL_IDS[CHOSEN_BP_ID]);
-      //check the df stats of the volume
+      // check the df stats of the volume
       assertEquals(spaceUsed, space);
       assertEquals(NUM_PROVIDED_BLKS, providedVolumes.get(i).getNumBlocks());
 
@@ -409,7 +378,7 @@ public class TestProvidedImpl {
       try {
         assertEquals(0, providedVolumes.get(i)
             .getBlockPoolUsed(BLOCK_POOL_IDS[1 - CHOSEN_BP_ID]));
-        //should not be triggered
+        // should not be triggered
         assertTrue(false);
       } catch (IOException e) {
         LOG.info("Expected exception: " + e);
@@ -428,7 +397,7 @@ public class TestProvidedImpl {
       assertEquals(vol.getBlockPoolList().length, BLOCK_POOL_IDS.length);
       for (int j = 0; j < BLOCK_POOL_IDS.length; j++) {
         if (j != CHOSEN_BP_ID) {
-          //this block pool should not have any blocks
+          // this block pool should not have any blocks
           assertEquals(null, volumeMap.replicas(BLOCK_POOL_IDS[j]));
         }
       }
@@ -445,7 +414,8 @@ public class TestProvidedImpl {
           HdfsConstants.GRANDFATHER_GENERATION_STAMP);
       InputStream ins = dataset.getBlockInputStream(eb, 0);
       String filepath = blkToPathMap.get((long) id);
-      compareBlkFile(ins, filepath);
+      TestProvidedReplicaImpl.verifyReplicaContents(new File(filepath), ins, 0,
+          BLK_LEN);
     }
   }
 
@@ -462,7 +432,7 @@ public class TestProvidedImpl {
         ExtendedBlock eb = iter.nextBlock();
         long blkId = eb.getBlockId();
         assertTrue(blkId >= MIN_BLK_ID && blkId < NUM_PROVIDED_BLKS);
-        //all block ids must be unique!
+        // all block ids must be unique!
         assertTrue(!blockIdsUsed.contains(blkId));
         blockIdsUsed.add(blkId);
       }
@@ -473,14 +443,14 @@ public class TestProvidedImpl {
       while(!iter.atEnd()) {
         ExtendedBlock eb = iter.nextBlock();
         long blkId = eb.getBlockId();
-        //the block should have already appeared in the first scan.
+        // the block should have already appeared in the first scan.
         assertTrue(blockIdsUsed.contains(blkId));
         blockIdsUsed.remove(blkId);
       }
-      //none of the blocks should remain in blockIdsUsed
+      // none of the blocks should remain in blockIdsUsed
       assertEquals(0, blockIdsUsed.size());
 
-      //the other block pool should not contain any blocks!
+      // the other block pool should not contain any blocks!
       BlockIterator nonProvidedBpIter =
           vol.newBlockIterator(BLOCK_POOL_IDS[1 - CHOSEN_BP_ID], "temp");
       assertEquals(null, nonProvidedBpIter.nextBlock());
@@ -513,8 +483,8 @@ public class TestProvidedImpl {
   public void testProvidedVolumeContents() throws IOException {
     int expectedBlocks = 5;
     int minId = 0;
-    //use a path which has the same prefix as providedBasePath
-    //all these blocks can belong to the provided volume
+    // use a path which has the same prefix as providedBasePath
+    // all these blocks can belong to the provided volume
     int blocksFound = getBlocksInProvidedVolumes(providedBasePath + "/test1/",
         expectedBlocks, minId);
     assertEquals(
@@ -525,8 +495,8 @@ public class TestProvidedImpl {
     assertEquals(
         "Number of blocks in provided volumes should be " + expectedBlocks,
         expectedBlocks, blocksFound);
-    //use a path that is entirely different from the providedBasePath
-    //none of these blocks can belong to the volume
+    // use a path that is entirely different from the providedBasePath
+    // none of these blocks can belong to the volume
     blocksFound =
         getBlocksInProvidedVolumes("randomtest1/", expectedBlocks, minId);
     assertEquals("Number of blocks in provided volumes should be 0", 0,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8239e3af/hadoop-tools/hadoop-fs2img/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/pom.xml b/hadoop-tools/hadoop-fs2img/pom.xml
index 8661c82..2e3e66a 100644
--- a/hadoop-tools/hadoop-fs2img/pom.xml
+++ b/hadoop-tools/hadoop-fs2img/pom.xml
@@ -23,8 +23,8 @@
   <groupId>org.apache.hadoop</groupId>
   <artifactId>hadoop-fs2img</artifactId>
   <version>3.1.0-SNAPSHOT</version>
-  <description>fs2img</description>
-  <name>fs2img</name>
+  <description>Apache Hadoop Image Generation Tool</description>
+  <name>Apache Hadoop Image Generation Tool</name>
   <packaging>jar</packaging>
 
   <properties>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8239e3af/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileSystemImage.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileSystemImage.java b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileSystemImage.java
index 80bbaf9..afe10ff 100644
--- a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileSystemImage.java
+++ b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileSystemImage.java
@@ -125,7 +125,8 @@ public class FileSystemImage implements Tool {
         opts.blockPoolID(o.getValue());
         break;
       default:
-        throw new UnsupportedOperationException("Internal error");
+        throw new UnsupportedOperationException(
+            "Unknown option: " + o.getOpt());
       }
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8239e3af/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageWriter.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageWriter.java b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageWriter.java
index c21c282..1be5190 100644
--- a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageWriter.java
+++ b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageWriter.java
@@ -236,7 +236,7 @@ public class ImageWriter implements Closeable {
       if (null == e) {
         return super.put(p, b);
       }
-      //merge
+      // merge
       e.addAllChildren(b.getChildrenList());
       // not strictly conforming
       return e;
@@ -265,7 +265,6 @@ public class ImageWriter implements Closeable {
     e.writeDelimitedTo(dirs);
   }
 
-  // from FSImageFormatProtobuf... why not just read position from the stream?
   private static int getOndiskSize(com.google.protobuf.GeneratedMessage s) {
     return CodedOutputStream.computeRawVarint32Size(s.getSerializedSize())
         + s.getSerializedSize();
@@ -283,7 +282,7 @@ public class ImageWriter implements Closeable {
     dircache.clear();
 
     // close side files
-    IOUtils.cleanup(null, dirs, inodes, blocks);
+    IOUtils.cleanupWithLogger(null, dirs, inodes, blocks);
     if (null == dirs || null == inodes) {
       // init failed
       if (raw != null) {
@@ -317,7 +316,6 @@ public class ImageWriter implements Closeable {
    */
   void writeMD5(String imagename) throws IOException {
     if (null == outdir) {
-      //LOG.warn("Not writing MD5");
       return;
     }
     MD5Hash md5 = new MD5Hash(digest.digest());
@@ -382,7 +380,6 @@ public class ImageWriter implements Closeable {
 
   void writeDirSection() throws IOException {
     // No header, so dirs can be written/compressed independently
-    //INodeDirectorySection.Builder b = INodeDirectorySection.newBuilder();
     OutputStream sec = raw;
     // copy dirs
     try (FileInputStream in = new FileInputStream(dirsTmp)) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8239e3af/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/SingleUGIResolver.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/SingleUGIResolver.java b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/SingleUGIResolver.java
index d60806f..9c42c11 100644
--- a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/SingleUGIResolver.java
+++ b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/SingleUGIResolver.java
@@ -84,11 +84,11 @@ public class SingleUGIResolver extends UGIResolver implements Configurable {
 
   @Override
   public void addUser(String name) {
-    //do nothing
+    // do nothing
   }
 
   @Override
   public void addGroup(String name) {
-    //do nothing
+    // do nothing
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8239e3af/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/TreePath.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/TreePath.java b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/TreePath.java
index dde351f..fd4dbff 100644
--- a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/TreePath.java
+++ b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/TreePath.java
@@ -121,7 +121,6 @@ public class TreePath {
   INode toFile(UGIResolver ugi, BlockResolver blk,
       BlockAliasMap.Writer<FileRegion> out) throws IOException {
     final FileStatus s = getFileStatus();
-    // TODO should this store resolver's user/group?
     ugi.addUser(s.getOwner());
     ugi.addGroup(s.getGroup());
     INodeFile.Builder b = INodeFile.newBuilder()
@@ -142,7 +141,7 @@ public class TreePath {
             "Exact path handle not supported by filesystem " + fs.toString());
       }
     }
-    //TODO: storage policy should be configurable per path; use BlockResolver
+    // TODO: storage policy should be configurable per path; use BlockResolver
     long off = 0L;
     for (BlockProto block : blk.resolve(s)) {
       b.addBlocks(block);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8239e3af/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/ITestProvidedImplementation.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/ITestProvidedImplementation.java b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/ITestProvidedImplementation.java
new file mode 100644
index 0000000..49c9bcf
--- /dev/null
+++ b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/ITestProvidedImplementation.java
@@ -0,0 +1,927 @@
+/**
+ * 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.namenode;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.io.Writer;
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.file.Files;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Random;
+import java.util.Set;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.DFSClient;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMap;
+import org.apache.hadoop.hdfs.server.aliasmap.InMemoryLevelDBAliasMapServer;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStatistics;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.ProvidedStorageMap;
+import org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap;
+import org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.InMemoryLevelDBAliasMapClient;
+import org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.TextFileRegionAliasMap;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_REPLICATOR_CLASSNAME_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY;
+
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsVolumeImpl;
+import org.apache.hadoop.hdfs.server.protocol.StorageReport;
+import org.apache.hadoop.net.NodeBase;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.TextFileRegionAliasMap.fileNameFromBlockPoolID;
+import static org.apache.hadoop.net.NodeBase.PATH_SEPARATOR_STR;
+import static org.junit.Assert.*;
+
+/**
+ * Integration tests for the Provided implementation.
+ */
+public class ITestProvidedImplementation {
+
+  @Rule public TestName name = new TestName();
+  public static final Logger LOG =
+      LoggerFactory.getLogger(ITestProvidedImplementation.class);
+
+  private final Random r = new Random();
+  private final File fBASE = new File(MiniDFSCluster.getBaseDirectory());
+  private final Path pBASE = new Path(fBASE.toURI().toString());
+  private final Path providedPath = new Path(pBASE, "providedDir");
+  private final Path nnDirPath = new Path(pBASE, "nnDir");
+  private final String singleUser = "usr1";
+  private final String singleGroup = "grp1";
+  private final int numFiles = 10;
+  private final String filePrefix = "file";
+  private final String fileSuffix = ".dat";
+  private final int baseFileLen = 1024;
+  private long providedDataSize = 0;
+  private final String bpid = "BP-1234-10.1.1.1-1224";
+
+  private Configuration conf;
+  private MiniDFSCluster cluster;
+
+  @Before
+  public void setSeed() throws Exception {
+    if (fBASE.exists() && !FileUtil.fullyDelete(fBASE)) {
+      throw new IOException("Could not fully delete " + fBASE);
+    }
+    long seed = r.nextLong();
+    r.setSeed(seed);
+    System.out.println(name.getMethodName() + " seed: " + seed);
+    conf = new HdfsConfiguration();
+    conf.set(SingleUGIResolver.USER, singleUser);
+    conf.set(SingleUGIResolver.GROUP, singleGroup);
+
+    conf.set(DFSConfigKeys.DFS_PROVIDER_STORAGEUUID,
+        DFSConfigKeys.DFS_PROVIDER_STORAGEUUID_DEFAULT);
+    conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_PROVIDED_ENABLED, true);
+
+    conf.setClass(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_CLASS,
+        TextFileRegionAliasMap.class, BlockAliasMap.class);
+    conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_WRITE_DIR,
+        nnDirPath.toString());
+    conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_READ_FILE,
+        new Path(nnDirPath, fileNameFromBlockPoolID(bpid)).toString());
+    conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_TEXT_DELIMITER, ",");
+
+    conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR_PROVIDED,
+        new File(providedPath.toUri()).toString());
+    File imageDir = new File(providedPath.toUri());
+    if (!imageDir.exists()) {
+      LOG.info("Creating directory: " + imageDir);
+      imageDir.mkdirs();
+    }
+
+    File nnDir = new File(nnDirPath.toUri());
+    if (!nnDir.exists()) {
+      nnDir.mkdirs();
+    }
+
+    // create 10 random files under pBASE
+    for (int i=0; i < numFiles; i++) {
+      File newFile = new File(
+          new Path(providedPath, filePrefix + i + fileSuffix).toUri());
+      if(!newFile.exists()) {
+        try {
+          LOG.info("Creating " + newFile.toString());
+          newFile.createNewFile();
+          Writer writer = new OutputStreamWriter(
+              new FileOutputStream(newFile.getAbsolutePath()), "utf-8");
+          for(int j=0; j < baseFileLen*i; j++) {
+            writer.write("0");
+          }
+          writer.flush();
+          writer.close();
+          providedDataSize += newFile.length();
+        } catch (IOException e) {
+          e.printStackTrace();
+        }
+      }
+    }
+  }
+
+  @After
+  public void shutdown() throws Exception {
+    try {
+      if (cluster != null) {
+        cluster.shutdown(true, true);
+      }
+    } finally {
+      cluster = null;
+    }
+  }
+
+  void createImage(TreeWalk t, Path out,
+      Class<? extends BlockResolver> blockIdsClass) throws Exception {
+    createImage(t, out, blockIdsClass, "", TextFileRegionAliasMap.class);
+  }
+
+  void createImage(TreeWalk t, Path out,
+      Class<? extends BlockResolver> blockIdsClass, String clusterID,
+      Class<? extends BlockAliasMap> aliasMapClass) throws Exception {
+    ImageWriter.Options opts = ImageWriter.defaults();
+    opts.setConf(conf);
+    opts.output(out.toString())
+        .blocks(aliasMapClass)
+        .blockIds(blockIdsClass)
+        .clusterID(clusterID)
+        .blockPoolID(bpid);
+    try (ImageWriter w = new ImageWriter(opts)) {
+      for (TreePath e : t) {
+        w.accept(e);
+      }
+    }
+  }
+  void startCluster(Path nspath, int numDatanodes,
+      StorageType[] storageTypes,
+      StorageType[][] storageTypesPerDatanode,
+      boolean doFormat) throws IOException {
+    startCluster(nspath, numDatanodes, storageTypes, storageTypesPerDatanode,
+        doFormat, null);
+  }
+
+  void startCluster(Path nspath, int numDatanodes,
+      StorageType[] storageTypes,
+      StorageType[][] storageTypesPerDatanode,
+      boolean doFormat, String[] racks) throws IOException {
+    conf.set(DFS_NAMENODE_NAME_DIR_KEY, nspath.toString());
+
+    if (storageTypesPerDatanode != null) {
+      cluster = new MiniDFSCluster.Builder(conf)
+          .format(doFormat)
+          .manageNameDfsDirs(doFormat)
+          .numDataNodes(numDatanodes)
+          .storageTypes(storageTypesPerDatanode)
+          .racks(racks)
+          .build();
+    } else if (storageTypes != null) {
+      cluster = new MiniDFSCluster.Builder(conf)
+          .format(doFormat)
+          .manageNameDfsDirs(doFormat)
+          .numDataNodes(numDatanodes)
+          .storagesPerDatanode(storageTypes.length)
+          .storageTypes(storageTypes)
+          .racks(racks)
+          .build();
+    } else {
+      cluster = new MiniDFSCluster.Builder(conf)
+          .format(doFormat)
+          .manageNameDfsDirs(doFormat)
+          .numDataNodes(numDatanodes)
+          .racks(racks)
+          .build();
+    }
+    cluster.waitActive();
+  }
+
+  @Test(timeout=20000)
+  public void testLoadImage() throws Exception {
+    final long seed = r.nextLong();
+    LOG.info("providedPath: " + providedPath);
+    createImage(new RandomTreeWalk(seed), nnDirPath, FixedBlockResolver.class);
+    startCluster(nnDirPath, 0,
+        new StorageType[] {StorageType.PROVIDED, StorageType.DISK}, null,
+        false);
+
+    FileSystem fs = cluster.getFileSystem();
+    for (TreePath e : new RandomTreeWalk(seed)) {
+      FileStatus rs = e.getFileStatus();
+      Path hp = new Path(rs.getPath().toUri().getPath());
+      assertTrue(fs.exists(hp));
+      FileStatus hs = fs.getFileStatus(hp);
+      assertEquals(rs.getPath().toUri().getPath(),
+                   hs.getPath().toUri().getPath());
+      assertEquals(rs.getPermission(), hs.getPermission());
+      assertEquals(rs.getLen(), hs.getLen());
+      assertEquals(singleUser, hs.getOwner());
+      assertEquals(singleGroup, hs.getGroup());
+      assertEquals(rs.getAccessTime(), hs.getAccessTime());
+      assertEquals(rs.getModificationTime(), hs.getModificationTime());
+    }
+  }
+
+  @Test(timeout=30000)
+  public void testProvidedReporting() throws Exception {
+    conf.setClass(ImageWriter.Options.UGI_CLASS,
+        SingleUGIResolver.class, UGIResolver.class);
+    createImage(new FSTreeWalk(providedPath, conf), nnDirPath,
+        FixedBlockResolver.class);
+    int numDatanodes = 10;
+    startCluster(nnDirPath, numDatanodes,
+        new StorageType[] {StorageType.PROVIDED, StorageType.DISK}, null,
+        false);
+    long diskCapacity = 1000;
+    // set the DISK capacity for testing
+    for (DataNode dn: cluster.getDataNodes()) {
+      for (FsVolumeSpi ref : dn.getFSDataset().getFsVolumeReferences()) {
+        if (ref.getStorageType() == StorageType.DISK) {
+          ((FsVolumeImpl) ref).setCapacityForTesting(diskCapacity);
+        }
+      }
+    }
+    // trigger heartbeats to update the capacities
+    cluster.triggerHeartbeats();
+    Thread.sleep(10000);
+    // verify namenode stats
+    FSNamesystem namesystem = cluster.getNameNode().getNamesystem();
+    DatanodeStatistics dnStats = namesystem.getBlockManager()
+        .getDatanodeManager().getDatanodeStatistics();
+
+    // total capacity reported includes only the local volumes and
+    // not the provided capacity
+    assertEquals(diskCapacity * numDatanodes, namesystem.getTotal());
+
+    // total storage used should be equal to the totalProvidedStorage
+    // no capacity should be remaining!
+    assertEquals(providedDataSize, dnStats.getProvidedCapacity());
+    assertEquals(providedDataSize, namesystem.getProvidedCapacityTotal());
+    assertEquals(providedDataSize, dnStats.getStorageTypeStats()
+        .get(StorageType.PROVIDED).getCapacityTotal());
+    assertEquals(providedDataSize, dnStats.getStorageTypeStats()
+        .get(StorageType.PROVIDED).getCapacityUsed());
+
+    // verify datanode stats
+    for (DataNode dn: cluster.getDataNodes()) {
+      for (StorageReport report : dn.getFSDataset()
+          .getStorageReports(namesystem.getBlockPoolId())) {
+        if (report.getStorage().getStorageType() == StorageType.PROVIDED) {
+          assertEquals(providedDataSize, report.getCapacity());
+          assertEquals(providedDataSize, report.getDfsUsed());
+          assertEquals(providedDataSize, report.getBlockPoolUsed());
+          assertEquals(0, report.getNonDfsUsed());
+          assertEquals(0, report.getRemaining());
+        }
+      }
+    }
+
+    DFSClient client = new DFSClient(new InetSocketAddress("localhost",
+            cluster.getNameNodePort()), cluster.getConfiguration(0));
+    BlockManager bm = namesystem.getBlockManager();
+    for (int fileId = 0; fileId < numFiles; fileId++) {
+      String filename = "/" + filePrefix + fileId + fileSuffix;
+      LocatedBlocks locatedBlocks = client.getLocatedBlocks(
+          filename, 0, baseFileLen);
+      for (LocatedBlock locatedBlock : locatedBlocks.getLocatedBlocks()) {
+        BlockInfo blockInfo =
+            bm.getStoredBlock(locatedBlock.getBlock().getLocalBlock());
+        Iterator<DatanodeStorageInfo> storagesItr = blockInfo.getStorageInfos();
+
+        DatanodeStorageInfo info = storagesItr.next();
+        assertEquals(StorageType.PROVIDED, info.getStorageType());
+        DatanodeDescriptor dnDesc = info.getDatanodeDescriptor();
+        // check the locations that are returned by FSCK have the right name
+        assertEquals(ProvidedStorageMap.ProvidedDescriptor.NETWORK_LOCATION
+            + PATH_SEPARATOR_STR + ProvidedStorageMap.ProvidedDescriptor.NAME,
+            NodeBase.getPath(dnDesc));
+        // no DatanodeStorageInfos should remain
+        assertFalse(storagesItr.hasNext());
+      }
+    }
+  }
+
+  @Test(timeout=500000)
+  public void testDefaultReplication() throws Exception {
+    int targetReplication = 2;
+    conf.setInt(FixedBlockMultiReplicaResolver.REPLICATION, targetReplication);
+    createImage(new FSTreeWalk(providedPath, conf), nnDirPath,
+        FixedBlockMultiReplicaResolver.class);
+    // make the last Datanode with only DISK
+    startCluster(nnDirPath, 3, null,
+        new StorageType[][] {
+            {StorageType.PROVIDED, StorageType.DISK},
+            {StorageType.PROVIDED, StorageType.DISK},
+            {StorageType.DISK}},
+        false);
+    // wait for the replication to finish
+    Thread.sleep(50000);
+
+    FileSystem fs = cluster.getFileSystem();
+    int count = 0;
+    for (TreePath e : new FSTreeWalk(providedPath, conf)) {
+      FileStatus rs = e.getFileStatus();
+      Path hp = removePrefix(providedPath, rs.getPath());
+      LOG.info("path: " + hp.toUri().getPath());
+      e.accept(count++);
+      assertTrue(fs.exists(hp));
+      FileStatus hs = fs.getFileStatus(hp);
+
+      if (rs.isFile()) {
+        BlockLocation[] bl = fs.getFileBlockLocations(
+            hs.getPath(), 0, hs.getLen());
+        int i = 0;
+        for(; i < bl.length; i++) {
+          int currentRep = bl[i].getHosts().length;
+          assertEquals(targetReplication, currentRep);
+        }
+      }
+    }
+  }
+
+
+  static Path removePrefix(Path base, Path walk) {
+    Path wpath = new Path(walk.toUri().getPath());
+    Path bpath = new Path(base.toUri().getPath());
+    Path ret = new Path("/");
+    while (!(bpath.equals(wpath) || "".equals(wpath.getName()))) {
+      ret = "".equals(ret.getName())
+        ? new Path("/", wpath.getName())
+        : new Path(new Path("/", wpath.getName()),
+                   new Path(ret.toString().substring(1)));
+      wpath = wpath.getParent();
+    }
+    if (!bpath.equals(wpath)) {
+      throw new IllegalArgumentException(base + " not a prefix of " + walk);
+    }
+    return ret;
+  }
+
+  private void verifyFileSystemContents() throws Exception {
+    FileSystem fs = cluster.getFileSystem();
+    int count = 0;
+    // read NN metadata, verify contents match
+    for (TreePath e : new FSTreeWalk(providedPath, conf)) {
+      FileStatus rs = e.getFileStatus();
+      Path hp = removePrefix(providedPath, rs.getPath());
+      LOG.info("path: " + hp.toUri().getPath());
+      e.accept(count++);
+      assertTrue(fs.exists(hp));
+      FileStatus hs = fs.getFileStatus(hp);
+      assertEquals(hp.toUri().getPath(), hs.getPath().toUri().getPath());
+      assertEquals(rs.getPermission(), hs.getPermission());
+      assertEquals(rs.getOwner(), hs.getOwner());
+      assertEquals(rs.getGroup(), hs.getGroup());
+
+      if (rs.isFile()) {
+        assertEquals(rs.getLen(), hs.getLen());
+        try (ReadableByteChannel i = Channels.newChannel(
+              new FileInputStream(new File(rs.getPath().toUri())))) {
+          try (ReadableByteChannel j = Channels.newChannel(
+                fs.open(hs.getPath()))) {
+            ByteBuffer ib = ByteBuffer.allocate(4096);
+            ByteBuffer jb = ByteBuffer.allocate(4096);
+            while (true) {
+              int il = i.read(ib);
+              int jl = j.read(jb);
+              if (il < 0 || jl < 0) {
+                assertEquals(il, jl);
+                break;
+              }
+              ib.flip();
+              jb.flip();
+              int cmp = Math.min(ib.remaining(), jb.remaining());
+              for (int k = 0; k < cmp; ++k) {
+                assertEquals(ib.get(), jb.get());
+              }
+              ib.compact();
+              jb.compact();
+            }
+
+          }
+        }
+      }
+    }
+  }
+
+  private BlockLocation[] createFile(Path path, short replication,
+      long fileLen, long blockLen) throws IOException {
+    FileSystem fs = cluster.getFileSystem();
+    // create a file that is not provided
+    DFSTestUtil.createFile(fs, path, false, (int) blockLen,
+        fileLen, blockLen, replication, 0, true);
+    return fs.getFileBlockLocations(path, 0, fileLen);
+  }
+
+  @Test(timeout=30000)
+  public void testClusterWithEmptyImage() throws IOException {
+    // start a cluster with 2 datanodes without any provided storage
+    startCluster(nnDirPath, 2, null,
+        new StorageType[][] {
+            {StorageType.DISK},
+            {StorageType.DISK}},
+        true);
+    assertTrue(cluster.isClusterUp());
+    assertTrue(cluster.isDataNodeUp());
+
+    BlockLocation[] locations = createFile(new Path("/testFile1.dat"),
+        (short) 2, 1024*1024, 1024*1024);
+    assertEquals(1, locations.length);
+    assertEquals(2, locations[0].getHosts().length);
+  }
+
+  private DatanodeInfo[] getAndCheckBlockLocations(DFSClient client,
+      String filename, long fileLen, long expectedBlocks, int expectedLocations)
+      throws IOException {
+    LocatedBlocks locatedBlocks = client.getLocatedBlocks(filename, 0, fileLen);
+    // given the start and length in the above call,
+    // only one LocatedBlock in LocatedBlocks
+    assertEquals(expectedBlocks, locatedBlocks.getLocatedBlocks().size());
+    DatanodeInfo[] locations =
+        locatedBlocks.getLocatedBlocks().get(0).getLocations();
+    assertEquals(expectedLocations, locations.length);
+    checkUniqueness(locations);
+    return locations;
+  }
+
+  /**
+   * verify that the given locations are all unique.
+   * @param locations
+   */
+  private void checkUniqueness(DatanodeInfo[] locations) {
+    Set<String> set = new HashSet<>();
+    for (DatanodeInfo info: locations) {
+      assertFalse("All locations should be unique",
+          set.contains(info.getDatanodeUuid()));
+      set.add(info.getDatanodeUuid());
+    }
+  }
+
+  /**
+   * Tests setting replication of provided files.
+   * @throws Exception
+   */
+  @Test(timeout=50000)
+  public void testSetReplicationForProvidedFiles() throws Exception {
+    createImage(new FSTreeWalk(providedPath, conf), nnDirPath,
+        FixedBlockResolver.class);
+    // 10 Datanodes with both DISK and PROVIDED storage
+    startCluster(nnDirPath, 10,
+        new StorageType[]{
+            StorageType.PROVIDED, StorageType.DISK},
+        null,
+        false);
+    setAndUnsetReplication("/" + filePrefix + (numFiles - 1) + fileSuffix);
+  }
+
+  private void setAndUnsetReplication(String filename) throws Exception {
+    Path file = new Path(filename);
+    FileSystem fs = cluster.getFileSystem();
+    // set the replication to 4, and test that the file has
+    // the required replication.
+    short newReplication = 4;
+    LOG.info("Setting replication of file {} to {}", filename, newReplication);
+    fs.setReplication(file, newReplication);
+    DFSTestUtil.waitForReplication((DistributedFileSystem) fs,
+        file, newReplication, 10000);
+    DFSClient client = new DFSClient(new InetSocketAddress("localhost",
+        cluster.getNameNodePort()), cluster.getConfiguration(0));
+    getAndCheckBlockLocations(client, filename, baseFileLen, 1, newReplication);
+
+    // set the replication back to 1
+    newReplication = 1;
+    LOG.info("Setting replication of file {} back to {}",
+        filename, newReplication);
+    fs.setReplication(file, newReplication);
+    // defaultReplication number of replicas should be returned
+    int defaultReplication = conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY,
+        DFSConfigKeys.DFS_REPLICATION_DEFAULT);
+    DFSTestUtil.waitForReplication((DistributedFileSystem) fs,
+        file, (short) defaultReplication, 10000);
+    getAndCheckBlockLocations(client, filename, baseFileLen, 1,
+        defaultReplication);
+  }
+
+  @Test(timeout=30000)
+  public void testProvidedDatanodeFailures() throws Exception {
+    createImage(new FSTreeWalk(providedPath, conf), nnDirPath,
+            FixedBlockResolver.class);
+    startCluster(nnDirPath, 3, null,
+        new StorageType[][] {
+            {StorageType.PROVIDED, StorageType.DISK},
+            {StorageType.PROVIDED, StorageType.DISK},
+            {StorageType.DISK}},
+        false);
+
+    DataNode providedDatanode1 = cluster.getDataNodes().get(0);
+    DataNode providedDatanode2 = cluster.getDataNodes().get(1);
+
+    DFSClient client = new DFSClient(new InetSocketAddress("localhost",
+        cluster.getNameNodePort()), cluster.getConfiguration(0));
+
+    DatanodeStorageInfo providedDNInfo = getProvidedDatanodeStorageInfo();
+
+    if (numFiles >= 1) {
+      String filename = "/" + filePrefix + (numFiles - 1) + fileSuffix;
+      // 2 locations returned as there are 2 PROVIDED datanodes
+      DatanodeInfo[] dnInfos =
+          getAndCheckBlockLocations(client, filename, baseFileLen, 1, 2);
+      // the location should be one of the provided DNs available
+      assertTrue(
+          dnInfos[0].getDatanodeUuid().equals(
+              providedDatanode1.getDatanodeUuid())
+          || dnInfos[0].getDatanodeUuid().equals(
+              providedDatanode2.getDatanodeUuid()));
+
+      // stop the 1st provided datanode
+      MiniDFSCluster.DataNodeProperties providedDNProperties1 =
+          cluster.stopDataNode(0);
+
+      // make NameNode detect that datanode is down
+      BlockManagerTestUtil.noticeDeadDatanode(
+          cluster.getNameNode(),
+          providedDatanode1.getDatanodeId().getXferAddr());
+
+      // should find the block on the 2nd provided datanode
+      dnInfos = getAndCheckBlockLocations(client, filename, baseFileLen, 1, 1);
+      assertEquals(providedDatanode2.getDatanodeUuid(),
+          dnInfos[0].getDatanodeUuid());
+
+      // stop the 2nd provided datanode
+      MiniDFSCluster.DataNodeProperties providedDNProperties2 =
+          cluster.stopDataNode(0);
+      // make NameNode detect that datanode is down
+      BlockManagerTestUtil.noticeDeadDatanode(
+          cluster.getNameNode(),
+          providedDatanode2.getDatanodeId().getXferAddr());
+      getAndCheckBlockLocations(client, filename, baseFileLen, 1, 0);
+
+      // BR count for the provided ProvidedDatanodeStorageInfo should reset to
+      // 0, when all DNs with PROVIDED storage fail.
+      assertEquals(0, providedDNInfo.getBlockReportCount());
+      // restart the provided datanode
+      cluster.restartDataNode(providedDNProperties1, true);
+      cluster.waitActive();
+
+      assertEquals(1, providedDNInfo.getBlockReportCount());
+
+      // should find the block on the 1st provided datanode now
+      dnInfos = getAndCheckBlockLocations(client, filename, baseFileLen, 1, 1);
+      // not comparing UUIDs as the datanode can now have a different one.
+      assertEquals(providedDatanode1.getDatanodeId().getXferAddr(),
+          dnInfos[0].getXferAddr());
+    }
+  }
+
+  @Test(timeout=300000)
+  public void testTransientDeadDatanodes() throws Exception {
+    createImage(new FSTreeWalk(providedPath, conf), nnDirPath,
+            FixedBlockResolver.class);
+    // 3 Datanodes, 2 PROVIDED and other DISK
+    startCluster(nnDirPath, 3, null,
+        new StorageType[][] {
+            {StorageType.PROVIDED, StorageType.DISK},
+            {StorageType.PROVIDED, StorageType.DISK},
+            {StorageType.DISK}},
+        false);
+
+    DataNode providedDatanode = cluster.getDataNodes().get(0);
+    DatanodeStorageInfo providedDNInfo = getProvidedDatanodeStorageInfo();
+    int initialBRCount = providedDNInfo.getBlockReportCount();
+    for (int i= 0; i < numFiles; i++) {
+      // expect to have 2 locations as we have 2 provided Datanodes.
+      verifyFileLocation(i, 2);
+      // NameNode thinks the datanode is down
+      BlockManagerTestUtil.noticeDeadDatanode(
+          cluster.getNameNode(),
+          providedDatanode.getDatanodeId().getXferAddr());
+      cluster.waitActive();
+      cluster.triggerHeartbeats();
+      Thread.sleep(1000);
+      // the report count should just continue to increase.
+      assertEquals(initialBRCount + i + 1,
+          providedDNInfo.getBlockReportCount());
+      verifyFileLocation(i, 2);
+    }
+  }
+
+  private DatanodeStorageInfo getProvidedDatanodeStorageInfo() {
+    ProvidedStorageMap providedStorageMap =
+        cluster.getNamesystem().getBlockManager().getProvidedStorageMap();
+    return providedStorageMap.getProvidedStorageInfo();
+  }
+
+  @Test(timeout=30000)
+  public void testNamenodeRestart() throws Exception {
+    createImage(new FSTreeWalk(providedPath, conf), nnDirPath,
+        FixedBlockResolver.class);
+    // 3 Datanodes, 2 PROVIDED and other DISK
+    startCluster(nnDirPath, 3, null,
+        new StorageType[][] {
+            {StorageType.PROVIDED, StorageType.DISK},
+            {StorageType.PROVIDED, StorageType.DISK},
+            {StorageType.DISK}},
+        false);
+
+    verifyFileLocation(numFiles - 1, 2);
+    cluster.restartNameNodes();
+    cluster.waitActive();
+    verifyFileLocation(numFiles - 1, 2);
+  }
+
+  /**
+   * verify that the specified file has a valid provided location.
+   * @param fileIndex the index of the file to verify.
+   * @throws Exception
+   */
+  private void verifyFileLocation(int fileIndex, int replication)
+      throws Exception {
+    DFSClient client = new DFSClient(
+        new InetSocketAddress("localhost", cluster.getNameNodePort()),
+        cluster.getConfiguration(0));
+    if (fileIndex < numFiles && fileIndex >= 0) {
+      String filename = filePrefix + fileIndex + fileSuffix;
+      File file = new File(new Path(providedPath, filename).toUri());
+      long fileLen = file.length();
+      long blockSize = conf.getLong(FixedBlockResolver.BLOCKSIZE,
+          FixedBlockResolver.BLOCKSIZE_DEFAULT);
+      long numLocatedBlocks =
+          fileLen == 0 ? 1 : (long) Math.ceil(fileLen * 1.0 / blockSize);
+      getAndCheckBlockLocations(client, "/" + filename, fileLen,
+          numLocatedBlocks, replication);
+    }
+  }
+
+  @Test(timeout=30000)
+  public void testSetClusterID() throws Exception {
+    String clusterID = "PROVIDED-CLUSTER";
+    createImage(new FSTreeWalk(providedPath, conf), nnDirPath,
+        FixedBlockResolver.class, clusterID, TextFileRegionAliasMap.class);
+    // 2 Datanodes, 1 PROVIDED and other DISK
+    startCluster(nnDirPath, 2, null,
+        new StorageType[][] {
+            {StorageType.PROVIDED, StorageType.DISK},
+            {StorageType.DISK}},
+        false);
+    NameNode nn = cluster.getNameNode();
+    assertEquals(clusterID, nn.getNamesystem().getClusterId());
+  }
+
+  @Test(timeout=30000)
+  public void testNumberOfProvidedLocations() throws Exception {
+    // set default replication to 4
+    conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, 4);
+    createImage(new FSTreeWalk(providedPath, conf), nnDirPath,
+        FixedBlockResolver.class);
+    // start with 4 PROVIDED location
+    startCluster(nnDirPath, 4,
+        new StorageType[]{
+            StorageType.PROVIDED, StorageType.DISK},
+        null,
+        false);
+    int expectedLocations = 4;
+    for (int i = 0; i < numFiles; i++) {
+      verifyFileLocation(i, expectedLocations);
+    }
+    // stop 2 datanodes, one after the other and verify number of locations.
+    for (int i = 1; i <= 2; i++) {
+      DataNode dn = cluster.getDataNodes().get(0);
+      cluster.stopDataNode(0);
+      // make NameNode detect that datanode is down
+      BlockManagerTestUtil.noticeDeadDatanode(cluster.getNameNode(),
+          dn.getDatanodeId().getXferAddr());
+
+      expectedLocations = 4 - i;
+      for (int j = 0; j < numFiles; j++) {
+        verifyFileLocation(j, expectedLocations);
+      }
+    }
+  }
+
+  @Test(timeout=30000)
+  public void testNumberOfProvidedLocationsManyBlocks() throws Exception {
+    // increase number of blocks per file to at least 10 blocks per file
+    conf.setLong(FixedBlockResolver.BLOCKSIZE, baseFileLen/10);
+    // set default replication to 4
+    conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, 4);
+    createImage(new FSTreeWalk(providedPath, conf), nnDirPath,
+        FixedBlockResolver.class);
+    // start with 4 PROVIDED location
+    startCluster(nnDirPath, 4,
+        new StorageType[]{
+            StorageType.PROVIDED, StorageType.DISK},
+        null,
+        false);
+    int expectedLocations = 4;
+    for (int i = 0; i < numFiles; i++) {
+      verifyFileLocation(i, expectedLocations);
+    }
+  }
+
+
+  @Test
+  public void testInMemoryAliasMap() throws Exception {
+    conf.setClass(ImageWriter.Options.UGI_CLASS,
+        FsUGIResolver.class, UGIResolver.class);
+    conf.setClass(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_CLASS,
+        InMemoryLevelDBAliasMapClient.class, BlockAliasMap.class);
+    conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS,
+        "localhost:32445");
+    File tempDirectory =
+        Files.createTempDirectory("in-memory-alias-map").toFile();
+    File leveDBPath = new File(tempDirectory, bpid);
+    leveDBPath.mkdirs();
+    conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_LEVELDB_DIR,
+        tempDirectory.getAbsolutePath());
+    conf.setBoolean(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_ENABLED, true);
+    conf.setInt(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_LOAD_RETRIES, 10);
+    InMemoryLevelDBAliasMapServer levelDBAliasMapServer =
+        new InMemoryLevelDBAliasMapServer(InMemoryAliasMap::init, bpid);
+    levelDBAliasMapServer.setConf(conf);
+    levelDBAliasMapServer.start();
+
+    createImage(new FSTreeWalk(providedPath, conf),
+        nnDirPath,
+        FixedBlockResolver.class, "",
+        InMemoryLevelDBAliasMapClient.class);
+    levelDBAliasMapServer.close();
+
+    // start cluster with two datanodes,
+    // each with 1 PROVIDED volume and other DISK volume
+    startCluster(nnDirPath, 2,
+        new StorageType[] {StorageType.PROVIDED, StorageType.DISK},
+        null, false);
+    verifyFileSystemContents();
+    FileUtils.deleteDirectory(tempDirectory);
+  }
+
+  private DatanodeDescriptor getDatanodeDescriptor(DatanodeManager dnm,
+      int dnIndex) throws Exception {
+    return dnm.getDatanode(cluster.getDataNodes().get(dnIndex).getDatanodeId());
+  }
+
+  private void startDecommission(FSNamesystem namesystem, DatanodeManager dnm,
+      int dnIndex) throws Exception {
+    namesystem.writeLock();
+    DatanodeDescriptor dnDesc = getDatanodeDescriptor(dnm, dnIndex);
+    dnm.getDatanodeAdminManager().startDecommission(dnDesc);
+    namesystem.writeUnlock();
+  }
+
+  private void startMaintenance(FSNamesystem namesystem, DatanodeManager dnm,
+      int dnIndex) throws Exception {
+    namesystem.writeLock();
+    DatanodeDescriptor dnDesc = getDatanodeDescriptor(dnm, dnIndex);
+    dnm.getDatanodeAdminManager().startMaintenance(dnDesc, Long.MAX_VALUE);
+    namesystem.writeUnlock();
+  }
+
+  private void stopMaintenance(FSNamesystem namesystem, DatanodeManager dnm,
+      int dnIndex) throws Exception {
+    namesystem.writeLock();
+    DatanodeDescriptor dnDesc = getDatanodeDescriptor(dnm, dnIndex);
+    dnm.getDatanodeAdminManager().stopMaintenance(dnDesc);
+    namesystem.writeUnlock();
+  }
+
+  @Test
+  public void testDatanodeLifeCycle() throws Exception {
+    createImage(new FSTreeWalk(providedPath, conf), nnDirPath,
+        FixedBlockResolver.class);
+    startCluster(nnDirPath, 3,
+        new StorageType[] {StorageType.PROVIDED, StorageType.DISK},
+        null, false);
+
+    int fileIndex = numFiles - 1;
+
+    final BlockManager blockManager = cluster.getNamesystem().getBlockManager();
+    final DatanodeManager dnm = blockManager.getDatanodeManager();
+
+    // to start, all 3 DNs are live in ProvidedDatanodeDescriptor.
+    verifyFileLocation(fileIndex, 3);
+
+    // de-commision first DN; still get 3 replicas.
+    startDecommission(cluster.getNamesystem(), dnm, 0);
+    verifyFileLocation(fileIndex, 3);
+
+    // remains the same even after heartbeats.
+    cluster.triggerHeartbeats();
+    verifyFileLocation(fileIndex, 3);
+
+    // start maintenance for 2nd DN; still get 3 replicas.
+    startMaintenance(cluster.getNamesystem(), dnm, 1);
+    verifyFileLocation(fileIndex, 3);
+
+    DataNode dn1 = cluster.getDataNodes().get(0);
+    DataNode dn2 = cluster.getDataNodes().get(1);
+
+    // stop the 1st DN while being decomissioned.
+    MiniDFSCluster.DataNodeProperties dn1Properties = cluster.stopDataNode(0);
+    BlockManagerTestUtil.noticeDeadDatanode(cluster.getNameNode(),
+        dn1.getDatanodeId().getXferAddr());
+
+    // get 2 locations
+    verifyFileLocation(fileIndex, 2);
+
+    // stop dn2 while in maintenance.
+    MiniDFSCluster.DataNodeProperties dn2Properties = cluster.stopDataNode(1);
+    BlockManagerTestUtil.noticeDeadDatanode(cluster.getNameNode(),
+        dn2.getDatanodeId().getXferAddr());
+
+    // 2 valid locations will be found as blocks on nodes that die during
+    // maintenance are not marked for removal.
+    verifyFileLocation(fileIndex, 2);
+
+    // stop the maintenance; get only 1 replicas
+    stopMaintenance(cluster.getNamesystem(), dnm, 0);
+    verifyFileLocation(fileIndex, 1);
+
+    // restart the stopped DN.
+    cluster.restartDataNode(dn1Properties, true);
+    cluster.waitActive();
+
+    // reports all 3 replicas
+    verifyFileLocation(fileIndex, 2);
+
+    cluster.restartDataNode(dn2Properties, true);
+    cluster.waitActive();
+
+    // reports all 3 replicas
+    verifyFileLocation(fileIndex, 3);
+  }
+
+  @Test
+  public void testProvidedWithHierarchicalTopology() throws Exception {
+    conf.setClass(ImageWriter.Options.UGI_CLASS, FsUGIResolver.class,
+        UGIResolver.class);
+    String packageName = "org.apache.hadoop.hdfs.server.blockmanagement";
+    String[] policies = new String[] {
+        "BlockPlacementPolicyDefault",
+        "BlockPlacementPolicyRackFaultTolerant",
+        "BlockPlacementPolicyWithNodeGroup",
+        "BlockPlacementPolicyWithUpgradeDomain"};
+    createImage(new FSTreeWalk(providedPath, conf), nnDirPath,
+        FixedBlockResolver.class);
+    String[] racks =
+        {"/pod0/rack0", "/pod0/rack0", "/pod0/rack1", "/pod0/rack1",
+            "/pod1/rack0", "/pod1/rack0", "/pod1/rack1", "/pod1/rack1" };
+    for (String policy: policies) {
+      LOG.info("Using policy: " + packageName + "." + policy);
+      conf.set(DFS_BLOCK_REPLICATOR_CLASSNAME_KEY, packageName + "." + policy);
+      startCluster(nnDirPath, racks.length,
+          new StorageType[]{StorageType.PROVIDED, StorageType.DISK},
+          null, false, racks);
+      verifyFileSystemContents();
+      setAndUnsetReplication("/" + filePrefix + (numFiles - 1) + fileSuffix);
+      cluster.shutdown();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8239e3af/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/RandomTreeWalk.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/RandomTreeWalk.java b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/RandomTreeWalk.java
index c9d109a..6e5b166 100644
--- a/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/RandomTreeWalk.java
+++ b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/RandomTreeWalk.java
@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
-import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
@@ -43,7 +42,6 @@ public class RandomTreeWalk extends TreeWalk {
   private final float depth;
   private final int children;
   private final Map<Long, Long> mSeed;
-  //private final AtomicLong blockIds = new AtomicLong(1L << 30);
 
   RandomTreeWalk(long seed) {
     this(seed, 10);
@@ -54,7 +52,7 @@ public class RandomTreeWalk extends TreeWalk {
   }
 
   RandomTreeWalk(long seed, int children, float depth) {
-    this(randomRoot(seed), seed, children, 0.15f);
+    this(randomRoot(seed), seed, children, depth);
   }
 
   RandomTreeWalk(Path root, long seed, int children, float depth) {


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


[14/50] [abbrv] hadoop git commit: HADOOP-15106. FileSystem::open(PathHandle) should throw a specific exception on validation failure

Posted by na...@apache.org.
HADOOP-15106. FileSystem::open(PathHandle) should throw a specific exception on validation failure


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

Branch: refs/heads/yarn-3409
Commit: 5e81f32d1155ea96c892099008cfeb50799082eb
Parents: fc7ec80
Author: Chris Douglas <cd...@apache.org>
Authored: Sat Dec 16 10:53:10 2017 -0800
Committer: Chris Douglas <cd...@apache.org>
Committed: Sat Dec 16 10:53:10 2017 -0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/fs/FileSystem.java   |  6 +++
 .../hadoop/fs/InvalidPathHandleException.java   | 46 ++++++++++++++++++++
 .../src/site/markdown/filesystem/filesystem.md  |  2 +-
 .../fs/contract/AbstractContractOpenTest.java   |  7 +--
 .../hadoop/hdfs/DistributedFileSystem.java      |  3 ++
 .../hadoop/hdfs/protocol/HdfsPathHandle.java    | 16 +++----
 6 files changed, 67 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e81f32d/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
index a364921..6b7dead 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
@@ -957,6 +957,8 @@ public abstract class FileSystem extends Configured implements Closeable {
    * resource directly and verify that the resource referenced
    * satisfies constraints specified at its construciton.
    * @param fd PathHandle object returned by the FS authority.
+   * @throws InvalidPathHandleException If {@link PathHandle} constraints are
+   *                                    not satisfied
    * @throws IOException IO failure
    * @throws UnsupportedOperationException If {@link #open(PathHandle, int)}
    *                                       not overridden by subclass
@@ -973,6 +975,8 @@ public abstract class FileSystem extends Configured implements Closeable {
    * satisfies constraints specified at its construciton.
    * @param fd PathHandle object returned by the FS authority.
    * @param bufferSize the size of the buffer to use
+   * @throws InvalidPathHandleException If {@link PathHandle} constraints are
+   *                                    not satisfied
    * @throws IOException IO failure
    * @throws UnsupportedOperationException If not overridden by subclass
    */
@@ -994,6 +998,8 @@ public abstract class FileSystem extends Configured implements Closeable {
    *         the specified constraints.
    */
   public final PathHandle getPathHandle(FileStatus stat, HandleOpt... opt) {
+    // method is final with a default so clients calling getPathHandle(stat)
+    // get the same semantics for all FileSystem implementations
     if (null == opt || 0 == opt.length) {
       return createPathHandle(stat, HandleOpt.path());
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e81f32d/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/InvalidPathHandleException.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/InvalidPathHandleException.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/InvalidPathHandleException.java
new file mode 100644
index 0000000..8e26ea7
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/InvalidPathHandleException.java
@@ -0,0 +1,46 @@
+/**
+ * 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;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Thrown when the constraints enoded in a {@link PathHandle} do not hold.
+ * For example, if a handle were created with the default
+ * {@link Options.HandleOpt#path()} constraints, a call to
+ * {@link FileSystem#open(PathHandle)} would succeed if the file were
+ * modified, but if a different file was at that location then it would throw
+ * this exception.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class InvalidPathHandleException extends IOException {
+  private static final long serialVersionUID = 0xcd8ac329L;
+
+  public InvalidPathHandleException(String str) {
+    super(str);
+  }
+
+  public InvalidPathHandleException(String message, Throwable cause) {
+    super(message, cause);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e81f32d/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md
index c0e2a2c..2637f54 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md
@@ -790,7 +790,7 @@ Implementaions without a compliant call MUST throw `UnsupportedOperationExceptio
       (FS.Directories', FS.Files', FS.Symlinks')
       p' in FS.Files' where:
         FS.Files'[p'] = fd
-    if not exists(FS', p') : raise FileNotFoundException
+    if not exists(FS', p') : raise InvalidPathHandleException
 
 The implementation MUST resolve the referent of the `PathHandle` following
 the constraints specified at its creation by `getPathHandle(FileStatus)`.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e81f32d/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractOpenTest.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractOpenTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractOpenTest.java
index 7528312..ab179eb 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractOpenTest.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractOpenTest.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.InvalidPathHandleException;
 import org.apache.hadoop.fs.Options.HandleOpt;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathHandle;
@@ -241,7 +242,7 @@ public abstract class AbstractContractOpenTest
       // fail to resolve if path1 had been modified
       instream = getFileSystem().open(fd1);
       fail("Expected an exception");
-    } catch (IOException e) {
+    } catch (InvalidPathHandleException e) {
       // expected
     }
 
@@ -290,7 +291,7 @@ public abstract class AbstractContractOpenTest
       // handle should not resolve when content changed
       instream = getFileSystem().open(fd);
       fail("Failed to detect change to content");
-    } catch (IOException e) {
+    } catch (InvalidPathHandleException e) {
       // expected
     }
   }
@@ -330,7 +331,7 @@ public abstract class AbstractContractOpenTest
       // verify attempt to resolve the handle fails
       instream = getFileSystem().open(fd1);
       fail("Expected an exception");
-    } catch (IOException e) {
+    } catch (InvalidPathHandleException e) {
       // expected
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e81f32d/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 2449fb6..6b0c57a 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
@@ -46,6 +46,7 @@ import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.FsStatus;
 import org.apache.hadoop.fs.GlobalStorageStatistics;
 import org.apache.hadoop.fs.GlobalStorageStatistics.StorageStatisticsProvider;
+import org.apache.hadoop.fs.InvalidPathHandleException;
 import org.apache.hadoop.fs.PathHandle;
 import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.Options;
@@ -336,6 +337,8 @@ public class DistributedFileSystem extends FileSystem
    * the {@link PathHandle}.
    * @param fd Reference to entity in this FileSystem.
    * @param bufferSize the size of the buffer to be used.
+   * @throws InvalidPathHandleException If PathHandle constraints do not hold
+   * @throws IOException On I/O errors
    */
   @Override
   public FSDataInputStream open(PathHandle fd, int bufferSize)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e81f32d/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsPathHandle.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsPathHandle.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsPathHandle.java
index f80a067..14661ae 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsPathHandle.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsPathHandle.java
@@ -23,6 +23,7 @@ import java.util.Optional;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.InvalidPathHandleException;
 import org.apache.hadoop.fs.PathHandle;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.HdfsPathHandleProto;
 
@@ -55,24 +56,21 @@ public final class HdfsPathHandle implements PathHandle {
     HdfsPathHandleProto p =
         HdfsPathHandleProto.parseFrom(ByteString.copyFrom(bytes));
     path = p.getPath();
-    mtime = p.hasMtime()
-        ? p.getMtime()
-        : null;
-    inodeId = p.hasInodeId()
-        ? p.getInodeId()
-        : null;
+    mtime   = p.hasMtime()   ? p.getMtime()   : null;
+    inodeId = p.hasInodeId() ? p.getInodeId() : null;
   }
 
   public String getPath() {
     return path;
   }
 
-  public void verify(HdfsLocatedFileStatus stat) throws IOException {
+  public void verify(HdfsLocatedFileStatus stat)
+      throws InvalidPathHandleException {
     if (mtime != null && mtime != stat.getModificationTime()) {
-      throw new IOException("Content changed");
+      throw new InvalidPathHandleException("Content changed");
     }
     if (inodeId != null && inodeId != stat.getFileId()) {
-      throw new IOException("Wrong file");
+      throw new InvalidPathHandleException("Wrong file");
     }
   }
 


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


[40/50] [abbrv] hadoop git commit: HADOOP-14965. S3a input stream "normal" fadvise mode to be adaptive

Posted by na...@apache.org.
HADOOP-14965. S3a input stream "normal" fadvise mode to be adaptive


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

Branch: refs/heads/yarn-3409
Commit: 1ba491ff907fc5d2618add980734a3534e2be098
Parents: 13ad747
Author: Steve Loughran <st...@apache.org>
Authored: Wed Dec 20 18:25:33 2017 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Wed Dec 20 18:25:33 2017 +0000

----------------------------------------------------------------------
 .../apache/hadoop/fs/s3a/S3AInputStream.java    | 28 +++++++++++++++++---
 .../hadoop/fs/s3a/S3AInstrumentation.java       | 13 +++++++++
 .../src/site/markdown/tools/hadoop-aws/index.md | 13 ++++++++-
 .../scale/ITestS3AInputStreamPerformance.java   |  6 ++++-
 4 files changed, 54 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1ba491ff/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java
index 7e6d640..0074143 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java
@@ -83,7 +83,7 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead {
   private final S3AInstrumentation.InputStreamStatistics streamStatistics;
   private S3AEncryptionMethods serverSideEncryptionAlgorithm;
   private String serverSideEncryptionKey;
-  private final S3AInputPolicy inputPolicy;
+  private S3AInputPolicy inputPolicy;
   private long readahead = Constants.DEFAULT_READAHEAD_RANGE;
   private final Invoker invoker;
 
@@ -139,12 +139,22 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead {
     this.serverSideEncryptionAlgorithm =
         s3Attributes.getServerSideEncryptionAlgorithm();
     this.serverSideEncryptionKey = s3Attributes.getServerSideEncryptionKey();
-    this.inputPolicy = inputPolicy;
+    setInputPolicy(inputPolicy);
     setReadahead(readahead);
     this.invoker = invoker;
   }
 
   /**
+   * Set/update the input policy of the stream.
+   * This updates the stream statistics.
+   * @param inputPolicy new input policy.
+   */
+  private void setInputPolicy(S3AInputPolicy inputPolicy) {
+    this.inputPolicy = inputPolicy;
+    streamStatistics.inputPolicySet(inputPolicy.ordinal());
+  }
+
+  /**
    * Opens up the stream at specified target position and for given length.
    *
    * @param reason reason for reopen
@@ -162,8 +172,9 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead {
     contentRangeFinish = calculateRequestLimit(inputPolicy, targetPos,
         length, contentLength, readahead);
     LOG.debug("reopen({}) for {} range[{}-{}], length={}," +
-        " streamPosition={}, nextReadPosition={}",
-        uri, reason, targetPos, contentRangeFinish, length,  pos, nextReadPos);
+        " streamPosition={}, nextReadPosition={}, policy={}",
+        uri, reason, targetPos, contentRangeFinish, length,  pos, nextReadPos,
+        inputPolicy);
 
     long opencount = streamStatistics.streamOpened();
     GetObjectRequest request = new GetObjectRequest(bucket, key)
@@ -274,6 +285,12 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead {
     } else if (diff < 0) {
       // backwards seek
       streamStatistics.seekBackwards(diff);
+      // if the stream is in "Normal" mode, switch to random IO at this
+      // point, as it is indicative of columnar format IO
+      if (inputPolicy.equals(S3AInputPolicy.Normal)) {
+        LOG.info("Switching to Random IO seek policy");
+        setInputPolicy(S3AInputPolicy.Random);
+      }
     } else {
       // targetPos == pos
       if (remainingInCurrentRequest() > 0) {
@@ -443,6 +460,7 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead {
       try {
         // close or abort the stream
         closeStream("close() operation", this.contentRangeFinish, false);
+        LOG.debug("Statistics of stream {}\n{}", key, streamStatistics);
         // this is actually a no-op
         super.close();
       } finally {
@@ -713,6 +731,8 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead {
       break;
 
     case Normal:
+      // normal is considered sequential until a backwards seek switches
+      // it to 'Random'
     default:
       rangeLimit = contentLength;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1ba491ff/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
index 0fbcc00..d843347 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java
@@ -667,6 +667,8 @@ public class S3AInstrumentation implements Closeable, MetricsSource {
     public long readsIncomplete;
     public long bytesReadInClose;
     public long bytesDiscardedInAbort;
+    public long policySetCount;
+    public long inputPolicy;
 
     private InputStreamStatistics() {
     }
@@ -783,6 +785,15 @@ public class S3AInstrumentation implements Closeable, MetricsSource {
     }
 
     /**
+     * The input policy has been switched.
+     * @param updatedPolicy enum value of new policy.
+     */
+    public void inputPolicySet(int updatedPolicy) {
+      policySetCount++;
+      inputPolicy = updatedPolicy;
+    }
+
+    /**
      * String operator describes all the current statistics.
      * <b>Important: there are no guarantees as to the stability
      * of this value.</b>
@@ -813,6 +824,8 @@ public class S3AInstrumentation implements Closeable, MetricsSource {
       sb.append(", ReadsIncomplete=").append(readsIncomplete);
       sb.append(", BytesReadInClose=").append(bytesReadInClose);
       sb.append(", BytesDiscardedInAbort=").append(bytesDiscardedInAbort);
+      sb.append(", InputPolicy=").append(inputPolicy);
+      sb.append(", InputPolicySetCount=").append(policySetCount);
       sb.append('}');
       return sb.toString();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1ba491ff/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
index fbcd54a..7eebf5c 100644
--- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
+++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
@@ -1553,7 +1553,18 @@ backward seeks.
 
 *"normal" (default)*
 
-This is currently the same as "sequential", though it may evolve in future.
+The "Normal" policy starts off reading a file  in "sequential" mode,
+but if the caller seeks backwards in the stream, it switches from
+sequential to "random".
+
+This policy effectively recognizes the initial read pattern of columnar
+storage formats (e.g. Apache ORC and Apache Parquet), which seek to the end
+of a file, read in index data and then seek backwards to selectively read
+columns. The first seeks may be be expensive compared to the random policy,
+however the overall process is much less expensive than either sequentially
+reading through a file with the "random" policy, or reading columnar data
+with the "sequential" policy. When the exact format/recommended
+seek policy of data are known in advance, this policy
 
 *"random"*
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1ba491ff/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java
index 83ab210..efd96c4 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java
@@ -427,7 +427,11 @@ public class ITestS3AInputStreamPerformance extends S3AScaleTestBase {
     long expectedOpenCount = RANDOM_IO_SEQUENCE.length;
     executeRandomIO(S3AInputPolicy.Normal, expectedOpenCount);
     assertEquals("streams aborted in " + streamStatistics,
-        4, streamStatistics.aborted);
+        1, streamStatistics.aborted);
+    assertEquals("policy changes in " + streamStatistics,
+        2, streamStatistics.policySetCount);
+    assertEquals("input policy in " + streamStatistics,
+        S3AInputPolicy.Random.ordinal(), streamStatistics.inputPolicy);
   }
 
   /**


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


[27/50] [abbrv] hadoop git commit: Add 2.8.3 release jdiff files.

Posted by na...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/a7f8caf5/hadoop-mapreduce-project/dev-support/jdiff/Apache_Hadoop_MapReduce_Core_2.8.3.xml
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/dev-support/jdiff/Apache_Hadoop_MapReduce_Core_2.8.3.xml b/hadoop-mapreduce-project/dev-support/jdiff/Apache_Hadoop_MapReduce_Core_2.8.3.xml
new file mode 100644
index 0000000..e96d018
--- /dev/null
+++ b/hadoop-mapreduce-project/dev-support/jdiff/Apache_Hadoop_MapReduce_Core_2.8.3.xml
@@ -0,0 +1,27495 @@
+<?xml version="1.0" encoding="iso-8859-1" standalone="no"?>
+<!-- Generated by the JDiff Javadoc doclet -->
+<!-- (http://www.jdiff.org) -->
+<!-- on Tue Dec 05 05:47:49 UTC 2017 -->
+
+<api
+  xmlns:xsi='http://www.w3.org/2001/XMLSchema-instance'
+  xsi:noNamespaceSchemaLocation='api.xsd'
+  name="Apache Hadoop MapReduce Core 2.8.3"
+  jdversion="1.0.9">
+
+<!--  Command line arguments =  -doclet org.apache.hadoop.classification.tools.IncludePublicAnnotationsJDiffDoclet -docletpath /build/source/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/target/hadoop-annotations.jar:/build/source/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/target/jdiff.jar -verbose -classpath /build/source/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/target/classes:/build/source/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/target/hadoop-yarn-common-2.8.3.jar:/build/source/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/target/hadoop-yarn-api-2.8.3.jar:/maven/javax/xml/bind/jaxb-api/2.2.2/jaxb-api-2.2.2.jar:/maven/javax/xml/stream/stax-api/1.0-2/stax-api-1.0-2.jar:/maven/javax/activation/activation/1.1/activation-1.1.jar:/maven/org/apache/commons/commons-compress/1.4.1/commons-compress-1.4.1.jar:/maven/org/tukaani/xz/1.0/xz-1.0.jar:/maven/javax/servlet/servlet-ap
 i/2.5/servlet-api-2.5.jar:/maven/org/mortbay/jetty/jetty-util/6.1.26/jetty-util-6.1.26.jar:/maven/com/sun/jersey/jersey-core/1.9/jersey-core-1.9.jar:/maven/com/sun/jersey/jersey-client/1.9/jersey-client-1.9.jar:/maven/org/codehaus/jackson/jackson-core-asl/1.9.13/jackson-core-asl-1.9.13.jar:/maven/org/codehaus/jackson/jackson-mapper-asl/1.9.13/jackson-mapper-asl-1.9.13.jar:/maven/org/codehaus/jackson/jackson-jaxrs/1.9.13/jackson-jaxrs-1.9.13.jar:/maven/org/codehaus/jackson/jackson-xc/1.9.13/jackson-xc-1.9.13.jar:/maven/commons-io/commons-io/2.4/commons-io-2.4.jar:/maven/com/google/inject/guice/3.0/guice-3.0.jar:/maven/javax/inject/javax.inject/1/javax.inject-1.jar:/maven/aopalliance/aopalliance/1.0/aopalliance-1.0.jar:/maven/com/sun/jersey/jersey-server/1.9/jersey-server-1.9.jar:/maven/asm/asm/3.2/asm-3.2.jar:/maven/com/sun/jersey/jersey-json/1.9/jersey-json-1.9.jar:/maven/org/codehaus/jettison/jettison/1.1/jettison-1.1.jar:/maven/com/sun/xml/bind/jaxb-impl/2.2.3-1/jaxb-impl-2.2.3-1.
 jar:/maven/com/sun/jersey/contribs/jersey-guice/1.9/jersey-guice-1.9.jar:/maven/log4j/log4j/1.2.17/log4j-1.2.17.jar:/maven/org/mortbay/jetty/jetty/6.1.26/jetty-6.1.26.jar:/maven/xmlenc/xmlenc/0.52/xmlenc-0.52.jar:/maven/org/apache/htrace/htrace-core4/4.0.1-incubating/htrace-core4-4.0.1-incubating.jar:/maven/com/google/protobuf/protobuf-java/2.5.0/protobuf-java-2.5.0.jar:/maven/org/apache/avro/avro/1.7.4/avro-1.7.4.jar:/maven/com/thoughtworks/paranamer/paranamer/2.3/paranamer-2.3.jar:/maven/org/xerial/snappy/snappy-java/1.0.4.1/snappy-java-1.0.4.1.jar:/build/source/hadoop-common-project/hadoop-common/target/hadoop-common-2.8.3.jar:/maven/org/apache/commons/commons-math3/3.1.1/commons-math3-3.1.1.jar:/maven/org/apache/httpcomponents/httpclient/4.5.2/httpclient-4.5.2.jar:/maven/org/apache/httpcomponents/httpcore/4.4.4/httpcore-4.4.4.jar:/maven/commons-net/commons-net/3.1/commons-net-3.1.jar:/maven/org/mortbay/jetty/jetty-sslengine/6.1.26/jetty-sslengine-6.1.26.jar:/maven/javax/servlet/
 jsp/jsp-api/2.1/jsp-api-2.1.jar:/maven/net/java/dev/jets3t/jets3t/0.9.0/jets3t-0.9.0.jar:/maven/com/jamesmurty/utils/java-xmlbuilder/0.4/java-xmlbuilder-0.4.jar:/maven/commons-configuration/commons-configuration/1.6/commons-configuration-1.6.jar:/maven/commons-digester/commons-digester/1.8/commons-digester-1.8.jar:/maven/commons-beanutils/commons-beanutils/1.7.0/commons-beanutils-1.7.0.jar:/maven/commons-beanutils/commons-beanutils-core/1.8.0/commons-beanutils-core-1.8.0.jar:/maven/com/google/code/gson/gson/2.2.4/gson-2.2.4.jar:/build/source/hadoop-common-project/hadoop-auth/target/hadoop-auth-2.8.3.jar:/maven/com/nimbusds/nimbus-jose-jwt/3.9/nimbus-jose-jwt-3.9.jar:/maven/net/jcip/jcip-annotations/1.0/jcip-annotations-1.0.jar:/maven/net/minidev/json-smart/1.1.1/json-smart-1.1.1.jar:/maven/org/apache/directory/server/apacheds-kerberos-codec/2.0.0-M15/apacheds-kerberos-codec-2.0.0-M15.jar:/maven/org/apache/directory/server/apacheds-i18n/2.0.0-M15/apacheds-i18n-2.0.0-M15.jar:/maven/or
 g/apache/directory/api/api-asn1-api/1.0.0-M20/api-asn1-api-1.0.0-M20.jar:/maven/org/apache/directory/api/api-util/1.0.0-M20/api-util-1.0.0-M20.jar:/maven/org/apache/curator/curator-framework/2.7.1/curator-framework-2.7.1.jar:/maven/com/jcraft/jsch/0.1.54/jsch-0.1.54.jar:/maven/org/apache/curator/curator-client/2.7.1/curator-client-2.7.1.jar:/maven/org/apache/curator/curator-recipes/2.7.1/curator-recipes-2.7.1.jar:/maven/com/google/code/findbugs/jsr305/3.0.0/jsr305-3.0.0.jar:/maven/org/apache/zookeeper/zookeeper/3.4.6/zookeeper-3.4.6.jar:/maven/org/slf4j/slf4j-api/1.7.10/slf4j-api-1.7.10.jar:/maven/org/slf4j/slf4j-log4j12/1.7.10/slf4j-log4j12-1.7.10.jar:/build/source/hadoop-common-project/hadoop-annotations/target/hadoop-annotations-2.8.3.jar:/usr/lib/jvm/java-7-openjdk-amd64/lib/tools.jar:/maven/com/google/inject/extensions/guice-servlet/3.0/guice-servlet-3.0.jar:/maven/io/netty/netty/3.6.2.Final/netty-3.6.2.Final.jar:/maven/commons-logging/commons-logging/1.1.3/commons-logging-1.1.
 3.jar:/maven/com/google/guava/guava/11.0.2/guava-11.0.2.jar:/maven/commons-codec/commons-codec/1.4/commons-codec-1.4.jar:/maven/commons-cli/commons-cli/1.2/commons-cli-1.2.jar:/maven/commons-lang/commons-lang/2.6/commons-lang-2.6.jar:/maven/commons-collections/commons-collections/3.2.2/commons-collections-3.2.2.jar -sourcepath /build/source/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java -apidir /build/source/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/target/site/jdiff/xml -apiname Apache Hadoop MapReduce Core 2.8.3 -->
+<package name="org.apache.hadoop.filecache">
+  <!-- start class org.apache.hadoop.filecache.DistributedCache -->
+  <class name="DistributedCache" extends="org.apache.hadoop.mapreduce.filecache.DistributedCache"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="DistributedCache"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="addLocalArchives"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <param name="str" type="java.lang.String"/>
+      <doc>
+      <![CDATA[Add a archive that has been localized to the conf.  Used
+ by internal DistributedCache code.
+ @param conf The conf to modify to contain the localized caches
+ @param str a comma separated list of local archives]]>
+      </doc>
+    </method>
+    <method name="addLocalFiles"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <param name="str" type="java.lang.String"/>
+      <doc>
+      <![CDATA[Add a file that has been localized to the conf..  Used
+ by internal DistributedCache code.
+ @param conf The conf to modify to contain the localized caches
+ @param str a comma separated list of local files]]>
+      </doc>
+    </method>
+    <method name="createAllSymlink"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="Internal to MapReduce framework.  Use DistributedCacheManager
+ instead.">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <param name="jobCacheDir" type="java.io.File"/>
+      <param name="workDir" type="java.io.File"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[This method create symlinks for all files in a given dir in another
+ directory. Currently symlinks cannot be disabled. This is a NO-OP.
+
+ @param conf the configuration
+ @param jobCacheDir the target directory for creating symlinks
+ @param workDir the directory in which the symlinks are created
+ @throws IOException
+ @deprecated Internal to MapReduce framework.  Use DistributedCacheManager
+ instead.]]>
+      </doc>
+    </method>
+    <method name="getFileStatus" return="org.apache.hadoop.fs.FileStatus"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <param name="cache" type="java.net.URI"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Returns {@link FileStatus} of a given cache file on hdfs. Internal to
+ MapReduce.
+ @param conf configuration
+ @param cache cache file
+ @return <code>FileStatus</code> of a given cache file on hdfs
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getTimestamp" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <param name="cache" type="java.net.URI"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Returns mtime of a given cache file on hdfs. Internal to MapReduce.
+ @param conf configuration
+ @param cache cache file
+ @return mtime of a given cache file on hdfs
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="setArchiveTimestamps"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <param name="timestamps" type="java.lang.String"/>
+      <doc>
+      <![CDATA[This is to check the timestamp of the archives to be localized.
+ Used by internal MapReduce code.
+ @param conf Configuration which stores the timestamp's
+ @param timestamps comma separated list of timestamps of archives.
+ The order should be the same as the order in which the archives are added.]]>
+      </doc>
+    </method>
+    <method name="setFileTimestamps"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <param name="timestamps" type="java.lang.String"/>
+      <doc>
+      <![CDATA[This is to check the timestamp of the files to be localized.
+ Used by internal MapReduce code.
+ @param conf Configuration which stores the timestamp's
+ @param timestamps comma separated list of timestamps of files.
+ The order should be the same as the order in which the files are added.]]>
+      </doc>
+    </method>
+    <method name="setLocalArchives"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <param name="str" type="java.lang.String"/>
+      <doc>
+      <![CDATA[Set the conf to contain the location for localized archives.  Used
+ by internal DistributedCache code.
+ @param conf The conf to modify to contain the localized caches
+ @param str a comma separated list of local archives]]>
+      </doc>
+    </method>
+    <method name="setLocalFiles"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <param name="str" type="java.lang.String"/>
+      <doc>
+      <![CDATA[Set the conf to contain the location for localized files.  Used
+ by internal DistributedCache code.
+ @param conf The conf to modify to contain the localized caches
+ @param str a comma separated list of local files]]>
+      </doc>
+    </method>
+    <field name="CACHE_FILES_SIZES" type="java.lang.String"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Warning: {@link #CACHE_FILES_SIZES} is not a *public* constant.
+ The variable is kept for M/R 1.x applications, M/R 2.x applications should
+ use {@link MRJobConfig#CACHE_FILES_SIZES}]]>
+      </doc>
+    </field>
+    <field name="CACHE_ARCHIVES_SIZES" type="java.lang.String"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Warning: {@link #CACHE_ARCHIVES_SIZES} is not a *public* constant.
+ The variable is kept for M/R 1.x applications, M/R 2.x applications should
+ use {@link MRJobConfig#CACHE_ARCHIVES_SIZES}]]>
+      </doc>
+    </field>
+    <field name="CACHE_ARCHIVES_TIMESTAMPS" type="java.lang.String"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Warning: {@link #CACHE_ARCHIVES_TIMESTAMPS} is not a *public* constant.
+ The variable is kept for M/R 1.x applications, M/R 2.x applications should
+ use {@link MRJobConfig#CACHE_ARCHIVES_TIMESTAMPS}]]>
+      </doc>
+    </field>
+    <field name="CACHE_FILES_TIMESTAMPS" type="java.lang.String"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Warning: {@link #CACHE_FILES_TIMESTAMPS} is not a *public* constant.
+ The variable is kept for M/R 1.x applications, M/R 2.x applications should
+ use {@link MRJobConfig#CACHE_FILE_TIMESTAMPS}]]>
+      </doc>
+    </field>
+    <field name="CACHE_ARCHIVES" type="java.lang.String"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Warning: {@link #CACHE_ARCHIVES} is not a *public* constant.
+ The variable is kept for M/R 1.x applications, M/R 2.x applications should
+ use {@link MRJobConfig#CACHE_ARCHIVES}]]>
+      </doc>
+    </field>
+    <field name="CACHE_FILES" type="java.lang.String"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Warning: {@link #CACHE_FILES} is not a *public* constant.
+ The variable is kept for M/R 1.x applications, M/R 2.x applications should
+ use {@link MRJobConfig#CACHE_FILES}]]>
+      </doc>
+    </field>
+    <field name="CACHE_LOCALARCHIVES" type="java.lang.String"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Warning: {@link #CACHE_LOCALARCHIVES} is not a *public* constant.
+ The variable is kept for M/R 1.x applications, M/R 2.x applications should
+ use {@link MRJobConfig#CACHE_LOCALARCHIVES}]]>
+      </doc>
+    </field>
+    <field name="CACHE_LOCALFILES" type="java.lang.String"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Warning: {@link #CACHE_LOCALFILES} is not a *public* constant.
+ The variable is kept for M/R 1.x applications, M/R 2.x applications should
+ use {@link MRJobConfig#CACHE_LOCALFILES}]]>
+      </doc>
+    </field>
+    <field name="CACHE_SYMLINK" type="java.lang.String"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Warning: {@link #CACHE_SYMLINK} is not a *public* constant.
+ The variable is kept for M/R 1.x applications, M/R 2.x applications should
+ use {@link MRJobConfig#CACHE_SYMLINK}]]>
+      </doc>
+    </field>
+    <doc>
+    <![CDATA[Distribute application-specific large, read-only files efficiently.
+ 
+ <p><code>DistributedCache</code> is a facility provided by the Map-Reduce
+ framework to cache files (text, archives, jars etc.) needed by applications.
+ </p>
+ 
+ <p>Applications specify the files, via urls (hdfs:// or http://) to be cached 
+ via the {@link org.apache.hadoop.mapred.JobConf}. The
+ <code>DistributedCache</code> assumes that the files specified via urls are
+ already present on the {@link FileSystem} at the path specified by the url
+ and are accessible by every machine in the cluster.</p>
+ 
+ <p>The framework will copy the necessary files on to the slave node before 
+ any tasks for the job are executed on that node. Its efficiency stems from 
+ the fact that the files are only copied once per job and the ability to 
+ cache archives which are un-archived on the slaves.</p> 
+
+ <p><code>DistributedCache</code> can be used to distribute simple, read-only
+ data/text files and/or more complex types such as archives, jars etc. 
+ Archives (zip, tar and tgz/tar.gz files) are un-archived at the slave nodes. 
+ Jars may be optionally added to the classpath of the tasks, a rudimentary 
+ software distribution mechanism.  Files have execution permissions.
+ In older version of Hadoop Map/Reduce users could optionally ask for symlinks
+ to be created in the working directory of the child task.  In the current 
+ version symlinks are always created.  If the URL does not have a fragment 
+ the name of the file or directory will be used. If multiple files or 
+ directories map to the same link name, the last one added, will be used.  All
+ others will not even be downloaded.</p>
+ 
+ <p><code>DistributedCache</code> tracks modification timestamps of the cache 
+ files. Clearly the cache files should not be modified by the application 
+ or externally while the job is executing.</p>
+ 
+ <p>Here is an illustrative example on how to use the 
+ <code>DistributedCache</code>:</p>
+ <p><blockquote><pre>
+     // Setting up the cache for the application
+     
+     1. Copy the requisite files to the <code>FileSystem</code>:
+     
+     $ bin/hadoop fs -copyFromLocal lookup.dat /myapp/lookup.dat  
+     $ bin/hadoop fs -copyFromLocal map.zip /myapp/map.zip  
+     $ bin/hadoop fs -copyFromLocal mylib.jar /myapp/mylib.jar
+     $ bin/hadoop fs -copyFromLocal mytar.tar /myapp/mytar.tar
+     $ bin/hadoop fs -copyFromLocal mytgz.tgz /myapp/mytgz.tgz
+     $ bin/hadoop fs -copyFromLocal mytargz.tar.gz /myapp/mytargz.tar.gz
+     
+     2. Setup the application's <code>JobConf</code>:
+     
+     JobConf job = new JobConf();
+     DistributedCache.addCacheFile(new URI("/myapp/lookup.dat#lookup.dat"), 
+                                   job);
+     DistributedCache.addCacheArchive(new URI("/myapp/map.zip", job);
+     DistributedCache.addFileToClassPath(new Path("/myapp/mylib.jar"), job);
+     DistributedCache.addCacheArchive(new URI("/myapp/mytar.tar", job);
+     DistributedCache.addCacheArchive(new URI("/myapp/mytgz.tgz", job);
+     DistributedCache.addCacheArchive(new URI("/myapp/mytargz.tar.gz", job);
+     
+     3. Use the cached files in the {@link org.apache.hadoop.mapred.Mapper}
+     or {@link org.apache.hadoop.mapred.Reducer}:
+     
+     public static class MapClass extends MapReduceBase  
+     implements Mapper&lt;K, V, K, V&gt; {
+     
+       private Path[] localArchives;
+       private Path[] localFiles;
+       
+       public void configure(JobConf job) {
+         // Get the cached archives/files
+         File f = new File("./map.zip/some/file/in/zip.txt");
+       }
+       
+       public void map(K key, V value, 
+                       OutputCollector&lt;K, V&gt; output, Reporter reporter) 
+       throws IOException {
+         // Use data from the cached archives/files here
+         // ...
+         // ...
+         output.collect(k, v);
+       }
+     }
+     
+ </pre></blockquote>
+
+ It is also very common to use the DistributedCache by using
+ {@link org.apache.hadoop.util.GenericOptionsParser}.
+
+ This class includes methods that should be used by users
+ (specifically those mentioned in the example above, as well
+ as {@link DistributedCache#addArchiveToClassPath(Path, Configuration)}),
+ as well as methods intended for use by the MapReduce framework
+ (e.g., {@link org.apache.hadoop.mapred.JobClient}).
+
+ @see org.apache.hadoop.mapred.JobConf
+ @see org.apache.hadoop.mapred.JobClient
+ @see org.apache.hadoop.mapreduce.Job]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.filecache.DistributedCache -->
+</package>
+<package name="org.apache.hadoop.mapred">
+  <!-- start class org.apache.hadoop.mapred.ClusterStatus -->
+  <class name="ClusterStatus" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="org.apache.hadoop.io.Writable"/>
+    <method name="getTaskTrackers" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the number of task trackers in the cluster.
+ 
+ @return the number of task trackers in the cluster.]]>
+      </doc>
+    </method>
+    <method name="getActiveTrackerNames" return="java.util.Collection"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the names of task trackers in the cluster.
+ 
+ @return the active task trackers in the cluster.]]>
+      </doc>
+    </method>
+    <method name="getBlacklistedTrackerNames" return="java.util.Collection"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the names of task trackers in the cluster.
+ 
+ @return the blacklisted task trackers in the cluster.]]>
+      </doc>
+    </method>
+    <method name="getGraylistedTrackerNames" return="java.util.Collection"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the names of graylisted task trackers in the cluster.
+
+ The gray list of trackers is no longer available on M/R 2.x. The function
+ is kept to be compatible with M/R 1.x applications.
+
+ @return an empty graylisted task trackers in the cluster.]]>
+      </doc>
+    </method>
+    <method name="getGraylistedTrackers" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the number of graylisted task trackers in the cluster.
+
+ The gray list of trackers is no longer available on M/R 2.x. The function
+ is kept to be compatible with M/R 1.x applications.
+
+ @return 0 graylisted task trackers in the cluster.]]>
+      </doc>
+    </method>
+    <method name="getBlacklistedTrackers" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the number of blacklisted task trackers in the cluster.
+ 
+ @return the number of blacklisted task trackers in the cluster.]]>
+      </doc>
+    </method>
+    <method name="getNumExcludedNodes" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the number of excluded hosts in the cluster.
+ @return the number of excluded hosts in the cluster.]]>
+      </doc>
+    </method>
+    <method name="getTTExpiryInterval" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the tasktracker expiry interval for the cluster
+ @return the expiry interval in msec]]>
+      </doc>
+    </method>
+    <method name="getMapTasks" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the number of currently running map tasks in the cluster.
+ 
+ @return the number of currently running map tasks in the cluster.]]>
+      </doc>
+    </method>
+    <method name="getReduceTasks" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the number of currently running reduce tasks in the cluster.
+ 
+ @return the number of currently running reduce tasks in the cluster.]]>
+      </doc>
+    </method>
+    <method name="getMaxMapTasks" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the maximum capacity for running map tasks in the cluster.
+ 
+ @return the maximum capacity for running map tasks in the cluster.]]>
+      </doc>
+    </method>
+    <method name="getMaxReduceTasks" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the maximum capacity for running reduce tasks in the cluster.
+ 
+ @return the maximum capacity for running reduce tasks in the cluster.]]>
+      </doc>
+    </method>
+    <method name="getJobTrackerStatus" return="org.apache.hadoop.mapreduce.Cluster.JobTrackerStatus"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the JobTracker's status.
+ 
+ @return {@link JobTrackerStatus} of the JobTracker]]>
+      </doc>
+    </method>
+    <method name="getMaxMemory" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Returns UNINITIALIZED_MEMORY_VALUE (-1)]]>
+      </doc>
+    </method>
+    <method name="getUsedMemory" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Returns UNINITIALIZED_MEMORY_VALUE (-1)]]>
+      </doc>
+    </method>
+    <method name="getBlackListedTrackersInfo" return="java.util.Collection"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Gets the list of blacklisted trackers along with reasons for blacklisting.
+ 
+ @return the collection of {@link BlackListInfo} objects.]]>
+      </doc>
+    </method>
+    <method name="getJobTrackerState" return="org.apache.hadoop.mapred.JobTracker.State"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the current state of the <code>JobTracker</code>,
+ as {@link JobTracker.State}
+
+ {@link JobTracker.State} should no longer be used on M/R 2.x. The function
+ is kept to be compatible with M/R 1.x applications.
+
+ @return the invalid state of the <code>JobTracker</code>.]]>
+      </doc>
+    </method>
+    <method name="write"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="out" type="java.io.DataOutput"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="readFields"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="in" type="java.io.DataInput"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <field name="UNINITIALIZED_MEMORY_VALUE" type="long"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+    </field>
+    <doc>
+    <![CDATA[Status information on the current state of the Map-Reduce cluster.
+ 
+ <p><code>ClusterStatus</code> provides clients with information such as:
+ <ol>
+   <li>
+   Size of the cluster. 
+   </li>
+   <li>
+   Name of the trackers. 
+   </li>
+   <li>
+   Task capacity of the cluster. 
+   </li>
+   <li>
+   The number of currently running map and reduce tasks.
+   </li>
+   <li>
+   State of the <code>JobTracker</code>.
+   </li>
+   <li>
+   Details regarding black listed trackers.
+   </li>
+ </ol>
+ 
+ <p>Clients can query for the latest <code>ClusterStatus</code>, via 
+ {@link JobClient#getClusterStatus()}.</p>
+ 
+ @see JobClient]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.mapred.ClusterStatus -->
+  <!-- start class org.apache.hadoop.mapred.Counters -->
+  <class name="Counters" extends="org.apache.hadoop.mapreduce.counters.AbstractCounters"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="Counters"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="Counters" type="org.apache.hadoop.mapreduce.Counters"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="getGroup" return="org.apache.hadoop.mapred.Counters.Group"
+      abstract="false" native="false" synchronized="true"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="groupName" type="java.lang.String"/>
+    </method>
+    <method name="getGroupNames" return="java.util.Collection"
+      abstract="false" native="false" synchronized="true"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="makeCompactString" return="java.lang.String"
+      abstract="false" native="false" synchronized="true"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="findCounter" return="org.apache.hadoop.mapred.Counters.Counter"
+      abstract="false" native="false" synchronized="true"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="group" type="java.lang.String"/>
+      <param name="name" type="java.lang.String"/>
+    </method>
+    <method name="findCounter" return="org.apache.hadoop.mapred.Counters.Counter"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="use {@link #findCounter(String, String)} instead">
+      <param name="group" type="java.lang.String"/>
+      <param name="id" type="int"/>
+      <param name="name" type="java.lang.String"/>
+      <doc>
+      <![CDATA[Find a counter by using strings
+ @param group the name of the group
+ @param id the id of the counter within the group (0 to N-1)
+ @param name the internal name of the counter
+ @return the counter for that name
+ @deprecated use {@link #findCounter(String, String)} instead]]>
+      </doc>
+    </method>
+    <method name="incrCounter"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="key" type="java.lang.Enum"/>
+      <param name="amount" type="long"/>
+      <doc>
+      <![CDATA[Increments the specified counter by the specified amount, creating it if
+ it didn't already exist.
+ @param key identifies a counter
+ @param amount amount by which counter is to be incremented]]>
+      </doc>
+    </method>
+    <method name="incrCounter"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="group" type="java.lang.String"/>
+      <param name="counter" type="java.lang.String"/>
+      <param name="amount" type="long"/>
+      <doc>
+      <![CDATA[Increments the specified counter by the specified amount, creating it if
+ it didn't already exist.
+ @param group the name of the group
+ @param counter the internal name of the counter
+ @param amount amount by which counter is to be incremented]]>
+      </doc>
+    </method>
+    <method name="getCounter" return="long"
+      abstract="false" native="false" synchronized="true"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="key" type="java.lang.Enum"/>
+      <doc>
+      <![CDATA[Returns current value of the specified counter, or 0 if the counter
+ does not exist.
+ @param key the counter enum to lookup
+ @return the counter value or 0 if counter not found]]>
+      </doc>
+    </method>
+    <method name="incrAllCounters"
+      abstract="false" native="false" synchronized="true"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="other" type="org.apache.hadoop.mapred.Counters"/>
+      <doc>
+      <![CDATA[Increments multiple counters by their amounts in another Counters
+ instance.
+ @param other the other Counters instance]]>
+      </doc>
+    </method>
+    <method name="size" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="use {@link #countCounters()} instead">
+      <doc>
+      <![CDATA[@return the total number of counters
+ @deprecated use {@link #countCounters()} instead]]>
+      </doc>
+    </method>
+    <method name="sum" return="org.apache.hadoop.mapred.Counters"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="a" type="org.apache.hadoop.mapred.Counters"/>
+      <param name="b" type="org.apache.hadoop.mapred.Counters"/>
+      <doc>
+      <![CDATA[Convenience method for computing the sum of two sets of counters.
+ @param a the first counters
+ @param b the second counters
+ @return a new summed counters object]]>
+      </doc>
+    </method>
+    <method name="log"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="log" type="org.apache.commons.logging.Log"/>
+      <doc>
+      <![CDATA[Logs the current counter values.
+ @param log The log to use.]]>
+      </doc>
+    </method>
+    <method name="makeEscapedCompactString" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Represent the counter in a textual format that can be converted back to
+ its object form
+ @return the string in the following format
+ {(groupName)(group-displayName)[(counterName)(displayName)(value)][]*}*]]>
+      </doc>
+    </method>
+    <method name="fromEscapedCompactString" return="org.apache.hadoop.mapred.Counters"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="compactString" type="java.lang.String"/>
+      <exception name="ParseException" type="java.text.ParseException"/>
+      <doc>
+      <![CDATA[Convert a stringified (by {@link #makeEscapedCompactString()} counter
+ representation into a counter object.
+ @param compactString to parse
+ @return a new counters object
+ @throws ParseException]]>
+      </doc>
+    </method>
+    <field name="MAX_COUNTER_LIMIT" type="int"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+    </field>
+    <field name="MAX_GROUP_LIMIT" type="int"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+    </field>
+    <doc>
+    <![CDATA[A set of named counters.
+
+ <p><code>Counters</code> represent global counters, defined either by the
+ Map-Reduce framework or applications. Each <code>Counter</code> can be of
+ any {@link Enum} type.</p>
+
+ <p><code>Counters</code> are bunched into {@link Group}s, each comprising of
+ counters from a particular <code>Enum</code> class.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.mapred.Counters -->
+  <!-- start class org.apache.hadoop.mapred.Counters.Counter -->
+  <class name="Counters.Counter" extends="java.lang.Object"
+    abstract="false"
+    static="true" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="org.apache.hadoop.mapreduce.Counter"/>
+    <constructor name="Counters.Counter"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="setDisplayName"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="displayName" type="java.lang.String"/>
+    </method>
+    <method name="getName" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getDisplayName" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getValue" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="setValue"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="value" type="long"/>
+    </method>
+    <method name="increment"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="incr" type="long"/>
+    </method>
+    <method name="write"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="out" type="java.io.DataOutput"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="readFields"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="in" type="java.io.DataInput"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="makeEscapedCompactString" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Returns the compact stringified version of the counter in the format
+ [(actual-name)(display-name)(value)]
+ @return the stringified result]]>
+      </doc>
+    </method>
+    <method name="contentEquals" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="deprecated, no comment">
+      <param name="counter" type="org.apache.hadoop.mapred.Counters.Counter"/>
+      <doc>
+      <![CDATA[Checks for (content) equality of two (basic) counters
+ @param counter to compare
+ @return true if content equals
+ @deprecated]]>
+      </doc>
+    </method>
+    <method name="getCounter" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[@return the value of the counter]]>
+      </doc>
+    </method>
+    <method name="getUnderlyingCounter" return="org.apache.hadoop.mapreduce.Counter"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="equals" return="boolean"
+      abstract="false" native="false" synchronized="true"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="genericRight" type="java.lang.Object"/>
+    </method>
+    <method name="hashCode" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <doc>
+    <![CDATA[A counter record, comprising its name and value.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.mapred.Counters.Counter -->
+  <!-- start class org.apache.hadoop.mapred.Counters.Group -->
+  <class name="Counters.Group" extends="java.lang.Object"
+    abstract="false"
+    static="true" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="org.apache.hadoop.mapreduce.counters.CounterGroupBase"/>
+    <constructor name="Counters.Group"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="getCounter" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="counterName" type="java.lang.String"/>
+      <doc>
+      <![CDATA[@param counterName the name of the counter
+ @return the value of the specified counter, or 0 if the counter does
+ not exist.]]>
+      </doc>
+    </method>
+    <method name="makeEscapedCompactString" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[@return the compact stringified version of the group in the format
+ {(actual-name)(display-name)(value)[][][]} where [] are compact strings
+ for the counters within.]]>
+      </doc>
+    </method>
+    <method name="getCounter" return="org.apache.hadoop.mapred.Counters.Counter"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="use {@link #findCounter(String)} instead">
+      <param name="id" type="int"/>
+      <param name="name" type="java.lang.String"/>
+      <doc>
+      <![CDATA[Get the counter for the given id and create it if it doesn't exist.
+ @param id the numeric id of the counter within the group
+ @param name the internal counter name
+ @return the counter
+ @deprecated use {@link #findCounter(String)} instead]]>
+      </doc>
+    </method>
+    <method name="getCounterForName" return="org.apache.hadoop.mapred.Counters.Counter"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <doc>
+      <![CDATA[Get the counter for the given name and create it if it doesn't exist.
+ @param name the internal counter name
+ @return the counter]]>
+      </doc>
+    </method>
+    <method name="write"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="out" type="java.io.DataOutput"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="readFields"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="in" type="java.io.DataInput"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="iterator" return="java.util.Iterator"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getName" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getDisplayName" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="setDisplayName"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="displayName" type="java.lang.String"/>
+    </method>
+    <method name="addCounter"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="counter" type="org.apache.hadoop.mapred.Counters.Counter"/>
+    </method>
+    <method name="addCounter" return="org.apache.hadoop.mapred.Counters.Counter"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <param name="displayName" type="java.lang.String"/>
+      <param name="value" type="long"/>
+    </method>
+    <method name="findCounter" return="org.apache.hadoop.mapred.Counters.Counter"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="counterName" type="java.lang.String"/>
+      <param name="displayName" type="java.lang.String"/>
+    </method>
+    <method name="findCounter" return="org.apache.hadoop.mapred.Counters.Counter"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="counterName" type="java.lang.String"/>
+      <param name="create" type="boolean"/>
+    </method>
+    <method name="findCounter" return="org.apache.hadoop.mapred.Counters.Counter"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="counterName" type="java.lang.String"/>
+    </method>
+    <method name="size" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="incrAllCounters"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="rightGroup" type="org.apache.hadoop.mapreduce.counters.CounterGroupBase"/>
+    </method>
+    <method name="getUnderlyingGroup" return="org.apache.hadoop.mapreduce.counters.CounterGroupBase"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="equals" return="boolean"
+      abstract="false" native="false" synchronized="true"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="genericRight" type="java.lang.Object"/>
+    </method>
+    <method name="hashCode" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <doc>
+    <![CDATA[<code>Group</code> of counters, comprising of counters from a particular
+  counter {@link Enum} class.
+
+  <p><code>Group</code>handles localization of the class name and the
+  counter names.</p>]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.mapred.Counters.Group -->
+  <!-- start class org.apache.hadoop.mapred.FileAlreadyExistsException -->
+  <class name="FileAlreadyExistsException" extends="java.io.IOException"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="FileAlreadyExistsException"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="FileAlreadyExistsException" type="java.lang.String"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <doc>
+    <![CDATA[Used when target file already exists for any operation and 
+ is not configured to be overwritten.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.mapred.FileAlreadyExistsException -->
+  <!-- start class org.apache.hadoop.mapred.FileInputFormat -->
+  <class name="FileInputFormat" extends="java.lang.Object"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="org.apache.hadoop.mapred.InputFormat"/>
+    <constructor name="FileInputFormat"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="setMinSplitSize"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="minSplitSize" type="long"/>
+    </method>
+    <method name="isSplitable" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="fs" type="org.apache.hadoop.fs.FileSystem"/>
+      <param name="filename" type="org.apache.hadoop.fs.Path"/>
+      <doc>
+      <![CDATA[Is the given filename splittable? Usually, true, but if the file is
+ stream compressed, it will not be.
+
+ The default implementation in <code>FileInputFormat</code> always returns
+ true. Implementations that may deal with non-splittable files <i>must</i>
+ override this method.
+
+ <code>FileInputFormat</code> implementations can override this and return
+ <code>false</code> to ensure that individual input files are never split-up
+ so that {@link Mapper}s process entire files.
+ 
+ @param fs the file system that the file is on
+ @param filename the file name to check
+ @return is this file splitable?]]>
+      </doc>
+    </method>
+    <method name="getRecordReader" return="org.apache.hadoop.mapred.RecordReader"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="split" type="org.apache.hadoop.mapred.InputSplit"/>
+      <param name="job" type="org.apache.hadoop.mapred.JobConf"/>
+      <param name="reporter" type="org.apache.hadoop.mapred.Reporter"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="setInputPathFilter"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.mapred.JobConf"/>
+      <param name="filter" type="java.lang.Class"/>
+      <doc>
+      <![CDATA[Set a PathFilter to be applied to the input paths for the map-reduce job.
+
+ @param filter the PathFilter class use for filtering the input paths.]]>
+      </doc>
+    </method>
+    <method name="getInputPathFilter" return="org.apache.hadoop.fs.PathFilter"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.mapred.JobConf"/>
+      <doc>
+      <![CDATA[Get a PathFilter instance of the filter set for the input paths.
+
+ @return the PathFilter instance set for the job, NULL if none has been set.]]>
+      </doc>
+    </method>
+    <method name="addInputPathRecursively"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="result" type="java.util.List"/>
+      <param name="fs" type="org.apache.hadoop.fs.FileSystem"/>
+      <param name="path" type="org.apache.hadoop.fs.Path"/>
+      <param name="inputFilter" type="org.apache.hadoop.fs.PathFilter"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Add files in the input path recursively into the results.
+ @param result
+          The List to store all files.
+ @param fs
+          The FileSystem.
+ @param path
+          The input path.
+ @param inputFilter
+          The input filter that can be used to filter files/dirs. 
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="listStatus" return="org.apache.hadoop.fs.FileStatus[]"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="job" type="org.apache.hadoop.mapred.JobConf"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[List input directories.
+ Subclasses may override to, e.g., select only files matching a regular
+ expression. 
+ 
+ @param job the job to list input paths for
+ @return array of FileStatus objects
+ @throws IOException if zero items.]]>
+      </doc>
+    </method>
+    <method name="makeSplit" return="org.apache.hadoop.mapred.FileSplit"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="file" type="org.apache.hadoop.fs.Path"/>
+      <param name="start" type="long"/>
+      <param name="length" type="long"/>
+      <param name="hosts" type="java.lang.String[]"/>
+      <doc>
+      <![CDATA[A factory that makes the split for this class. It can be overridden
+ by sub-classes to make sub-types]]>
+      </doc>
+    </method>
+    <method name="makeSplit" return="org.apache.hadoop.mapred.FileSplit"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="file" type="org.apache.hadoop.fs.Path"/>
+      <param name="start" type="long"/>
+      <param name="length" type="long"/>
+      <param name="hosts" type="java.lang.String[]"/>
+      <param name="inMemoryHosts" type="java.lang.String[]"/>
+      <doc>
+      <![CDATA[A factory that makes the split for this class. It can be overridden
+ by sub-classes to make sub-types]]>
+      </doc>
+    </method>
+    <method name="getSplits" return="org.apache.hadoop.mapred.InputSplit[]"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="job" type="org.apache.hadoop.mapred.JobConf"/>
+      <param name="numSplits" type="int"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Splits files returned by {@link #listStatus(JobConf)} when
+ they're too big.]]>
+      </doc>
+    </method>
+    <method name="computeSplitSize" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="goalSize" type="long"/>
+      <param name="minSize" type="long"/>
+      <param name="blockSize" type="long"/>
+    </method>
+    <method name="getBlockIndex" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="blkLocations" type="org.apache.hadoop.fs.BlockLocation[]"/>
+      <param name="offset" type="long"/>
+    </method>
+    <method name="setInputPaths"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.mapred.JobConf"/>
+      <param name="commaSeparatedPaths" type="java.lang.String"/>
+      <doc>
+      <![CDATA[Sets the given comma separated paths as the list of inputs 
+ for the map-reduce job.
+ 
+ @param conf Configuration of the job
+ @param commaSeparatedPaths Comma separated paths to be set as 
+        the list of inputs for the map-reduce job.]]>
+      </doc>
+    </method>
+    <method name="addInputPaths"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.mapred.JobConf"/>
+      <param name="commaSeparatedPaths" type="java.lang.String"/>
+      <doc>
+      <![CDATA[Add the given comma separated paths to the list of inputs for
+  the map-reduce job.
+ 
+ @param conf The configuration of the job 
+ @param commaSeparatedPaths Comma separated paths to be added to
+        the list of inputs for the map-reduce job.]]>
+      </doc>
+    </method>
+    <method name="setInputPaths"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.mapred.JobConf"/>
+      <param name="inputPaths" type="org.apache.hadoop.fs.Path[]"/>
+      <doc>
+      <![CDATA[Set the array of {@link Path}s as the list of inputs
+ for the map-reduce job.
+ 
+ @param conf Configuration of the job. 
+ @param inputPaths the {@link Path}s of the input directories/files 
+ for the map-reduce job.]]>
+      </doc>
+    </method>
+    <method name="addInputPath"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.mapred.JobConf"/>
+      <param name="path" type="org.apache.hadoop.fs.Path"/>
+      <doc>
+      <![CDATA[Add a {@link Path} to the list of inputs for the map-reduce job.
+ 
+ @param conf The configuration of the job 
+ @param path {@link Path} to be added to the list of inputs for 
+            the map-reduce job.]]>
+      </doc>
+    </method>
+    <method name="getInputPaths" return="org.apache.hadoop.fs.Path[]"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.mapred.JobConf"/>
+      <doc>
+      <![CDATA[Get the list of input {@link Path}s for the map-reduce job.
+ 
+ @param conf The configuration of the job 
+ @return the list of input {@link Path}s for the map-reduce job.]]>
+      </doc>
+    </method>
+    <method name="getSplitHosts" return="java.lang.String[]"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="blkLocations" type="org.apache.hadoop.fs.BlockLocation[]"/>
+      <param name="offset" type="long"/>
+      <param name="splitSize" type="long"/>
+      <param name="clusterMap" type="org.apache.hadoop.net.NetworkTopology"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[This function identifies and returns the hosts that contribute 
+ most for a given split. For calculating the contribution, rack
+ locality is treated on par with host locality, so hosts from racks
+ that contribute the most are preferred over hosts on racks that 
+ contribute less
+ @param blkLocations The list of block locations
+ @param offset 
+ @param splitSize 
+ @return an array of hosts that contribute most to this split
+ @throws IOException]]>
+      </doc>
+    </method>
+    <field name="LOG" type="org.apache.commons.logging.Log"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+    </field>
+    <field name="NUM_INPUT_FILES" type="java.lang.String"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+    </field>
+    <field name="INPUT_DIR_RECURSIVE" type="java.lang.String"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+    </field>
+    <doc>
+    <![CDATA[A base class for file-based {@link InputFormat}.
+ 
+ <p><code>FileInputFormat</code> is the base class for all file-based 
+ <code>InputFormat</code>s. This provides a generic implementation of
+ {@link #getSplits(JobConf, int)}.
+
+ Implementations of <code>FileInputFormat</code> can also override the
+ {@link #isSplitable(FileSystem, Path)} method to prevent input files
+ from being split-up in certain situations. Implementations that may
+ deal with non-splittable files <i>must</i> override this method, since
+ the default implementation assumes splitting is always possible.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.mapred.FileInputFormat -->
+  <!-- start class org.apache.hadoop.mapred.FileOutputCommitter -->
+  <class name="FileOutputCommitter" extends="org.apache.hadoop.mapred.OutputCommitter"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="FileOutputCommitter"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="getWorkPath" return="org.apache.hadoop.fs.Path"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="context" type="org.apache.hadoop.mapred.TaskAttemptContext"/>
+      <param name="outputPath" type="org.apache.hadoop.fs.Path"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="setupJob"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="context" type="org.apache.hadoop.mapred.JobContext"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="commitJob"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="context" type="org.apache.hadoop.mapred.JobContext"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="cleanupJob"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="context" type="org.apache.hadoop.mapred.JobContext"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="abortJob"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="context" type="org.apache.hadoop.mapred.JobContext"/>
+      <param name="runState" type="int"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="setupTask"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="context" type="org.apache.hadoop.mapred.TaskAttemptContext"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="commitTask"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="context" type="org.apache.hadoop.mapred.TaskAttemptContext"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="abortTask"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="context" type="org.apache.hadoop.mapred.TaskAttemptContext"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="needsTaskCommit" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="context" type="org.apache.hadoop.mapred.TaskAttemptContext"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="isRecoverySupported" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="isCommitJobRepeatable" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="context" type="org.apache.hadoop.mapred.JobContext"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="isRecoverySupported" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="context" type="org.apache.hadoop.mapred.JobContext"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="recoverTask"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="context" type="org.apache.hadoop.mapred.TaskAttemptContext"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <field name="LOG" type="org.apache.commons.logging.Log"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+    </field>
+    <field name="TEMP_DIR_NAME" type="java.lang.String"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Temporary directory name]]>
+      </doc>
+    </field>
+    <field name="SUCCEEDED_FILE_NAME" type="java.lang.String"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+    </field>
+    <doc>
+    <![CDATA[An {@link OutputCommitter} that commits files specified 
+ in job output directory i.e. ${mapreduce.output.fileoutputformat.outputdir}.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.mapred.FileOutputCommitter -->
+  <!-- start class org.apache.hadoop.mapred.FileOutputFormat -->
+  <class name="FileOutputFormat" extends="java.lang.Object"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="org.apache.hadoop.mapred.OutputFormat"/>
+    <constructor name="FileOutputFormat"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="setCompressOutput"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.mapred.JobConf"/>
+      <param name="compress" type="boolean"/>
+      <doc>
+      <![CDATA[Set whether the output of the job is compressed.
+ @param conf the {@link JobConf} to modify
+ @param compress should the output of the job be compressed?]]>
+      </doc>
+    </method>
+    <method name="getCompressOutput" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.mapred.JobConf"/>
+      <doc>
+      <![CDATA[Is the job output compressed?
+ @param conf the {@link JobConf} to look in
+ @return <code>true</code> if the job output should be compressed,
+         <code>false</code> otherwise]]>
+      </doc>
+    </method>
+    <method name="setOutputCompressorClass"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.mapred.JobConf"/>
+      <param name="codecClass" type="java.lang.Class"/>
+      <doc>
+      <![CDATA[Set the {@link CompressionCodec} to be used to compress job outputs.
+ @param conf the {@link JobConf} to modify
+ @param codecClass the {@link CompressionCodec} to be used to
+                   compress the job outputs]]>
+      </doc>
+    </method>
+    <method name="getOutputCompressorClass" return="java.lang.Class"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.mapred.JobConf"/>
+      <param name="defaultValue" type="java.lang.Class"/>
+      <doc>
+      <![CDATA[Get the {@link CompressionCodec} for compressing the job outputs.
+ @param conf the {@link JobConf} to look in
+ @param defaultValue the {@link CompressionCodec} to return if not set
+ @return the {@link CompressionCodec} to be used to compress the 
+         job outputs
+ @throws IllegalArgumentException if the class was specified, but not found]]>
+      </doc>
+    </method>
+    <method name="getRecordWriter" return="org.apache.hadoop.mapred.RecordWriter"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="ignored" type="org.apache.hadoop.fs.FileSystem"/>
+      <param name="job" type="org.apache.hadoop.mapred.JobConf"/>
+      <param name="name" type="java.lang.String"/>
+      <param name="progress" type="org.apache.hadoop.util.Progressable"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="checkOutputSpecs"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="ignored" type="org.apache.hadoop.fs.FileSystem"/>
+      <param name="job" type="org.apache.hadoop.mapred.JobConf"/>
+      <exception name="FileAlreadyExistsException" type="org.apache.hadoop.mapred.FileAlreadyExistsException"/>
+      <exception name="InvalidJobConfException" type="org.apache.hadoop.mapred.InvalidJobConfException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="setOutputPath"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.mapred.JobConf"/>
+      <param name="outputDir" type="org.apache.hadoop.fs.Path"/>
+      <doc>
+      <![CDATA[Set the {@link Path} of the output directory for the map-reduce job.
+
+ @param conf The configuration of the job.
+ @param outputDir the {@link Path} of the output directory for 
+ the map-reduce job.]]>
+      </doc>
+    </method>
+    <method name="getOutputPath" return="org.apache.hadoop.fs.Path"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.mapred.JobConf"/>
+      <doc>
+      <![CDATA[Get the {@link Path} to the output directory for the map-reduce job.
+ 
+ @return the {@link Path} to the output directory for the map-reduce job.
+ @see FileOutputFormat#getWorkOutputPath(JobConf)]]>
+      </doc>
+    </method>
+    <method name="getWorkOutputPath" return="org.apache.hadoop.fs.Path"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.mapred.JobConf"/>
+      <doc>
+      <![CDATA[Get the {@link Path} to the task's temporary output directory 
+  for the map-reduce job
+  
+ <b id="SideEffectFiles">Tasks' Side-Effect Files</b>
+ 
+ <p><i>Note:</i> The following is valid only if the {@link OutputCommitter}
+  is {@link FileOutputCommitter}. If <code>OutputCommitter</code> is not 
+  a <code>FileOutputCommitter</code>, the task's temporary output
+  directory is same as {@link #getOutputPath(JobConf)} i.e.
+  <tt>${mapreduce.output.fileoutputformat.outputdir}$</tt></p>
+  
+ <p>Some applications need to create/write-to side-files, which differ from
+ the actual job-outputs.
+ 
+ <p>In such cases there could be issues with 2 instances of the same TIP 
+ (running simultaneously e.g. speculative tasks) trying to open/write-to the
+ same file (path) on HDFS. Hence the application-writer will have to pick 
+ unique names per task-attempt (e.g. using the attemptid, say 
+ <tt>attempt_200709221812_0001_m_000000_0</tt>), not just per TIP.</p> 
+ 
+ <p>To get around this the Map-Reduce framework helps the application-writer 
+ out by maintaining a special 
+ <tt>${mapreduce.output.fileoutputformat.outputdir}/_temporary/_${taskid}</tt> 
+ sub-directory for each task-attempt on HDFS where the output of the 
+ task-attempt goes. On successful completion of the task-attempt the files 
+ in the <tt>${mapreduce.output.fileoutputformat.outputdir}/_temporary/_${taskid}</tt> (only) 
+ are <i>promoted</i> to <tt>${mapreduce.output.fileoutputformat.outputdir}</tt>. Of course, the 
+ framework discards the sub-directory of unsuccessful task-attempts. This 
+ is completely transparent to the application.</p>
+ 
+ <p>The application-writer can take advantage of this by creating any 
+ side-files required in <tt>${mapreduce.task.output.dir}</tt> during execution 
+ of his reduce-task i.e. via {@link #getWorkOutputPath(JobConf)}, and the 
+ framework will move them out similarly - thus she doesn't have to pick 
+ unique paths per task-attempt.</p>
+ 
+ <p><i>Note</i>: the value of <tt>${mapreduce.task.output.dir}</tt> during 
+ execution of a particular task-attempt is actually 
+ <tt>${mapreduce.output.fileoutputformat.outputdir}/_temporary/_{$taskid}</tt>, and this value is 
+ set by the map-reduce framework. So, just create any side-files in the 
+ path  returned by {@link #getWorkOutputPath(JobConf)} from map/reduce 
+ task to take advantage of this feature.</p>
+ 
+ <p>The entire discussion holds true for maps of jobs with 
+ reducer=NONE (i.e. 0 reduces) since output of the map, in that case, 
+ goes directly to HDFS.</p> 
+ 
+ @return the {@link Path} to the task's temporary output directory 
+ for the map-reduce job.]]>
+      </doc>
+    </method>
+    <method name="getTaskOutputPath" return="org.apache.hadoop.fs.Path"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.mapred.JobConf"/>
+      <param name="name" type="java.lang.String"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Helper function to create the task's temporary output directory and 
+ return the path to the task's output file.
+ 
+ @param conf job-configuration
+ @param name temporary task-output filename
+ @return path to the task's temporary output file
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getUniqueName" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.mapred.JobConf"/>
+      <param name="name" type="java.lang.String"/>
+      <doc>
+      <![CDATA[Helper function to generate a name that is unique for the task.
+
+ <p>The generated name can be used to create custom files from within the
+ different tasks for the job, the names for different tasks will not collide
+ with each other.</p>
+
+ <p>The given name is postfixed with the task type, 'm' for maps, 'r' for
+ reduces and the task partition number. For example, give a name 'test'
+ running on the first map o the job the generated name will be
+ 'test-m-00000'.</p>
+
+ @param conf the configuration for the job.
+ @param name the name to make unique.
+ @return a unique name accross all tasks of the job.]]>
+      </doc>
+    </method>
+    <method name="getPathForCustomFile" return="org.apache.hadoop.fs.Path"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.mapred.JobConf"/>
+      <param name="name" type="java.lang.String"/>
+      <doc>
+      <![CDATA[Helper function to generate a {@link Path} for a file that is unique for
+ the task within the job output directory.
+
+ <p>The path can be used to create custom files from within the map and
+ reduce tasks. The path name will be unique for each task. The path parent
+ will be the job output directory.</p>ls
+
+ <p>This method uses the {@link #getUniqueName} method to make the file name
+ unique for the task.</p>
+
+ @param conf the configuration for the job.
+ @param name the name for the file.
+ @return a unique path accross all tasks of the job.]]>
+      </doc>
+    </method>
+    <doc>
+    <![CDATA[A base class for {@link OutputFormat}.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.mapred.FileOutputFormat -->
+  <!-- start class org.apache.hadoop.mapred.FileSplit -->
+  <class name="FileSplit" extends="org.apache.hadoop.mapreduce.InputSplit"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="org.apache.hadoop.mapred.InputSplitWithLocationInfo"/>
+    <constructor name="FileSplit"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="FileSplit" type="org.apache.hadoop.fs.Path, long, long, org.apache.hadoop.mapred.JobConf"
+      static="false" final="false" visibility="public"
+      deprecated="deprecated, no comment">
+      <doc>
+      <![CDATA[Constructs a split.
+ @deprecated
+ @param file the file name
+ @param start the position of the first byte in the file to process
+ @param length the number of bytes in the file to process]]>
+      </doc>
+    </constructor>
+    <constructor name="FileSplit" type="org.apache.hadoop.fs.Path, long, long, java.lang.String[]"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Constructs a split with host information
+
+ @param file the file name
+ @param start the position of the first byte in the file to process
+ @param length the number of bytes in the file to process
+ @param hosts the list of hosts containing the block, possibly null]]>
+      </doc>
+    </constructor>
+    <constructor name="FileSplit" type="org.apache.hadoop.fs.Path, long, long, java.lang.String[], java.lang.String[]"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Constructs a split with host information
+
+ @param file the file name
+ @param start the position of the first byte in the file to process
+ @param length the number of bytes in the file to process
+ @param hosts the list of hosts containing the block, possibly null
+ @param inMemoryHosts the list of hosts containing the block in memory]]>
+      </doc>
+    </constructor>
+    <constructor name="FileSplit" type="org.apache.hadoop.mapreduce.lib.input.FileSplit"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="getPath" return="org.apache.hadoop.fs.Path"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[The file containing this split's data.]]>
+      </doc>
+    </method>
+    <method name="getStart" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[The position of the first byte in the file to process.]]>
+      </doc>
+    </method>
+    <method name="getLength" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[The number of bytes in the file to process.]]>
+      </doc>
+    </method>
+    <method name="toString" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="write"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="out" type="java.io.DataOutput"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="readFields"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="in" type="java.io.DataInput"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="getLocations" return="java.lang.String[]"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="getLocationInfo" return="org.apache.hadoop.mapred.SplitLocationInfo[]"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <doc>
+    <![CDATA[A section of an input file.  Returned by {@link
+ InputFormat#getSplits(JobConf, int)} and passed to
+ {@link InputFormat#getRecordReader(InputSplit,JobConf,Reporter)}.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.mapred.FileSplit -->
+  <!-- start class org.apache.hadoop.mapred.FixedLengthInputFormat -->
+  <class name="FixedLengthInputFormat" extends="org.apache.hadoop.mapred.FileInputFormat"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="org.apache.hadoop.mapred.JobConfigurable"/>
+    <constructor name="FixedLengthInputFormat"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="setRecordLength"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <param name="recordLength" type="int"/>
+      <doc>
+      <![CDATA[Set the length of each record
+ @param conf configuration
+ @param recordLength the length of a record]]>
+      </doc>
+    </method>
+    <method name="getRecordLength" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <doc>
+      <![CDATA[Get record length value
+ @param conf configuration
+ @return the record length, zero means none was set]]>
+      </doc>
+    </method>
+    <method name="configure"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.mapred.JobConf"/>
+    </method>
+    <method name="getRecordReader" return="org.apache.hadoop.mapred.RecordReader"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="genericSplit" type="org.apache.hadoop.mapred.InputSplit"/>
+      <param name="job" type="org.apache.hadoop.mapred.JobConf"/>
+      <param name="reporter" type="org.apache.hadoop.mapred.Reporter"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="isSplitable" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="fs" type="org.apache.hadoop.fs.FileSystem"/>
+      <param name="file" type="org.apache.hadoop.fs.Path"/>
+    </method>
+    <field name="FIXED_RECORD_LENGTH" type="java.lang.String"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+    </field>
+    <doc>
+    <![CDATA[FixedLengthInputFormat is an input format used to read input files
+ which contain fixed length records.  The content of a record need not be
+ text.  It can be arbitrary binary data.  Users must configure the record
+ length property by calling:
+ FixedLengthInputFormat.setRecordLength(conf, recordLength);<br><br> or
+ conf.setInt(FixedLengthInputFormat.FIXED_RECORD_LENGTH, recordLength);
+ <br><br>
+ @see FixedLengthRecordReader]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.mapred.FixedLengthInputFormat -->
+  <!-- start class org.apache.hadoop.mapred.ID -->
+  <class name="ID" extends="org.apache.hadoop.mapreduce.ID"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="ID" type="int"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[constructs an ID object from the given int]]>
+      </doc>
+    </constructor>
+    <constructor name="ID"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </const

<TRUNCATED>

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


[25/50] [abbrv] hadoop git commit: Add 2.8.3 release jdiff files.

Posted by na...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/a7f8caf5/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Client_2.8.3.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Client_2.8.3.xml b/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Client_2.8.3.xml
new file mode 100644
index 0000000..3f6c5eb
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Client_2.8.3.xml
@@ -0,0 +1,2316 @@
+<?xml version="1.0" encoding="iso-8859-1" standalone="no"?>
+<!-- Generated by the JDiff Javadoc doclet -->
+<!-- (http://www.jdiff.org) -->
+<!-- on Tue Dec 05 05:39:30 UTC 2017 -->
+
+<api
+  xmlns:xsi='http://www.w3.org/2001/XMLSchema-instance'
+  xsi:noNamespaceSchemaLocation='api.xsd'
+  name="Apache Hadoop YARN Client 2.8.3"
+  jdversion="1.0.9">
+
+<!--  Command line arguments =  -doclet org.apache.hadoop.classification.tools.IncludePublicAnnotationsJDiffDoclet -docletpath /build/source/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/target/hadoop-annotations.jar:/build/source/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/target/jdiff.jar -verbose -classpath /build/source/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/target/classes:/build/source/hadoop-common-project/hadoop-common/target/hadoop-common-2.8.3.jar:/maven/org/apache/commons/commons-math3/3.1.1/commons-math3-3.1.1.jar:/maven/xmlenc/xmlenc/0.52/xmlenc-0.52.jar:/maven/org/apache/httpcomponents/httpclient/4.5.2/httpclient-4.5.2.jar:/maven/org/apache/httpcomponents/httpcore/4.4.4/httpcore-4.4.4.jar:/maven/commons-codec/commons-codec/1.4/commons-codec-1.4.jar:/maven/commons-io/commons-io/2.4/commons-io-2.4.jar:/maven/commons-net/commons-net/3.1/commons-net-3.1.jar:/maven/commons-collections/commons-collections/3.2.2/commons-collections-3.2.2.jar:/maven/java
 x/servlet/servlet-api/2.5/servlet-api-2.5.jar:/maven/org/mortbay/jetty/jetty/6.1.26/jetty-6.1.26.jar:/maven/org/mortbay/jetty/jetty-util/6.1.26/jetty-util-6.1.26.jar:/maven/org/mortbay/jetty/jetty-sslengine/6.1.26/jetty-sslengine-6.1.26.jar:/maven/javax/servlet/jsp/jsp-api/2.1/jsp-api-2.1.jar:/maven/com/sun/jersey/jersey-core/1.9/jersey-core-1.9.jar:/maven/com/sun/jersey/jersey-json/1.9/jersey-json-1.9.jar:/maven/com/sun/xml/bind/jaxb-impl/2.2.3-1/jaxb-impl-2.2.3-1.jar:/maven/com/sun/jersey/jersey-server/1.9/jersey-server-1.9.jar:/maven/asm/asm/3.2/asm-3.2.jar:/maven/net/java/dev/jets3t/jets3t/0.9.0/jets3t-0.9.0.jar:/maven/com/jamesmurty/utils/java-xmlbuilder/0.4/java-xmlbuilder-0.4.jar:/maven/commons-configuration/commons-configuration/1.6/commons-configuration-1.6.jar:/maven/commons-digester/commons-digester/1.8/commons-digester-1.8.jar:/maven/commons-beanutils/commons-beanutils/1.7.0/commons-beanutils-1.7.0.jar:/maven/commons-beanutils/commons-beanutils-core/1.8.0/commons-beanuti
 ls-core-1.8.0.jar:/maven/org/slf4j/slf4j-api/1.7.10/slf4j-api-1.7.10.jar:/maven/org/slf4j/slf4j-log4j12/1.7.10/slf4j-log4j12-1.7.10.jar:/maven/org/codehaus/jackson/jackson-core-asl/1.9.13/jackson-core-asl-1.9.13.jar:/maven/org/codehaus/jackson/jackson-mapper-asl/1.9.13/jackson-mapper-asl-1.9.13.jar:/maven/org/apache/avro/avro/1.7.4/avro-1.7.4.jar:/maven/com/thoughtworks/paranamer/paranamer/2.3/paranamer-2.3.jar:/maven/org/xerial/snappy/snappy-java/1.0.4.1/snappy-java-1.0.4.1.jar:/maven/com/google/protobuf/protobuf-java/2.5.0/protobuf-java-2.5.0.jar:/maven/com/google/code/gson/gson/2.2.4/gson-2.2.4.jar:/build/source/hadoop-common-project/hadoop-auth/target/hadoop-auth-2.8.3.jar:/maven/com/nimbusds/nimbus-jose-jwt/3.9/nimbus-jose-jwt-3.9.jar:/maven/net/jcip/jcip-annotations/1.0/jcip-annotations-1.0.jar:/maven/net/minidev/json-smart/1.1.1/json-smart-1.1.1.jar:/maven/org/apache/directory/server/apacheds-kerberos-codec/2.0.0-M15/apacheds-kerberos-codec-2.0.0-M15.jar:/maven/org/apache/dir
 ectory/server/apacheds-i18n/2.0.0-M15/apacheds-i18n-2.0.0-M15.jar:/maven/org/apache/directory/api/api-asn1-api/1.0.0-M20/api-asn1-api-1.0.0-M20.jar:/maven/org/apache/directory/api/api-util/1.0.0-M20/api-util-1.0.0-M20.jar:/maven/org/apache/curator/curator-framework/2.7.1/curator-framework-2.7.1.jar:/maven/com/jcraft/jsch/0.1.54/jsch-0.1.54.jar:/maven/org/apache/curator/curator-client/2.7.1/curator-client-2.7.1.jar:/maven/org/apache/curator/curator-recipes/2.7.1/curator-recipes-2.7.1.jar:/maven/com/google/code/findbugs/jsr305/3.0.0/jsr305-3.0.0.jar:/maven/org/apache/htrace/htrace-core4/4.0.1-incubating/htrace-core4-4.0.1-incubating.jar:/maven/org/apache/zookeeper/zookeeper/3.4.6/zookeeper-3.4.6.jar:/maven/org/apache/commons/commons-compress/1.4.1/commons-compress-1.4.1.jar:/maven/org/tukaani/xz/1.0/xz-1.0.jar:/maven/com/google/guava/guava/11.0.2/guava-11.0.2.jar:/maven/commons-logging/commons-logging/1.1.3/commons-logging-1.1.3.jar:/maven/commons-lang/commons-lang/2.6/commons-lang-2.
 6.jar:/maven/commons-cli/commons-cli/1.2/commons-cli-1.2.jar:/maven/log4j/log4j/1.2.17/log4j-1.2.17.jar:/build/source/hadoop-common-project/hadoop-annotations/target/hadoop-annotations-2.8.3.jar:/usr/lib/jvm/java-7-openjdk-amd64/lib/tools.jar:/maven/io/netty/netty/3.6.2.Final/netty-3.6.2.Final.jar:/build/source/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/target/hadoop-yarn-api-2.8.3.jar:/build/source/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/target/hadoop-yarn-common-2.8.3.jar:/maven/javax/xml/bind/jaxb-api/2.2.2/jaxb-api-2.2.2.jar:/maven/javax/xml/stream/stax-api/1.0-2/stax-api-1.0-2.jar:/maven/javax/activation/activation/1.1/activation-1.1.jar:/maven/com/sun/jersey/jersey-client/1.9/jersey-client-1.9.jar:/maven/org/codehaus/jackson/jackson-jaxrs/1.9.13/jackson-jaxrs-1.9.13.jar:/maven/org/codehaus/jackson/jackson-xc/1.9.13/jackson-xc-1.9.13.jar:/maven/com/google/inject/extensions/guice-servlet/3.0/guice-servlet-3.0.jar:/maven/com/google/inject/guice/3.0/guice-3.0.jar:/
 maven/javax/inject/javax.inject/1/javax.inject-1.jar:/maven/aopalliance/aopalliance/1.0/aopalliance-1.0.jar:/maven/com/sun/jersey/contribs/jersey-guice/1.9/jersey-guice-1.9.jar:/maven/org/codehaus/jettison/jettison/1.1/jettison-1.1.jar -sourcepath /build/source/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java -apidir /build/source/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/target/site/jdiff/xml -apiname Apache Hadoop YARN Client 2.8.3 -->
+<package name="org.apache.hadoop.yarn.client">
+</package>
+<package name="org.apache.hadoop.yarn.client.api">
+  <!-- start class org.apache.hadoop.yarn.client.api.AHSClient -->
+  <class name="AHSClient" extends="org.apache.hadoop.service.AbstractService"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="AHSClient" type="java.lang.String"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="createAHSClient" return="org.apache.hadoop.yarn.client.api.AHSClient"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Create a new instance of AHSClient.]]>
+      </doc>
+    </method>
+    <method name="getApplicationReport" return="org.apache.hadoop.yarn.api.records.ApplicationReport"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appId" type="org.apache.hadoop.yarn.api.records.ApplicationId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Get a report of the given Application.
+ <p>
+ In secure mode, <code>YARN</code> verifies access to the application, queue
+ etc. before accepting the request.
+ <p>
+ If the user does not have <code>VIEW_APP</code> access then the following
+ fields in the report will be set to stubbed values:
+ <ul>
+   <li>host - set to "N/A"</li>
+   <li>RPC port - set to -1</li>
+   <li>client token - set to "N/A"</li>
+   <li>diagnostics - set to "N/A"</li>
+   <li>tracking URL - set to "N/A"</li>
+   <li>original tracking URL - set to "N/A"</li>
+   <li>resource usage report - all values are -1</li>
+ </ul>
+ 
+ @param appId
+          {@link ApplicationId} of the application that needs a report
+ @return application report
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getApplications" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report (ApplicationReport) of all Applications in the cluster.
+ </p>
+ 
+ <p>
+ If the user does not have <code>VIEW_APP</code> access for an application
+ then the corresponding report will be filtered as described in
+ {@link #getApplicationReport(ApplicationId)}.
+ </p>
+ 
+ @return a list of reports for all applications
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getApplicationAttemptReport" return="org.apache.hadoop.yarn.api.records.ApplicationAttemptReport"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationAttemptId" type="org.apache.hadoop.yarn.api.records.ApplicationAttemptId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report of the given ApplicationAttempt.
+ </p>
+ 
+ <p>
+ In secure mode, <code>YARN</code> verifies access to the application, queue
+ etc. before accepting the request.
+ </p>
+ 
+ @param applicationAttemptId
+          {@link ApplicationAttemptId} of the application attempt that needs
+          a report
+ @return application attempt report
+ @throws YarnException
+ @throws ApplicationAttemptNotFoundException if application attempt
+         not found
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getApplicationAttempts" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationId" type="org.apache.hadoop.yarn.api.records.ApplicationId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report of all (ApplicationAttempts) of Application in the cluster.
+ </p>
+ 
+ @param applicationId
+ @return a list of reports for all application attempts for specified
+         application
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getContainerReport" return="org.apache.hadoop.yarn.api.records.ContainerReport"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="containerId" type="org.apache.hadoop.yarn.api.records.ContainerId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report of the given Container.
+ </p>
+ 
+ <p>
+ In secure mode, <code>YARN</code> verifies access to the application, queue
+ etc. before accepting the request.
+ </p>
+ 
+ @param containerId
+          {@link ContainerId} of the container that needs a report
+ @return container report
+ @throws YarnException
+ @throws ContainerNotFoundException if container not found
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getContainers" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationAttemptId" type="org.apache.hadoop.yarn.api.records.ApplicationAttemptId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report of all (Containers) of ApplicationAttempt in the cluster.
+ </p>
+ 
+ @param applicationAttemptId
+ @return a list of reports of all containers for specified application
+         attempt
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.api.AHSClient -->
+  <!-- start class org.apache.hadoop.yarn.client.api.AMRMClient -->
+  <class name="AMRMClient" extends="org.apache.hadoop.service.AbstractService"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="AMRMClient" type="java.lang.String"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="createAMRMClient" return="org.apache.hadoop.yarn.client.api.AMRMClient"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Create a new instance of AMRMClient.
+ For usage:
+ <pre>
+ {@code
+ AMRMClient.<T>createAMRMClientContainerRequest()
+ }</pre>
+ @return the newly create AMRMClient instance.]]>
+      </doc>
+    </method>
+    <method name="registerApplicationMaster" return="org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appHostName" type="java.lang.String"/>
+      <param name="appHostPort" type="int"/>
+      <param name="appTrackingUrl" type="java.lang.String"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Register the application master. This must be called before any 
+ other interaction
+ @param appHostName Name of the host on which master is running
+ @param appHostPort Port master is listening on
+ @param appTrackingUrl URL at which the master info can be seen
+ @return <code>RegisterApplicationMasterResponse</code>
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="allocate" return="org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="progressIndicator" type="float"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Request additional containers and receive new container allocations.
+ Requests made via <code>addContainerRequest</code> are sent to the
+ <code>ResourceManager</code>. New containers assigned to the master are
+ retrieved. Status of completed containers and node health updates are also
+ retrieved. This also doubles up as a heartbeat to the ResourceManager and
+ must be made periodically. The call may not always return any new
+ allocations of containers. App should not make concurrent allocate
+ requests. May cause request loss.
+ 
+ <p>
+ Note : If the user has not removed container requests that have already
+ been satisfied, then the re-register may end up sending the entire
+ container requests to the RM (including matched requests). Which would mean
+ the RM could end up giving it a lot of new allocated containers.
+ </p>
+ 
+ @param progressIndicator Indicates progress made by the master
+ @return the response of the allocate request
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="unregisterApplicationMaster"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appStatus" type="org.apache.hadoop.yarn.api.records.FinalApplicationStatus"/>
+      <param name="appMessage" type="java.lang.String"/>
+      <param name="appTrackingUrl" type="java.lang.String"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Unregister the application master. This must be called in the end.
+ @param appStatus Success/Failure status of the master
+ @param appMessage Diagnostics message on failure
+ @param appTrackingUrl New URL to get master info
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="addContainerRequest"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="req" type="T"/>
+      <doc>
+      <![CDATA[Request containers for resources before calling <code>allocate</code>
+ @param req Resource request]]>
+      </doc>
+    </method>
+    <method name="removeContainerRequest"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="req" type="T"/>
+      <doc>
+      <![CDATA[Remove previous container request. The previous container request may have 
+ already been sent to the ResourceManager. So even after the remove request 
+ the app must be prepared to receive an allocation for the previous request 
+ even after the remove request
+ @param req Resource request]]>
+      </doc>
+    </method>
+    <method name="requestContainerResourceChange"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="container" type="org.apache.hadoop.yarn.api.records.Container"/>
+      <param name="capability" type="org.apache.hadoop.yarn.api.records.Resource"/>
+      <doc>
+      <![CDATA[Request container resource change before calling <code>allocate</code>.
+ Any previous pending resource change request of the same container will be
+ removed.
+
+ Application that calls this method is expected to maintain the
+ <code>Container</code>s that are returned from previous successful
+ allocations or resource changes. By passing in the existing container and a
+ target resource capability to this method, the application requests the
+ ResourceManager to change the existing resource allocation to the target
+ resource allocation.
+
+ @param container The container returned from the last successful resource
+                  allocation or resource change
+ @param capability  The target resource capability of the container]]>
+      </doc>
+    </method>
+    <method name="releaseAssignedContainer"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="containerId" type="org.apache.hadoop.yarn.api.records.ContainerId"/>
+      <doc>
+      <![CDATA[Release containers assigned by the Resource Manager. If the app cannot use
+ the container or wants to give up the container then it can release them.
+ The app needs to make new requests for the released resource capability if
+ it still needs it. eg. it released non-local resources
+ @param containerId]]>
+      </doc>
+    </method>
+    <method name="getAvailableResources" return="org.apache.hadoop.yarn.api.records.Resource"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the currently available resources in the cluster.
+ A valid value is available after a call to allocate has been made
+ @return Currently available resources]]>
+      </doc>
+    </method>
+    <method name="getClusterNodeCount" return="int"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the current number of nodes in the cluster.
+ A valid values is available after a call to allocate has been made
+ @return Current number of nodes in the cluster]]>
+      </doc>
+    </method>
+    <method name="getMatchingRequests" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="priority" type="org.apache.hadoop.yarn.api.records.Priority"/>
+      <param name="resourceName" type="java.lang.String"/>
+      <param name="capability" type="org.apache.hadoop.yarn.api.records.Resource"/>
+      <doc>
+      <![CDATA[Get outstanding <code>ContainerRequest</code>s matching the given 
+ parameters. These ContainerRequests should have been added via
+ <code>addContainerRequest</code> earlier in the lifecycle. For performance,
+ the AMRMClient may return its internal collection directly without creating 
+ a copy. Users should not perform mutable operations on the return value.
+ Each collection in the list contains requests with identical 
+ <code>Resource</code> size that fit in the given capability. In a 
+ collection, requests will be returned in the same order as they were added.
+ @return Collection of request matching the parameters]]>
+      </doc>
+    </method>
+    <method name="updateBlacklist"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="blacklistAdditions" type="java.util.List"/>
+      <param name="blacklistRemovals" type="java.util.List"/>
+      <doc>
+      <![CDATA[Update application's blacklist with addition or removal resources.
+ 
+ @param blacklistAdditions list of resources which should be added to the 
+        application blacklist
+ @param blacklistRemovals list of resources which should be removed from the 
+        application blacklist]]>
+      </doc>
+    </method>
+    <method name="setNMTokenCache"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="nmTokenCache" type="org.apache.hadoop.yarn.client.api.NMTokenCache"/>
+      <doc>
+      <![CDATA[Set the NM token cache for the <code>AMRMClient</code>. This cache must
+ be shared with the {@link NMClient} used to manage containers for the
+ <code>AMRMClient</code>
+ <p>
+ If a NM token cache is not set, the {@link NMTokenCache#getSingleton()}
+ singleton instance will be used.
+
+ @param nmTokenCache the NM token cache to use.]]>
+      </doc>
+    </method>
+    <method name="getNMTokenCache" return="org.apache.hadoop.yarn.client.api.NMTokenCache"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the NM token cache of the <code>AMRMClient</code>. This cache must be
+ shared with the {@link NMClient} used to manage containers for the
+ <code>AMRMClient</code>.
+ <p>
+ If a NM token cache is not set, the {@link NMTokenCache#getSingleton()}
+ singleton instance will be used.
+
+ @return the NM token cache.]]>
+      </doc>
+    </method>
+    <method name="waitFor"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="check" type="com.google.common.base.Supplier"/>
+      <exception name="InterruptedException" type="java.lang.InterruptedException"/>
+      <doc>
+      <![CDATA[Wait for <code>check</code> to return true for each 1000 ms.
+ See also {@link #waitFor(com.google.common.base.Supplier, int)}
+ and {@link #waitFor(com.google.common.base.Supplier, int, int)}
+ @param check]]>
+      </doc>
+    </method>
+    <method name="waitFor"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="check" type="com.google.common.base.Supplier"/>
+      <param name="checkEveryMillis" type="int"/>
+      <exception name="InterruptedException" type="java.lang.InterruptedException"/>
+      <doc>
+      <![CDATA[Wait for <code>check</code> to return true for each
+ <code>checkEveryMillis</code> ms.
+ See also {@link #waitFor(com.google.common.base.Supplier, int, int)}
+ @param check user defined checker
+ @param checkEveryMillis interval to call <code>check</code>]]>
+      </doc>
+    </method>
+    <method name="waitFor"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="check" type="com.google.common.base.Supplier"/>
+      <param name="checkEveryMillis" type="int"/>
+      <param name="logInterval" type="int"/>
+      <exception name="InterruptedException" type="java.lang.InterruptedException"/>
+      <doc>
+      <![CDATA[Wait for <code>check</code> to return true for each
+ <code>checkEveryMillis</code> ms. In the main loop, this method will log
+ the message "waiting in main loop" for each <code>logInterval</code> times
+ iteration to confirm the thread is alive.
+ @param check user defined checker
+ @param checkEveryMillis interval to call <code>check</code>
+ @param logInterval interval to log for each]]>
+      </doc>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.api.AMRMClient -->
+  <!-- start class org.apache.hadoop.yarn.client.api.NMClient -->
+  <class name="NMClient" extends="org.apache.hadoop.service.AbstractService"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="NMClient" type="java.lang.String"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="createNMClient" return="org.apache.hadoop.yarn.client.api.NMClient"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Create a new instance of NMClient.]]>
+      </doc>
+    </method>
+    <method name="createNMClient" return="org.apache.hadoop.yarn.client.api.NMClient"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="name" type="java.lang.String"/>
+      <doc>
+      <![CDATA[Create a new instance of NMClient.]]>
+      </doc>
+    </method>
+    <method name="startContainer" return="java.util.Map"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="container" type="org.apache.hadoop.yarn.api.records.Container"/>
+      <param name="containerLaunchContext" type="org.apache.hadoop.yarn.api.records.ContainerLaunchContext"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>Start an allocated container.</p>
+
+ <p>The <code>ApplicationMaster</code> or other applications that use the
+ client must provide the details of the allocated container, including the
+ Id, the assigned node's Id and the token via {@link Container}. In
+ addition, the AM needs to provide the {@link ContainerLaunchContext} as
+ well.</p>
+
+ @param container the allocated container
+ @param containerLaunchContext the context information needed by the
+                               <code>NodeManager</code> to launch the
+                               container
+ @return a map between the auxiliary service names and their outputs
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="increaseContainerResource"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="container" type="org.apache.hadoop.yarn.api.records.Container"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>Increase the resource of a container.</p>
+
+ <p>The <code>ApplicationMaster</code> or other applications that use the
+ client must provide the details of the container, including the Id and
+ the target resource encapsulated in the updated container token via
+ {@link Container}.
+ </p>
+
+ @param container the container with updated token
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="stopContainer"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="containerId" type="org.apache.hadoop.yarn.api.records.ContainerId"/>
+      <param name="nodeId" type="org.apache.hadoop.yarn.api.records.NodeId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>Stop an started container.</p>
+
+ @param containerId the Id of the started container
+ @param nodeId the Id of the <code>NodeManager</code>
+ 
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getContainerStatus" return="org.apache.hadoop.yarn.api.records.ContainerStatus"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="containerId" type="org.apache.hadoop.yarn.api.records.ContainerId"/>
+      <param name="nodeId" type="org.apache.hadoop.yarn.api.records.NodeId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>Query the status of a container.</p>
+
+ @param containerId the Id of the started container
+ @param nodeId the Id of the <code>NodeManager</code>
+ 
+ @return the status of a container
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="cleanupRunningContainersOnStop"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="enabled" type="boolean"/>
+      <doc>
+      <![CDATA[<p>Set whether the containers that are started by this client, and are
+ still running should be stopped when the client stops. By default, the
+ feature should be enabled.</p> However, containers will be stopped only  
+ when service is stopped. i.e. after {@link NMClient#stop()}. 
+
+ @param enabled whether the feature is enabled or not]]>
+      </doc>
+    </method>
+    <method name="setNMTokenCache"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="nmTokenCache" type="org.apache.hadoop.yarn.client.api.NMTokenCache"/>
+      <doc>
+      <![CDATA[Set the NM Token cache of the <code>NMClient</code>. This cache must be
+ shared with the {@link AMRMClient} that requested the containers managed
+ by this <code>NMClient</code>
+ <p>
+ If a NM token cache is not set, the {@link NMTokenCache#getSingleton()}
+ singleton instance will be used.
+
+ @param nmTokenCache the NM token cache to use.]]>
+      </doc>
+    </method>
+    <method name="getNMTokenCache" return="org.apache.hadoop.yarn.client.api.NMTokenCache"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the NM token cache of the <code>NMClient</code>. This cache must be
+ shared with the {@link AMRMClient} that requested the containers managed
+ by this <code>NMClient</code>
+ <p>
+ If a NM token cache is not set, the {@link NMTokenCache#getSingleton()}
+ singleton instance will be used.
+
+ @return the NM token cache]]>
+      </doc>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.api.NMClient -->
+  <!-- start class org.apache.hadoop.yarn.client.api.NMTokenCache -->
+  <class name="NMTokenCache" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="NMTokenCache"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Creates a NM token cache instance.]]>
+      </doc>
+    </constructor>
+    <method name="getSingleton" return="org.apache.hadoop.yarn.client.api.NMTokenCache"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Returns the singleton NM token cache.
+
+ @return the singleton NM token cache.]]>
+      </doc>
+    </method>
+    <method name="getNMToken" return="org.apache.hadoop.yarn.api.records.Token"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="nodeAddr" type="java.lang.String"/>
+      <doc>
+      <![CDATA[Returns NMToken, null if absent. Only the singleton obtained from
+ {@link #getSingleton()} is looked at for the tokens. If you are using your
+ own NMTokenCache that is different from the singleton, use
+ {@link #getToken(String) }
+ 
+ @param nodeAddr
+ @return {@link Token} NMToken required for communicating with node manager]]>
+      </doc>
+    </method>
+    <method name="setNMToken"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="nodeAddr" type="java.lang.String"/>
+      <param name="token" type="org.apache.hadoop.yarn.api.records.Token"/>
+      <doc>
+      <![CDATA[Sets the NMToken for node address only in the singleton obtained from
+ {@link #getSingleton()}. If you are using your own NMTokenCache that is
+ different from the singleton, use {@link #setToken(String, Token) }
+ 
+ @param nodeAddr
+          node address (host:port)
+ @param token
+          NMToken]]>
+      </doc>
+    </method>
+    <method name="getToken" return="org.apache.hadoop.yarn.api.records.Token"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="nodeAddr" type="java.lang.String"/>
+      <doc>
+      <![CDATA[Returns NMToken, null if absent
+ @param nodeAddr
+ @return {@link Token} NMToken required for communicating with node
+         manager]]>
+      </doc>
+    </method>
+    <method name="setToken"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="nodeAddr" type="java.lang.String"/>
+      <param name="token" type="org.apache.hadoop.yarn.api.records.Token"/>
+      <doc>
+      <![CDATA[Sets the NMToken for node address
+ @param nodeAddr node address (host:port)
+ @param token NMToken]]>
+      </doc>
+    </method>
+    <doc>
+    <![CDATA[NMTokenCache manages NMTokens required for an Application Master
+ communicating with individual NodeManagers.
+ <p>
+ By default Yarn client libraries {@link AMRMClient} and {@link NMClient} use
+ {@link #getSingleton()} instance of the cache.
+ <ul>
+   <li>
+     Using the singleton instance of the cache is appropriate when running a
+     single ApplicationMaster in the same JVM.
+   </li>
+   <li>
+     When using the singleton, users don't need to do anything special,
+     {@link AMRMClient} and {@link NMClient} are already set up to use the
+     default singleton {@link NMTokenCache}
+     </li>
+ </ul>
+ If running multiple Application Masters in the same JVM, a different cache
+ instance should be used for each Application Master.
+ <ul>
+   <li>
+     If using the {@link AMRMClient} and the {@link NMClient}, setting up
+     and using an instance cache is as follows:
+ <pre>
+   NMTokenCache nmTokenCache = new NMTokenCache();
+   AMRMClient rmClient = AMRMClient.createAMRMClient();
+   NMClient nmClient = NMClient.createNMClient();
+   nmClient.setNMTokenCache(nmTokenCache);
+   ...
+ </pre>
+   </li>
+   <li>
+     If using the {@link AMRMClientAsync} and the {@link NMClientAsync},
+     setting up and using an instance cache is as follows:
+ <pre>
+   NMTokenCache nmTokenCache = new NMTokenCache();
+   AMRMClient rmClient = AMRMClient.createAMRMClient();
+   NMClient nmClient = NMClient.createNMClient();
+   nmClient.setNMTokenCache(nmTokenCache);
+   AMRMClientAsync rmClientAsync = new AMRMClientAsync(rmClient, 1000, [AMRM_CALLBACK]);
+   NMClientAsync nmClientAsync = new NMClientAsync("nmClient", nmClient, [NM_CALLBACK]);
+   ...
+ </pre>
+   </li>
+   <li>
+     If using {@link ApplicationMasterProtocol} and
+     {@link ContainerManagementProtocol} directly, setting up and using an
+     instance cache is as follows:
+ <pre>
+   NMTokenCache nmTokenCache = new NMTokenCache();
+   ...
+   ApplicationMasterProtocol amPro = ClientRMProxy.createRMProxy(conf, ApplicationMasterProtocol.class);
+   ...
+   AllocateRequest allocateRequest = ...
+   ...
+   AllocateResponse allocateResponse = rmClient.allocate(allocateRequest);
+   for (NMToken token : allocateResponse.getNMTokens()) {
+     nmTokenCache.setToken(token.getNodeId().toString(), token.getToken());
+   }
+   ...
+   ContainerManagementProtocolProxy nmPro = ContainerManagementProtocolProxy(conf, nmTokenCache);
+   ...
+   nmPro.startContainer(container, containerContext);
+   ...
+ </pre>
+   </li>
+ </ul>
+ It is also possible to mix the usage of a client ({@code AMRMClient} or
+ {@code NMClient}, or the async versions of them) with a protocol proxy
+ ({@code ContainerManagementProtocolProxy} or
+ {@code ApplicationMasterProtocol}).]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.api.NMTokenCache -->
+  <!-- start class org.apache.hadoop.yarn.client.api.SharedCacheClient -->
+  <class name="SharedCacheClient" extends="org.apache.hadoop.service.AbstractService"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="SharedCacheClient" type="java.lang.String"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="createSharedCacheClient" return="org.apache.hadoop.yarn.client.api.SharedCacheClient"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="use" return="org.apache.hadoop.fs.Path"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationId" type="org.apache.hadoop.yarn.api.records.ApplicationId"/>
+      <param name="resourceKey" type="java.lang.String"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <doc>
+      <![CDATA[<p>
+ The method to claim a resource with the <code>SharedCacheManager.</code>
+ The client uses a checksum to identify the resource and an
+ {@link ApplicationId} to identify which application will be using the
+ resource.
+ </p>
+ 
+ <p>
+ The <code>SharedCacheManager</code> responds with whether or not the
+ resource exists in the cache. If the resource exists, a <code>Path</code>
+ to the resource in the shared cache is returned. If the resource does not
+ exist, null is returned instead.
+ </p>
+ 
+ @param applicationId ApplicationId of the application using the resource
+ @param resourceKey the key (i.e. checksum) that identifies the resource
+ @return Path to the resource, or null if it does not exist]]>
+      </doc>
+    </method>
+    <method name="release"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationId" type="org.apache.hadoop.yarn.api.records.ApplicationId"/>
+      <param name="resourceKey" type="java.lang.String"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <doc>
+      <![CDATA[<p>
+ The method to release a resource with the <code>SharedCacheManager.</code>
+ This method is called once an application is no longer using a claimed
+ resource in the shared cache. The client uses a checksum to identify the
+ resource and an {@link ApplicationId} to identify which application is
+ releasing the resource.
+ </p>
+ 
+ <p>
+ Note: This method is an optimization and the client is not required to call
+ it for correctness.
+ </p>
+ 
+ @param applicationId ApplicationId of the application releasing the
+          resource
+ @param resourceKey the key (i.e. checksum) that identifies the resource]]>
+      </doc>
+    </method>
+    <method name="getFileChecksum" return="java.lang.String"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="sourceFile" type="org.apache.hadoop.fs.Path"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[A convenience method to calculate the checksum of a specified file.
+ 
+ @param sourceFile A path to the input file
+ @return A hex string containing the checksum digest
+ @throws IOException]]>
+      </doc>
+    </method>
+    <doc>
+    <![CDATA[This is the client for YARN's shared cache.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.api.SharedCacheClient -->
+  <!-- start class org.apache.hadoop.yarn.client.api.YarnClient -->
+  <class name="YarnClient" extends="org.apache.hadoop.service.AbstractService"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="YarnClient" type="java.lang.String"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="createYarnClient" return="org.apache.hadoop.yarn.client.api.YarnClient"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Create a new instance of YarnClient.]]>
+      </doc>
+    </method>
+    <method name="createApplication" return="org.apache.hadoop.yarn.client.api.YarnClientApplication"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Obtain a {@link YarnClientApplication} for a new application,
+ which in turn contains the {@link ApplicationSubmissionContext} and
+ {@link org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse}
+ objects.
+ </p>
+
+ @return {@link YarnClientApplication} built for a new application
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="submitApplication" return="org.apache.hadoop.yarn.api.records.ApplicationId"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appContext" type="org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Submit a new application to <code>YARN.</code> It is a blocking call - it
+ will not return {@link ApplicationId} until the submitted application is
+ submitted successfully and accepted by the ResourceManager.
+ </p>
+ 
+ <p>
+ Users should provide an {@link ApplicationId} as part of the parameter
+ {@link ApplicationSubmissionContext} when submitting a new application,
+ otherwise it will throw the {@link ApplicationIdNotProvidedException}.
+ </p>
+
+ <p>This internally calls {@link ApplicationClientProtocol#submitApplication
+ (SubmitApplicationRequest)}, and after that, it internally invokes
+ {@link ApplicationClientProtocol#getApplicationReport
+ (GetApplicationReportRequest)} and waits till it can make sure that the
+ application gets properly submitted. If RM fails over or RM restart
+ happens before ResourceManager saves the application's state,
+ {@link ApplicationClientProtocol
+ #getApplicationReport(GetApplicationReportRequest)} will throw
+ the {@link ApplicationNotFoundException}. This API automatically resubmits
+ the application with the same {@link ApplicationSubmissionContext} when it
+ catches the {@link ApplicationNotFoundException}</p>
+
+ @param appContext
+          {@link ApplicationSubmissionContext} containing all the details
+          needed to submit a new application
+ @return {@link ApplicationId} of the accepted application
+ @throws YarnException
+ @throws IOException
+ @see #createApplication()]]>
+      </doc>
+    </method>
+    <method name="failApplicationAttempt"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationAttemptId" type="org.apache.hadoop.yarn.api.records.ApplicationAttemptId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Fail an application attempt identified by given ID.
+ </p>
+
+ @param applicationAttemptId
+          {@link ApplicationAttemptId} of the attempt to fail.
+ @throws YarnException
+           in case of errors or if YARN rejects the request due to
+           access-control restrictions.
+ @throws IOException
+ @see #getQueueAclsInfo()]]>
+      </doc>
+    </method>
+    <method name="killApplication"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationId" type="org.apache.hadoop.yarn.api.records.ApplicationId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Kill an application identified by given ID.
+ </p>
+ 
+ @param applicationId
+          {@link ApplicationId} of the application that needs to be killed
+ @throws YarnException
+           in case of errors or if YARN rejects the request due to
+           access-control restrictions.
+ @throws IOException
+ @see #getQueueAclsInfo()]]>
+      </doc>
+    </method>
+    <method name="killApplication"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationId" type="org.apache.hadoop.yarn.api.records.ApplicationId"/>
+      <param name="diagnostics" type="java.lang.String"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Kill an application identified by given ID.
+ </p>
+ @param applicationId {@link ApplicationId} of the application that needs to
+          be killed
+ @param diagnostics for killing an application.
+ @throws YarnException in case of errors or if YARN rejects the request due
+           to access-control restrictions.
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getApplicationReport" return="org.apache.hadoop.yarn.api.records.ApplicationReport"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appId" type="org.apache.hadoop.yarn.api.records.ApplicationId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report of the given Application.
+ </p>
+ 
+ <p>
+ In secure mode, <code>YARN</code> verifies access to the application, queue
+ etc. before accepting the request.
+ </p>
+ 
+ <p>
+ If the user does not have <code>VIEW_APP</code> access then the following
+ fields in the report will be set to stubbed values:
+ <ul>
+ <li>host - set to "N/A"</li>
+ <li>RPC port - set to -1</li>
+ <li>client token - set to "N/A"</li>
+ <li>diagnostics - set to "N/A"</li>
+ <li>tracking URL - set to "N/A"</li>
+ <li>original tracking URL - set to "N/A"</li>
+ <li>resource usage report - all values are -1</li>
+ </ul>
+ 
+ @param appId
+          {@link ApplicationId} of the application that needs a report
+ @return application report
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getAMRMToken" return="org.apache.hadoop.security.token.Token"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appId" type="org.apache.hadoop.yarn.api.records.ApplicationId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Get the AMRM token of the application.
+ <p>
+ The AMRM token is required for AM to RM scheduling operations. For 
+ managed Application Masters Yarn takes care of injecting it. For unmanaged
+ Applications Masters, the token must be obtained via this method and set
+ in the {@link org.apache.hadoop.security.UserGroupInformation} of the
+ current user.
+ <p>
+ The AMRM token will be returned only if all the following conditions are
+ met:
+ <ul>
+   <li>the requester is the owner of the ApplicationMaster</li>
+   <li>the application master is an unmanaged ApplicationMaster</li>
+   <li>the application master is in ACCEPTED state</li>
+ </ul>
+ Else this method returns NULL.
+
+ @param appId {@link ApplicationId} of the application to get the AMRM token
+ @return the AMRM token if available
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getApplications" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report (ApplicationReport) of all Applications in the cluster.
+ </p>
+
+ <p>
+ If the user does not have <code>VIEW_APP</code> access for an application
+ then the corresponding report will be filtered as described in
+ {@link #getApplicationReport(ApplicationId)}.
+ </p>
+
+ @return a list of reports of all running applications
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getApplications" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationTypes" type="java.util.Set"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report (ApplicationReport) of Applications
+ matching the given application types in the cluster.
+ </p>
+
+ <p>
+ If the user does not have <code>VIEW_APP</code> access for an application
+ then the corresponding report will be filtered as described in
+ {@link #getApplicationReport(ApplicationId)}.
+ </p>
+
+ @param applicationTypes set of application types you are interested in
+ @return a list of reports of applications
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getApplications" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationStates" type="java.util.EnumSet"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report (ApplicationReport) of Applications matching the given
+ application states in the cluster.
+ </p>
+
+ <p>
+ If the user does not have <code>VIEW_APP</code> access for an application
+ then the corresponding report will be filtered as described in
+ {@link #getApplicationReport(ApplicationId)}.
+ </p>
+
+ @param applicationStates set of application states you are interested in
+ @return a list of reports of applications
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getApplications" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationTypes" type="java.util.Set"/>
+      <param name="applicationStates" type="java.util.EnumSet"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report (ApplicationReport) of Applications matching the given
+ application types and application states in the cluster.
+ </p>
+
+ <p>
+ If the user does not have <code>VIEW_APP</code> access for an application
+ then the corresponding report will be filtered as described in
+ {@link #getApplicationReport(ApplicationId)}.
+ </p>
+
+ @param applicationTypes set of application types you are interested in
+ @param applicationStates set of application states you are interested in
+ @return a list of reports of applications
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getApplications" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="queues" type="java.util.Set"/>
+      <param name="users" type="java.util.Set"/>
+      <param name="applicationTypes" type="java.util.Set"/>
+      <param name="applicationStates" type="java.util.EnumSet"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report (ApplicationReport) of Applications matching the given users,
+ queues, application types and application states in the cluster. If any of
+ the params is set to null, it is not used when filtering.
+ </p>
+
+ <p>
+ If the user does not have <code>VIEW_APP</code> access for an application
+ then the corresponding report will be filtered as described in
+ {@link #getApplicationReport(ApplicationId)}.
+ </p>
+
+ @param queues set of queues you are interested in
+ @param users set of users you are interested in
+ @param applicationTypes set of application types you are interested in
+ @param applicationStates set of application states you are interested in
+ @return a list of reports of applications
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getYarnClusterMetrics" return="org.apache.hadoop.yarn.api.records.YarnClusterMetrics"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get metrics ({@link YarnClusterMetrics}) about the cluster.
+ </p>
+ 
+ @return cluster metrics
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getNodeReports" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="states" type="org.apache.hadoop.yarn.api.records.NodeState[]"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report of nodes ({@link NodeReport}) in the cluster.
+ </p>
+ 
+ @param states The {@link NodeState}s to filter on. If no filter states are
+          given, nodes in all states will be returned.
+ @return A list of node reports
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getRMDelegationToken" return="org.apache.hadoop.yarn.api.records.Token"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="renewer" type="org.apache.hadoop.io.Text"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a delegation token so as to be able to talk to YARN using those tokens.
+ 
+ @param renewer
+          Address of the renewer who can renew these tokens when needed by
+          securely talking to YARN.
+ @return a delegation token ({@link Token}) that can be used to
+         talk to YARN
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getQueueInfo" return="org.apache.hadoop.yarn.api.records.QueueInfo"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="queueName" type="java.lang.String"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get information ({@link QueueInfo}) about a given <em>queue</em>.
+ </p>
+ 
+ @param queueName
+          Name of the queue whose information is needed
+ @return queue information
+ @throws YarnException
+           in case of errors or if YARN rejects the request due to
+           access-control restrictions.
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getAllQueues" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get information ({@link QueueInfo}) about all queues, recursively if there
+ is a hierarchy
+ </p>
+ 
+ @return a list of queue-information for all queues
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getRootQueueInfos" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get information ({@link QueueInfo}) about top level queues.
+ </p>
+ 
+ @return a list of queue-information for all the top-level queues
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getChildQueueInfos" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="parent" type="java.lang.String"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get information ({@link QueueInfo}) about all the immediate children queues
+ of the given queue
+ </p>
+ 
+ @param parent
+          Name of the queue whose child-queues' information is needed
+ @return a list of queue-information for all queues who are direct children
+         of the given parent queue.
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getQueueAclsInfo" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get information about <em>acls</em> for <em>current user</em> on all the
+ existing queues.
+ </p>
+ 
+ @return a list of queue acls ({@link QueueUserACLInfo}) for
+         <em>current user</em>
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getApplicationAttemptReport" return="org.apache.hadoop.yarn.api.records.ApplicationAttemptReport"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationAttemptId" type="org.apache.hadoop.yarn.api.records.ApplicationAttemptId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report of the given ApplicationAttempt.
+ </p>
+ 
+ <p>
+ In secure mode, <code>YARN</code> verifies access to the application, queue
+ etc. before accepting the request.
+ </p>
+ 
+ @param applicationAttemptId
+          {@link ApplicationAttemptId} of the application attempt that needs
+          a report
+ @return application attempt report
+ @throws YarnException
+ @throws ApplicationAttemptNotFoundException if application attempt
+         not found
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getApplicationAttempts" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationId" type="org.apache.hadoop.yarn.api.records.ApplicationId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report of all (ApplicationAttempts) of Application in the cluster.
+ </p>
+ 
+ @param applicationId application id of the app
+ @return a list of reports for all application attempts for specified
+         application.
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getContainerReport" return="org.apache.hadoop.yarn.api.records.ContainerReport"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="containerId" type="org.apache.hadoop.yarn.api.records.ContainerId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report of the given Container.
+ </p>
+ 
+ <p>
+ In secure mode, <code>YARN</code> verifies access to the application, queue
+ etc. before accepting the request.
+ </p>
+ 
+ @param containerId
+          {@link ContainerId} of the container that needs a report
+ @return container report
+ @throws YarnException
+ @throws ContainerNotFoundException if container not found.
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getContainers" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationAttemptId" type="org.apache.hadoop.yarn.api.records.ApplicationAttemptId"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a report of all (Containers) of ApplicationAttempt in the cluster.
+ </p>
+ 
+ @param applicationAttemptId application attempt id
+ @return a list of reports of all containers for specified application
+         attempts
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="moveApplicationAcrossQueues"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appId" type="org.apache.hadoop.yarn.api.records.ApplicationId"/>
+      <param name="queue" type="java.lang.String"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Attempts to move the given application to the given queue.
+ </p>
+ 
+ @param appId
+    Application to move.
+ @param queue
+    Queue to place it in to.
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="createReservation" return="org.apache.hadoop.yarn.api.protocolrecords.GetNewReservationResponse"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Obtain a {@link GetNewReservationResponse} for a new reservation,
+ which contains the {@link ReservationId} object.
+ </p>
+
+ @return The {@link GetNewReservationResponse} containing a new
+         {@link ReservationId} object.
+ @throws YarnException if reservation cannot be created.
+ @throws IOException if reservation cannot be created.]]>
+      </doc>
+    </method>
+    <method name="submitReservation" return="org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionResponse"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ The interface used by clients to submit a new reservation to the
+ {@code ResourceManager}.
+ </p>
+ 
+ <p>
+ The client packages all details of its request in a
+ {@link ReservationSubmissionRequest} object. This contains information
+ about the amount of capacity, temporal constraints, and gang needs.
+ Furthermore, the reservation might be composed of multiple stages, with
+ ordering dependencies among them.
+ </p>
+ 
+ <p>
+ In order to respond, a new admission control component in the
+ {@code ResourceManager} performs an analysis of the resources that have
+ been committed over the period of time the user is requesting, verify that
+ the user requests can be fulfilled, and that it respect a sharing policy
+ (e.g., {@code CapacityOverTimePolicy}). Once it has positively determined
+ that the ReservationRequest is satisfiable the {@code ResourceManager}
+ answers with a {@link ReservationSubmissionResponse} that includes a
+ {@link ReservationId}. Upon failure to find a valid allocation the response
+ is an exception with the message detailing the reason of failure.
+ </p>
+ 
+ <p>
+ The semantics guarantees that the {@link ReservationId} returned,
+ corresponds to a valid reservation existing in the time-range request by
+ the user. The amount of capacity dedicated to such reservation can vary
+ overtime, depending of the allocation that has been determined. But it is
+ guaranteed to satisfy all the constraint expressed by the user in the
+ {@link ReservationDefinition}
+ </p>
+ 
+ @param request request to submit a new Reservation
+ @return response contains the {@link ReservationId} on accepting the
+         submission
+ @throws YarnException if the reservation cannot be created successfully
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="updateReservation" return="org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateResponse"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ The interface used by clients to update an existing Reservation. This is
+ referred to as a re-negotiation process, in which a user that has
+ previously submitted a Reservation.
+ </p>
+ 
+ <p>
+ The allocation is attempted by virtually substituting all previous
+ allocations related to this Reservation with new ones, that satisfy the new
+ {@link ReservationDefinition}. Upon success the previous allocation is
+ atomically substituted by the new one, and on failure (i.e., if the system
+ cannot find a valid allocation for the updated request), the previous
+ allocation remains valid.
+ </p>
+ 
+ @param request to update an existing Reservation (the
+          {@link ReservationUpdateRequest} should refer to an existing valid
+          {@link ReservationId})
+ @return response empty on successfully updating the existing reservation
+ @throws YarnException if the request is invalid or reservation cannot be
+           updated successfully
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="deleteReservation" return="org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteResponse"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ The interface used by clients to remove an existing Reservation.
+ </p>
+ 
+ @param request to remove an existing Reservation (the
+          {@link ReservationDeleteRequest} should refer to an existing valid
+          {@link ReservationId})
+ @return response empty on successfully deleting the existing reservation
+ @throws YarnException if the request is invalid or reservation cannot be
+           deleted successfully
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="listReservations" return="org.apache.hadoop.yarn.api.protocolrecords.ReservationListResponse"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="request" type="org.apache.hadoop.yarn.api.protocolrecords.ReservationListRequest"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ The interface used by clients to get the list of reservations in a plan.
+ The reservationId will be used to search for reservations to list if it is
+ provided. Otherwise, it will select active reservations within the
+ startTime and endTime (inclusive).
+ </p>
+
+ @param request to list reservations in a plan. Contains fields to select
+                String queue, ReservationId reservationId, long startTime,
+                long endTime, and a bool includeReservationAllocations.
+
+                queue: Required. Cannot be null or empty. Refers to the
+                reservable queue in the scheduler that was selected when
+                creating a reservation submission
+                {@link ReservationSubmissionRequest}.
+
+                reservationId: Optional. If provided, other fields will
+                be ignored.
+
+                startTime: Optional. If provided, only reservations that
+                end after the startTime will be selected. This defaults
+                to 0 if an invalid number is used.
+
+                endTime: Optional. If provided, only reservations that
+                start on or before endTime will be selected. This defaults
+                to Long.MAX_VALUE if an invalid number is used.
+
+                includeReservationAllocations: Optional. Flag that
+                determines whether the entire reservation allocations are
+                to be returned. Reservation allocations are subject to
+                change in the event of re-planning as described by
+                {@link ReservationDefinition}.
+
+ @return response that contains information about reservations that are
+                being searched for.
+ @throws YarnException if the request is invalid
+ @throws IOException if the request failed otherwise]]>
+      </doc>
+    </method>
+    <method name="getNodeToLabels" return="java.util.Map"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ The interface used by client to get node to labels mappings in existing cluster
+ </p>
+ 
+ @return node to labels mappings
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getLabelsToNodes" return="java.util.Map"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ The interface used by client to get labels to nodes mapping
+ in existing cluster
+ </p>
+
+ @return node to labels mappings
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getLabelsToNodes" return="java.util.Map"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="labels" type="java.util.Set"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ The interface used by client to get labels to nodes mapping
+ for specified labels in existing cluster
+ </p>
+
+ @param labels labels for which labels to nodes mapping has to be retrieved
+ @return labels to nodes mappings for specific labels
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getClusterNodeLabels" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ The interface used by client to get node labels in the cluster
+ </p>
+
+ @return cluster node labels collection
+ @throws YarnException when there is a failure in
+           {@link ApplicationClientProtocol}
+ @throws IOException when there is a failure in
+           {@link ApplicationClientProtocol}]]>
+      </doc>
+    </method>
+    <method name="updateApplicationPriority" return="org.apache.hadoop.yarn.api.records.Priority"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationId" type="org.apache.hadoop.yarn.api.records.ApplicationId"/>
+      <param name="priority" type="org.apache.hadoop.yarn.api.records.Priority"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ The interface used by client to set priority of an application
+ </p>
+ @param applicationId
+ @param priority
+ @return updated priority of an application.
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="signalToContainer"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="containerId" type="org.apache.hadoop.yarn.api.records.ContainerId"/>
+      <param name="command" type="org.apache.hadoop.yarn.api.records.SignalContainerCommand"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[<p>
+ Signal a container identified by given ID.
+ </p>
+
+ @param containerId
+          {@link ContainerId} of the container that needs to be signaled
+ @param command the signal container command
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.api.YarnClient -->
+  <!-- start class org.apache.hadoop.yarn.client.api.YarnClientApplication -->
+  <class name="YarnClientApplication" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="YarnClientApplication" type="org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse, org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="getNewApplicationResponse" return="org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getApplicationSubmissionContext" return="org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.api.YarnClientApplication -->
+</package>
+<package name="org.apache.hadoop.yarn.client.api.async">
+  <!-- start class org.apache.hadoop.yarn.client.api.async.AMRMClientAsync -->
+  <class name="AMRMClientAsync" extends="org.apache.hadoop.service.AbstractService"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="AMRMClientAsync" type="int, org.apache.hadoop.yarn.client.api.async.AMRMClientAsync.AbstractCallbackHandler"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="AMRMClientAsync" type="org.apache.hadoop.yarn.client.api.AMRMClient, int, org.apache.hadoop.yarn.client.api.async.AMRMClientAsync.AbstractCallbackHandler"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="AMRMClientAsync" type="int, org.apache.hadoop.yarn.client.api.async.AMRMClientAsync.CallbackHandler"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="AMRMClientAsync" type="org.apache.hadoop.yarn.client.api.AMRMClient, int, org.apache.hadoop.yarn.client.api.async.AMRMClientAsync.CallbackHandler"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="createAMRMClientAsync" return="org.apache.hadoop.yarn.client.api.async.AMRMClientAsync"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="intervalMs" type="int"/>
+      <param name="callbackHandler" type="org.apache.hadoop.yarn.client.api.async.AMRMClientAsync.AbstractCallbackHandler"/>
+      <doc>
+      <![CDATA[<p>Create a new instance of AMRMClientAsync.</p>
+
+ @param intervalMs heartbeat interval in milliseconds between AM and RM
+ @param callbackHandler callback handler that processes responses from
+                        the <code>ResourceManager</code>]]>
+      </doc>
+    </method>
+    <method name="createAMRMClientAsync" return="org.apache.hadoop.yarn.client.api.async.AMRMClientAsync"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="client" type="org.apache.hadoop.yarn.client.api.AMRMClient"/>
+      <param name="intervalMs" type="int"/>
+      <param name="callbackHandler" type="org.apache.hadoop.yarn.client.api.async.AMRMClientAsync.AbstractCallbackHandler"/>
+      <doc>
+      <![CDATA[<p>Create a new instance of AMRMClientAsync.</p>
+
+ @param client the AMRMClient instance
+ @param intervalMs heartbeat interval in milliseconds between AM and RM
+ @param callbackHandler callback handler that processes responses from
+                        the <code>ResourceManager</code>]]>
+      </doc>
+    </method>
+    <method name="createAMRMClientAsync" return="org.apache.hadoop.yarn.client.api.async.AMRMClientAsync"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="Use {@link #createAMRMClientAsync(int,
+             AMRMClientAsync.AbstractCallbackHandler)} instead.">
+      <param name="intervalMs" type="int"/>
+      <param name="callbackHandler" type="org.apache.hadoop.yarn.client.api.async.AMRMClientAsync.CallbackHandler"/>
+      <doc>
+      <![CDATA[@deprecated Use {@link #createAMRMClientAsync(int,
+             AMRMClientAsync.AbstractCallbackHandler)} instead.]]>
+      </doc>
+    </method>
+    <method name="createAMRMClientAsync" return="org.apache.hadoop.yarn.client.api.async.AMRMClientAsync"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="Use {@link #createAMRMClientAsync(AMRMClient,
+             int, AMRMClientAsync.AbstractCallbackHandler)} instead.">
+      <param name="client" type="org.apache.hadoop.yarn.client.api.AMRMClient"/>
+      <param name="intervalMs" type="int"/>
+      <param name="callbackHandler" type="org.apache.hadoop.yarn.client.api.async.AMRMClientAsync.CallbackHandler"/>
+      <doc>
+      <![CDATA[@deprecated Use {@link #createAMRMClientAsync(AMRMClient,
+             int, AMRMClientAsync.AbstractCallbackHandler)} instead.]]>
+      </doc>
+    </method>
+    <method name="setHeartbeatInterval"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="interval" type="int"/>
+    </method>
+    <method name="getMatchingRequests" return="java.util.List"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="priority" type="org.apache.hadoop.yarn.api.records.Priority"/>
+      <param name="resourceName" type="java.lang.String"/>
+      <param name="capability" type="org.apache.hadoop.yarn.api.records.Resource"/>
+    </method>
+    <method name="registerApplicationMaster" return="org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appHostName" type="java.lang.String"/>
+      <param name="appHostPort" type="int"/>
+      <param name="appTrackingUrl" type="java.lang.String"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Registers this application master with the resource manager. On successful
+ registration, starts the heartbeating thread.
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="unregisterApplicationMaster"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appStatus" type="org.apache.hadoop.yarn.api.records.FinalApplicationStatus"/>
+      <param name="appMessage" type="java.lang.String"/>
+      <param name="appTrackingUrl" type="java.lang.String"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Unregister the application master. This must be called in the end.
+ @param appStatus Success/Failure status of the master
+ @param appMessage Diagnostics message on failure
+ @param appTrackingUrl New URL to get master info
+ @throws YarnException
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="addContainerRequest"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="req" type="T"/>
+      <doc>

<TRUNCATED>

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


[30/50] [abbrv] hadoop git commit: Add 2.8.3 release jdiff files.

Posted by na...@apache.org.
Add 2.8.3 release jdiff files.

(cherry picked from commit c89f99aade575ab1f6a9836df719cda272293d90)


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

Branch: refs/heads/yarn-3409
Commit: a7f8caf58ed47574861d455a9d9e1521e35c10b9
Parents: 25a36b7
Author: Junping Du <ju...@apache.org>
Authored: Mon Dec 18 22:30:32 2017 -0800
Committer: Junping Du <ju...@apache.org>
Committed: Mon Dec 18 22:39:01 2017 -0800

----------------------------------------------------------------------
 .../jdiff/Apache_Hadoop_Common_2.8.3.xml        | 38433 +++++++++++++++++
 .../jdiff/Apache_Hadoop_HDFS_2.8.3.xml          |   312 +
 .../Apache_Hadoop_MapReduce_Common_2.8.3.xml    |   113 +
 .../Apache_Hadoop_MapReduce_Core_2.8.3.xml      | 27495 ++++++++++++
 .../Apache_Hadoop_MapReduce_JobClient_2.8.3.xml |    16 +
 .../jdiff/Apache_Hadoop_YARN_Client_2.8.3.xml   |  2316 +
 .../jdiff/Apache_Hadoop_YARN_Common_2.8.3.xml   |  2665 ++
 .../Apache_Hadoop_YARN_Server_Common_2.8.3.xml  |   829 +
 8 files changed, 72179 insertions(+)
----------------------------------------------------------------------



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


[31/50] [abbrv] hadoop git commit: HDFS-12937. RBF: Add more unit tests for router admin commands. Contributed by Yiqun Lin.

Posted by na...@apache.org.
HDFS-12937. RBF: Add more unit tests for router admin commands. 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/e040c97b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/e040c97b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/e040c97b

Branch: refs/heads/yarn-3409
Commit: e040c97b7743469f363eeae52c8abcf4fe7c65d5
Parents: a7f8caf
Author: Yiqun Lin <yq...@apache.org>
Authored: Tue Dec 19 15:31:34 2017 +0800
Committer: Yiqun Lin <yq...@apache.org>
Committed: Tue Dec 19 15:31:34 2017 +0800

----------------------------------------------------------------------
 .../hdfs/tools/federation/RouterAdmin.java      |   4 +-
 .../federation/router/TestRouterAdminCLI.java   | 121 ++++++++++++++++++-
 2 files changed, 120 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e040c97b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/federation/RouterAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/federation/RouterAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/federation/RouterAdmin.java
index a91a602..fd961f2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/federation/RouterAdmin.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/federation/RouterAdmin.java
@@ -132,11 +132,11 @@ public class RouterAdmin extends Configured implements Tool {
     try {
       if ("-add".equals(cmd)) {
         if (addMount(argv, i)) {
-          System.err.println("Successfuly added mount point " + argv[i]);
+          System.out.println("Successfuly added mount point " + argv[i]);
         }
       } else if ("-rm".equals(cmd)) {
         if (removeMount(argv[i])) {
-          System.err.println("Successfully removed mount point " + argv[i]);
+          System.out.println("Successfully removed mount point " + argv[i]);
         }
       } else if ("-ls".equals(cmd)) {
         if (argv.length > 1) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e040c97b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterAdminCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterAdminCLI.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterAdminCLI.java
index 3882b8b..9e82967 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterAdminCLI.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterAdminCLI.java
@@ -18,16 +18,20 @@
 package org.apache.hadoop.hdfs.server.federation.router;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 
 import java.io.ByteArrayOutputStream;
 import java.io.PrintStream;
 import java.net.InetSocketAddress;
+import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.server.federation.RouterConfigBuilder;
 import org.apache.hadoop.hdfs.server.federation.RouterDFSCluster.RouterContext;
 import org.apache.hadoop.hdfs.server.federation.StateStoreDFSCluster;
+import org.apache.hadoop.hdfs.server.federation.resolver.RemoteLocation;
 import org.apache.hadoop.hdfs.server.federation.store.StateStoreService;
 import org.apache.hadoop.hdfs.server.federation.store.impl.MountTableStoreImpl;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.GetMountTableEntriesRequest;
@@ -36,6 +40,7 @@ import org.apache.hadoop.hdfs.server.federation.store.records.MountTable;
 import org.apache.hadoop.hdfs.tools.federation.RouterAdmin;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.ToolRunner;
+import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
@@ -83,12 +88,123 @@ public class TestRouterAdminCLI {
   }
 
   @AfterClass
-  public static void tearDown() {
+  public static void tearDownCluster() {
     cluster.stopRouter(routerContext);
     cluster.shutdown();
     cluster = null;
   }
 
+  @After
+  public void tearDown() {
+    // set back system out
+    System.setOut(OLD_OUT);
+  }
+
+  @Test
+  public void testAddMountTable() throws Exception {
+    String nsId = "ns0";
+    String src = "/test-addmounttable";
+    String dest = "/addmounttable";
+    String[] argv = new String[] {"-add", src, nsId, dest};
+    Assert.assertEquals(0, ToolRunner.run(admin, argv));
+
+    stateStore.loadCache(MountTableStoreImpl.class, true);
+    GetMountTableEntriesRequest getRequest = GetMountTableEntriesRequest
+        .newInstance(src);
+    GetMountTableEntriesResponse getResponse = client.getMountTableManager()
+        .getMountTableEntries(getRequest);
+    MountTable mountTable = getResponse.getEntries().get(0);
+
+    List<RemoteLocation> destinations = mountTable.getDestinations();
+    assertEquals(1, destinations.size());
+
+    assertEquals(src, mountTable.getSourcePath());
+    assertEquals(nsId, destinations.get(0).getNameserviceId());
+    assertEquals(dest, destinations.get(0).getDest());
+    assertFalse(mountTable.isReadOnly());
+
+    // test mount table update behavior
+    dest = dest + "-new";
+    argv = new String[] {"-add", src, nsId, dest, "-readonly"};
+    Assert.assertEquals(0, ToolRunner.run(admin, argv));
+    stateStore.loadCache(MountTableStoreImpl.class, true);
+
+    getResponse = client.getMountTableManager()
+        .getMountTableEntries(getRequest);
+    mountTable = getResponse.getEntries().get(0);
+    assertEquals(2, mountTable.getDestinations().size());
+    assertEquals(nsId, mountTable.getDestinations().get(1).getNameserviceId());
+    assertEquals(dest, mountTable.getDestinations().get(1).getDest());
+    assertTrue(mountTable.isReadOnly());
+  }
+
+  @Test
+  public void testListMountTable() throws Exception {
+    String nsId = "ns0";
+    String src = "/test-lsmounttable";
+    String dest = "/lsmounttable";
+    String[] argv = new String[] {"-add", src, nsId, dest};
+    assertEquals(0, ToolRunner.run(admin, argv));
+
+    // re-set system out for testing
+    System.setOut(new PrintStream(out));
+    stateStore.loadCache(MountTableStoreImpl.class, true);
+    argv = new String[] {"-ls", src};
+    assertEquals(0, ToolRunner.run(admin, argv));
+    assertTrue(out.toString().contains(src));
+
+    out.reset();
+    GetMountTableEntriesRequest getRequest = GetMountTableEntriesRequest
+        .newInstance("/");
+    GetMountTableEntriesResponse getResponse = client.getMountTableManager()
+        .getMountTableEntries(getRequest);
+
+    // Test ls command without input path, it will list
+    // mount table under root path.
+    argv = new String[] {"-ls"};
+    assertEquals(0, ToolRunner.run(admin, argv));
+    assertTrue(out.toString().contains(src));
+    String outStr = out.toString();
+    // verify if all the mount table are listed
+    for(MountTable entry: getResponse.getEntries()) {
+      assertTrue(outStr.contains(entry.getSourcePath()));
+    }
+  }
+
+  @Test
+  public void testRemoveMountTable() throws Exception {
+    String nsId = "ns0";
+    String src = "/test-rmmounttable";
+    String dest = "/rmmounttable";
+    String[] argv = new String[] {"-add", src, nsId, dest};
+    assertEquals(0, ToolRunner.run(admin, argv));
+
+    stateStore.loadCache(MountTableStoreImpl.class, true);
+    GetMountTableEntriesRequest getRequest = GetMountTableEntriesRequest
+        .newInstance(src);
+    GetMountTableEntriesResponse getResponse = client.getMountTableManager()
+        .getMountTableEntries(getRequest);
+    // ensure mount table added successfully
+    MountTable mountTable = getResponse.getEntries().get(0);
+    assertEquals(src, mountTable.getSourcePath());
+
+    argv = new String[] {"-rm", src};
+    assertEquals(0, ToolRunner.run(admin, argv));
+
+    stateStore.loadCache(MountTableStoreImpl.class, true);
+    getResponse = client.getMountTableManager()
+        .getMountTableEntries(getRequest);
+    assertEquals(0, getResponse.getEntries().size());
+
+    // remove an invalid mount table
+    String invalidPath = "/invalid";
+    System.setOut(new PrintStream(out));
+    argv = new String[] {"-rm", invalidPath};
+    assertEquals(0, ToolRunner.run(admin, argv));
+    assertTrue(out.toString().contains(
+        "Cannot remove mount point " + invalidPath));
+  }
+
   @Test
   public void testMountTableDefaultACL() throws Exception {
     String[] argv = new String[] {"-add", "/testpath0", "ns0", "/testdir0"};
@@ -140,8 +256,7 @@ public class TestRouterAdminCLI {
     assertEquals(0, ToolRunner.run(admin, argv));
     verifyExecutionResult("/testpath2-3", true, 0, 0);
 
-    // set back system out and login user
-    System.setOut(OLD_OUT);
+    // set back login user
     remoteUser = UserGroupInformation.createRemoteUser(superUser);
     UserGroupInformation.setLoginUser(remoteUser);
   }


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


[12/50] [abbrv] hadoop git commit: Revert "HDFS-12903. [READ] Fix closing streams in ImageWriter"

Posted by na...@apache.org.
Revert "HDFS-12903. [READ] Fix closing streams in ImageWriter"

This reverts commit c1bf2654b0e9118985b8518b0254eac4dd302a2f.


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

Branch: refs/heads/yarn-3409
Commit: e515103a83e12ad4908c0ca0b4b1aa4a87e2a840
Parents: 8239e3a
Author: Chris Douglas <cd...@apache.org>
Authored: Fri Dec 15 17:40:50 2017 -0800
Committer: Chris Douglas <cd...@apache.org>
Committed: Fri Dec 15 17:51:42 2017 -0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/server/namenode/ImageWriter.java | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e515103a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageWriter.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageWriter.java b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageWriter.java
index 1be5190..14a5f8f 100644
--- a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageWriter.java
+++ b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageWriter.java
@@ -183,9 +183,9 @@ public class ImageWriter implements Closeable {
       dirsTmp.deleteOnExit();
       dirsTmpStream = new FileOutputStream(dirsTmp);
       dirs = beginSection(dirsTmpStream);
-    } catch (Throwable e) {
+    } catch (IOException e) {
       IOUtils.cleanupWithLogger(null, raw, dirsTmpStream);
-      throw new IOException(e);
+      throw e;
     }
 
     try {


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


[16/50] [abbrv] hadoop git commit: YARN-7664. Several javadoc errors. Contributed by Sean Mackrory.

Posted by na...@apache.org.
YARN-7664. Several javadoc errors. Contributed by Sean Mackrory.


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

Branch: refs/heads/yarn-3409
Commit: 001008958d8da008ed2e3be370ea4431fd023c97
Parents: 9289641
Author: Akira Ajisaka <aa...@apache.org>
Authored: Mon Dec 18 22:24:51 2017 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Mon Dec 18 22:24:51 2017 +0900

----------------------------------------------------------------------
 .../yarn/api/protocolrecords/AllocateResponse.java  |  4 ++--
 .../hadoop/yarn/util/resource/ResourceUtils.java    |  2 +-
 .../fpga/AbstractFpgaVendorPlugin.java              |  2 +-
 .../resourceplugin/fpga/IntelFpgaOpenclPlugin.java  | 16 ++++++++--------
 .../yarn/server/resourcemanager/rmapp/RMApp.java    |  2 +-
 .../scheduler/capacity/CSQueueUtils.java            | 10 ++++++----
 6 files changed, 19 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/00100895/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java
index 98346ce..655c6dc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/AllocateResponse.java
@@ -376,7 +376,7 @@ public abstract class AllocateResponse {
    * Get the list of running containers as viewed by
    * <code>ResourceManager</code> from previous application attempts which
    * have not been reported to the Application Master yet.
-   * <br/>
+   * <br>
    * These containers were recovered by the RM after the application master
    * had already registered. This may happen after RM restart when some NMs get
    * delayed in connecting to the RM and reporting the active containers.
@@ -394,7 +394,7 @@ public abstract class AllocateResponse {
    * Set the list of running containers as viewed by
    * <code>ResourceManager</code> from previous application attempts which have
    * not been reported to the Application Master yet.
-   * <br/>
+   * <br>
    * These containers were recovered by the RM after the application master
    * had already registered. This may happen after RM restart when some NMs get
    * delayed in connecting to the RM and reporting the active containers.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/00100895/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java
index 39fd0c5..17567e8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/resource/ResourceUtils.java
@@ -422,7 +422,7 @@ public class ResourceUtils {
    * Extract unit and actual value from resource value.
    * @param resourceValue Value of the resource
    * @return Array containing unit and value. [0]=unit, [1]=value
-   * @throws IllegalArgumentExcpetion if units contain non alpha characters
+   * @throws IllegalArgumentException if units contain non alpha characters
    */
   public static String[] parseResourceValue(String resourceValue) {
     String[] resource = new String[2];

http://git-wip-us.apache.org/repos/asf/hadoop/blob/00100895/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/resourceplugin/fpga/AbstractFpgaVendorPlugin.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/resourceplugin/fpga/AbstractFpgaVendorPlugin.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/resourceplugin/fpga/AbstractFpgaVendorPlugin.java
index 60ea57c..4db9729 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/resourceplugin/fpga/AbstractFpgaVendorPlugin.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/resourceplugin/fpga/AbstractFpgaVendorPlugin.java
@@ -77,7 +77,7 @@ public interface AbstractFpgaVendorPlugin extends Configurable{
   /**
    * The vendor plugin configure an IP file to a device
    * @param ipPath The absolute path of the IP file
-   * @param majorMinorNumber The device in format <major:minor>
+   * @param majorMinorNumber The device in format &lt;major:minor&gt;
    * @return configure device ok or not
    * */
   boolean configureIP(String ipPath, String majorMinorNumber);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/00100895/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/resourceplugin/fpga/IntelFpgaOpenclPlugin.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/resourceplugin/fpga/IntelFpgaOpenclPlugin.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/resourceplugin/fpga/IntelFpgaOpenclPlugin.java
index f2e82b8..2d6cf6f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/resourceplugin/fpga/IntelFpgaOpenclPlugin.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/resourceplugin/fpga/IntelFpgaOpenclPlugin.java
@@ -221,26 +221,26 @@ public class IntelFpgaOpenclPlugin implements AbstractFpgaVendorPlugin {
    * While per Intel's guide, the output(should be outdated or prior SDK version's) is as below:
    *
    * "
-   * aocl diagnose: Running diagnostic from ALTERAOCLSDKROOT/board/<board_name>/
-   * <platform>/libexec
+   * aocl diagnose: Running diagnostic from ALTERAOCLSDKROOT/board/&lt;board_name&gt;/
+   * &lt;platform&gt;/libexec
    * Verified that the kernel mode driver is installed on the host machine.
-   * Using board package from vendor: <board_vendor_name>
+   * Using board package from vendor: &lt;board_vendor_name&gt;
    * Querying information for all supported devices that are installed on the host
    * machine ...
    *
    * device_name Status Information
    *
-   * acl0 Passed <descriptive_board_name>
-   *             PCIe dev_id = <device_ID>, bus:slot.func = 02:00.00,
+   * acl0 Passed &lt;descriptive_board_name&gt;
+   *             PCIe dev_id = &lt;device_ID&gt;, bus:slot.func = 02:00.00,
    *               at Gen 2 with 8 lanes.
    *             FPGA temperature=43.0 degrees C.
-   * acl1 Passed <descriptive_board_name>
-   *             PCIe dev_id = <device_ID>, bus:slot.func = 03:00.00,
+   * acl1 Passed &lt;descriptive_board_name&gt;
+   *             PCIe dev_id = &lt;device_ID&gt;, bus:slot.func = 03:00.00,
    *               at Gen 2 with 8 lanes.
    *             FPGA temperature = 35.0 degrees C.
    *
    * Found 2 active device(s) installed on the host machine, to perform a full
-   * diagnostic on a specific device, please run aocl diagnose <device_name>
+   * diagnostic on a specific device, please run aocl diagnose &lt;device_name&gt;
    *
    * DIAGNOSTIC_PASSED
    * "

http://git-wip-us.apache.org/repos/asf/hadoop/blob/00100895/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.java
index b357d91..39321cc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.java
@@ -307,7 +307,7 @@ public interface RMApp extends EventHandler<RMAppEvent> {
   boolean isAppInCompletedStates();
 
   /**
-   * Get the application -> queue placement context
+   * Get the application -&gt; queue placement context
    * @return ApplicationPlacementContext
    */
   ApplicationPlacementContext getApplicationPlacementContext();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/00100895/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueueUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueueUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueueUtils.java
index 3901398..0dfce83 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueueUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueueUtils.java
@@ -279,10 +279,12 @@ public class CSQueueUtils {
    * <p>
    * Update Queue Statistics:
    * </p>
-   *  
-   * <li>used-capacity/absolute-used-capacity by partition</li> 
-   * <li>non-partitioned max-avail-resource to queue</li>
-   * 
+   *
+   * <ul>
+   *   <li>used-capacity/absolute-used-capacity by partition</li>
+   *   <li>non-partitioned max-avail-resource to queue</li>
+   * </ul>
+   *
    * <p>
    * When nodePartition is null, all partition of
    * used-capacity/absolute-used-capacity will be updated.


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


[44/50] [abbrv] hadoop git commit: HADOOP-15113. NPE in S3A getFileStatus: null instrumentation on using closed instance. Contributed by Steve Loughran.

Posted by na...@apache.org.
HADOOP-15113. NPE in S3A getFileStatus: null instrumentation on using closed instance.
Contributed by Steve Loughran.


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

Branch: refs/heads/yarn-3409
Commit: ef450df443f1dea1c52082cf281f25db7141972f
Parents: d2d8f4a
Author: Steve Loughran <st...@apache.org>
Authored: Thu Dec 21 14:15:53 2017 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Thu Dec 21 14:15:53 2017 +0000

----------------------------------------------------------------------
 .../org/apache/hadoop/fs/s3a/S3AFileSystem.java | 53 ++++++++---
 .../java/org/apache/hadoop/fs/s3a/S3AUtils.java |  3 +
 .../apache/hadoop/fs/s3a/ITestS3AClosedFS.java  | 92 ++++++++++++++++++++
 3 files changed, 135 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef450df4/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
index 9431f17..f461c9e 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
@@ -187,6 +187,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
   private long readAhead;
   private S3AInputPolicy inputPolicy;
   private final AtomicBoolean closed = new AtomicBoolean(false);
+  private volatile boolean isClosed = false;
   private MetadataStore metadataStore;
   private boolean allowAuthoritative;
 
@@ -678,7 +679,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
    */
   public FSDataInputStream open(Path f, int bufferSize)
       throws IOException {
-
+    checkNotClosed();
     LOG.debug("Opening '{}' for reading; input policy = {}", f, inputPolicy);
     final FileStatus fileStatus = getFileStatus(f);
     if (fileStatus.isDirectory()) {
@@ -722,6 +723,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
   public FSDataOutputStream create(Path f, FsPermission permission,
       boolean overwrite, int bufferSize, short replication, long blockSize,
       Progressable progress) throws IOException {
+    checkNotClosed();
     final Path path = qualify(f);
     String key = pathToKey(path);
     FileStatus status = null;
@@ -871,7 +873,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
     Path dst = qualify(dest);
 
     LOG.debug("Rename path {} to {}", src, dst);
-    incrementStatistic(INVOCATION_RENAME);
+    entryPoint(INVOCATION_RENAME);
 
     String srcKey = pathToKey(src);
     String dstKey = pathToKey(dst);
@@ -1098,6 +1100,17 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
   }
 
   /**
+   * Entry point to an operation.
+   * Increments the statistic; verifies the FS is active.
+   * @param operation The operation to increment
+   * @throws IOException if the
+   */
+  protected void entryPoint(Statistic operation) throws IOException {
+    checkNotClosed();
+    incrementStatistic(operation);
+  }
+
+  /**
    * Increment a statistic by 1.
    * @param statistic The operation to increment
    */
@@ -1660,6 +1673,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
   @Retries.RetryTranslated
   public boolean delete(Path f, boolean recursive) throws IOException {
     try {
+      checkNotClosed();
       return innerDelete(innerGetFileStatus(f, true), recursive);
     } catch (FileNotFoundException e) {
       LOG.debug("Couldn't delete {} - does not exist", f);
@@ -1838,7 +1852,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
     Path path = qualify(f);
     String key = pathToKey(path);
     LOG.debug("List status for path: {}", path);
-    incrementStatistic(INVOCATION_LIST_STATUS);
+    entryPoint(INVOCATION_LIST_STATUS);
 
     List<FileStatus> result;
     final FileStatus fileStatus =  getFileStatus(path);
@@ -1981,7 +1995,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
       throws IOException, FileAlreadyExistsException, AmazonClientException {
     Path f = qualify(p);
     LOG.debug("Making directory: {}", f);
-    incrementStatistic(INVOCATION_MKDIRS);
+    entryPoint(INVOCATION_MKDIRS);
     FileStatus fileStatus;
     List<Path> metadataStoreDirs = null;
     if (hasMetadataStore()) {
@@ -2058,7 +2072,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
   @Retries.RetryTranslated
   S3AFileStatus innerGetFileStatus(final Path f,
       boolean needEmptyDirectoryFlag) throws IOException {
-    incrementStatistic(INVOCATION_GET_FILE_STATUS);
+    entryPoint(INVOCATION_GET_FILE_STATUS);
     final Path path = qualify(f);
     String key = pathToKey(path);
     LOG.debug("Getting path status for {}  ({})", path, key);
@@ -2319,7 +2333,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
   private void innerCopyFromLocalFile(boolean delSrc, boolean overwrite,
       Path src, Path dst)
       throws IOException, FileAlreadyExistsException, AmazonClientException {
-    incrementStatistic(INVOCATION_COPY_FROM_LOCAL_FILE);
+    entryPoint(INVOCATION_COPY_FROM_LOCAL_FILE);
     LOG.debug("Copying local file from {} to {}", src, dst);
 
     // Since we have a local file, we don't need to stream into a temporary file
@@ -2418,6 +2432,8 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
       // already closed
       return;
     }
+    isClosed = true;
+    LOG.debug("Filesystem {} is closed", uri);
     try {
       super.close();
     } finally {
@@ -2435,6 +2451,17 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
   }
 
   /**
+   * Verify that the input stream is open. Non blocking; this gives
+   * the last state of the volatile {@link #closed} field.
+   * @throws IOException if the connection is closed.
+   */
+  private void checkNotClosed() throws IOException {
+    if (isClosed) {
+      throw new IOException(uri + ": " + E_FS_CLOSED);
+    }
+  }
+
+  /**
    * Override getCanonicalServiceName because we don't support token in S3A.
    */
   @Override
@@ -2860,7 +2887,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
    */
   @Override
   public FileStatus[] globStatus(Path pathPattern) throws IOException {
-    incrementStatistic(INVOCATION_GLOB_STATUS);
+    entryPoint(INVOCATION_GLOB_STATUS);
     return super.globStatus(pathPattern);
   }
 
@@ -2871,7 +2898,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
   @Override
   public FileStatus[] globStatus(Path pathPattern, PathFilter filter)
       throws IOException {
-    incrementStatistic(INVOCATION_GLOB_STATUS);
+    entryPoint(INVOCATION_GLOB_STATUS);
     return super.globStatus(pathPattern, filter);
   }
 
@@ -2881,7 +2908,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
    */
   @Override
   public boolean exists(Path f) throws IOException {
-    incrementStatistic(INVOCATION_EXISTS);
+    entryPoint(INVOCATION_EXISTS);
     return super.exists(f);
   }
 
@@ -2892,7 +2919,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
   @Override
   @SuppressWarnings("deprecation")
   public boolean isDirectory(Path f) throws IOException {
-    incrementStatistic(INVOCATION_IS_DIRECTORY);
+    entryPoint(INVOCATION_IS_DIRECTORY);
     return super.isDirectory(f);
   }
 
@@ -2903,7 +2930,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
   @Override
   @SuppressWarnings("deprecation")
   public boolean isFile(Path f) throws IOException {
-    incrementStatistic(INVOCATION_IS_FILE);
+    entryPoint(INVOCATION_IS_FILE);
     return super.isFile(f);
   }
 
@@ -2948,7 +2975,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
 
   private RemoteIterator<LocatedFileStatus> innerListFiles(Path f, boolean
       recursive, Listing.FileStatusAcceptor acceptor) throws IOException {
-    incrementStatistic(INVOCATION_LIST_FILES);
+    entryPoint(INVOCATION_LIST_FILES);
     Path path = qualify(f);
     LOG.debug("listFiles({}, {})", path, recursive);
     try {
@@ -3033,7 +3060,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
   public RemoteIterator<LocatedFileStatus> listLocatedStatus(final Path f,
       final PathFilter filter)
       throws FileNotFoundException, IOException {
-    incrementStatistic(INVOCATION_LIST_LOCATED_STATUS);
+    entryPoint(INVOCATION_LIST_LOCATED_STATUS);
     Path path = qualify(f);
     LOG.debug("listLocatedStatus({}, {}", path, filter);
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef450df4/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
index 2457217..6d66739 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
@@ -89,6 +89,9 @@ public final class S3AUtils {
       "is abstract and therefore cannot be created";
   static final String ENDPOINT_KEY = "Endpoint";
 
+  /** Filesystem is closed; kept here to keep the errors close. */
+  static final String E_FS_CLOSED = "FileSystem is closed!";
+
   /**
    * Core property for provider path. Duplicated here for consistent
    * code across Hadoop version: {@value}.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef450df4/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AClosedFS.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AClosedFS.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AClosedFS.java
new file mode 100644
index 0000000..6e81452
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AClosedFS.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a;
+
+import java.io.IOException;
+
+import org.junit.Test;
+
+import org.apache.hadoop.fs.Path;
+
+import static org.apache.hadoop.test.LambdaTestUtils.*;
+import static org.apache.hadoop.fs.s3a.S3AUtils.E_FS_CLOSED;
+
+/**
+ * Tests of the S3A FileSystem which is closed; just make sure
+ * that that basic file Ops fail meaningfully.
+ */
+public class ITestS3AClosedFS extends AbstractS3ATestBase {
+
+  private Path root = new Path("/");
+
+  @Override
+  public void setup() throws Exception {
+    super.setup();
+    root = getFileSystem().makeQualified(new Path("/"));
+    getFileSystem().close();
+  }
+
+  @Override
+  public void teardown()  {
+    // no op, as the FS is closed
+  }
+
+  @Test
+  public void testClosedGetFileStatus() throws Exception {
+    intercept(IOException.class, E_FS_CLOSED,
+        () -> getFileSystem().getFileStatus(root));
+  }
+
+  @Test
+  public void testClosedListStatus() throws Exception {
+    intercept(IOException.class, E_FS_CLOSED,
+        () -> getFileSystem().listStatus(root));
+  }
+
+  @Test
+  public void testClosedListFile() throws Exception {
+    intercept(IOException.class, E_FS_CLOSED,
+        () -> getFileSystem().listFiles(root, false));
+  }
+
+  @Test
+  public void testClosedListLocatedStatus() throws Exception {
+    intercept(IOException.class, E_FS_CLOSED,
+        () -> getFileSystem().listLocatedStatus(root));
+  }
+
+  @Test
+  public void testClosedCreate() throws Exception {
+    intercept(IOException.class, E_FS_CLOSED,
+        () -> getFileSystem().create(path("to-create")).close());
+  }
+
+  @Test
+  public void testClosedDelete() throws Exception {
+    intercept(IOException.class, E_FS_CLOSED,
+        () ->  getFileSystem().delete(path("to-delete"), false));
+  }
+
+  @Test
+  public void testClosedOpen() throws Exception {
+    intercept(IOException.class, E_FS_CLOSED,
+        () ->  getFileSystem().open(path("to-open")));
+  }
+
+}


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


[09/50] [abbrv] hadoop git commit: HDFS-12874. Documentation for provided storage. Contributed by Virajith Jalaparti

Posted by na...@apache.org.
HDFS-12874. Documentation for provided storage. Contributed by Virajith Jalaparti


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

Branch: refs/heads/yarn-3409
Commit: 2298f2d76b2cafd84c8f7421ae792336d6f2f37a
Parents: 962b5e7
Author: Chris Douglas <cd...@apache.org>
Authored: Thu Dec 7 17:41:00 2017 -0800
Committer: Chris Douglas <cd...@apache.org>
Committed: Fri Dec 15 17:51:41 2017 -0800

----------------------------------------------------------------------
 .../src/main/resources/hdfs-default.xml         |   2 +-
 .../src/site/markdown/HdfsProvidedStorage.md    | 247 +++++++++++++++++++
 2 files changed, 248 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2298f2d7/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 3dc583c..7b5ccbc 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
@@ -4643,7 +4643,7 @@
     <description>
       The class that is used to specify the input format of the blocks on
       provided storages. The default is
-      org.apache.hadoop.hdfs.server.common.TextFileRegionAliasMap which uses
+      org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.TextFileRegionAliasMap which uses
       file regions to describe blocks. The file regions are specified as a
       delimited text file. Each file region is a 6-tuple containing the
       block id, remote file path, offset into file, length of block, the

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2298f2d7/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsProvidedStorage.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsProvidedStorage.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsProvidedStorage.md
new file mode 100644
index 0000000..7455044
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsProvidedStorage.md
@@ -0,0 +1,247 @@
+<!---
+  Licensed 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. See accompanying LICENSE file.
+-->
+
+HDFS Provided Storage
+=====================
+
+Provided storage allows data *stored outside HDFS* to be mapped to and addressed
+from HDFS. It builds on [heterogeneous storage](./ArchivalStorage.html) by
+introducing a new storage type, `PROVIDED`, to the set of media in a datanode.
+Clients accessing data in
+`PROVIDED` storages can cache replicas in local media, enforce HDFS invariants
+(e.g., security, quotas), and address more data than the cluster could persist
+in the storage attached to DataNodes. This architecture is particularly useful
+in scenarios where HDFS clusters are ephemeral (e.g., cloud scenarios), and/or
+require to read data that lives in other storage systems (e.g., blob stores).
+
+Provided storage is an experimental feature in HDFS.
+
+<!-- MACRO{toc|fromDepth=0|toDepth=3} -->
+
+Introduction
+------------
+
+As of this writing, support for mounting external storage as `PROVIDED` blocks
+is limited to creating a *read-only image* of a remote namespace that implements the
+`org.apache.hadoop.fs.FileSystem` interface, and starting a NameNode
+to serve the image. Specifically, reads from a snapshot of a remote namespace are
+supported. Adding a remote namespace to an existing/running namenode, refreshing the
+remote snapshot, unmounting, and writes are not available in this release. One
+can use [ViewFs](./ViewFs.html) and [RBF](HDFSRouterFederation.html) to
+integrate namespaces with `PROVIDED` storage into an existing deployment.
+
+Creating HDFS Clusters with `PROVIDED` Storage
+----------------------------------------------
+
+One can create snapshots of the remote namespace using the `fs2img` tool. Given
+a path to a remote `FileSystem`, the tool creates an _image_ mirroring the
+namespace and an _alias map_ that maps blockIDs in the generated image to a
+`FileRegion` in the remote filesystem. A `FileRegion` contains sufficient information to
+address a fixed sequence of bytes in the remote `FileSystem` (e.g., file, offset, length)
+and a nonce to verify that the region is unchanged since the image was generated.
+
+After the NameNode image and alias map are created, the NameNode and DataNodes
+must be configured to consistently reference this address space. When a DataNode
+registers with an attached, `PROVIDED` storage, the NameNode considers all the
+external blocks to be addressable through that DataNode, and may begin to direct
+clients to it. Symmetrically, the DataNode must be able to map every block in
+the `PROVIDED` storage to remote data.
+
+Deployment details vary depending on the configured alias map implementation.
+
+### `PROVIDED` Configuration
+
+Each NameNode supports one alias map. When `PROVIDED` storage is enabled,
+the storage ID configured on the NameNode and DataNodes must match.
+All other details are internal to the alias map implementation.
+
+The configuration to enable `PROVIDED` storage is as follows.
+The configuration options available for the alias map implementations are
+available below.
+
+```xml
+<configuration>
+
+  <property>
+    <name>dfs.namenode.provided.enabled</name>
+    <value>true</value>
+    <description>Enabled provided storage on the Namenode</description>
+  </property>
+
+  <property>
+     <name>dfs.datanode.data.dir</name>
+     <value>[DISK]/local/path/to/blocks/, [PROVIDED]remoteFS://remoteFS-authority/path/to/data/</value>
+  </property>
+
+  <property>
+      <name>dfs.provided.storage.id</name>
+      <value>DS-PROVIDED</value>
+      <description>The storage ID used for provided storages in the cluster.</description>
+  </property>
+
+  <property>
+    <name>dfs.provided.aliasmap.class</name>
+    <value>org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.TextFileRegionAliasMap</value>
+  </property>
+
+</configuration>
+```
+
+### fs2img tool
+
+The `fs2img` tool "walks" over a remote namespace by recursively enumerating
+children of a remote URI to produce an FSImage. Some attributes can be
+controlled by plugins, such as owner/group mappings from the remote filesystem
+to HDFS and the mapping of files to HDFS blocks.
+
+The various options available in running the tool are:
+
+| Option                  | Property                    | Default           | Description |
+|:------------------------|:--------------------------- |:----------------- |:---- |
+| `-o`, `--outdir`        | dfs.namenode.name.dir       | file://${hadoop.tmp.dir}/dfs/name | Output directory |
+| `-b`, `--blockclass`    | dfs.provided.aliasmap.class | NullBlocksMap     | Block output class |
+| `-u`, `--ugiclass`      | hdfs.image.writer.ugi.class | SingleUGIResolver | UGI resolver class |
+| `-i`, `--blockidclass`  | hdfs.image.writer.blockresolver.class | FixedBlockResolver | Block resolver class |
+| `-c`, `--cachedirs`     | hdfs.image.writer.cache.entries | 100           | Max active dirents |
+| `-cid`, `--clusterID`   |                             |                   | Cluster ID |
+| `-bpid`, `--blockPoolID`|                             |                   | Block pool ID |
+
+#### Examples
+
+Assign all files to be owned by "rmarathe", write to gzip compressed text:
+```
+hadoop org.apache.hadoop.hdfs.server.namenode.FileSystemImage \
+  -Dhdfs.image.writer.ugi.single.user=rmarathe \
+  -Ddfs.provided.aliasmap.text.codec=gzip \
+  -Ddfs.provided.aliasmap.text.write.dir=file:///tmp/
+  -b org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.TextFileRegionAliasMap \
+  -u org.apache.hadoop.hdfs.server.namenode.SingleUGIResolver \
+  -o file:///tmp/name \
+  hdfs://afreast/projects/ydau/onan
+```
+
+Assign ownership based on a custom `UGIResolver`, in LevelDB:
+```
+hadoop org.apache.hadoop.hdfs.server.namenode.FileSystemImage \
+  -Ddfs.provided.aliasmap.leveldb.path=file:///path/to/leveldb/map/dingos.db \
+  -b org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.LevelDBFileRegionAliasMap \
+  -o file:///tmp/name \
+  -u CustomResolver \
+  hdfs://enfield/projects/ywqmd/incandenza
+```
+
+
+Alias Map Implementations
+-------------------------
+
+The alias map implementation to use is configured using the `dfs.provided.aliasmap.class` parameter.
+Currently, the following two types of alias maps are supported.
+
+### InMemoryAliasMap
+
+This is a LevelDB-based alias map that runs as a separate server in Namenode.
+The alias map itself can be created using the `fs2img` tool using the option
+`-Ddfs.provided.aliasmap.leveldb.path=file:///path/to/leveldb/map/dingos.db -o org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.LevelDBFileRegionAliasMap`
+as in the example above.
+
+Datanodes contact this alias map using the `org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMapProtocol` protocol.
+
+#### Configuration
+
+
+```xml
+<configuration>
+  <property>
+    <name>dfs.provided.aliasmap.inmemory.batch-size</name>
+    <value>500</value>
+    <description>
+      The batch size when iterating over the database backing the aliasmap
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.provided.aliasmap.inmemory.dnrpc-address</name>
+    <value>namenode:rpc-port</value>
+    <description>
+      The address where the aliasmap server will be running
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.provided.aliasmap.inmemory.leveldb.dir</name>
+    <value>file:///path/to/leveldb/map/dingos.db</value>
+    <description>
+      The directory where the leveldb files will be kept
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.provided.aliasmap.inmemory.enabled</name>
+    <value>true</value>
+    <description>Enable the inmemory alias map on the NameNode. Defaults to false.</description>
+  </property>
+
+  <property>
+    <name>dfs.provided.aliasmap.class</name>
+    <value>org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.InMemoryLevelDBAliasMapClient</value>
+  </property>
+</configuration>
+```
+
+### TextFileRegionAliasMap
+
+This alias map implementation stores the mapping from `blockID`s to `FileRegion`s
+in a delimited text file. This format is useful for test environments,
+particularly single-node.
+
+#### Configuration
+```xml
+<configuration>
+  <property>
+    <name>dfs.provided.aliasmap.text.delimiter</name>
+    <value>,</value>
+    <description>
+        The delimiter used when the alias map is specified as
+        a text file.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.provided.aliasmap.text.read.file</name>
+    <value>file:///path/to/aliasmap/blocks_blocPoolID.csv</value>
+    <description>
+        The path specifying the alias map as a text file,
+        specified as a URI.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.provided.aliasmap.text.codec</name>
+    <value></value>
+    <description>
+        The codec used to de-compress the alias map. Default value is empty.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.provided.aliasmap.text.write.dir</name>
+    <value>file:///path/to/aliasmap/</value>
+    <description>
+        The path to which the alias map should be written as a text
+        file, specified as a URI.
+    </description>
+  </property>
+</configuration>
+```
+


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


[15/50] [abbrv] hadoop git commit: YARN-7617. Add a flag in distributed shell to automatically PROMOTE opportunistic containers to guaranteed once they are started. Contributed by Weiwei Yang.

Posted by na...@apache.org.
YARN-7617. Add a flag in distributed shell to automatically PROMOTE opportunistic containers to guaranteed once they are started. Contributed by Weiwei Yang.


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

Branch: refs/heads/yarn-3409
Commit: 928964102029e96406f5482e8900802f38164501
Parents: 5e81f32
Author: Weiwei Yang <ww...@apache.org>
Authored: Mon Dec 18 10:07:16 2017 +0800
Committer: Weiwei Yang <ww...@apache.org>
Committed: Mon Dec 18 10:07:16 2017 +0800

----------------------------------------------------------------------
 .../distributedshell/ApplicationMaster.java     | 49 +++++++++++++++++++-
 .../applications/distributedshell/Client.java   | 11 +++++
 .../site/markdown/OpportunisticContainers.md.vm |  2 +-
 3 files changed, 59 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/92896410/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
index 926de50..b3fa0ff 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
@@ -93,6 +93,8 @@ import org.apache.hadoop.yarn.api.records.URL;
 import org.apache.hadoop.yarn.api.records.UpdatedContainer;
 import org.apache.hadoop.yarn.api.records.ExecutionType;
 import org.apache.hadoop.yarn.api.records.ExecutionTypeRequest;
+import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
+import org.apache.hadoop.yarn.api.records.ContainerUpdateType;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntityGroupId;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent;
@@ -247,6 +249,8 @@ public class ApplicationMaster {
   // Execution type of the containers.
   // Default GUARANTEED.
   private ExecutionType containerType = ExecutionType.GUARANTEED;
+  // Whether to automatically promote opportunistic containers.
+  private boolean autoPromoteContainers = false;
 
   // Resource profile for the container
   private String containerResourceProfile = "";
@@ -420,6 +424,9 @@ public class ApplicationMaster {
         "Environment for shell script. Specified as env_key=env_val pairs");
     opts.addOption("container_type", true,
         "Container execution type, GUARANTEED or OPPORTUNISTIC");
+    opts.addOption("promote_opportunistic_after_start", false,
+        "Flag to indicate whether to automatically promote opportunistic"
+            + " containers to guaranteed.");
     opts.addOption("container_memory", true,
         "Amount of memory in MB to be requested to run the shell command");
     opts.addOption("container_vcores", true,
@@ -576,6 +583,9 @@ public class ApplicationMaster {
       }
       containerType = ExecutionType.valueOf(containerTypeStr);
     }
+    if (cliParser.hasOption("promote_opportunistic_after_start")) {
+      autoPromoteContainers = true;
+    }
     containerMemory = Integer.parseInt(cliParser.getOptionValue(
         "container_memory", "-1"));
     containerVirtualCores = Integer.parseInt(cliParser.getOptionValue(
@@ -977,7 +987,24 @@ public class ApplicationMaster {
 
     @Override
     public void onContainersUpdated(
-        List<UpdatedContainer> containers) {}
+        List<UpdatedContainer> containers) {
+      for (UpdatedContainer container : containers) {
+        LOG.info("Container {} updated, updateType={}, resource={}, "
+                + "execType={}",
+            container.getContainer().getId(),
+            container.getUpdateType().toString(),
+            container.getContainer().getResource().toString(),
+            container.getContainer().getExecutionType());
+
+        // TODO Remove this line with finalized updateContainer API.
+        // Currently nm client needs to notify the NM to update container
+        // execution type via NMClient#updateContainerResource() or
+        // NMClientAsync#updateContainerResourceAsync() when
+        // auto-update.containers is disabled, but this API is
+        // under evolving and will need to be replaced by a proper new API.
+        nmClientAsync.updateContainerResourceAsync(container.getContainer());
+      }
+    }
 
     @Override
     public void onShutdownRequest() {
@@ -1004,7 +1031,7 @@ public class ApplicationMaster {
   }
 
   @VisibleForTesting
-  static class NMCallbackHandler extends NMClientAsync.AbstractCallbackHandler {
+  class NMCallbackHandler extends NMClientAsync.AbstractCallbackHandler {
 
     private ConcurrentMap<ContainerId, Container> containers =
         new ConcurrentHashMap<ContainerId, Container>();
@@ -1033,6 +1060,24 @@ public class ApplicationMaster {
         LOG.debug("Container Status: id=" + containerId + ", status=" +
             containerStatus);
       }
+
+      // If promote_opportunistic_after_start is set, automatically promote
+      // opportunistic containers to guaranteed.
+      if (autoPromoteContainers) {
+        if (containerStatus.getState() == ContainerState.RUNNING) {
+          Container container = containers.get(containerId);
+          if (container.getExecutionType() == ExecutionType.OPPORTUNISTIC) {
+            // Promote container
+            LOG.info("Promoting container {} to {}", container.getId(),
+                container.getExecutionType());
+            UpdateContainerRequest updateRequest = UpdateContainerRequest
+                .newInstance(container.getVersion(), container.getId(),
+                    ContainerUpdateType.PROMOTE_EXECUTION_TYPE, null,
+                    ExecutionType.GUARANTEED);
+            amRMClient.requestContainerUpdate(container, updateRequest);
+          }
+        }
+      }
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/92896410/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java
index 16bf0fd..e299acc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java
@@ -175,6 +175,8 @@ public class Client {
   private String nodeLabelExpression = null;
   // Container type, default GUARANTEED.
   private ExecutionType containerType = ExecutionType.GUARANTEED;
+  // Whether to auto promote opportunistic containers
+  private boolean autoPromoteContainers = false;
 
   // log4j.properties file 
   // if available, add to local resources and set into classpath 
@@ -292,6 +294,9 @@ public class Client {
     opts.addOption("container_vcores", true, "Amount of virtual cores to be requested to run the shell command");
     opts.addOption("container_resource_profile", true, "Resource profile for the shell command");
     opts.addOption("num_containers", true, "No. of containers on which the shell command needs to be executed");
+    opts.addOption("promote_opportunistic_after_start", false,
+        "Flag to indicate whether to automatically promote opportunistic"
+            + " containers to guaranteed.");
     opts.addOption("log_properties", true, "log4j.properties file");
     opts.addOption("keep_containers_across_application_attempts", false,
       "Flag to indicate whether to keep containers across application attempts." +
@@ -449,6 +454,9 @@ public class Client {
       }
       containerType = ExecutionType.valueOf(containerTypeStr);
     }
+    if (cliParser.hasOption("promote_opportunistic_after_start")) {
+      autoPromoteContainers = true;
+    }
     containerMemory =
         Integer.parseInt(cliParser.getOptionValue("container_memory", "-1"));
     containerVirtualCores =
@@ -759,6 +767,9 @@ public class Client {
     if (containerType != null) {
       vargs.add("--container_type " + String.valueOf(containerType));
     }
+    if (autoPromoteContainers) {
+      vargs.add("--promote_opportunistic_after_start");
+    }
     if (containerMemory > 0) {
       vargs.add("--container_memory " + String.valueOf(containerMemory));
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/92896410/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/OpportunisticContainers.md.vm
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/OpportunisticContainers.md.vm b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/OpportunisticContainers.md.vm
index 7882b87..f1c75ae 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/OpportunisticContainers.md.vm
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/OpportunisticContainers.md.vm
@@ -83,7 +83,7 @@ Another sample job is the distributed shell, it allows us to run a given shell c
 $ yarn org.apache.hadoop.yarn.applications.distributedshell.Client -jar share/hadoop/yarn/hadoop-yarn-applications-distributedshell-${project.version}.jar.jar -shell_command sleep -shell_args 10 -num_containers 10 -container_type OPPORTUNISTIC
 ```
 
-By change the value of `container_type` to `OPPORTUNISTIC` or `GUARANTEED` in the above command, we can specify the tasks to be running in opportunistic or guaranteed containers. The default type is `GUARANTEED`.
+By change the value of `container_type` to `OPPORTUNISTIC` or `GUARANTEED` in the above command, we can specify the tasks to be running in opportunistic or guaranteed containers. The default type is `GUARANTEED`. By adding flag `-promote_opportunistic_after_start` to the above command, application master will attempt to promote all opportunistic containers to guaranteed once they are started.
 
 $H3 Opportunistic Containers in Web UI
 


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


[32/50] [abbrv] hadoop git commit: YARN-7620. Allow node partition filters on Queues page of new YARN UI. Contributed by Vasudevan Skm.

Posted by na...@apache.org.
YARN-7620. Allow node partition filters on Queues page of new YARN UI. Contributed by Vasudevan Skm.


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

Branch: refs/heads/yarn-3409
Commit: fe5b057c8144d01ef9fdfb2639a2cba97ead8144
Parents: e040c97
Author: Sunil G <su...@apache.org>
Authored: Tue Dec 19 20:27:25 2017 +0530
Committer: Sunil G <su...@apache.org>
Committed: Tue Dec 19 20:27:25 2017 +0530

----------------------------------------------------------------------
 .../webapp/app/components/queue-navigator.js    |  26 +-
 .../main/webapp/app/components/tree-selector.js | 301 ++++++++++++-------
 .../yarn-queue-partition-capacity-labels.js     |  48 +++
 .../src/main/webapp/app/constants.js            |   4 +-
 .../main/webapp/app/controllers/yarn-queues.js  |  36 ++-
 .../app/models/yarn-queue/capacity-queue.js     |  56 ++--
 .../serializers/yarn-queue/capacity-queue.js    |   6 +
 .../src/main/webapp/app/styles/app.scss         |   1 +
 .../src/main/webapp/app/styles/yarn-queues.scss |  29 ++
 .../templates/components/queue-navigator.hbs    |  20 +-
 .../yarn-queue-partition-capacity-labels.hbs    |  54 ++++
 .../components/yarn-queue/capacity-queue.hbs    |  34 +--
 .../main/webapp/app/templates/yarn-queues.hbs   |   3 +-
 ...yarn-queue-partition-capacity-labels-test.js |  43 +++
 14 files changed, 478 insertions(+), 183 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fe5b057c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/queue-navigator.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/queue-navigator.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/queue-navigator.js
index 4b741b8..2cecefb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/queue-navigator.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/queue-navigator.js
@@ -16,7 +16,27 @@
  * limitations under the License.
  */
 
-import Ember from 'ember';
+import Ember from "ember";
 
-export default Ember.Component.extend({
-});
\ No newline at end of file
+export default Ember.Component.extend(Ember.TargetActionSupport,{
+  actions: {
+    filterQueuesByPartition(filter) {
+      this.set("filteredPartition", filter);
+      this.sendAction("setFilter", filter);
+    }
+  },
+  didInsertElement: function() {
+    $(".js-filter-queue-by-labels").select2({
+      width: "350px",
+      multiple: false
+    });
+
+    $(".js-filter-queue-by-labels").on("select2:select", e => {
+      this.triggerAction({
+        action: "filterQueuesByPartition",
+        target: this,
+        actionContext: e.params.data.text
+      });
+    });
+  }
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fe5b057c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/tree-selector.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/tree-selector.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/tree-selector.js
index 4645a48..5168c0e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/tree-selector.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/tree-selector.js
@@ -16,19 +16,20 @@
  * limitations under the License.
  */
 
-import Ember from 'ember';
+import Ember from "ember";
+import {PARTITION_LABEL} from '../constants';
 
 const INBETWEEN_HEIGHT = 130;
 
 export default Ember.Component.extend({
   // Map: <queue-name, queue>
-  map : undefined,
+  map: undefined,
 
   // Normalized data for d3
   treeData: undefined,
 
   // folded queues, folded[<queue-name>] == true means <queue-name> is folded
-  foldedQueues: { },
+  foldedQueues: {},
 
   // maxDepth
   maxDepth: 0,
@@ -42,17 +43,23 @@ export default Ember.Component.extend({
   used: undefined,
   max: undefined,
 
+  didUpdateAttrs: function({ oldAttrs, newAttrs }) {
+    if (oldAttrs.filteredPartition.value !== newAttrs.filteredPartition.value) {
+      this.reDraw();
+    }
+  },
   // Init data
   initData: function() {
-    this.map = { };
-    this.treeData = { };
+    this.map = {};
+    this.treeData = {};
     this.maxDepth = 0;
     this.numOfLeafQueue = 0;
 
-    this.get("model")
-      .forEach(function(o) {
+    this.get("model").forEach(
+      function(o) {
         this.map[o.id] = o;
-      }.bind(this));
+      }.bind(this)
+    );
 
     // var selected = this.get("selected");
     this.used = this.get("used");
@@ -81,9 +88,9 @@ export default Ember.Component.extend({
 
   // Init queues
   initQueue: function(queueName, depth, node) {
-    if ((!queueName) || (!this.map[queueName])) {
+    if (!queueName || !this.map[queueName]) {
       // Queue is not existed
-      return;
+      return false;
     }
     if (depth > this.maxDepth) {
       this.maxDepth = this.maxDepth + 1;
@@ -91,6 +98,13 @@ export default Ember.Component.extend({
 
     var queue = this.map[queueName];
 
+    if (
+      this.filteredPartition &&
+      !queue.get("partitions").contains(this.filteredPartition)
+    ) {
+      return false;
+    }
+
     var names = this.getChildrenNamesArray(queue);
 
     node.name = queueName;
@@ -100,14 +114,21 @@ export default Ember.Component.extend({
     if (names.length > 0) {
       node.children = [];
 
-      names.forEach(function(name) {
-        var childQueueData = {};
-        node.children.push(childQueueData);
-        this.initQueue(name, depth + 1, childQueueData);
-      }.bind(this));
+      names.forEach(
+        function(name) {
+          var childQueueData = {};
+          node.children.push(childQueueData);
+          const status = this.initQueue(name, depth + 1, childQueueData);
+          if (!status) {
+            node.children.pop();
+          }
+        }.bind(this)
+      );
     } else {
       this.numOfLeafQueue = this.numOfLeafQueue + 1;
     }
+
+    return true;
   },
 
   update: function(source, root, tree, diagonal) {
@@ -119,141 +140,183 @@ export default Ember.Component.extend({
     var links = tree.links(nodes);
 
     // Normalize for fixed-depth.
-    nodes.forEach(function(d) { d.y = d.depth * 200; });
+    nodes.forEach(function(d) {
+      d.y = d.depth * 200;
+    });
 
     // Update the nodes…
-    var node = this.mainSvg.selectAll("g.node")
-      .data(nodes, function(d) { return d.id || (d.id = ++i); });
+    var node = this.mainSvg.selectAll("g.node").data(nodes, function(d) {
+      return d.id || (d.id = ++i);
+    });
 
     // Enter any new nodes at the parent's previous position.
-    var nodeEnter = node.enter().append("g")
+    var nodeEnter = node
+      .enter()
+      .append("g")
       .attr("class", "node")
-      .attr("transform", function() { return "translate(" + source.y0 + "," + source.x0 + ")"; })
-      .on("click", function(d){
-        if (d.queueData.get("name") !== this.get("selected")) {
-            document.location.href = "#/yarn-queues/" + d.queueData.get("name") + "!";
-        }
-
-        Ember.run.later(this, function () {
-          var treeWidth = this.maxDepth * 200;
-          var treeHeight = this.numOfLeafQueue * INBETWEEN_HEIGHT;
-          var tree = d3.layout.tree().size([treeHeight, treeWidth]);
-          var diagonal = d3.svg.diagonal()
-            .projection(function(d) { return [d.y, d.x]; });
-
-          this.update(this.treeData, this.treeData, tree, diagonal);
-        }, 100);
-
-      }.bind(this))
-    .on("dblclick", function (d) {
-      document.location.href = "#/yarn-queue/" + d.queueData.get("name") + "/apps";
-    });
+      .attr("transform", function() {
+        return `translate(${source.y0 + 50}, ${source.x0})`;
+      })
+      .on(
+        "click",
+        function(d) {
+          if (d.queueData.get("name") !== this.get("selected")) {
+            document.location.href =
+              "#/yarn-queues/" + d.queueData.get("name") + "!";
+          }
+
+          Ember.run.later(
+            this,
+            function() {
+              var treeWidth = this.maxDepth * 200;
+              var treeHeight = this.numOfLeafQueue * INBETWEEN_HEIGHT;
+              var tree = d3.layout.tree().size([treeHeight, treeWidth]);
+              var diagonal = d3.svg.diagonal().projection(function(d) {
+                return [d.y + 50, d.x];
+              });
+
+              this.update(this.treeData, this.treeData, tree, diagonal);
+            },
+            100
+          );
+        }.bind(this)
+      )
+      .on("dblclick", function(d) {
+        document.location.href =
+          "#/yarn-queue/" + d.queueData.get("name") + "/apps";
+      });
 
-    nodeEnter.append("circle")
+    nodeEnter
+      .append("circle")
       .attr("r", 1e-6)
-      .style("fill", function(d) {
-        var maxCap = d.queueData.get(this.max);
-        maxCap = maxCap === undefined ? 100 : maxCap;
-        var usedCap = d.queueData.get(this.used) / maxCap * 100.0;
-        if (usedCap <= 60.0) {
-          return "mediumaquamarine";
-        } else if (usedCap <= 100.0) {
-          return "coral";
-        } else {
-          return "salmon";
-        }
-      }.bind(this));
+      .style(
+        "fill",
+        function(d) {
+          const usedCapacity = getUsedCapacity(d.queueData.get("partitionMap"), this.filteredPartition);
+          if (usedCapacity <= 60.0) {
+            return "#60cea5";
+          } else if (usedCapacity <= 100.0) {
+            return "#ffbc0b";
+          } else {
+            return "#ef6162";
+          }
+        }.bind(this)
+      );
 
     // append percentage
-    nodeEnter.append("text")
-      .attr("x", function() { return 0; })
+    nodeEnter
+      .append("text")
+      .attr("x", function() {
+        return 0;
+      })
       .attr("dy", ".35em")
       .attr("fill", "white")
-      .attr("text-anchor", function() { return "middle"; })
-      .text(function(d) {
-        var maxCap = d.queueData.get(this.max);
-        maxCap = maxCap === undefined ? 100 : maxCap;
-        var usedCap = d.queueData.get(this.used) / maxCap * 100.0;
-        if (usedCap >= 100.0) {
-          return usedCap.toFixed(0) + "%";
-        } else {
-          return usedCap.toFixed(1) + "%";
-        }
-      }.bind(this))
+      .attr("text-anchor", function() {
+        return "middle";
+      })
+      .text(
+        function(d) {
+          const usedCapacity = getUsedCapacity(d.queueData.get("partitionMap"), this.filteredPartition);
+          if (usedCapacity >= 100.0) {
+            return usedCapacity.toFixed(0) + "%";
+          } else {
+            return usedCapacity.toFixed(1) + "%";
+          }
+        }.bind(this)
+      )
       .style("fill-opacity", 1e-6);
 
     // append queue name
-    nodeEnter.append("text")
+    nodeEnter
+      .append("text")
       .attr("x", "0px")
       .attr("dy", "45px")
       .attr("text-anchor", "middle")
-      .text(function(d) { return d.name; })
+      .text(function(d) {
+        return d.name;
+      })
       .style("fill-opacity", 1e-6);
 
     // Transition nodes to their new position.
-    var nodeUpdate = node.transition()
+    var nodeUpdate = node
+      .transition()
       .duration(duration)
-      .attr("transform", function(d) { return "translate(" + d.y + "," + d.x + ")"; });
+      .attr("transform", function(d) {
+        return `translate(${d.y + 50}, ${d.x})`;
+      });
 
-    nodeUpdate.select("circle")
+    nodeUpdate
+      .select("circle")
       .attr("r", 30)
-      .attr("href",
+      .attr("href", function(d) {
+        return "#/yarn-queues/" + d.queueData.get("name");
+      })
+      .style(
+        "stroke-width",
+        function(d) {
+          if (d.queueData.get("name") === this.get("selected")) {
+            return 7;
+          } else {
+            return 2;
+          }
+        }.bind(this)
+      )
+      .style(
+        "stroke",
         function(d) {
-          return "#/yarn-queues/" + d.queueData.get("name");
-        })
-      .style("stroke-width", function(d) {
-        if (d.queueData.get("name") === this.get("selected")) {
-          return 7;
-        } else {
-          return 2;
-        }
-      }.bind(this))
-      .style("stroke", function(d) {
-        if (d.queueData.get("name") === this.get("selected")) {
-          return "gray";
-        } else {
-          return "gray";
-        }
-      }.bind(this));
-
-    nodeUpdate.selectAll("text")
-      .style("fill-opacity", 1);
+          if (d.queueData.get("name") === this.get("selected")) {
+            return "gray";
+          } else {
+            return "gray";
+          }
+        }.bind(this)
+      );
+
+    nodeUpdate.selectAll("text").style("fill-opacity", 1);
 
     // Transition exiting nodes to the parent's new position.
-    var nodeExit = node.exit().transition()
+    var nodeExit = node
+      .exit()
+      .transition()
       .duration(duration)
-      .attr("transform", function() { return "translate(" + source.y + "," + source.x + ")"; })
+      .attr("transform", function() {
+        return `translate(${source.y}, ${source.x})`;
+      })
       .remove();
 
-    nodeExit.select("circle")
-      .attr("r", 1e-6);
+    nodeExit.select("circle").attr("r", 1e-6);
 
-    nodeExit.select("text")
-      .style("fill-opacity", 1e-6);
+    nodeExit.select("text").style("fill-opacity", 1e-6);
 
     // Update the links…
-    var link = this.mainSvg.selectAll("path.link")
-      .data(links, function(d) { return d.target.id; });
+    var link = this.mainSvg.selectAll("path.link").data(links, function(d) {
+      return d.target.id;
+    });
 
     // Enter any new links at the parent's previous position.
-    link.enter().insert("path", "g")
+    link
+      .enter()
+      .insert("path", "g")
       .attr("class", "link")
       .attr("d", function() {
-        var o = {x: source.x0, y: source.y0};
-        return diagonal({source: o, target: o});
+        var o = { x: source.x0, y: source.y0 + 50 };
+        return diagonal({ source: o, target: o });
       });
 
     // Transition links to their new position.
-    link.transition()
+    link
+      .transition()
       .duration(duration)
       .attr("d", diagonal);
 
     // Transition exiting nodes to the parent's new position.
-    link.exit().transition()
+    link
+      .exit()
+      .transition()
       .duration(duration)
       .attr("d", function() {
-        var o = {x: source.x, y: source.y};
-        return diagonal({source: o, target: o});
+        var o = { x: source.x, y: source.y };
+        return diagonal({ source: o, target: o });
       })
       .remove();
 
@@ -267,27 +330,32 @@ export default Ember.Component.extend({
   reDraw: function() {
     this.initData();
 
-    var margin = {top: 20, right: 120, bottom: 20, left: 120};
+    var margin = { top: 20, right: 120, bottom: 20, left: 120 };
     var treeWidth = this.maxDepth * 200;
     var treeHeight = this.numOfLeafQueue * INBETWEEN_HEIGHT;
     var width = treeWidth + margin.left + margin.right;
     var height = treeHeight + margin.top + margin.bottom;
 
     if (this.mainSvg) {
-      this.mainSvg.remove();
+      this.mainSvg.selectAll("*").remove();
+    } else {
+      this.mainSvg = d3
+        .select("#" + this.get("parentId"))
+        .append("svg")
+        .attr("width", width)
+        .attr("height", height)
+        .attr("class", "tree-selector");
     }
 
-    this.mainSvg = d3.select("#" + this.get("parentId")).append("svg")
-      .attr("width", width)
-      .attr("height", height)
-      .attr("class", "tree-selector")
+    this.mainSvg
       .append("g")
       .attr("transform", "translate(" + margin.left + "," + margin.top + ")");
 
     var tree = d3.layout.tree().size([treeHeight, treeWidth]);
 
-    var diagonal = d3.svg.diagonal()
-      .projection(function(d) { return [d.y, d.x]; });
+    var diagonal = d3.svg.diagonal().projection(function(d) {
+      return [d.y + 50, d.x];
+    });
 
     var root = this.treeData;
     root.x0 = height / 2;
@@ -299,6 +367,11 @@ export default Ember.Component.extend({
   },
 
   didInsertElement: function() {
-   this.reDraw();
+    this.reDraw();
   }
 });
+
+
+const getUsedCapacity = (partitionMap, filter=PARTITION_LABEL) => {
+  return partitionMap[filter].absoluteUsedCapacity;
+};
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fe5b057c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/yarn-queue-partition-capacity-labels.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/yarn-queue-partition-capacity-labels.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/yarn-queue-partition-capacity-labels.js
new file mode 100644
index 0000000..e7f9c03
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/components/yarn-queue-partition-capacity-labels.js
@@ -0,0 +1,48 @@
+/**
+ * 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.
+ */
+
+import Ember from "ember";
+import { PARTITION_LABEL } from "../constants";
+
+export default Ember.Component.extend({
+  didUpdateAttrs: function({ oldAttrs, newAttrs }) {
+    this._super(...arguments);
+    this.set("data", this.initData());
+  },
+
+  init() {
+    this._super(...arguments);
+    this.set("data", this.initData());
+  },
+
+  initData() {
+    const queue = this.get("queue");
+    const partitionMap = this.get("partitionMap");
+    const filteredParition = this.get("filteredPartition") || PARTITION_LABEL;
+    const userLimit = queue.get("userLimit");
+    const userLimitFactor = queue.get("userLimitFactor");
+    const isLeafQueue = queue.get("isLeafQueue");
+
+    return {
+      ...partitionMap[filteredParition],
+      userLimit,
+      userLimitFactor,
+      isLeafQueue
+    };
+  }
+});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fe5b057c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/constants.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/constants.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/constants.js
index 29ad4bc..6b37b7f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/constants.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/constants.js
@@ -34,4 +34,6 @@ export const Entities = {
   Memory:'memory',
   Resource: 'resource',
   Unit: 'unit'
-}
\ No newline at end of file
+}
+
+export const PARTITION_LABEL = 'Default partition';
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fe5b057c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-queues.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-queues.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-queues.js
index 9658ded..6cc8767 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-queues.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/controllers/yarn-queues.js
@@ -17,19 +17,39 @@
  */
 
 import Ember from 'ember';
+import {PARTITION_LABEL} from '../constants';
 
 export default Ember.Controller.extend({
   needReload: true,
   selectedQueue: undefined,
   showLoading: true,
+  filteredPartition: PARTITION_LABEL,
 
-  breadcrumbs: [{
-    text: "Home",
-    routeName: 'application'
-  }, {
-    text: "Queues",
-    routeName: 'yarn-queues',
-    model: 'root'
-  }]
+  breadcrumbs: [
+    {
+      text: "Home",
+      routeName: "application"
+    },
+    {
+      text: "Queues",
+      routeName: "yarn-queues",
+      model: "root"
+    }
+  ],
 
+  actions: {
+    setFilter(partition) {
+      this.set("filteredPartition", partition);
+      const model = this.get('model');
+      const {selectedQueue} = model;
+      // If the selected queue does not have the filtered partition
+      // reset it to root
+      if (!selectedQueue.get('partitions').contains(partition)) {
+        const root = model.queues.get('firstObject');
+        document.location.href = "#/yarn-queues/" + root.get("id") + "!";
+        this.set("model.selectedQueue", root);
+        this.set("model.selected", root.get('id'));
+      }
+    }
+  }
 });

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fe5b057c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-queue/capacity-queue.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-queue/capacity-queue.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-queue/capacity-queue.js
index f892c2b..c123989 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-queue/capacity-queue.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/models/yarn-queue/capacity-queue.js
@@ -20,24 +20,26 @@ import DS from 'ember-data';
 import Converter from 'yarn-ui/utils/converter';
 
 export default DS.Model.extend({
-  name: DS.attr('string'),
-  children: DS.attr('array'),
-  parent: DS.attr('string'),
-  capacity: DS.attr('number'),
-  maxCapacity: DS.attr('number'),
-  usedCapacity: DS.attr('number'),
-  absCapacity: DS.attr('number'),
-  absMaxCapacity: DS.attr('number'),
-  absUsedCapacity: DS.attr('number'),
-  state: DS.attr('string'),
-  userLimit: DS.attr('number'),
-  userLimitFactor: DS.attr('number'),
-  preemptionDisabled: DS.attr('number'),
-  numPendingApplications: DS.attr('number'),
-  numActiveApplications: DS.attr('number'),
-  users: DS.hasMany('YarnUser'),
-  type: DS.attr('string'),
-  resources: DS.attr('object'),
+  name: DS.attr("string"),
+  children: DS.attr("array"),
+  parent: DS.attr("string"),
+  capacity: DS.attr("number"),
+  partitions: DS.attr("array"),
+  partitionMap: DS.attr("object"),
+  maxCapacity: DS.attr("number"),
+  usedCapacity: DS.attr("number"),
+  absCapacity: DS.attr("number"),
+  absMaxCapacity: DS.attr("number"),
+  absUsedCapacity: DS.attr("number"),
+  state: DS.attr("string"),
+  userLimit: DS.attr("number"),
+  userLimitFactor: DS.attr("number"),
+  preemptionDisabled: DS.attr("number"),
+  numPendingApplications: DS.attr("number"),
+  numActiveApplications: DS.attr("number"),
+  users: DS.hasMany("YarnUser"),
+  type: DS.attr("string"),
+  resources: DS.attr("object"),
 
   isLeafQueue: function() {
     var len = this.get("children.length");
@@ -53,21 +55,29 @@ export default DS.Model.extend({
       {
         label: "Absolute Used",
         style: "primary",
-        value: this.get("name") === "root" ? floatToFixed(this.get("usedCapacity")) : floatToFixed(this.get("absUsedCapacity"))
+        value:
+          this.get("name") === "root"
+            ? floatToFixed(this.get("usedCapacity"))
+            : floatToFixed(this.get("absUsedCapacity"))
       },
       {
         label: "Absolute Capacity",
         style: "primary",
-        value: this.get("name") === "root" ? 100 : floatToFixed(this.get("absCapacity"))
+        value:
+          this.get("name") === "root"
+            ? 100
+            : floatToFixed(this.get("absCapacity"))
       },
       {
         label: "Absolute Max Capacity",
         style: "secondary",
-        value: this.get("name") === "root" ? 100 : floatToFixed(this.get("absMaxCapacity"))
+        value:
+          this.get("name") === "root"
+            ? 100
+            : floatToFixed(this.get("absMaxCapacity"))
       }
     ];
   }.property("absCapacity", "usedCapacity", "absMaxCapacity"),
-
   userUsagesDonutChartData: function() {
     var data = [];
     if (this.get("users")) {
@@ -97,5 +107,5 @@ export default DS.Model.extend({
         value: this.get("numActiveApplications") || 0
       }
     ];
-  }.property("numPendingApplications", "numActiveApplications"),
+  }.property("numPendingApplications", "numActiveApplications")
 });

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fe5b057c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-queue/capacity-queue.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-queue/capacity-queue.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-queue/capacity-queue.js
index 7626598..b171c6e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-queue/capacity-queue.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/serializers/yarn-queue/capacity-queue.js
@@ -17,6 +17,7 @@
  */
 
 import DS from 'ember-data';
+import {PARTITION_LABEL} from '../../constants';
 
 export default DS.JSONAPISerializer.extend({
 
@@ -73,6 +74,11 @@ export default DS.JSONAPISerializer.extend({
           numPendingApplications: payload.numPendingApplications,
           numActiveApplications: payload.numActiveApplications,
           resources: payload.resources,
+          partitions: payload.capacities.queueCapacitiesByPartition.map(cap => cap.partitionName || PARTITION_LABEL),
+          partitionMap: payload.capacities.queueCapacitiesByPartition.reduce((init, cap) => {
+            init[cap.partitionName || PARTITION_LABEL] = cap;
+            return init;
+          }, {}),
           type: "capacity",
         },
         // Relationships

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fe5b057c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/styles/app.scss
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/styles/app.scss b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/styles/app.scss
index 3919ac3..5d99d8e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/styles/app.scss
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/styles/app.scss
@@ -3,6 +3,7 @@
 @import 'yarn-app.scss';
 @import './compose-box.scss';
 @import 'em-table.scss';
+@import './yarn-queues.scss';
 
 /**
  * Licensed to the Apache Software Foundation (ASF) under one

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fe5b057c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/styles/yarn-queues.scss
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/styles/yarn-queues.scss b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/styles/yarn-queues.scss
new file mode 100644
index 0000000..8852270
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/styles/yarn-queues.scss
@@ -0,0 +1,29 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+.filter-partitions {
+  padding: 15px;
+  margin-left: auto;
+  label {
+    font-weight: 500;
+  }
+  .filter-queue-by-labels {
+    display: inline-block;
+    max-width: 350px;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fe5b057c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/queue-navigator.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/queue-navigator.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/queue-navigator.hbs
index e3b0a90..b063aae 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/queue-navigator.hbs
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/queue-navigator.hbs
@@ -21,9 +21,25 @@
   <div class="col-md-12 container-fluid">
     <div class="panel panel-default" id="tree-selector-container">
       <div class="panel-heading">
-        Scheduler: {{model.firstObject.type}}
+        {{#if filteredPartition}}
+           {{model.firstObject.type}} scheduler - Showing queues from partition {{lower filteredPartition}}
+        {{else}}
+          {{model.firstObject.type}} scheduler - Showing queues from all partitions
+        {{/if}}
       </div>
-     {{tree-selector model=model parentId="tree-selector-container" selected=selected used=used max=max}}
+       {{#if (eq model.firstObject.type "capacity")}}
+       <div class="flex">
+        <div class="filter-partitions flex-right">
+          <label><i class="glyphicon glyphicon-filter"/> Partitions: </label>
+            <select onchange={{action "filterQueuesByPartition" value="target.value"}} class="form-control js-filter-queue-by-labels">
+              {{#each model.firstObject.partitions as |part|}}
+                <option value={{part}}>{{part}}</option>
+              {{/each}}
+            </select>
+        </div>
+       </div>
+      {{/if}}
+     {{tree-selector model=model parentId="tree-selector-container" selected=selected used=used max=max filteredPartition=filteredPartition}}
     </div>
   </div>
 </div>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fe5b057c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue-partition-capacity-labels.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue-partition-capacity-labels.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue-partition-capacity-labels.hbs
new file mode 100644
index 0000000..fdecb2d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue-partition-capacity-labels.hbs
@@ -0,0 +1,54 @@
+{{!
+ * 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.
+}}
+
+<div class="top-1">
+  <span class="yarn-label primary">
+    <span class="label-key">absolute used</span>
+    <span class="label-value">{{data.absoluteUsedCapacity}}%</span>
+  </span>
+  <span class="yarn-label primary">
+    <span class="label-key">absolute capacity</span>
+    <span class="label-value">{{data.absoluteCapacity}}%</span>
+  </span>
+  <span class="yarn-label secondary">
+    <span class="label-key">absolute max capacity</span>
+    <span class="label-value">{{data.absoluteMaxCapacity}}%</span>
+  </span>
+</div>
+<div class="top-1">
+  <span class="yarn-label secondary">
+    <span class="label-key">configured capacity</span>
+    <span class="label-value">{{data.capacity}}%</span>
+  </span>
+  <span class="yarn-label secondary">
+    <span class="label-key">configured max capacity</span>
+    <span class="label-value">{{data.maxCapacity}}%</span>
+  </span>
+</div>
+{{#if data.isLeafQueue}}
+<div class="top-1">
+  <span class="yarn-label secondary">
+    <span class="label-key">user limit</span>
+    <span class="label-value">{{data.userLimit}}%</span>
+  </span>
+  <span class="yarn-label secondary">
+    <span class="label-key">user limit factor</span>
+    <span class="label-value">{{data.userLimitFactor}}</span>
+  </span>
+</div>
+{{/if}}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fe5b057c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/capacity-queue.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/capacity-queue.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/capacity-queue.hbs
index bb9a87e..9ad2a6f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/capacity-queue.hbs
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/components/yarn-queue/capacity-queue.hbs
@@ -17,7 +17,7 @@
 }}
 
 {{queue-navigator model=model.queues selected=model.selected
-  used="usedCapacity" max="absMaxCapacity"}}
+  used="usedCapacity" max="absMaxCapacity" setFilter=(action setFilter)}}
 
 <div class="yarn-compose-box yarn-queues-container">
   <div>
@@ -31,36 +31,8 @@
         {{em-table-simple-status-cell content=model.selectedQueue.state}}
       </div>
     {{/if}}
-    <div class="top-1">
-      {{#each model.selectedQueue.capacitiesBarChartData as |item|}}
-        <span class="yarn-label {{item.style}}">
-          <span class="label-key"> {{lower item.label}}</span>
-          <span class="label-value">{{item.value}}%</span>
-        </span>
-      {{/each}}
-    </div>
-    <div class="top-1">
-      <span class="yarn-label secondary">
-        <span class="label-key">configured capacity</span>
-        <span class="label-value">{{model.selectedQueue.capacity}}%</span>
-      </span>
-      <span class="yarn-label secondary">
-        <span class="label-key">configured max capacity</span>
-        <span class="label-value">{{model.selectedQueue.maxCapacity}}%</span>
-      </span>
-    </div>
-    {{#if model.selectedQueue.isLeafQueue}}
-      <div class="top-1">
-        <span class="yarn-label secondary">
-          <span class="label-key">user limit</span>
-          <span class="label-value">{{model.selectedQueue.userLimit}}%</span>
-        </span>
-        <span class="yarn-label secondary">
-          <span class="label-key">user limit factor</span>
-          <span class="label-value">{{model.selectedQueue.userLimitFactor}}</span>
-        </span>
-      </div>
-    {{/if}}
+
+    {{yarn-queue-partition-capacity-labels partitionMap=model.selectedQueue.partitionMap queue=model.selectedQueue filteredPartition=filteredPartition}}
   </div>
 
   <h5> Running Apps </h5>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fe5b057c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-queues.hbs
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-queues.hbs b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-queues.hbs
index b3165d5..ede2994 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-queues.hbs
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/templates/yarn-queues.hbs
@@ -18,9 +18,10 @@
 <div class="queue-page-breadcrumb">
   {{breadcrumb-bar breadcrumbs=breadcrumbs}}
 </div>
+
 <div class="container-fluid">
   {{#if (eq model.queues.firstObject.type "capacity")}}
-    {{yarn-queue.capacity-queue model=model}}
+    {{yarn-queue.capacity-queue model=model setFilter=(action "setFilter") filteredPartition=filteredPartition}}
   {{else if (eq model.queues.firstObject.type "fair")}}
     {{yarn-queue.fair-queue model=model}}
   {{else}}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fe5b057c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/integration/components/yarn-queue-partition-capacity-labels-test.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/integration/components/yarn-queue-partition-capacity-labels-test.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/integration/components/yarn-queue-partition-capacity-labels-test.js
new file mode 100644
index 0000000..414e326
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/tests/integration/components/yarn-queue-partition-capacity-labels-test.js
@@ -0,0 +1,43 @@
+/**
+ * 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.
+ */
+
+ import { moduleForComponent, test } from 'ember-qunit';
+import hbs from 'htmlbars-inline-precompile';
+
+moduleForComponent('yarn-queue-partition-capacity-labels', 'Integration | Component | yarn queue partition capacity labels', {
+  integration: true
+});
+
+test('it renders', function(assert) {
+
+  // Set any properties with this.set('myProperty', 'value');
+  // Handle any actions with this.on('myAction', function(val) { ... });" + EOL + EOL +
+
+  this.render(hbs`{{yarn-queue-partition-capacity-labels}}`);
+
+  assert.equal(this.$().text().trim(), '');
+
+  // Template block usage:" + EOL +
+  this.render(hbs`
+    {{#yarn-queue-partition-capacity-labels}}
+      template block text
+    {{/yarn-queue-partition-capacity-labels}}
+  `);
+
+  assert.equal(this.$().text().trim(), 'template block text');
+});


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


[13/50] [abbrv] hadoop git commit: Merge branch 'HDFS-9806' into trunk

Posted by na...@apache.org.
Merge branch 'HDFS-9806' into trunk


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

Branch: refs/heads/yarn-3409
Commit: fc7ec80d85a751b2b2b261a2b97ec38c7b58f1df
Parents: 44825f0 4b3a785
Author: Chris Douglas <cd...@apache.org>
Authored: Fri Dec 15 18:06:24 2017 -0800
Committer: Chris Douglas <cd...@apache.org>
Committed: Fri Dec 15 18:06:24 2017 -0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/fs/StorageType.java  |   3 +-
 .../org/apache/hadoop/fs/shell/TestCount.java   |   3 +-
 .../hadoop/hdfs/protocol/HdfsConstants.java     |   3 +
 .../hadoop/hdfs/protocol/LocatedBlock.java      | 103 ++-
 .../hdfs/protocol/ProvidedStorageLocation.java  |  89 ++
 .../hadoop/hdfs/protocolPB/PBHelperClient.java  |  36 +
 .../src/main/proto/hdfs.proto                   |  15 +
 hadoop-hdfs-project/hadoop-hdfs/pom.xml         |   7 +-
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |  29 +
 .../hdfs/protocolPB/AliasMapProtocolPB.java     |  37 +
 .../AliasMapProtocolServerSideTranslatorPB.java | 134 +++
 ...yAliasMapProtocolClientSideTranslatorPB.java | 174 ++++
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java |  28 +
 .../hdfs/server/aliasmap/InMemoryAliasMap.java  | 222 +++++
 .../aliasmap/InMemoryAliasMapProtocol.java      | 103 +++
 .../aliasmap/InMemoryLevelDBAliasMapServer.java | 153 +++
 .../hdfs/server/blockmanagement/BlockInfo.java  |  17 +-
 .../server/blockmanagement/BlockManager.java    | 149 ++-
 .../BlockStoragePolicySuite.java                |   6 +
 .../blockmanagement/DatanodeDescriptor.java     |  44 +-
 .../server/blockmanagement/DatanodeManager.java |   2 +
 .../blockmanagement/DatanodeStatistics.java     |   3 +
 .../server/blockmanagement/DatanodeStats.java   |   4 +-
 .../blockmanagement/DatanodeStorageInfo.java    |  15 +-
 .../blockmanagement/HeartbeatManager.java       |   9 +-
 .../blockmanagement/LocatedBlockBuilder.java    | 109 +++
 .../blockmanagement/ProvidedStorageMap.java     | 540 +++++++++++
 .../blockmanagement/StorageTypeStats.java       |  33 +-
 .../hadoop/hdfs/server/common/BlockAlias.java   |  33 +
 .../hadoop/hdfs/server/common/FileRegion.java   |  85 ++
 .../hadoop/hdfs/server/common/Storage.java      |  71 +-
 .../hadoop/hdfs/server/common/StorageInfo.java  |   6 +
 .../common/blockaliasmap/BlockAliasMap.java     | 113 +++
 .../impl/InMemoryLevelDBAliasMapClient.java     | 178 ++++
 .../impl/LevelDBFileRegionAliasMap.java         | 274 ++++++
 .../impl/TextFileRegionAliasMap.java            | 490 ++++++++++
 .../common/blockaliasmap/package-info.java      |  27 +
 .../server/datanode/BlockPoolSliceStorage.java  |  20 +-
 .../hdfs/server/datanode/DataStorage.java       |  44 +-
 .../hdfs/server/datanode/DirectoryScanner.java  |  26 +-
 .../datanode/FinalizedProvidedReplica.java      | 122 +++
 .../hdfs/server/datanode/ProvidedReplica.java   | 350 +++++++
 .../hdfs/server/datanode/ReplicaBuilder.java    | 141 ++-
 .../hdfs/server/datanode/ReplicaInfo.java       |  20 +-
 .../hdfs/server/datanode/StorageLocation.java   |  54 +-
 .../server/datanode/fsdataset/FsDatasetSpi.java |   4 +-
 .../server/datanode/fsdataset/FsVolumeSpi.java  |  38 +-
 .../datanode/fsdataset/impl/FsDatasetImpl.java  |  65 +-
 .../datanode/fsdataset/impl/FsDatasetUtil.java  |  25 +-
 .../datanode/fsdataset/impl/FsVolumeImpl.java   |  19 +-
 .../fsdataset/impl/FsVolumeImplBuilder.java     |   6 +
 .../fsdataset/impl/ProvidedVolumeImpl.java      | 718 ++++++++++++++
 .../federation/metrics/FederationMBean.java     |   6 +
 .../federation/metrics/FederationMetrics.java   |   5 +
 .../federation/metrics/NamenodeBeanMetrics.java |  10 +
 .../resolver/MembershipNamenodeResolver.java    |   1 +
 .../resolver/NamenodeStatusReport.java          |  12 +-
 .../router/NamenodeHeartbeatService.java        |   3 +-
 .../store/records/MembershipStats.java          |   4 +
 .../records/impl/pb/MembershipStatsPBImpl.java  |  10 +
 .../apache/hadoop/hdfs/server/mover/Mover.java  |   2 +-
 .../server/namenode/FSImageCompression.java     |   2 +-
 .../hdfs/server/namenode/FSNamesystem.java      |  12 +
 .../hadoop/hdfs/server/namenode/NNStorage.java  |  10 +-
 .../hadoop/hdfs/server/namenode/NameNode.java   |  21 +
 .../hdfs/server/namenode/NameNodeMXBean.java    |  10 +-
 .../namenode/metrics/FSNamesystemMBean.java     |   7 +-
 .../hdfs/server/protocol/NamespaceInfo.java     |   8 +
 .../src/main/proto/AliasMapProtocol.proto       |  68 ++
 .../src/main/proto/FederationProtocol.proto     |   1 +
 .../src/main/resources/hdfs-default.xml         | 119 +++
 .../src/main/webapps/hdfs/dfshealth.html        |   1 +
 .../src/site/markdown/HdfsProvidedStorage.md    | 247 +++++
 .../org/apache/hadoop/hdfs/MiniDFSCluster.java  |  30 +-
 .../hadoop/hdfs/TestBlockStoragePolicy.java     |   6 +
 .../org/apache/hadoop/hdfs/TestDFSRollback.java |   6 +-
 .../hadoop/hdfs/TestDFSStartupVersions.java     |   2 +-
 .../org/apache/hadoop/hdfs/TestDFSUpgrade.java  |   4 +-
 .../apache/hadoop/hdfs/UpgradeUtilities.java    |  16 +-
 .../server/aliasmap/ITestInMemoryAliasMap.java  | 129 +++
 .../server/aliasmap/TestInMemoryAliasMap.java   |  45 +
 .../blockmanagement/TestDatanodeManager.java    |  66 +-
 .../blockmanagement/TestProvidedStorageMap.java | 120 +++
 .../impl/TestInMemoryLevelDBAliasMapClient.java | 344 +++++++
 .../impl/TestLevelDBFileRegionAliasMap.java     | 117 +++
 .../impl/TestLevelDbMockAliasMapClient.java     | 121 +++
 .../impl/TestTextBlockAliasMap.java             | 200 ++++
 .../server/datanode/SimulatedFSDataset.java     |   6 +-
 .../datanode/TestProvidedReplicaImpl.java       | 162 ++++
 .../extdataset/ExternalDatasetImpl.java         |   5 +-
 .../fsdataset/impl/TestFsDatasetImpl.java       |  17 +-
 .../fsdataset/impl/TestProvidedImpl.java        | 649 +++++++++++++
 .../metrics/TestFederationMetrics.java          |   2 +
 .../hdfs/server/namenode/TestClusterId.java     |   5 +-
 hadoop-project/pom.xml                          |   8 +-
 .../dev-support/findbugs-exclude.xml            |  28 +
 hadoop-tools/hadoop-fs2img/pom.xml              |  93 ++
 .../hdfs/server/namenode/BlockResolver.java     |  99 ++
 .../hadoop/hdfs/server/namenode/FSTreeWalk.java | 109 +++
 .../hdfs/server/namenode/FileSystemImage.java   | 152 +++
 .../FixedBlockMultiReplicaResolver.java         |  48 +
 .../server/namenode/FixedBlockResolver.java     |  98 ++
 .../hdfs/server/namenode/FsUGIResolver.java     |  63 ++
 .../hdfs/server/namenode/ImageWriter.java       | 628 +++++++++++++
 .../hdfs/server/namenode/NullBlockAliasMap.java |  97 ++
 .../hdfs/server/namenode/SingleUGIResolver.java |  94 ++
 .../hadoop/hdfs/server/namenode/TreePath.java   | 187 ++++
 .../hadoop/hdfs/server/namenode/TreeWalk.java   | 108 +++
 .../hdfs/server/namenode/UGIResolver.java       | 135 +++
 .../hdfs/server/namenode/package-info.java      |  23 +
 .../namenode/ITestProvidedImplementation.java   | 927 +++++++++++++++++++
 .../hdfs/server/namenode/RandomTreeWalk.java    | 184 ++++
 .../server/namenode/TestFixedBlockResolver.java | 121 +++
 .../server/namenode/TestRandomTreeWalk.java     | 130 +++
 .../server/namenode/TestSingleUGIResolver.java  | 148 +++
 .../src/test/resources/log4j.properties         |  24 +
 hadoop-tools/hadoop-tools-dist/pom.xml          |   6 +
 hadoop-tools/pom.xml                            |   1 +
 118 files changed, 10918 insertions(+), 203 deletions(-)
----------------------------------------------------------------------



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


[19/50] [abbrv] hadoop git commit: YARN-7661. NodeManager metrics return wrong value after update node resource. Contributed by Yang Wang

Posted by na...@apache.org.
YARN-7661. NodeManager metrics return wrong value after update node resource. Contributed by Yang Wang


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

Branch: refs/heads/yarn-3409
Commit: 811fabdebe881248756c0165bf7667bfc22be9bb
Parents: 35ad9b1
Author: Jason Lowe <jl...@apache.org>
Authored: Mon Dec 18 14:28:27 2017 -0600
Committer: Jason Lowe <jl...@apache.org>
Committed: Mon Dec 18 15:20:06 2017 -0600

----------------------------------------------------------------------
 .../yarn/server/nodemanager/metrics/NodeManagerMetrics.java    | 2 +-
 .../server/nodemanager/metrics/TestNodeManagerMetrics.java     | 6 ++++++
 2 files changed, 7 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/811fabde/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/metrics/NodeManagerMetrics.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/metrics/NodeManagerMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/metrics/NodeManagerMetrics.java
index f0abfd4..1e7149b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/metrics/NodeManagerMetrics.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/metrics/NodeManagerMetrics.java
@@ -211,7 +211,7 @@ public class NodeManagerMetrics {
 
   public void addResource(Resource res) {
     availableMB = availableMB + res.getMemorySize();
-    availableGB.incr((int)Math.floor(availableMB/1024d));
+    availableGB.set((int)Math.floor(availableMB/1024d));
     availableVCores.incr(res.getVirtualCores());
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/811fabde/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/metrics/TestNodeManagerMetrics.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/metrics/TestNodeManagerMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/metrics/TestNodeManagerMetrics.java
index a08ee82..5dead91 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/metrics/TestNodeManagerMetrics.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/metrics/TestNodeManagerMetrics.java
@@ -84,6 +84,12 @@ public class TestNodeManagerMetrics {
     // allocatedGB: 3.75GB allocated memory is shown as 4GB
     // availableGB: 4.25GB available memory is shown as 4GB
     checkMetrics(10, 1, 1, 1, 1, 1, 4, 7, 4, 13, 3);
+
+    // Update resource and check available resource again
+    metrics.addResource(total);
+    MetricsRecordBuilder rb = getMetrics("NodeManagerMetrics");
+    assertGauge("AvailableGB", 12, rb);
+    assertGauge("AvailableVCores", 19, rb);
   }
 
   private void checkMetrics(int launched, int completed, int failed, int killed,


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


[39/50] [abbrv] hadoop git commit: YARN-7674. Update Timeline Reader web app address in UI2. Contributed by Sunil G.

Posted by na...@apache.org.
YARN-7674. Update Timeline Reader web app address in UI2. Contributed by Sunil G.


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

Branch: refs/heads/yarn-3409
Commit: 13ad7479b0e35a2c2d398e28c676871d9e672dc3
Parents: a78db99
Author: Rohith Sharma K S <ro...@apache.org>
Authored: Wed Dec 20 22:26:49 2017 +0530
Committer: Rohith Sharma K S <ro...@apache.org>
Committed: Wed Dec 20 22:28:28 2017 +0530

----------------------------------------------------------------------
 .../hadoop-yarn-ui/src/main/webapp/app/initializers/loader.js      | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/13ad7479/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/initializers/loader.js
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/initializers/loader.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/initializers/loader.js
index 55f6e1b..9d63de3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/initializers/loader.js
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/app/initializers/loader.js
@@ -24,7 +24,7 @@ function getTimeLineURL(rmhost) {
   var url = window.location.protocol + '//' +
     (ENV.hosts.localBaseAddress? ENV.hosts.localBaseAddress + '/' : '') + rmhost;
 
-  url += '/conf?name=yarn.timeline-service.webapp.address';
+  url += '/conf?name=yarn.timeline-service.reader.webapp.address';
   Ember.Logger.log("Get Timeline Address URL: " + url);
   return url;
 }


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


[47/50] [abbrv] hadoop git commit: HDFS-12951. Incorrect javadoc in SaslDataTransferServer.java#receive. Contributed by Mukul Kumar Singh.

Posted by na...@apache.org.
HDFS-12951. Incorrect javadoc in SaslDataTransferServer.java#receive. Contributed by Mukul Kumar Singh.


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

Branch: refs/heads/yarn-3409
Commit: 826507c41b7dd89ce5b53d2245d09c2443423670
Parents: b318bed
Author: Chen Liang <cl...@apache.org>
Authored: Thu Dec 21 11:20:30 2017 -0800
Committer: Chen Liang <cl...@apache.org>
Committed: Thu Dec 21 11:20:30 2017 -0800

----------------------------------------------------------------------
 .../hdfs/protocol/datatransfer/sasl/SaslDataTransferServer.java    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/826507c4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferServer.java
index e67d873..e3a72d0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferServer.java
@@ -97,7 +97,7 @@ public class SaslDataTransferServer {
    * @param peer connection peer
    * @param underlyingOut connection output stream
    * @param underlyingIn connection input stream
-   * @param int xferPort data transfer port of DataNode accepting connection
+   * @param xferPort data transfer port of DataNode accepting connection
    * @param datanodeId ID of DataNode accepting connection
    * @return new pair of streams, wrapped after SASL negotiation
    * @throws IOException for any error


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


[37/50] [abbrv] hadoop git commit: YARN-7032. [ATSv2] NPE while starting hbase co-processor when HBase authorization is enabled. Contributed by Rohith Sharma K S.

Posted by na...@apache.org.
YARN-7032. [ATSv2] NPE while starting hbase co-processor when HBase authorization is enabled. Contributed by Rohith Sharma K S.


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

Branch: refs/heads/yarn-3409
Commit: d62932c3b2fcacc81dc1f5048cdeb60fb0d38504
Parents: 41b5810
Author: Sunil G <su...@apache.org>
Authored: Wed Dec 20 11:31:15 2017 +0530
Committer: Sunil G <su...@apache.org>
Committed: Wed Dec 20 11:31:15 2017 +0530

----------------------------------------------------------------------
 .../server/timelineservice/storage/flow/FlowRunCoprocessor.java  | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d62932c3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunCoprocessor.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunCoprocessor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunCoprocessor.java
index 359eec9..96a7cf3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunCoprocessor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunCoprocessor.java
@@ -98,7 +98,9 @@ public class FlowRunCoprocessor extends BaseRegionObserver {
     if ((attributes != null) && (attributes.size() > 0)) {
       for (Map.Entry<String, byte[]> attribute : attributes.entrySet()) {
         Tag t = HBaseTimelineStorageUtils.getTagFromAttribute(attribute);
-        tags.add(t);
+        if (t != null) {
+          tags.add(t);
+        }
       }
       byte[] tagByteArray = Tag.fromList(tags);
       NavigableMap<byte[], List<Cell>> newFamilyMap = new TreeMap<>(


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


[38/50] [abbrv] hadoop git commit: HDFS-12932. Fix confusing LOG message for block replication. Contributed by Chao Sun.

Posted by na...@apache.org.
HDFS-12932. Fix confusing LOG message for block replication. Contributed by Chao Sun.


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

Branch: refs/heads/yarn-3409
Commit: a78db9919065d06ced8122229530f44cc7369857
Parents: d62932c
Author: Wei Yan <we...@apache.org>
Authored: Wed Dec 20 08:55:46 2017 -0800
Committer: Wei Yan <we...@apache.org>
Committed: Wed Dec 20 08:55:46 2017 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java     | 5 ++++-
 1 file changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a78db991/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
index 0dfaa8e..201605f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
@@ -414,9 +414,12 @@ public class FSDirAttrOp {
       if (oldBR > targetReplication) {
         FSDirectory.LOG.info("Decreasing replication from {} to {} for {}",
                              oldBR, targetReplication, iip.getPath());
-      } else {
+      } else if (oldBR < targetReplication) {
         FSDirectory.LOG.info("Increasing replication from {} to {} for {}",
                              oldBR, targetReplication, iip.getPath());
+      } else {
+        FSDirectory.LOG.info("Replication remains unchanged at {} for {}",
+                             oldBR, iip.getPath());
       }
     }
     return file.getBlocks();


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


[08/50] [abbrv] hadoop git commit: HDFS-12905. [READ] Handle decommissioning and under-maintenance Datanodes with Provided storage.

Posted by na...@apache.org.
HDFS-12905. [READ] Handle decommissioning and under-maintenance Datanodes with Provided storage.


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

Branch: refs/heads/yarn-3409
Commit: 0f6aa9564cbe0812a8cab36d999e353269dd6bc9
Parents: 2298f2d
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Fri Dec 8 10:07:40 2017 -0800
Committer: Chris Douglas <cd...@apache.org>
Committed: Fri Dec 15 17:51:41 2017 -0800

----------------------------------------------------------------------
 .../blockmanagement/ProvidedStorageMap.java     | 13 ++-
 .../TestNameNodeProvidedImplementation.java     | 95 ++++++++++++++++++++
 2 files changed, 107 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0f6aa956/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ProvidedStorageMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ProvidedStorageMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ProvidedStorageMap.java
index 7fbc71a..208ed3e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ProvidedStorageMap.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ProvidedStorageMap.java
@@ -342,14 +342,25 @@ public class ProvidedStorageMap {
           return dn;
         }
       }
+      // prefer live nodes first.
+      DatanodeDescriptor dn = chooseRandomNode(excludedUUids, true);
+      if (dn == null) {
+        dn = chooseRandomNode(excludedUUids, false);
+      }
+      return dn;
+    }
 
+    private DatanodeDescriptor chooseRandomNode(Set<String> excludedUUids,
+        boolean preferLiveNodes) {
       Random r = new Random();
       for (int i = dnR.size() - 1; i >= 0; --i) {
         int pos = r.nextInt(i + 1);
         DatanodeDescriptor node = dnR.get(pos);
         String uuid = node.getDatanodeUuid();
         if (!excludedUUids.contains(uuid)) {
-          return node;
+          if (!preferLiveNodes || node.getAdminState() == AdminStates.NORMAL) {
+            return node;
+          }
         }
         Collections.swap(dnR, i, pos);
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0f6aa956/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java
index d057247..394e8d8 100644
--- a/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java
+++ b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java
@@ -56,6 +56,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStatistics;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.ProvidedStorageMap;
@@ -795,4 +796,98 @@ public class TestNameNodeProvidedImplementation {
     FileUtils.deleteDirectory(tempDirectory);
   }
 
+  private DatanodeDescriptor getDatanodeDescriptor(DatanodeManager dnm,
+      int dnIndex) throws Exception {
+    return dnm.getDatanode(cluster.getDataNodes().get(dnIndex).getDatanodeId());
+  }
+
+  private void startDecommission(FSNamesystem namesystem, DatanodeManager dnm,
+      int dnIndex) throws Exception {
+    namesystem.writeLock();
+    DatanodeDescriptor dnDesc = getDatanodeDescriptor(dnm, dnIndex);
+    dnm.getDatanodeAdminManager().startDecommission(dnDesc);
+    namesystem.writeUnlock();
+  }
+
+  private void startMaintenance(FSNamesystem namesystem, DatanodeManager dnm,
+      int dnIndex) throws Exception {
+    namesystem.writeLock();
+    DatanodeDescriptor dnDesc = getDatanodeDescriptor(dnm, dnIndex);
+    dnm.getDatanodeAdminManager().startMaintenance(dnDesc, Long.MAX_VALUE);
+    namesystem.writeUnlock();
+  }
+
+  private void stopMaintenance(FSNamesystem namesystem, DatanodeManager dnm,
+      int dnIndex) throws Exception {
+    namesystem.writeLock();
+    DatanodeDescriptor dnDesc = getDatanodeDescriptor(dnm, dnIndex);
+    dnm.getDatanodeAdminManager().stopMaintenance(dnDesc);
+    namesystem.writeUnlock();
+  }
+
+  @Test
+  public void testDatanodeLifeCycle() throws Exception {
+    createImage(new FSTreeWalk(NAMEPATH, conf), NNDIRPATH,
+        FixedBlockResolver.class);
+    startCluster(NNDIRPATH, 3,
+        new StorageType[] {StorageType.PROVIDED, StorageType.DISK},
+        null, false);
+
+    int fileIndex = numFiles -1;
+
+    final BlockManager blockManager = cluster.getNamesystem().getBlockManager();
+    final DatanodeManager dnm = blockManager.getDatanodeManager();
+
+    // to start, all 3 DNs are live in ProvidedDatanodeDescriptor.
+    verifyFileLocation(fileIndex, 3);
+
+    // de-commision first DN; still get 3 replicas.
+    startDecommission(cluster.getNamesystem(), dnm, 0);
+    verifyFileLocation(fileIndex, 3);
+
+    // remains the same even after heartbeats.
+    cluster.triggerHeartbeats();
+    verifyFileLocation(fileIndex, 3);
+
+    // start maintenance for 2nd DN; still get 3 replicas.
+    startMaintenance(cluster.getNamesystem(), dnm, 1);
+    verifyFileLocation(fileIndex, 3);
+
+    DataNode dn1 = cluster.getDataNodes().get(0);
+    DataNode dn2 = cluster.getDataNodes().get(1);
+
+    // stop the 1st DN while being decomissioned.
+    MiniDFSCluster.DataNodeProperties dn1Properties = cluster.stopDataNode(0);
+    BlockManagerTestUtil.noticeDeadDatanode(cluster.getNameNode(),
+        dn1.getDatanodeId().getXferAddr());
+
+    // get 2 locations
+    verifyFileLocation(fileIndex, 2);
+
+    // stop dn2 while in maintenance.
+    MiniDFSCluster.DataNodeProperties dn2Properties = cluster.stopDataNode(1);
+    BlockManagerTestUtil.noticeDeadDatanode(cluster.getNameNode(),
+        dn2.getDatanodeId().getXferAddr());
+
+    // 2 valid locations will be found as blocks on nodes that die during
+    // maintenance are not marked for removal.
+    verifyFileLocation(fileIndex, 2);
+
+    // stop the maintenance; get only 1 replicas
+    stopMaintenance(cluster.getNamesystem(), dnm, 0);
+    verifyFileLocation(fileIndex, 1);
+
+    // restart the stopped DN.
+    cluster.restartDataNode(dn1Properties, true);
+    cluster.waitActive();
+
+    // reports all 3 replicas
+    verifyFileLocation(fileIndex, 2);
+
+    cluster.restartDataNode(dn2Properties, true);
+    cluster.waitActive();
+
+    // reports all 3 replicas
+    verifyFileLocation(fileIndex, 3);
+  }
 }


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


[21/50] [abbrv] hadoop git commit: HDFS-12347. TestBalancerRPCDelay#testBalancerRPCDelay fails very frequently. Contributed by Bharat Viswanadham

Posted by na...@apache.org.
HDFS-12347. TestBalancerRPCDelay#testBalancerRPCDelay fails very frequently.  Contributed by Bharat Viswanadham


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

Branch: refs/heads/yarn-3409
Commit: c7499f2d242c64bee8f822a22161d956525f7153
Parents: c7a4dda
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Tue Dec 19 10:02:30 2017 +0800
Committer: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Committed: Tue Dec 19 10:02:30 2017 +0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c7499f2d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
index a900ad1..9452b8b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
@@ -2084,7 +2084,7 @@ public class TestBalancer {
     initConf(conf);
     conf.setInt(DFSConfigKeys.DFS_BALANCER_DISPATCHERTHREADS_KEY, 30);
 
-    int numDNs = 40;
+    int numDNs = 20;
     long[] capacities = new long[numDNs];
     String[] racks = new String[numDNs];
     for(int i = 0; i < numDNs; i++) {


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


[10/50] [abbrv] hadoop git commit: HDFS-12903. [READ] Fix closing streams in ImageWriter

Posted by na...@apache.org.
HDFS-12903. [READ] Fix closing streams in ImageWriter


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

Branch: refs/heads/yarn-3409
Commit: 962b5e722ba86d1c012be11280c6b8fb5e0a2043
Parents: 71ec170
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Thu Dec 7 14:21:24 2017 -0800
Committer: Chris Douglas <cd...@apache.org>
Committed: Fri Dec 15 17:51:41 2017 -0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/server/namenode/ImageWriter.java | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/962b5e72/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageWriter.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageWriter.java b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageWriter.java
index 0abc7a7..c21c282 100644
--- a/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageWriter.java
+++ b/hadoop-tools/hadoop-fs2img/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageWriter.java
@@ -183,9 +183,9 @@ public class ImageWriter implements Closeable {
       dirsTmp.deleteOnExit();
       dirsTmpStream = new FileOutputStream(dirsTmp);
       dirs = beginSection(dirsTmpStream);
-    } catch (IOException e) {
+    } catch (Throwable e) {
       IOUtils.cleanupWithLogger(null, raw, dirsTmpStream);
-      throw e;
+      throw new IOException(e);
     }
 
     try {


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


[05/50] [abbrv] hadoop git commit: HDFS-12887. [READ] Allow Datanodes with Provided volumes to start when blocks with the same id exist locally

Posted by na...@apache.org.
HDFS-12887. [READ] Allow Datanodes with Provided volumes to start when blocks with the same id exist locally


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

Branch: refs/heads/yarn-3409
Commit: 71ec170107e67e42cdbc5052c3f7b23c64751835
Parents: 4531588
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Wed Dec 6 09:42:31 2017 -0800
Committer: Chris Douglas <cd...@apache.org>
Committed: Fri Dec 15 17:51:41 2017 -0800

----------------------------------------------------------------------
 .../hdfs/server/datanode/fsdataset/impl/ProvidedVolumeImpl.java  | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/71ec1701/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeImpl.java
index f65fbbc..59ec100 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeImpl.java
@@ -208,8 +208,8 @@ class ProvidedVolumeImpl extends FsVolumeImpl {
             incrNumBlocks();
             incDfsUsed(region.getBlock().getNumBytes());
           } else {
-            throw new IOException("A block with id " + newReplica.getBlockId()
-                + " already exists in the volumeMap");
+            LOG.warn("A block with id " + newReplica.getBlockId()
+                + " exists locally. Skipping PROVIDED replica");
           }
         }
       }


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


[34/50] [abbrv] hadoop git commit: YARN-7543. Add check for max cpu limit and missing file for YARN service. (Contributed by Jian He)

Posted by na...@apache.org.
YARN-7543.  Add check for max cpu limit and missing file for YARN service.
            (Contributed by Jian He)


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

Branch: refs/heads/yarn-3409
Commit: 989c75109a619deeaee7461864e7cb3c289c9421
Parents: c0aeb66
Author: Eric Yang <ey...@apache.org>
Authored: Tue Dec 19 16:45:04 2017 -0500
Committer: Eric Yang <ey...@apache.org>
Committed: Tue Dec 19 16:45:04 2017 -0500

----------------------------------------------------------------------
 .../hadoop/yarn/service/utils/ServiceApiUtil.java     | 14 ++++++++++----
 .../hadoop/yarn/service/utils/ServiceUtils.java       |  4 ++++
 2 files changed, 14 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/989c7510/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/ServiceApiUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/ServiceApiUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/ServiceApiUtil.java
index d5ea45c..7f85c95 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/ServiceApiUtil.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/ServiceApiUtil.java
@@ -330,13 +330,19 @@ public class ServiceApiUtil {
       org.apache.hadoop.yarn.api.records.Resource maxResource,
       Service service) throws YarnException {
     for (Component component : service.getComponents()) {
-      // only handle mem now.
       long mem = Long.parseLong(component.getResource().getMemory());
       if (mem > maxResource.getMemorySize()) {
         throw new YarnException(
-            "Component " + component.getName() + " memory size (" + mem
-                + ") is larger than configured max container memory size ("
-                + maxResource.getMemorySize() + ")");
+            "Component " + component.getName() + ": specified memory size ("
+                + mem + ") is larger than configured max container memory " +
+                "size (" + maxResource.getMemorySize() + ")");
+      }
+      int cpu = component.getResource().getCpus();
+      if (cpu > maxResource.getVirtualCores()) {
+        throw new YarnException(
+            "Component " + component.getName() + ": specified number of " +
+                "virtual core (" + cpu + ") is larger than configured max " +
+                "virtual core size (" + maxResource.getVirtualCores() + ")");
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/989c7510/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/ServiceUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/ServiceUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/ServiceUtils.java
index 173001b..915b836 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/ServiceUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/ServiceUtils.java
@@ -411,6 +411,10 @@ public final class ServiceUtils {
         return;
       }
       for (File jarFile : listOfJars) {
+        if (!jarFile.exists()) {
+          log.debug("File does not exist, skipping: " + jarFile);
+          continue;
+        }
         LocalResource res = sliderFileSystem.submitFile(jarFile, tempPath, libDir, jarFile.getName());
         providerResources.put(libDir + "/" + jarFile.getName(), res);
       }


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


[24/50] [abbrv] hadoop git commit: Add 2.8.3 release jdiff files.

Posted by na...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/a7f8caf5/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Common_2.8.3.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Common_2.8.3.xml b/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Common_2.8.3.xml
new file mode 100644
index 0000000..6826c25
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Common_2.8.3.xml
@@ -0,0 +1,2665 @@
+<?xml version="1.0" encoding="iso-8859-1" standalone="no"?>
+<!-- Generated by the JDiff Javadoc doclet -->
+<!-- (http://www.jdiff.org) -->
+<!-- on Tue Dec 05 05:29:28 UTC 2017 -->
+
+<api
+  xmlns:xsi='http://www.w3.org/2001/XMLSchema-instance'
+  xsi:noNamespaceSchemaLocation='api.xsd'
+  name="Apache Hadoop YARN Common 2.8.3"
+  jdversion="1.0.9">
+
+<!--  Command line arguments =  -doclet org.apache.hadoop.classification.tools.IncludePublicAnnotationsJDiffDoclet -docletpath /build/source/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/target/hadoop-annotations.jar:/build/source/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/target/jdiff.jar -verbose -classpath /build/source/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/target/classes:/build/source/hadoop-common-project/hadoop-common/target/hadoop-common-2.8.3.jar:/maven/org/apache/commons/commons-math3/3.1.1/commons-math3-3.1.1.jar:/maven/xmlenc/xmlenc/0.52/xmlenc-0.52.jar:/maven/org/apache/httpcomponents/httpclient/4.5.2/httpclient-4.5.2.jar:/maven/org/apache/httpcomponents/httpcore/4.4.4/httpcore-4.4.4.jar:/maven/commons-net/commons-net/3.1/commons-net-3.1.jar:/maven/commons-collections/commons-collections/3.2.2/commons-collections-3.2.2.jar:/maven/org/mortbay/jetty/jetty/6.1.26/jetty-6.1.26.jar:/maven/org/mortbay/jetty/jetty-sslengine/6.1.26/jetty-sslengine-6.1.2
 6.jar:/maven/javax/servlet/jsp/jsp-api/2.1/jsp-api-2.1.jar:/maven/net/java/dev/jets3t/jets3t/0.9.0/jets3t-0.9.0.jar:/maven/com/jamesmurty/utils/java-xmlbuilder/0.4/java-xmlbuilder-0.4.jar:/maven/commons-configuration/commons-configuration/1.6/commons-configuration-1.6.jar:/maven/commons-digester/commons-digester/1.8/commons-digester-1.8.jar:/maven/commons-beanutils/commons-beanutils/1.7.0/commons-beanutils-1.7.0.jar:/maven/commons-beanutils/commons-beanutils-core/1.8.0/commons-beanutils-core-1.8.0.jar:/maven/org/slf4j/slf4j-log4j12/1.7.10/slf4j-log4j12-1.7.10.jar:/maven/org/apache/avro/avro/1.7.4/avro-1.7.4.jar:/maven/com/thoughtworks/paranamer/paranamer/2.3/paranamer-2.3.jar:/maven/org/xerial/snappy/snappy-java/1.0.4.1/snappy-java-1.0.4.1.jar:/maven/com/google/code/gson/gson/2.2.4/gson-2.2.4.jar:/build/source/hadoop-common-project/hadoop-auth/target/hadoop-auth-2.8.3.jar:/maven/com/nimbusds/nimbus-jose-jwt/3.9/nimbus-jose-jwt-3.9.jar:/maven/net/jcip/jcip-annotations/1.0/jcip-annota
 tions-1.0.jar:/maven/net/minidev/json-smart/1.1.1/json-smart-1.1.1.jar:/maven/org/apache/directory/server/apacheds-kerberos-codec/2.0.0-M15/apacheds-kerberos-codec-2.0.0-M15.jar:/maven/org/apache/directory/server/apacheds-i18n/2.0.0-M15/apacheds-i18n-2.0.0-M15.jar:/maven/org/apache/directory/api/api-asn1-api/1.0.0-M20/api-asn1-api-1.0.0-M20.jar:/maven/org/apache/directory/api/api-util/1.0.0-M20/api-util-1.0.0-M20.jar:/maven/org/apache/curator/curator-framework/2.7.1/curator-framework-2.7.1.jar:/maven/com/jcraft/jsch/0.1.54/jsch-0.1.54.jar:/maven/org/apache/curator/curator-client/2.7.1/curator-client-2.7.1.jar:/maven/org/apache/curator/curator-recipes/2.7.1/curator-recipes-2.7.1.jar:/maven/com/google/code/findbugs/jsr305/3.0.0/jsr305-3.0.0.jar:/maven/org/apache/htrace/htrace-core4/4.0.1-incubating/htrace-core4-4.0.1-incubating.jar:/maven/org/apache/zookeeper/zookeeper/3.4.6/zookeeper-3.4.6.jar:/maven/io/netty/netty/3.6.2.Final/netty-3.6.2.Final.jar:/build/source/hadoop-yarn-project/h
 adoop-yarn/hadoop-yarn-api/target/hadoop-yarn-api-2.8.3.jar:/maven/javax/xml/bind/jaxb-api/2.2.2/jaxb-api-2.2.2.jar:/maven/javax/xml/stream/stax-api/1.0-2/stax-api-1.0-2.jar:/maven/javax/activation/activation/1.1/activation-1.1.jar:/maven/org/apache/commons/commons-compress/1.4.1/commons-compress-1.4.1.jar:/maven/org/tukaani/xz/1.0/xz-1.0.jar:/maven/commons-lang/commons-lang/2.6/commons-lang-2.6.jar:/maven/javax/servlet/servlet-api/2.5/servlet-api-2.5.jar:/maven/commons-codec/commons-codec/1.4/commons-codec-1.4.jar:/maven/org/mortbay/jetty/jetty-util/6.1.26/jetty-util-6.1.26.jar:/maven/com/sun/jersey/jersey-core/1.9/jersey-core-1.9.jar:/maven/com/sun/jersey/jersey-client/1.9/jersey-client-1.9.jar:/maven/org/codehaus/jackson/jackson-core-asl/1.9.13/jackson-core-asl-1.9.13.jar:/maven/org/codehaus/jackson/jackson-mapper-asl/1.9.13/jackson-mapper-asl-1.9.13.jar:/maven/org/codehaus/jackson/jackson-jaxrs/1.9.13/jackson-jaxrs-1.9.13.jar:/maven/org/codehaus/jackson/jackson-xc/1.9.13/jackson
 -xc-1.9.13.jar:/maven/com/google/guava/guava/11.0.2/guava-11.0.2.jar:/maven/commons-logging/commons-logging/1.1.3/commons-logging-1.1.3.jar:/maven/commons-cli/commons-cli/1.2/commons-cli-1.2.jar:/maven/org/slf4j/slf4j-api/1.7.10/slf4j-api-1.7.10.jar:/build/source/hadoop-common-project/hadoop-annotations/target/hadoop-annotations-2.8.3.jar:/usr/lib/jvm/java-7-openjdk-amd64/lib/tools.jar:/maven/com/google/inject/extensions/guice-servlet/3.0/guice-servlet-3.0.jar:/maven/com/google/protobuf/protobuf-java/2.5.0/protobuf-java-2.5.0.jar:/maven/commons-io/commons-io/2.4/commons-io-2.4.jar:/maven/com/google/inject/guice/3.0/guice-3.0.jar:/maven/javax/inject/javax.inject/1/javax.inject-1.jar:/maven/aopalliance/aopalliance/1.0/aopalliance-1.0.jar:/maven/com/sun/jersey/jersey-server/1.9/jersey-server-1.9.jar:/maven/asm/asm/3.2/asm-3.2.jar:/maven/com/sun/jersey/jersey-json/1.9/jersey-json-1.9.jar:/maven/org/codehaus/jettison/jettison/1.1/jettison-1.1.jar:/maven/com/sun/xml/bind/jaxb-impl/2.2.3-1
 /jaxb-impl-2.2.3-1.jar:/maven/com/sun/jersey/contribs/jersey-guice/1.9/jersey-guice-1.9.jar:/maven/log4j/log4j/1.2.17/log4j-1.2.17.jar -sourcepath /build/source/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java -apidir /build/source/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/target/site/jdiff/xml -apiname Apache Hadoop YARN Common 2.8.3 -->
+<package name="org.apache.hadoop.yarn">
+  <!-- start class org.apache.hadoop.yarn.ContainerLogAppender -->
+  <class name="ContainerLogAppender" extends="org.apache.log4j.FileAppender"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="java.io.Flushable"/>
+    <constructor name="ContainerLogAppender"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="activateOptions"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="append"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="event" type="org.apache.log4j.spi.LoggingEvent"/>
+    </method>
+    <method name="flush"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="close"
+      abstract="false" native="false" synchronized="true"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getContainerLogDir" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Getter/Setter methods for log4j.]]>
+      </doc>
+    </method>
+    <method name="setContainerLogDir"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="containerLogDir" type="java.lang.String"/>
+    </method>
+    <method name="getContainerLogFile" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="setContainerLogFile"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="containerLogFile" type="java.lang.String"/>
+    </method>
+    <method name="getTotalLogFileSize" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="setTotalLogFileSize"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="logSize" type="long"/>
+    </method>
+    <doc>
+    <![CDATA[A simple log4j-appender for container's logs.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.ContainerLogAppender -->
+  <!-- start class org.apache.hadoop.yarn.ContainerRollingLogAppender -->
+  <class name="ContainerRollingLogAppender" extends="org.apache.log4j.RollingFileAppender"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="java.io.Flushable"/>
+    <constructor name="ContainerRollingLogAppender"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="activateOptions"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="flush"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getContainerLogDir" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Getter/Setter methods for log4j.]]>
+      </doc>
+    </method>
+    <method name="setContainerLogDir"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="containerLogDir" type="java.lang.String"/>
+    </method>
+    <method name="getContainerLogFile" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="setContainerLogFile"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="containerLogFile" type="java.lang.String"/>
+    </method>
+    <doc>
+    <![CDATA[A simple log4j-appender for container's logs.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.ContainerRollingLogAppender -->
+  <!-- start class org.apache.hadoop.yarn.YarnUncaughtExceptionHandler -->
+  <class name="YarnUncaughtExceptionHandler" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="java.lang.Thread.UncaughtExceptionHandler"/>
+    <constructor name="YarnUncaughtExceptionHandler"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="uncaughtException"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="t" type="java.lang.Thread"/>
+      <param name="e" type="java.lang.Throwable"/>
+    </method>
+    <doc>
+    <![CDATA[This class is intended to be installed by calling 
+ {@link Thread#setDefaultUncaughtExceptionHandler(UncaughtExceptionHandler)}
+ In the main entry point.  It is intended to try and cleanly shut down
+ programs using the Yarn Event framework.
+ 
+ Note: Right now it only will shut down the program if a Error is caught, but
+ not any other exception.  Anything else is just logged.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.YarnUncaughtExceptionHandler -->
+</package>
+<package name="org.apache.hadoop.yarn.api">
+</package>
+<package name="org.apache.hadoop.yarn.client">
+  <!-- start class org.apache.hadoop.yarn.client.AHSProxy -->
+  <class name="AHSProxy" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="AHSProxy"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="createAHSProxy" return="T"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <param name="protocol" type="java.lang.Class"/>
+      <param name="ahsAddress" type="java.net.InetSocketAddress"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="getProxy" return="T"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <param name="protocol" type="java.lang.Class"/>
+      <param name="rmAddress" type="java.net.InetSocketAddress"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.AHSProxy -->
+  <!-- start class org.apache.hadoop.yarn.client.ClientRMProxy -->
+  <class name="ClientRMProxy" extends="org.apache.hadoop.yarn.client.RMProxy"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <method name="createRMProxy" return="T"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="configuration" type="org.apache.hadoop.conf.Configuration"/>
+      <param name="protocol" type="java.lang.Class"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Create a proxy to the ResourceManager for the specified protocol.
+ @param configuration Configuration with all the required information.
+ @param protocol Client protocol for which proxy is being requested.
+ @param <T> Type of proxy.
+ @return Proxy to the ResourceManager for the specified client protocol.
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getRMDelegationTokenService" return="org.apache.hadoop.io.Text"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <doc>
+      <![CDATA[Get the token service name to be used for RMDelegationToken. Depending
+ on whether HA is enabled or not, this method generates the appropriate
+ service name as a comma-separated list of service addresses.
+
+ @param conf Configuration corresponding to the cluster we need the
+             RMDelegationToken for
+ @return - Service name for RMDelegationToken]]>
+      </doc>
+    </method>
+    <method name="getAMRMTokenService" return="org.apache.hadoop.io.Text"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+    </method>
+    <method name="getTokenService" return="org.apache.hadoop.io.Text"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <param name="address" type="java.lang.String"/>
+      <param name="defaultAddr" type="java.lang.String"/>
+      <param name="defaultPort" type="int"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.ClientRMProxy -->
+  <!-- start class org.apache.hadoop.yarn.client.NMProxy -->
+  <class name="NMProxy" extends="org.apache.hadoop.yarn.client.ServerProxy"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="NMProxy"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="createNMProxy" return="T"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <param name="protocol" type="java.lang.Class"/>
+      <param name="ugi" type="org.apache.hadoop.security.UserGroupInformation"/>
+      <param name="rpc" type="org.apache.hadoop.yarn.ipc.YarnRPC"/>
+      <param name="serverAddress" type="java.net.InetSocketAddress"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.NMProxy -->
+  <!-- start class org.apache.hadoop.yarn.client.RMProxy -->
+  <class name="RMProxy" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="RMProxy"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="createRMProxy" return="T"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="This method is deprecated and is not used by YARN internally any more.
+ To create a proxy to the RM, use ClientRMProxy#createRMProxy or
+ ServerRMProxy#createRMProxy.
+
+ Create a proxy to the ResourceManager at the specified address.">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <param name="protocol" type="java.lang.Class"/>
+      <param name="rmAddress" type="java.net.InetSocketAddress"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[@deprecated
+ This method is deprecated and is not used by YARN internally any more.
+ To create a proxy to the RM, use ClientRMProxy#createRMProxy or
+ ServerRMProxy#createRMProxy.
+
+ Create a proxy to the ResourceManager at the specified address.
+
+ @param conf Configuration to generate retry policy
+ @param protocol Protocol for the proxy
+ @param rmAddress Address of the ResourceManager
+ @param <T> Type information of the proxy
+ @return Proxy to the RM
+ @throws IOException]]>
+      </doc>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.RMProxy -->
+  <!-- start class org.apache.hadoop.yarn.client.ServerProxy -->
+  <class name="ServerProxy" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="ServerProxy"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="createRetryPolicy" return="org.apache.hadoop.io.retry.RetryPolicy"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <param name="maxWaitTimeStr" type="java.lang.String"/>
+      <param name="defMaxWaitTime" type="long"/>
+      <param name="connectRetryIntervalStr" type="java.lang.String"/>
+      <param name="defRetryInterval" type="long"/>
+    </method>
+    <method name="createRetriableProxy" return="T"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <param name="protocol" type="java.lang.Class"/>
+      <param name="user" type="org.apache.hadoop.security.UserGroupInformation"/>
+      <param name="rpc" type="org.apache.hadoop.yarn.ipc.YarnRPC"/>
+      <param name="serverAddress" type="java.net.InetSocketAddress"/>
+      <param name="retryPolicy" type="org.apache.hadoop.io.retry.RetryPolicy"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.ServerProxy -->
+</package>
+<package name="org.apache.hadoop.yarn.client.api">
+  <!-- start class org.apache.hadoop.yarn.client.api.TimelineClient -->
+  <class name="TimelineClient" extends="org.apache.hadoop.service.AbstractService"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="java.io.Flushable"/>
+    <constructor name="TimelineClient" type="java.lang.String"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="createTimelineClient" return="org.apache.hadoop.yarn.client.api.TimelineClient"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Create a timeline client. The current UGI when the user initialize the
+ client will be used to do the put and the delegation token operations. The
+ current user may use {@link UserGroupInformation#doAs} another user to
+ construct and initialize a timeline client if the following operations are
+ supposed to be conducted by that user.
+
+ @return a timeline client]]>
+      </doc>
+    </method>
+    <method name="putEntities" return="org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="entities" type="org.apache.hadoop.yarn.api.records.timeline.TimelineEntity[]"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <doc>
+      <![CDATA[<p>
+ Send the information of a number of conceptual entities to the timeline
+ server. It is a blocking API. The method will not return until it gets the
+ response from the timeline server.
+ </p>
+ 
+ @param entities
+          the collection of {@link TimelineEntity}
+ @return the error information if the sent entities are not correctly stored
+ @throws IOException
+ @throws YarnException]]>
+      </doc>
+    </method>
+    <method name="putEntities" return="org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appAttemptId" type="org.apache.hadoop.yarn.api.records.ApplicationAttemptId"/>
+      <param name="groupId" type="org.apache.hadoop.yarn.api.records.timeline.TimelineEntityGroupId"/>
+      <param name="entities" type="org.apache.hadoop.yarn.api.records.timeline.TimelineEntity[]"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <doc>
+      <![CDATA[<p>
+ Send the information of a number of conceptual entities to the timeline
+ server. It is a blocking API. The method will not return until it gets the
+ response from the timeline server.
+
+ This API is only for timeline service v1.5
+ </p>
+
+ @param appAttemptId {@link ApplicationAttemptId}
+ @param groupId {@link TimelineEntityGroupId}
+ @param entities
+          the collection of {@link TimelineEntity}
+ @return the error information if the sent entities are not correctly stored
+ @throws IOException
+ @throws YarnException]]>
+      </doc>
+    </method>
+    <method name="putDomain"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="domain" type="org.apache.hadoop.yarn.api.records.timeline.TimelineDomain"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <doc>
+      <![CDATA[<p>
+ Send the information of a domain to the timeline server. It is a
+ blocking API. The method will not return until it gets the response from
+ the timeline server.
+ </p>
+ 
+ @param domain
+          an {@link TimelineDomain} object
+ @throws IOException
+ @throws YarnException]]>
+      </doc>
+    </method>
+    <method name="putDomain"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appAttemptId" type="org.apache.hadoop.yarn.api.records.ApplicationAttemptId"/>
+      <param name="domain" type="org.apache.hadoop.yarn.api.records.timeline.TimelineDomain"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <doc>
+      <![CDATA[<p>
+ Send the information of a domain to the timeline server. It is a
+ blocking API. The method will not return until it gets the response from
+ the timeline server.
+
+ This API is only for timeline service v1.5
+ </p>
+
+ @param domain
+          an {@link TimelineDomain} object
+ @param appAttemptId {@link ApplicationAttemptId}
+ @throws IOException
+ @throws YarnException]]>
+      </doc>
+    </method>
+    <method name="getDelegationToken" return="org.apache.hadoop.security.token.Token"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="renewer" type="java.lang.String"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <doc>
+      <![CDATA[<p>
+ Get a delegation token so as to be able to talk to the timeline server in a
+ secure way.
+ </p>
+ 
+ @param renewer
+          Address of the renewer who can renew these tokens when needed by
+          securely talking to the timeline server
+ @return a delegation token ({@link Token}) that can be used to talk to the
+         timeline server
+ @throws IOException
+ @throws YarnException]]>
+      </doc>
+    </method>
+    <method name="renewDelegationToken" return="long"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="timelineDT" type="org.apache.hadoop.security.token.Token"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <doc>
+      <![CDATA[<p>
+ Renew a timeline delegation token.
+ </p>
+ 
+ @param timelineDT
+          the delegation token to renew
+ @return the new expiration time
+ @throws IOException
+ @throws YarnException]]>
+      </doc>
+    </method>
+    <method name="cancelDelegationToken"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="timelineDT" type="org.apache.hadoop.security.token.Token"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <exception name="YarnException" type="org.apache.hadoop.yarn.exceptions.YarnException"/>
+      <doc>
+      <![CDATA[<p>
+ Cancel a timeline delegation token.
+ </p>
+ 
+ @param timelineDT
+          the delegation token to cancel
+ @throws IOException
+ @throws YarnException]]>
+      </doc>
+    </method>
+    <doc>
+    <![CDATA[A client library that can be used to post some information in terms of a
+ number of conceptual entities.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.client.api.TimelineClient -->
+</package>
+<package name="org.apache.hadoop.yarn.client.api.impl">
+</package>
+<package name="org.apache.hadoop.yarn.event">
+  <!-- start class org.apache.hadoop.yarn.event.AbstractEvent -->
+  <class name="AbstractEvent" extends="java.lang.Object"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="org.apache.hadoop.yarn.event.Event"/>
+    <constructor name="AbstractEvent" type="TYPE"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="AbstractEvent" type="TYPE, long"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="getTimestamp" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getType" return="TYPE"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="toString" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <doc>
+    <![CDATA[Parent class of all the events. All events extend this class.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.event.AbstractEvent -->
+  <!-- start class org.apache.hadoop.yarn.event.AsyncDispatcher -->
+  <class name="AsyncDispatcher" extends="org.apache.hadoop.service.AbstractService"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="org.apache.hadoop.yarn.event.Dispatcher"/>
+    <constructor name="AsyncDispatcher"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="AsyncDispatcher" type="java.util.concurrent.BlockingQueue"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="serviceInit"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <exception name="Exception" type="java.lang.Exception"/>
+    </method>
+    <method name="serviceStart"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <exception name="Exception" type="java.lang.Exception"/>
+    </method>
+    <method name="setDrainEventsOnStop"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="serviceStop"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <exception name="Exception" type="java.lang.Exception"/>
+    </method>
+    <method name="dispatch"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+      <param name="event" type="org.apache.hadoop.yarn.event.Event"/>
+    </method>
+    <method name="register"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="eventType" type="java.lang.Class"/>
+      <param name="handler" type="org.apache.hadoop.yarn.event.EventHandler"/>
+    </method>
+    <method name="getEventHandler" return="org.apache.hadoop.yarn.event.EventHandler"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="isEventThreadWaiting" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </method>
+    <method name="isDrained" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </method>
+    <field name="eventDispatchers" type="java.util.Map"
+      transient="false" volatile="false"
+      static="false" final="true" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <doc>
+    <![CDATA[Dispatches {@link Event}s in a separate thread. Currently only single thread
+ does that. Potentially there could be multiple channels for each event type
+ class and a thread pool can be used to dispatch the events.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.event.AsyncDispatcher -->
+  <!-- start interface org.apache.hadoop.yarn.event.Dispatcher -->
+  <interface name="Dispatcher"    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <method name="getEventHandler" return="org.apache.hadoop.yarn.event.EventHandler"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="register"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="eventType" type="java.lang.Class"/>
+      <param name="handler" type="org.apache.hadoop.yarn.event.EventHandler"/>
+    </method>
+    <field name="DISPATCHER_EXIT_ON_ERROR_KEY" type="java.lang.String"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+    </field>
+    <field name="DEFAULT_DISPATCHER_EXIT_ON_ERROR" type="boolean"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+    </field>
+    <doc>
+    <![CDATA[Event Dispatcher interface. It dispatches events to registered 
+ event handlers based on event types.]]>
+    </doc>
+  </interface>
+  <!-- end interface org.apache.hadoop.yarn.event.Dispatcher -->
+  <!-- start interface org.apache.hadoop.yarn.event.Event -->
+  <interface name="Event"    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <method name="getType" return="TYPE"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getTimestamp" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="toString" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <doc>
+    <![CDATA[Interface defining events api.]]>
+    </doc>
+  </interface>
+  <!-- end interface org.apache.hadoop.yarn.event.Event -->
+  <!-- start interface org.apache.hadoop.yarn.event.EventHandler -->
+  <interface name="EventHandler"    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <method name="handle"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="event" type="T"/>
+    </method>
+    <doc>
+    <![CDATA[Interface for handling events of type T
+
+ @param <T> parameterized event of type T]]>
+    </doc>
+  </interface>
+  <!-- end interface org.apache.hadoop.yarn.event.EventHandler -->
+</package>
+<package name="org.apache.hadoop.yarn.factories">
+</package>
+<package name="org.apache.hadoop.yarn.factory.providers">
+</package>
+<package name="org.apache.hadoop.yarn.logaggregation">
+  <!-- start class org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat -->
+  <class name="AggregatedLogFormat" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="AggregatedLogFormat"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat -->
+  <!-- start class org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogKey -->
+  <class name="AggregatedLogFormat.LogKey" extends="java.lang.Object"
+    abstract="false"
+    static="true" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="org.apache.hadoop.io.Writable"/>
+    <constructor name="AggregatedLogFormat.LogKey"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="AggregatedLogFormat.LogKey" type="org.apache.hadoop.yarn.api.records.ContainerId"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="AggregatedLogFormat.LogKey" type="java.lang.String"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="hashCode" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="equals" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="obj" type="java.lang.Object"/>
+    </method>
+    <method name="toString" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogKey -->
+  <!-- start class org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogReader -->
+  <class name="AggregatedLogFormat.LogReader" extends="java.lang.Object"
+    abstract="false"
+    static="true" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="AggregatedLogFormat.LogReader" type="org.apache.hadoop.conf.Configuration, org.apache.hadoop.fs.Path"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="IOException" type="java.io.IOException"/>
+    </constructor>
+    <method name="getApplicationOwner" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Returns the owner of the application.
+ 
+ @return the application owner.
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="getApplicationAcls" return="java.util.Map"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Returns ACLs for the application. An empty map is returned if no ACLs are
+ found.
+ 
+ @return a map of the Application ACLs.
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="next" return="java.io.DataInputStream"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="key" type="org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogKey"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Read the next key and return the value-stream.
+ 
+ @param key
+ @return the valueStream if there are more keys or null otherwise.
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="readAcontainerLogs"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="valueStream" type="java.io.DataInputStream"/>
+      <param name="writer" type="java.io.Writer"/>
+      <param name="logUploadedTime" type="long"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Writes all logs for a single container to the provided writer.
+ @param valueStream
+ @param writer
+ @param logUploadedTime
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="readAcontainerLogs"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="valueStream" type="java.io.DataInputStream"/>
+      <param name="writer" type="java.io.Writer"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Writes all logs for a single container to the provided writer.
+ @param valueStream
+ @param writer
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="readAContainerLogsForALogType"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="valueStream" type="java.io.DataInputStream"/>
+      <param name="out" type="java.io.PrintStream"/>
+      <param name="logUploadedTime" type="long"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Keep calling this till you get a {@link EOFException} for getting logs of
+ all types for a single container.
+ 
+ @param valueStream
+ @param out
+ @param logUploadedTime
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="readAContainerLogsForALogType"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="valueStream" type="java.io.DataInputStream"/>
+      <param name="out" type="java.io.PrintStream"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Keep calling this till you get a {@link EOFException} for getting logs of
+ all types for a single container.
+ 
+ @param valueStream
+ @param out
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="readContainerLogsForALogType" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="valueStream" type="java.io.DataInputStream"/>
+      <param name="out" type="java.io.PrintStream"/>
+      <param name="logUploadedTime" type="long"/>
+      <param name="logType" type="java.util.List"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Keep calling this till you get a {@link EOFException} for getting logs of
+ the specific types for a single container.
+ @param valueStream
+ @param out
+ @param logUploadedTime
+ @param logType
+ @throws IOException]]>
+      </doc>
+    </method>
+    <method name="close"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.logaggregation.AggregatedLogFormat.LogReader -->
+</package>
+<package name="org.apache.hadoop.yarn.nodelabels">
+</package>
+<package name="org.apache.hadoop.yarn.nodelabels.event">
+</package>
+<package name="org.apache.hadoop.yarn.security">
+  <!-- start class org.apache.hadoop.yarn.security.AMRMTokenIdentifier -->
+  <class name="AMRMTokenIdentifier" extends="org.apache.hadoop.security.token.TokenIdentifier"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="AMRMTokenIdentifier"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="AMRMTokenIdentifier" type="org.apache.hadoop.yarn.api.records.ApplicationAttemptId, int"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="write"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="out" type="java.io.DataOutput"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="readFields"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="in" type="java.io.DataInput"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="getKind" return="org.apache.hadoop.io.Text"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getUser" return="org.apache.hadoop.security.UserGroupInformation"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getKeyId" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getProto" return="org.apache.hadoop.yarn.proto.YarnSecurityTokenProtos.AMRMTokenIdentifierProto"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="hashCode" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="equals" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="other" type="java.lang.Object"/>
+    </method>
+    <method name="toString" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <field name="KIND_NAME" type="org.apache.hadoop.io.Text"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+    </field>
+    <doc>
+    <![CDATA[AMRMTokenIdentifier is the TokenIdentifier to be used by
+ ApplicationMasters to authenticate to the ResourceManager.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.security.AMRMTokenIdentifier -->
+  <!-- start class org.apache.hadoop.yarn.security.AMRMTokenSelector -->
+  <class name="AMRMTokenSelector" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="org.apache.hadoop.security.token.TokenSelector"/>
+    <constructor name="AMRMTokenSelector"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="selectToken" return="org.apache.hadoop.security.token.Token"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="service" type="org.apache.hadoop.io.Text"/>
+      <param name="tokens" type="java.util.Collection"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.security.AMRMTokenSelector -->
+  <!-- start class org.apache.hadoop.yarn.security.ContainerManagerSecurityInfo -->
+  <class name="ContainerManagerSecurityInfo" extends="org.apache.hadoop.security.SecurityInfo"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="ContainerManagerSecurityInfo"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="getKerberosInfo" return="org.apache.hadoop.security.KerberosInfo"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="protocol" type="java.lang.Class"/>
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+    </method>
+    <method name="getTokenInfo" return="org.apache.hadoop.security.token.TokenInfo"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="protocol" type="java.lang.Class"/>
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.security.ContainerManagerSecurityInfo -->
+  <!-- start class org.apache.hadoop.yarn.security.ContainerTokenIdentifier -->
+  <class name="ContainerTokenIdentifier" extends="org.apache.hadoop.security.token.TokenIdentifier"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="ContainerTokenIdentifier" type="org.apache.hadoop.yarn.api.records.ContainerId, java.lang.String, java.lang.String, org.apache.hadoop.yarn.api.records.Resource, long, int, long, org.apache.hadoop.yarn.api.records.Priority, long"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="ContainerTokenIdentifier" type="org.apache.hadoop.yarn.api.records.ContainerId, java.lang.String, java.lang.String, org.apache.hadoop.yarn.api.records.Resource, long, int, long, org.apache.hadoop.yarn.api.records.Priority, long, org.apache.hadoop.yarn.api.records.LogAggregationContext"
+      static="false" final="false" visibility="public"
+      deprecated="Use one of the other constructors instead.">
+      <doc>
+      <![CDATA[Creates a instance.
+
+ @param appSubmitter appSubmitter
+ @param containerID container ID
+ @param creationTime creation time
+ @param expiryTimeStamp expiry timestamp
+ @param hostName hostname
+ @param logAggregationContext log aggregation context
+ @param masterKeyId master key ID
+ @param priority priority
+ @param r resource needed by the container
+ @param rmIdentifier ResourceManager identifier
+ @deprecated Use one of the other constructors instead.]]>
+      </doc>
+    </constructor>
+    <constructor name="ContainerTokenIdentifier" type="org.apache.hadoop.yarn.api.records.ContainerId, java.lang.String, java.lang.String, org.apache.hadoop.yarn.api.records.Resource, long, int, long, org.apache.hadoop.yarn.api.records.Priority, long, org.apache.hadoop.yarn.api.records.LogAggregationContext, java.lang.String"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="ContainerTokenIdentifier" type="org.apache.hadoop.yarn.api.records.ContainerId, java.lang.String, java.lang.String, org.apache.hadoop.yarn.api.records.Resource, long, int, long, org.apache.hadoop.yarn.api.records.Priority, long, org.apache.hadoop.yarn.api.records.LogAggregationContext, java.lang.String, org.apache.hadoop.yarn.server.api.ContainerType"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="ContainerTokenIdentifier" type="org.apache.hadoop.yarn.api.records.ContainerId, int, java.lang.String, java.lang.String, org.apache.hadoop.yarn.api.records.Resource, long, int, long, org.apache.hadoop.yarn.api.records.Priority, long, org.apache.hadoop.yarn.api.records.LogAggregationContext, java.lang.String, org.apache.hadoop.yarn.server.api.ContainerType"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="ContainerTokenIdentifier"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Default constructor needed by RPC layer/SecretManager.]]>
+      </doc>
+    </constructor>
+    <method name="getContainerID" return="org.apache.hadoop.yarn.api.records.ContainerId"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getApplicationSubmitter" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getNmHostAddress" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getResource" return="org.apache.hadoop.yarn.api.records.Resource"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getExpiryTimeStamp" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getMasterKeyId" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getPriority" return="org.apache.hadoop.yarn.api.records.Priority"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getCreationTime" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getRMIdentifier" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the RMIdentifier of RM in which containers are allocated
+ @return RMIdentifier]]>
+      </doc>
+    </method>
+    <method name="getContainerType" return="org.apache.hadoop.yarn.server.api.ContainerType"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the ContainerType of container to allocate.
+ @return ContainerType]]>
+      </doc>
+    </method>
+    <method name="getProto" return="org.apache.hadoop.yarn.proto.YarnSecurityTokenProtos.ContainerTokenIdentifierProto"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getLogAggregationContext" return="org.apache.hadoop.yarn.api.records.LogAggregationContext"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="write"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="out" type="java.io.DataOutput"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="readFields"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="in" type="java.io.DataInput"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="getKind" return="org.apache.hadoop.io.Text"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getUser" return="org.apache.hadoop.security.UserGroupInformation"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getVersion" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the Container version
+ @return container version]]>
+      </doc>
+    </method>
+    <method name="getNodeLabelExpression" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the node-label-expression in the original ResourceRequest]]>
+      </doc>
+    </method>
+    <method name="hashCode" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="equals" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="other" type="java.lang.Object"/>
+    </method>
+    <method name="toString" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <field name="KIND" type="org.apache.hadoop.io.Text"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+    </field>
+    <doc>
+    <![CDATA[TokenIdentifier for a container. Encodes {@link ContainerId},
+ {@link Resource} needed by the container and the target NMs host-address.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.security.ContainerTokenIdentifier -->
+  <!-- start class org.apache.hadoop.yarn.security.ContainerTokenSelector -->
+  <class name="ContainerTokenSelector" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="org.apache.hadoop.security.token.TokenSelector"/>
+    <constructor name="ContainerTokenSelector"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="selectToken" return="org.apache.hadoop.security.token.Token"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="service" type="org.apache.hadoop.io.Text"/>
+      <param name="tokens" type="java.util.Collection"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.security.ContainerTokenSelector -->
+  <!-- start class org.apache.hadoop.yarn.security.NMTokenIdentifier -->
+  <class name="NMTokenIdentifier" extends="org.apache.hadoop.security.token.TokenIdentifier"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="NMTokenIdentifier" type="org.apache.hadoop.yarn.api.records.ApplicationAttemptId, org.apache.hadoop.yarn.api.records.NodeId, java.lang.String, int"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="NMTokenIdentifier"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Default constructor needed by RPC/Secret manager]]>
+      </doc>
+    </constructor>
+    <method name="getApplicationAttemptId" return="org.apache.hadoop.yarn.api.records.ApplicationAttemptId"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getNodeId" return="org.apache.hadoop.yarn.api.records.NodeId"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getApplicationSubmitter" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getKeyId" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="write"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="out" type="java.io.DataOutput"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="readFields"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="in" type="java.io.DataInput"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="getKind" return="org.apache.hadoop.io.Text"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getUser" return="org.apache.hadoop.security.UserGroupInformation"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getProto" return="org.apache.hadoop.yarn.proto.YarnSecurityTokenProtos.NMTokenIdentifierProto"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="hashCode" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="equals" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="other" type="java.lang.Object"/>
+    </method>
+    <method name="toString" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <field name="KIND" type="org.apache.hadoop.io.Text"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+    </field>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.security.NMTokenIdentifier -->
+  <!-- start class org.apache.hadoop.yarn.security.SchedulerSecurityInfo -->
+  <class name="SchedulerSecurityInfo" extends="org.apache.hadoop.security.SecurityInfo"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="SchedulerSecurityInfo"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="getKerberosInfo" return="org.apache.hadoop.security.KerberosInfo"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="protocol" type="java.lang.Class"/>
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+    </method>
+    <method name="getTokenInfo" return="org.apache.hadoop.security.token.TokenInfo"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="protocol" type="java.lang.Class"/>
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.security.SchedulerSecurityInfo -->
+</package>
+<package name="org.apache.hadoop.yarn.security.admin">
+  <!-- start class org.apache.hadoop.yarn.security.admin.AdminSecurityInfo -->
+  <class name="AdminSecurityInfo" extends="org.apache.hadoop.security.SecurityInfo"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="AdminSecurityInfo"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="getKerberosInfo" return="org.apache.hadoop.security.KerberosInfo"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="protocol" type="java.lang.Class"/>
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+    </method>
+    <method name="getTokenInfo" return="org.apache.hadoop.security.token.TokenInfo"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="protocol" type="java.lang.Class"/>
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.security.admin.AdminSecurityInfo -->
+</package>
+<package name="org.apache.hadoop.yarn.security.client">
+  <!-- start class org.apache.hadoop.yarn.security.client.BaseClientToAMTokenSecretManager -->
+  <class name="BaseClientToAMTokenSecretManager" extends="org.apache.hadoop.security.token.SecretManager"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="BaseClientToAMTokenSecretManager"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <doc>
+    <![CDATA[A base {@link SecretManager} for AMs to extend and validate Client-RM tokens
+ issued to clients by the RM using the underlying master-key shared by RM to
+ the AMs on their launch. All the methods are called by either Hadoop RPC or
+ YARN, so this class is strictly for the purpose of inherit/extend and
+ register with Hadoop RPC.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.security.client.BaseClientToAMTokenSecretManager -->
+  <!-- start class org.apache.hadoop.yarn.security.client.ClientRMSecurityInfo -->
+  <class name="ClientRMSecurityInfo" extends="org.apache.hadoop.security.SecurityInfo"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="ClientRMSecurityInfo"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="getKerberosInfo" return="org.apache.hadoop.security.KerberosInfo"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="protocol" type="java.lang.Class"/>
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+    </method>
+    <method name="getTokenInfo" return="org.apache.hadoop.security.token.TokenInfo"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="protocol" type="java.lang.Class"/>
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.security.client.ClientRMSecurityInfo -->
+  <!-- start class org.apache.hadoop.yarn.security.client.ClientTimelineSecurityInfo -->
+  <class name="ClientTimelineSecurityInfo" extends="org.apache.hadoop.security.SecurityInfo"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="ClientTimelineSecurityInfo"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="getKerberosInfo" return="org.apache.hadoop.security.KerberosInfo"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="protocol" type="java.lang.Class"/>
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+    </method>
+    <method name="getTokenInfo" return="org.apache.hadoop.security.token.TokenInfo"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="protocol" type="java.lang.Class"/>
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.security.client.ClientTimelineSecurityInfo -->
+  <!-- start class org.apache.hadoop.yarn.security.client.ClientToAMTokenIdentifier -->
+  <class name="ClientToAMTokenIdentifier" extends="org.apache.hadoop.security.token.TokenIdentifier"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="ClientToAMTokenIdentifier"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="ClientToAMTokenIdentifier" type="org.apache.hadoop.yarn.api.records.ApplicationAttemptId, java.lang.String"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="getApplicationAttemptID" return="org.apache.hadoop.yarn.api.records.ApplicationAttemptId"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getClientName" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getProto" return="org.apache.hadoop.yarn.proto.YarnSecurityTokenProtos.ClientToAMTokenIdentifierProto"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="write"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="out" type="java.io.DataOutput"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="readFields"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="in" type="java.io.DataInput"/>
+      <exception name="IOException" type="java.io.IOException"/>
+    </method>
+    <method name="getKind" return="org.apache.hadoop.io.Text"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getUser" return="org.apache.hadoop.security.UserGroupInformation"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="hashCode" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="equals" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="other" type="java.lang.Object"/>
+    </method>
+    <method name="toString" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <field name="KIND_NAME" type="org.apache.hadoop.io.Text"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+    </field>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.security.client.ClientToAMTokenIdentifier -->
+  <!-- start class org.apache.hadoop.yarn.security.client.ClientToAMTokenSecretManager -->
+  <class name="ClientToAMTokenSecretManager" extends="org.apache.hadoop.yarn.security.client.BaseClientToAMTokenSecretManager"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="ClientToAMTokenSecretManager" type="org.apache.hadoop.yarn.api.records.ApplicationAttemptId, byte[]"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="retrievePassword" return="byte[]"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="identifier" type="org.apache.hadoop.yarn.security.client.ClientToAMTokenIdentifier"/>
+      <exception name="SecretManager.InvalidToken" type="org.apache.hadoop.security.token.SecretManager.InvalidToken"/>
+    </method>
+    <method name="getMasterKey" return="javax.crypto.SecretKey"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="applicationAttemptID" type="org.apache.hadoop.yarn.api.records.ApplicationAttemptId"/>
+    </method>
+    <method name="setMasterKey"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="key" type="byte[]"/>
+    </method>
+    <doc>
+    <![CDATA[A simple {@link SecretManager} for AMs to validate Client-RM tokens issued to
+ clients by the RM using the underlying master-key shared by RM to the AMs on
+ their launch. All the methods are called by either Hadoop RPC or YARN, so
+ this class is strictly for the purpose of inherit/extend and register with
+ Hadoop RPC.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.security.client.ClientToAMTokenSecretManager -->
+  <!-- start class org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier -->
+  <class name="RMDelegationTokenIdentifier" extends="org.apache.hadoop.yarn.security.client.YARNDelegationTokenIdentifier"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="RMDelegationTokenIdentifier"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="RMDelegationTokenIdentifier" type="org.apache.hadoop.io.Text, org.apache.hadoop.io.Text, org.apache.hadoop.io.Text"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Create a new delegation token identifier
+ @param owner the effective username of the token owner
+ @param renewer the username of the renewer
+ @param realUser the real username of the token owner]]>
+      </doc>
+    </constructor>
+    <method name="getKind" return="org.apache.hadoop.io.Text"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <field name="KIND_NAME" type="org.apache.hadoop.io.Text"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+    </field>
+    <doc>
+    <![CDATA[Delegation Token Identifier that identifies the delegation tokens from the 
+ Resource Manager.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier -->
+  <!-- start class org.apache.hadoop.yarn.security.client.RMDelegationTokenSelector -->
+  <class name="RMDelegationTokenSelector" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="org.apache.hadoop.security.token.TokenSelector"/>
+    <constructor name="RMDelegationTokenSelector"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="selectToken" return="org.apache.hadoop.security.token.Token"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="service" type="org.apache.hadoop.io.Text"/>
+      <param name="tokens" type="java.util.Collection"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.security.client.RMDelegationTokenSelector -->
+  <!-- start class org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier -->
+  <class name="TimelineDelegationTokenIdentifier" extends="org.apache.hadoop.yarn.security.client.YARNDelegationTokenIdentifier"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="TimelineDelegationTokenIdentifier"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="TimelineDelegationTokenIdentifier" type="org.apache.hadoop.io.Text, org.apache.hadoop.io.Text, org.apache.hadoop.io.Text"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Create a new timeline delegation token identifier
+
+ @param owner the effective username of the token owner
+ @param renewer the username of the renewer
+ @param realUser the real username of the token owner]]>
+      </doc>
+    </constructor>
+    <method name="getKind" return="org.apache.hadoop.io.Text"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <field name="KIND_NAME" type="org.apache.hadoop.io.Text"
+      transient="false" volatile="false"
+      static="true" final="true" visibility="public"
+      deprecated="not deprecated">
+    </field>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier -->
+  <!-- start class org.apache.hadoop.yarn.security.client.TimelineDelegationTokenSelector -->
+  <class name="TimelineDelegationTokenSelector" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="org.apache.hadoop.security.token.TokenSelector"/>
+    <constructor name="TimelineDelegationTokenSelector"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="selectToken" return="org.apache.hadoop.security.token.Token"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="service" type="org.apache.hadoop.io.Text"/>
+      <param name="tokens" type="java.util.Collection"/>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.security.client.TimelineDelegationTokenSelector -->
+</package>
+<package name="org.apache.hadoop.yarn.server.api">
+</package>
+<package name="org.apache.hadoop.yarn.server.api.impl.pb.client">
+</package>
+<package name="org.apache.hadoop.yarn.server.api.impl.pb.service">
+</package>
+<package name="org.apache.hadoop.yarn.sharedcache">
+  <!-- start interface org.apache.hadoop.yarn.sharedcache.SharedCacheChecksum -->
+  <interface name="SharedCacheChecksum"    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <method name="computeChecksum" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="in" type="java.io.InputStream"/>
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Calculate the checksum of the passed input stream.
+
+ @param in <code>InputStream</code> to be checksumed
+ @return the message digest of the input stream
+ @throws IOException]]>
+      </doc>
+    </method>
+  </interface>
+  <!-- end interface org.apache.hadoop.yarn.sharedcache.SharedCacheChecksum -->
+  <!-- start class org.apache.hadoop.yarn.sharedcache.SharedCacheChecksumFactory -->
+  <class name="SharedCacheChecksumFactory" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="SharedCacheChecksumFactory"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="getChecksum" return="org.apache.hadoop.yarn.sharedcache.SharedCacheChecksum"
+      abstract="false" native="false" synchronized="false"
+      static="true" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="conf" type="org.apache.hadoop.conf.Configuration"/>
+      <doc>
+      <![CDATA[Get a new <code>SharedCacheChecksum</code> object based on the configurable
+ algorithm implementation
+ (see <code>yarn.sharedcache.checksum.algo.impl</code>)
+
+ @return <code>SharedCacheChecksum</code> object]]>
+      </doc>
+    </method>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.sharedcache.SharedCacheChecksumFactory -->
+</package>
+<package name="org.apache.hadoop.yarn.state">
+  <!-- start class org.apache.hadoop.yarn.state.InvalidStateTransitionException -->
+  <class name="InvalidStateTransitionException" extends="org.apache.hadoop.yarn.state.InvalidStateTransitonException"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="InvalidStateTransitionException" type="java.lang.Enum, java.lang.Enum"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <doc>
+    <![CDATA[The exception that happens when you call invalid state transition.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.state.InvalidStateTransitionException -->
+  <!-- start class org.apache.hadoop.yarn.state.InvalidStateTransitonException -->
+  <class name="InvalidStateTransitonException" extends="org.apache.hadoop.yarn.exceptions.YarnRuntimeException"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="Use {@link InvalidStateTransitionException} instead.">
+    <constructor name="InvalidStateTransitonException" type="java.lang.Enum, java.lang.Enum"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="getCurrentState" return="java.lang.Enum"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getEvent" return="java.lang.Enum"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <doc>
+    <![CDATA[@deprecated Use {@link InvalidStateTransitionException} instead.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.state.InvalidStateTransitonException -->
+  <!-- start interface org.apache.hadoop.yarn.state.MultipleArcTransition -->
+  <interface name="MultipleArcTransition"    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <method name="transition" return="STATE"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="operand" type="OPERAND"/>
+      <param name="event" type="EVENT"/>
+      <doc>
+      <![CDATA[Transition hook.
+ @return the postState. Post state must be one of the 
+                      valid post states registered in StateMachine.
+ @param operand the entity attached to the FSM, whose internal 
+                state may change.
+ @param event causal event]]>
+      </doc>
+    </method>
+    <doc>
+    <![CDATA[Hook for Transition. 
+ Post state is decided by Transition hook. Post state must be one of the 
+ valid post states registered in StateMachine.]]>
+    </doc>
+  </interface>
+  <!-- end interface org.apache.hadoop.yarn.state.MultipleArcTransition -->
+  <!-- start interface org.apache.hadoop.yarn.state.SingleArcTransition -->
+  <interface name="SingleArcTransition"    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <method name="transition"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="operand" type="OPERAND"/>
+      <param name="event" type="EVENT"/>
+      <doc>
+      <![CDATA[Transition hook.
+ 
+ @param operand the entity attached to the FSM, whose internal 
+                state may change.
+ @param event causal event]]>
+      </doc>
+    </method>
+    <doc>
+    <![CDATA[Hook for Transition. This lead to state machine to move to 
+ the post state as registered in the state machine.]]>
+    </doc>
+  </interface>
+  <!-- end interface org.apache.hadoop.yarn.state.SingleArcTransition -->
+  <!-- start interface org.apache.hadoop.yarn.state.StateMachine -->
+  <interface name="StateMachine"    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not

<TRUNCATED>

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


[48/50] [abbrv] hadoop git commit: HDFS-12959. Fix TestOpenFilesWithSnapshot redundant configurations.

Posted by na...@apache.org.
HDFS-12959. Fix TestOpenFilesWithSnapshot redundant configurations.


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

Branch: refs/heads/yarn-3409
Commit: 76e664e931bf0784620b69bc588bd51cf2a024e6
Parents: 826507c
Author: Manoj Govindassamy <ma...@apache.org>
Authored: Thu Dec 21 15:47:15 2017 -0800
Committer: Manoj Govindassamy <ma...@apache.org>
Committed: Thu Dec 21 15:47:15 2017 -0800

----------------------------------------------------------------------
 .../server/namenode/snapshot/TestOpenFilesWithSnapshot.java   | 7 ++-----
 1 file changed, 2 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/76e664e9/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestOpenFilesWithSnapshot.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestOpenFilesWithSnapshot.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestOpenFilesWithSnapshot.java
index be118a3..17082a1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestOpenFilesWithSnapshot.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestOpenFilesWithSnapshot.java
@@ -66,7 +66,8 @@ public class TestOpenFilesWithSnapshot {
   public void setup() throws IOException {
     conf.setBoolean(
         DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_CAPTURE_OPENFILES, true);
-    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
+    cluster =
+        new MiniDFSCluster.Builder(conf).numDataNodes(REPLICATION).build();
     conf.set("dfs.blocksize", "1048576");
     fs = cluster.getFileSystem();
   }
@@ -252,8 +253,6 @@ public class TestOpenFilesWithSnapshot {
    */
   @Test (timeout = 120000)
   public void testPointInTimeSnapshotCopiesForOpenFiles() throws Exception {
-    conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_CAPTURE_OPENFILES,
-        true);
     // Construct the directory tree
     final Path level0A = new Path("/level_0_A");
     final Path level0B = new Path("/level_0_B");
@@ -738,8 +737,6 @@ public class TestOpenFilesWithSnapshot {
    */
   @Test (timeout = 120000)
   public void testOpenFilesSnapChecksumWithTrunkAndAppend() throws Exception {
-    conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_CAPTURE_OPENFILES,
-        true);
     // Construct the directory tree
     final Path dir = new Path("/A/B/C");
     fs.mkdirs(dir);


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


[50/50] [abbrv] hadoop git commit: YARN-6855. [YARN-3409] CLI Proto Modifications to support Node Attributes. Contributed by Naganarasimha G R.

Posted by na...@apache.org.
YARN-6855. [YARN-3409] CLI Proto Modifications to support Node Attributes. Contributed by Naganarasimha G R.


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

Branch: refs/heads/yarn-3409
Commit: 84ebbfbfbcfe0f27117ceb46d52880f2106cd88d
Parents: 52babbb
Author: Sunil G <su...@apache.org>
Authored: Tue Oct 3 14:59:33 2017 +0530
Committer: naga <yo...@example.com>
Committed: Wed Dec 27 06:32:12 2017 +0800

----------------------------------------------------------------------
 .../hadoop/yarn/api/records/NodeAttribute.java  |  78 ++++++++
 .../yarn/api/records/NodeAttributeType.java     |  35 ++++
 .../ResourceManagerAdministrationProtocol.java  |  13 +-
 .../AttributeMappingOperationType.java          |  42 ++++
 .../api/protocolrecords/NodeToAttributes.java   |  59 ++++++
 .../NodesToAttributesMappingRequest.java        |  69 +++++++
 .../NodesToAttributesMappingResponse.java       |  27 +++
 ...esourcemanager_administration_protocol.proto |   1 +
 ..._server_resourcemanager_service_protos.proto |  21 ++
 .../src/main/proto/yarn_protos.proto            |  11 ++
 .../records/impl/pb/NodeAttributePBImpl.java    | 155 +++++++++++++++
 ...nagerAdministrationProtocolPBClientImpl.java |  26 ++-
 ...agerAdministrationProtocolPBServiceImpl.java |  31 ++-
 .../impl/pb/NodeToAttributesPBImpl.java         | 161 +++++++++++++++
 .../NodesToAttributesMappingRequestPBImpl.java  | 194 +++++++++++++++++++
 .../NodesToAttributesMappingResponsePBImpl.java |  47 +++++
 .../hadoop/yarn/api/TestPBImplRecords.java      |  46 ++++-
 .../yarn/server/MockResourceManagerFacade.java  |  21 +-
 .../server/resourcemanager/AdminService.java    |  10 +
 .../DefaultRMAdminRequestInterceptor.java       |   9 +
 .../router/rmadmin/RouterRMAdminService.java    |  10 +
 .../PassThroughRMAdminRequestInterceptor.java   |   9 +
 22 files changed, 1052 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/84ebbfbf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeAttribute.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeAttribute.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeAttribute.java
new file mode 100644
index 0000000..13081f3
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeAttribute.java
@@ -0,0 +1,78 @@
+/**
+ * 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.api.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * <p>
+ * Node Attribute is a kind of a label which represents one of the
+ * attribute/feature of a Node. Its different from node partition label as
+ * resource guarantees across the queues will not be maintained for these type
+ * of labels.
+ * </p>
+ * <p>
+ * A given Node can be mapped with any kind of attribute, few examples are
+ * HAS_SSD=true, JAVA_VERSION=JDK1.8, OS_TYPE=WINDOWS.
+ * </p>
+ * <p>
+ * Its not compulsory for all the attributes to have value, empty string is the
+ * default value of the <code>NodeAttributeType.STRING</code>
+ * </p>
+ *
+ */
+@Public
+@Unstable
+public abstract class NodeAttribute {
+
+  public static NodeAttribute newInstance(String attributeName,
+      NodeAttributeType attributeType, String attributeValue) {
+    NodeAttribute nodeAttribute = Records.newRecord(NodeAttribute.class);
+    nodeAttribute.setAttributeName(attributeName);
+    nodeAttribute.setAttributeType(attributeType);
+    nodeAttribute.setAttributeValue(attributeValue);
+    return nodeAttribute;
+  }
+
+  @Public
+  @Unstable
+  public abstract String getAttributeName();
+
+  @Public
+  @Unstable
+  public abstract void setAttributeName(String attributeName);
+
+  @Public
+  @Unstable
+  public abstract String getAttributeValue();
+
+  @Public
+  @Unstable
+  public abstract void setAttributeValue(String attributeValue);
+
+  @Public
+  @Unstable
+  public abstract NodeAttributeType getAttributeType();
+
+  @Public
+  @Unstable
+  public abstract void setAttributeType(NodeAttributeType attributeType);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/84ebbfbf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeAttributeType.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeAttributeType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeAttributeType.java
new file mode 100644
index 0000000..3f281c8
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeAttributeType.java
@@ -0,0 +1,35 @@
+/**
+ * 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.api.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+
+/**
+ * <p>
+ * Type of a <code>node Attribute</code>.
+ * </p>
+ * Based on this attribute expressions and values will be evaluated.
+ */
+@Public
+@Unstable
+public enum NodeAttributeType {
+  /** string type node attribute. */
+  STRING
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/84ebbfbf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ResourceManagerAdministrationProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ResourceManagerAdministrationProtocol.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ResourceManagerAdministrationProtocol.java
index 8523342..58bb270 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ResourceManagerAdministrationProtocol.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ResourceManagerAdministrationProtocol.java
@@ -30,6 +30,8 @@ import org.apache.hadoop.yarn.api.records.ResourceOption;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.api.protocolrecords.AddToClusterNodeLabelsRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.AddToClusterNodeLabelsResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodesToAttributesMappingRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodesToAttributesMappingResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.CheckForDecommissioningNodesRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.CheckForDecommissioningNodesResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshAdminAclsRequest;
@@ -37,6 +39,8 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshAdminAclsRespons
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshClusterMaxPriorityRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshClusterMaxPriorityResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshNodesRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshNodesResourcesRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshNodesResourcesResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshNodesResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshQueuesRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshQueuesResponse;
@@ -52,8 +56,6 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.ReplaceLabelsOnNodeRequ
 import org.apache.hadoop.yarn.server.api.protocolrecords.ReplaceLabelsOnNodeResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.UpdateNodeResourceRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.UpdateNodeResourceResponse;
-import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshNodesResourcesRequest;
-import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshNodesResourcesResponse;
 
 @Private
 public interface ResourceManagerAdministrationProtocol extends GetUserMappingsProtocol {
@@ -144,4 +146,11 @@ public interface ResourceManagerAdministrationProtocol extends GetUserMappingsPr
   public RefreshClusterMaxPriorityResponse refreshClusterMaxPriority(
       RefreshClusterMaxPriorityRequest request) throws YarnException,
       IOException;
+
+
+  @Private
+  @Idempotent
+  public NodesToAttributesMappingResponse mapAttributesToNodes(
+      NodesToAttributesMappingRequest request) throws YarnException,
+      IOException;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/84ebbfbf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/AttributeMappingOperationType.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/AttributeMappingOperationType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/AttributeMappingOperationType.java
new file mode 100644
index 0000000..5de1504
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/AttributeMappingOperationType.java
@@ -0,0 +1,42 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.api.protocolrecords;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+
+/**
+ * <p>
+ * Type of node to attribute mapping operation.
+ * </p>
+ *
+ */
+@Public
+@Unstable
+public enum AttributeMappingOperationType {
+  /** Replaces the existing node to attribute mapping with new mapping.*/
+  REPLACE,
+
+  /** Add attribute(s) to a node and if it already exists will update the
+   *  value.*/
+  ADD,
+
+  /** Removes attribute(s) mapped to a node. */
+  REMOVE
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/84ebbfbf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeToAttributes.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeToAttributes.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeToAttributes.java
new file mode 100644
index 0000000..b2e38b4
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodeToAttributes.java
@@ -0,0 +1,59 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.api.protocolrecords;
+
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * Represents a mapping of Node id to list of attributes.
+ */
+@Public
+@Unstable
+public abstract class NodeToAttributes {
+
+  public static NodeToAttributes newInstance(String node,
+      List<NodeAttribute> attributes) {
+    NodeToAttributes nodeIdToAttributes =
+        Records.newRecord(NodeToAttributes.class);
+    nodeIdToAttributes.setNode(node);
+    nodeIdToAttributes.setNodeAttributes(attributes);
+    return nodeIdToAttributes;
+  }
+
+  @Public
+  @Unstable
+  public abstract String getNode();
+
+  @Public
+  @Unstable
+  public abstract void setNode(String node);
+
+  @Public
+  @Unstable
+  public abstract List<NodeAttribute> getNodeAttributes();
+
+  @Public
+  @Unstable
+  public abstract void setNodeAttributes(List<NodeAttribute> attributes);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/84ebbfbf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodesToAttributesMappingRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodesToAttributesMappingRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodesToAttributesMappingRequest.java
new file mode 100644
index 0000000..71421ed
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodesToAttributesMappingRequest.java
@@ -0,0 +1,69 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.api.protocolrecords;
+
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * list of node-attribute mapping request info.
+ */
+@Public
+@Unstable
+public abstract class NodesToAttributesMappingRequest {
+
+  public static NodesToAttributesMappingRequest newInstance(
+      AttributeMappingOperationType operation,
+      List<NodeToAttributes> nodesToAttributes, boolean failOnUnknownNodes) {
+    NodesToAttributesMappingRequest request =
+        Records.newRecord(NodesToAttributesMappingRequest.class);
+    request.setNodesToAttributes(nodesToAttributes);
+    request.setFailOnUnknownNodes(failOnUnknownNodes);
+    request.setOperation(operation);
+    return request;
+  }
+
+  @Public
+  @Unstable
+  public abstract void setNodesToAttributes(
+      List<NodeToAttributes> nodesToAttributes);
+
+  @Public
+  @Unstable
+  public abstract List<NodeToAttributes> getNodesToAttributes();
+
+  @Public
+  @Unstable
+  public abstract void setFailOnUnknownNodes(boolean failOnUnknownNodes);
+
+  @Public
+  @Unstable
+  public abstract boolean getFailOnUnknownNodes();
+
+  @Public
+  @Unstable
+  public abstract void setOperation(AttributeMappingOperationType operation);
+
+  @Public
+  @Unstable
+  public abstract AttributeMappingOperationType getOperation();
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/84ebbfbf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodesToAttributesMappingResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodesToAttributesMappingResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodesToAttributesMappingResponse.java
new file mode 100644
index 0000000..8e44adf
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/NodesToAttributesMappingResponse.java
@@ -0,0 +1,27 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.api.protocolrecords;
+
+import org.apache.hadoop.yarn.util.Records;
+
+public class NodesToAttributesMappingResponse {
+  public static NodesToAttributesMappingResponse newInstance() {
+    return Records.newRecord(NodesToAttributesMappingResponse.class);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/84ebbfbf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/resourcemanager_administration_protocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/resourcemanager_administration_protocol.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/resourcemanager_administration_protocol.proto
index 1134623..032aa8e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/resourcemanager_administration_protocol.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/resourcemanager_administration_protocol.proto
@@ -45,4 +45,5 @@ service ResourceManagerAdministrationProtocolService {
   rpc replaceLabelsOnNodes(ReplaceLabelsOnNodeRequestProto) returns (ReplaceLabelsOnNodeResponseProto);
   rpc checkForDecommissioningNodes(CheckForDecommissioningNodesRequestProto) returns (CheckForDecommissioningNodesResponseProto);
   rpc refreshClusterMaxPriority(RefreshClusterMaxPriorityRequestProto) returns (RefreshClusterMaxPriorityResponseProto);
+  rpc mapAttributesToNodes(NodesToAttributesMappingRequestProto) returns (NodesToAttributesMappingResponseProto);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/84ebbfbf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/yarn_server_resourcemanager_service_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/yarn_server_resourcemanager_service_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/yarn_server_resourcemanager_service_protos.proto
index e8c92d9..5b93aec 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/yarn_server_resourcemanager_service_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/yarn_server_resourcemanager_service_protos.proto
@@ -130,6 +130,27 @@ enum DecommissionTypeProto {
   GRACEFUL = 2;
   FORCEFUL = 3;
 }
+
+
+enum AttributeMappingOperationTypeProto {
+  REPLACE = 1;
+  ADD = 2;
+  REMOVE = 3;
+}
+
+message NodesToAttributesMappingRequestProto {
+  optional AttributeMappingOperationTypeProto operation = 1 [default = REPLACE];
+  repeated NodeToAttributesProto nodeToAttributes = 2;
+  optional bool failOnUnknownNodes = 3;
+}
+
+message NodeToAttributesProto {
+  optional string node = 1;
+  repeated NodeAttributeProto nodeAttributes = 2;
+}
+
+message NodesToAttributesMappingResponseProto {
+}
 //////////////////////////////////////////////////////////////////
 ///////////// RM Failover related records ////////////////////////
 //////////////////////////////////////////////////////////////////

http://git-wip-us.apache.org/repos/asf/hadoop/blob/84ebbfbf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
index 3a9662b..6ecd608 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
@@ -370,6 +370,17 @@ message NodeLabelProto {
   optional bool isExclusive = 2 [default = true]; 
 }
 
+enum NodeAttributeTypeProto {
+  STRING = 1;
+}
+
+message NodeAttributeProto {
+  optional string attributeName = 1;
+  optional NodeAttributeTypeProto attributeType = 2;
+  optional string attributeValue = 3;
+}
+
+
 enum ContainerTypeProto {
   APPLICATION_MASTER = 1;
   TASK = 2;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/84ebbfbf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeAttributePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeAttributePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeAttributePBImpl.java
new file mode 100644
index 0000000..11c9c48
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeAttributePBImpl.java
@@ -0,0 +1,155 @@
+/**
+ * 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.api.records.impl.pb;
+
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
+import org.apache.hadoop.yarn.api.records.NodeAttributeType;
+import org.apache.hadoop.yarn.proto.YarnProtos.NodeAttributeProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.NodeAttributeProtoOrBuilder;
+import org.apache.hadoop.yarn.proto.YarnProtos.NodeAttributeTypeProto;
+
+public class NodeAttributePBImpl extends NodeAttribute {
+  private NodeAttributeProto proto = NodeAttributeProto.getDefaultInstance();
+  private NodeAttributeProto.Builder builder = null;
+  private boolean viaProto = false;
+
+  public NodeAttributePBImpl() {
+    builder = NodeAttributeProto.newBuilder();
+  }
+
+  public NodeAttributePBImpl(NodeAttributeProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public NodeAttributeProto getProto() {
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = NodeAttributeProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  @Override
+  public String getAttributeName() {
+    NodeAttributeProtoOrBuilder p = viaProto ? proto : builder;
+    if (!p.hasAttributeName()) {
+      return null;
+    }
+    return p.getAttributeName();
+  }
+
+  @Override
+  public void setAttributeName(String attributeName) {
+    maybeInitBuilder();
+    builder.setAttributeName(attributeName);
+  }
+
+  @Override
+  public String getAttributeValue() {
+    NodeAttributeProtoOrBuilder p = viaProto ? proto : builder;
+    if (!p.hasAttributeValue()) {
+      return null;
+    }
+    return p.getAttributeValue();
+  }
+
+  @Override
+  public void setAttributeValue(String attributeValue) {
+    maybeInitBuilder();
+    builder.setAttributeValue(attributeValue);
+  }
+
+  @Override
+  public NodeAttributeType getAttributeType() {
+    NodeAttributeProtoOrBuilder p = viaProto ? proto : builder;
+    if (!p.hasAttributeType()) {
+      return null;
+    }
+    return convertFromProtoFormat(p.getAttributeType());
+  }
+
+  @Override
+  public void setAttributeType(NodeAttributeType attributeType) {
+    maybeInitBuilder();
+    if (attributeType == null) {
+      builder.clearAttributeType();
+      return;
+    }
+    builder.setAttributeType(convertToProtoFormat(attributeType));
+  }
+
+  private NodeAttributeTypeProto convertToProtoFormat(
+      NodeAttributeType attributeType) {
+    return NodeAttributeTypeProto.valueOf(attributeType.name());
+  }
+
+  private NodeAttributeType convertFromProtoFormat(
+      NodeAttributeTypeProto containerState) {
+    return NodeAttributeType.valueOf(containerState.name());
+  }
+
+  @Override
+  public String toString() {
+    return " name-" + getAttributeName() + ":value-" + getAttributeValue()
+        + ":type-" + getAttributeType();
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (obj instanceof NodeAttribute) {
+      NodeAttribute other = (NodeAttribute) obj;
+      if (!compare(getAttributeName(), other.getAttributeName())) {
+        return false;
+      }
+      if (!compare(getAttributeValue(), other.getAttributeValue())) {
+        return false;
+      }
+      if (!compare(getAttributeType(), other.getAttributeType())) {
+        return false;
+      }
+      return true;
+    }
+    return false;
+  }
+
+  private static boolean compare(Object left, Object right) {
+    if (left == null) {
+      return right == null;
+    } else {
+      return left.equals(right);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/84ebbfbf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/client/ResourceManagerAdministrationProtocolPBClientImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/client/ResourceManagerAdministrationProtocolPBClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/client/ResourceManagerAdministrationProtocolPBClientImpl.java
index 077edf3..156ed05 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/client/ResourceManagerAdministrationProtocolPBClientImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/client/ResourceManagerAdministrationProtocolPBClientImpl.java
@@ -33,9 +33,11 @@ import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.AddTo
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.CheckForDecommissioningNodesRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.GetGroupsForUserRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.GetGroupsForUserResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.NodesToAttributesMappingRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshAdminAclsRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshClusterMaxPriorityRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshNodesRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshNodesResourcesRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshQueuesRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshServiceAclsRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshSuperUserGroupsConfigurationRequestProto;
@@ -43,13 +45,14 @@ import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.Refre
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RemoveFromClusterNodeLabelsRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.ReplaceLabelsOnNodeRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.UpdateNodeResourceRequestProto;
-import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshNodesResourcesRequestProto;
 import org.apache.hadoop.yarn.server.api.ResourceManagerAdministrationProtocol;
 import org.apache.hadoop.yarn.server.api.ResourceManagerAdministrationProtocolPB;
 import org.apache.hadoop.yarn.server.api.protocolrecords.AddToClusterNodeLabelsRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.AddToClusterNodeLabelsResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.CheckForDecommissioningNodesRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.CheckForDecommissioningNodesResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodesToAttributesMappingRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodesToAttributesMappingResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshAdminAclsRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshAdminAclsResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshClusterMaxPriorityRequest;
@@ -76,11 +79,15 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.AddToClusterNod
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.AddToClusterNodeLabelsResponsePBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.CheckForDecommissioningNodesRequestPBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.CheckForDecommissioningNodesResponsePBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.NodesToAttributesMappingRequestPBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.NodesToAttributesMappingResponsePBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshAdminAclsRequestPBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshAdminAclsResponsePBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshClusterMaxPriorityRequestPBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshClusterMaxPriorityResponsePBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshNodesRequestPBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshNodesResourcesRequestPBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshNodesResourcesResponsePBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshNodesResponsePBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshQueuesRequestPBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshQueuesResponsePBImpl;
@@ -96,8 +103,6 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.ReplaceLabelsOn
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.ReplaceLabelsOnNodeResponsePBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.UpdateNodeResourceRequestPBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.UpdateNodeResourceResponsePBImpl;
-import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshNodesResourcesRequestPBImpl;
-import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshNodesResourcesResponsePBImpl;
 
 import com.google.protobuf.ServiceException;
 
@@ -323,4 +328,19 @@ public class ResourceManagerAdministrationProtocolPBClientImpl implements Resour
       return null;
     }
   }
+
+  @Override
+  public NodesToAttributesMappingResponse mapAttributesToNodes(
+      NodesToAttributesMappingRequest request)
+      throws YarnException, IOException {
+    NodesToAttributesMappingRequestProto requestProto =
+        ((NodesToAttributesMappingRequestPBImpl) request).getProto();
+    try {
+      return new NodesToAttributesMappingResponsePBImpl(
+          proxy.mapAttributesToNodes(null, requestProto));
+    } catch (ServiceException e) {
+      RPCUtil.unwrapAndThrowException(e);
+      return null;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/84ebbfbf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/service/ResourceManagerAdministrationProtocolPBServiceImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/service/ResourceManagerAdministrationProtocolPBServiceImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/service/ResourceManagerAdministrationProtocolPBServiceImpl.java
index aafce08..0036339 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/service/ResourceManagerAdministrationProtocolPBServiceImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/service/ResourceManagerAdministrationProtocolPBServiceImpl.java
@@ -28,11 +28,15 @@ import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.Check
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.CheckForDecommissioningNodesResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.GetGroupsForUserRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.GetGroupsForUserResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.NodesToAttributesMappingRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.NodesToAttributesMappingResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshAdminAclsRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshAdminAclsResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshClusterMaxPriorityRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshClusterMaxPriorityResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshNodesRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshNodesResourcesRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshNodesResourcesResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshNodesResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshQueuesRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshQueuesResponseProto;
@@ -48,13 +52,13 @@ import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.Repla
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.ReplaceLabelsOnNodeResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.UpdateNodeResourceRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.UpdateNodeResourceResponseProto;
-import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshNodesResourcesRequestProto;
-import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshNodesResourcesResponseProto;
 import org.apache.hadoop.yarn.server.api.ResourceManagerAdministrationProtocol;
 import org.apache.hadoop.yarn.server.api.ResourceManagerAdministrationProtocolPB;
 import org.apache.hadoop.yarn.server.api.protocolrecords.AddToClusterNodeLabelsResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.CheckForDecommissioningNodesRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.CheckForDecommissioningNodesResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodesToAttributesMappingRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodesToAttributesMappingResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshAdminAclsResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshClusterMaxPriorityRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshClusterMaxPriorityResponse;
@@ -71,11 +75,15 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.AddToClusterNod
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.AddToClusterNodeLabelsResponsePBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.CheckForDecommissioningNodesRequestPBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.CheckForDecommissioningNodesResponsePBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.NodesToAttributesMappingRequestPBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.NodesToAttributesMappingResponsePBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshAdminAclsRequestPBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshAdminAclsResponsePBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshClusterMaxPriorityRequestPBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshClusterMaxPriorityResponsePBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshNodesRequestPBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshNodesResourcesRequestPBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshNodesResourcesResponsePBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshNodesResponsePBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshQueuesRequestPBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshQueuesResponsePBImpl;
@@ -91,8 +99,6 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.ReplaceLabelsOn
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.ReplaceLabelsOnNodeResponsePBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.UpdateNodeResourceRequestPBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.UpdateNodeResourceResponsePBImpl;
-import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshNodesResourcesRequestPBImpl;
-import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshNodesResourcesResponsePBImpl;
 
 import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
@@ -336,4 +342,21 @@ public class ResourceManagerAdministrationProtocolPBServiceImpl implements Resou
       throw new ServiceException(e);
     }
   }
+
+  @Override
+  public NodesToAttributesMappingResponseProto mapAttributesToNodes(
+      RpcController controller, NodesToAttributesMappingRequestProto proto)
+      throws ServiceException {
+    NodesToAttributesMappingRequest request =
+        new NodesToAttributesMappingRequestPBImpl(proto);
+    try {
+      NodesToAttributesMappingResponse response =
+          real.mapAttributesToNodes(request);
+      return ((NodesToAttributesMappingResponsePBImpl) response).getProto();
+    } catch (YarnException e) {
+      throw new ServiceException(e);
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/84ebbfbf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeToAttributesPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeToAttributesPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeToAttributesPBImpl.java
new file mode 100644
index 0000000..7b52d03
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeToAttributesPBImpl.java
@@ -0,0 +1,161 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
+import org.apache.hadoop.yarn.api.records.impl.pb.NodeAttributePBImpl;
+import org.apache.hadoop.yarn.proto.YarnProtos.NodeAttributeProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.NodeToAttributesProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.NodeToAttributesProtoOrBuilder;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodeToAttributes;
+
+public class NodeToAttributesPBImpl extends NodeToAttributes {
+  private NodeToAttributesProto proto =
+      NodeToAttributesProto.getDefaultInstance();
+  private NodeToAttributesProto.Builder builder = null;
+  private boolean viaProto = false;
+
+  private List<NodeAttribute> nodeAttributes = null;
+
+  public NodeToAttributesPBImpl() {
+    builder = NodeToAttributesProto.newBuilder();
+  }
+
+  public NodeToAttributesPBImpl(NodeToAttributesProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    if (this.nodeAttributes != null) {
+      for (NodeAttribute nodeAttribute : nodeAttributes) {
+        builder.addNodeAttributes(
+            ((NodeAttributePBImpl) nodeAttribute).getProto());
+      }
+    }
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  public NodeToAttributesProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = NodeToAttributesProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  @Override
+  public String getNode() {
+    NodeToAttributesProtoOrBuilder p = viaProto ? proto : builder;
+    if (!p.hasNode()) {
+      return null;
+    }
+    return p.getNode();
+  }
+
+  @Override
+  public void setNode(String node) {
+    maybeInitBuilder();
+    builder.setNode(node);
+  }
+
+  private void initNodeAttributes() {
+    if (this.nodeAttributes != null) {
+      return;
+    }
+
+    NodeToAttributesProtoOrBuilder p = viaProto ? proto : builder;
+    List<NodeAttributeProto> nodeAttributesProtoList =
+        p.getNodeAttributesList();
+    List<NodeAttribute> attributes = new ArrayList<>();
+    if (nodeAttributesProtoList == null
+        || nodeAttributesProtoList.size() == 0) {
+      this.nodeAttributes = attributes;
+      return;
+    }
+    for (NodeAttributeProto nodeAttributeProto : nodeAttributesProtoList) {
+      attributes.add(new NodeAttributePBImpl(nodeAttributeProto));
+    }
+    this.nodeAttributes = attributes;
+  }
+
+  @Override
+  public List<NodeAttribute> getNodeAttributes() {
+    initNodeAttributes();
+    return this.nodeAttributes;
+  }
+
+  @Override
+  public void setNodeAttributes(List<NodeAttribute> attributes) {
+    if (nodeAttributes == null) {
+      nodeAttributes = new ArrayList<>();
+    }
+    nodeAttributes.clear();
+    nodeAttributes.addAll(attributes);
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (obj instanceof NodeToAttributes) {
+      NodeToAttributes other = (NodeToAttributes) obj;
+      if (getNodeAttributes() == null) {
+        if (other.getNodeAttributes() != null) {
+          return false;
+        }
+      } else if (!getNodeAttributes().containsAll(other.getNodeAttributes())) {
+        return false;
+      }
+
+      if (getNode() == null) {
+        if (other.getNode() != null) {
+          return false;
+        }
+      } else if (!getNode().equals(other.getNode())) {
+        return false;
+      }
+
+      return true;
+    }
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/84ebbfbf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodesToAttributesMappingRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodesToAttributesMappingRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodesToAttributesMappingRequestPBImpl.java
new file mode 100644
index 0000000..b319b26
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodesToAttributesMappingRequestPBImpl.java
@@ -0,0 +1,194 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.AttributeMappingOperationTypeProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.NodeToAttributesProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.NodesToAttributesMappingRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.NodesToAttributesMappingRequestProtoOrBuilder;
+import org.apache.hadoop.yarn.server.api.protocolrecords.AttributeMappingOperationType;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodeToAttributes;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodesToAttributesMappingRequest;
+
+public class NodesToAttributesMappingRequestPBImpl
+    extends NodesToAttributesMappingRequest {
+  private NodesToAttributesMappingRequestProto proto =
+      NodesToAttributesMappingRequestProto.getDefaultInstance();
+  private NodesToAttributesMappingRequestProto.Builder builder = null;
+  private boolean viaProto = false;
+
+  private List<NodeToAttributes> nodeAttributesMapping = null;
+
+  public NodesToAttributesMappingRequestPBImpl() {
+    builder = NodesToAttributesMappingRequestProto.newBuilder();
+  }
+
+  public NodesToAttributesMappingRequestPBImpl(
+      NodesToAttributesMappingRequestProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    if (this.nodeAttributesMapping != null) {
+      for (NodeToAttributes nodeAttributes : nodeAttributesMapping) {
+        builder.addNodeToAttributes(
+            ((NodeToAttributesPBImpl) nodeAttributes).getProto());
+      }
+    }
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  public NodesToAttributesMappingRequestProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = NodesToAttributesMappingRequestProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  @Override
+  public void setNodesToAttributes(List<NodeToAttributes> nodesToAttributes) {
+    if (nodeAttributesMapping == null) {
+      nodeAttributesMapping = new ArrayList<>();
+    }
+    if(nodesToAttributes == null) {
+      throw new IllegalArgumentException("nodesToAttributes cannot be null");
+    }
+    nodeAttributesMapping.clear();
+    nodeAttributesMapping.addAll(nodesToAttributes);
+  }
+
+  private void initNodeAttributesMapping() {
+    if (this.nodeAttributesMapping != null) {
+      return;
+    }
+
+    NodesToAttributesMappingRequestProtoOrBuilder p =
+        viaProto ? proto : builder;
+    List<NodeToAttributesProto> nodeAttributesProtoList =
+        p.getNodeToAttributesList();
+    List<NodeToAttributes> attributes = new ArrayList<>();
+    if (nodeAttributesProtoList == null
+        || nodeAttributesProtoList.size() == 0) {
+      this.nodeAttributesMapping = attributes;
+      return;
+    }
+    for (NodeToAttributesProto nodeAttributeProto : nodeAttributesProtoList) {
+      attributes.add(new NodeToAttributesPBImpl(nodeAttributeProto));
+    }
+    this.nodeAttributesMapping = attributes;
+  }
+
+  @Override
+  public List<NodeToAttributes> getNodesToAttributes() {
+    initNodeAttributesMapping();
+    return this.nodeAttributesMapping;
+  }
+
+  @Override
+  public void setFailOnUnknownNodes(boolean failOnUnknownNodes) {
+    maybeInitBuilder();
+    builder.setFailOnUnknownNodes(failOnUnknownNodes);
+  }
+
+  @Override
+  public boolean getFailOnUnknownNodes() {
+    NodesToAttributesMappingRequestProtoOrBuilder p =
+        viaProto ? proto : builder;
+    return p.getFailOnUnknownNodes();
+  }
+
+  @Override
+  public void setOperation(AttributeMappingOperationType operation) {
+    maybeInitBuilder();
+    builder.setOperation(convertToProtoFormat(operation));
+  }
+
+  private AttributeMappingOperationTypeProto convertToProtoFormat(
+      AttributeMappingOperationType operation) {
+    return AttributeMappingOperationTypeProto.valueOf(operation.name());
+  }
+
+  private AttributeMappingOperationType convertFromProtoFormat(
+      AttributeMappingOperationTypeProto operationTypeProto) {
+    return AttributeMappingOperationType.valueOf(operationTypeProto.name());
+  }
+
+  @Override
+  public AttributeMappingOperationType getOperation() {
+    NodesToAttributesMappingRequestProtoOrBuilder p =
+        viaProto ? proto : builder;
+    if (!p.hasOperation()) {
+      return null;
+    }
+    return convertFromProtoFormat(p.getOperation());
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (obj instanceof NodesToAttributesMappingRequest) {
+      NodesToAttributesMappingRequest other =
+          (NodesToAttributesMappingRequest) obj;
+      if (getNodesToAttributes() == null) {
+        if (other.getNodesToAttributes() != null) {
+          return false;
+        }
+      } else if (!getNodesToAttributes()
+          .containsAll(other.getNodesToAttributes())) {
+        return false;
+      }
+
+      if (getOperation() == null) {
+        if (other.getOperation() != null) {
+          return false;
+        }
+      } else if (!getOperation().equals(other.getOperation())) {
+        return false;
+      }
+
+      return getFailOnUnknownNodes() == other.getFailOnUnknownNodes();
+    }
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/84ebbfbf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodesToAttributesMappingResponsePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodesToAttributesMappingResponsePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodesToAttributesMappingResponsePBImpl.java
new file mode 100644
index 0000000..0554ef8
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodesToAttributesMappingResponsePBImpl.java
@@ -0,0 +1,47 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb;
+
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.NodesToAttributesMappingResponseProto;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodesToAttributesMappingResponse;
+
+public class NodesToAttributesMappingResponsePBImpl
+    extends NodesToAttributesMappingResponse {
+
+  private NodesToAttributesMappingResponseProto proto =
+      NodesToAttributesMappingResponseProto.getDefaultInstance();
+  private NodesToAttributesMappingResponseProto.Builder builder = null;
+  private boolean viaProto = false;
+
+  public NodesToAttributesMappingResponsePBImpl() {
+    builder = NodesToAttributesMappingResponseProto.newBuilder();
+  }
+
+  public NodesToAttributesMappingResponsePBImpl(
+      NodesToAttributesMappingResponseProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public NodesToAttributesMappingResponseProto getProto() {
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/84ebbfbf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
index c5585c2..b951fd3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.yarn.api;
 import java.io.IOException;
+import java.util.Arrays;
 
 import org.apache.commons.lang.math.LongRange;
 import org.apache.hadoop.security.proto.SecurityProtos.CancelDelegationTokenRequestProto;
@@ -106,7 +107,6 @@ import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StopContainersRequestP
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StopContainersResponsePBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SubmitApplicationRequestPBImpl;
 import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SubmitApplicationResponsePBImpl;
-import org.apache.hadoop.yarn.api.records.CollectorInfo;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -114,6 +114,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.ApplicationTimeout;
+import org.apache.hadoop.yarn.api.records.CollectorInfo;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
@@ -124,6 +125,7 @@ import org.apache.hadoop.yarn.api.records.ExecutionTypeRequest;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LogAggregationContext;
 import org.apache.hadoop.yarn.api.records.NMToken;
+import org.apache.hadoop.yarn.api.records.NodeAttribute;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeLabel;
 import org.apache.hadoop.yarn.api.records.NodeReport;
@@ -145,8 +147,8 @@ import org.apache.hadoop.yarn.api.records.ReservationRequest;
 import org.apache.hadoop.yarn.api.records.ReservationRequests;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceAllocationRequest;
-import org.apache.hadoop.yarn.api.records.ResourceInformation;
 import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
+import org.apache.hadoop.yarn.api.records.ResourceInformation;
 import org.apache.hadoop.yarn.api.records.ResourceOption;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.ResourceTypeInfo;
@@ -174,6 +176,7 @@ import org.apache.hadoop.yarn.api.records.impl.pb.ContainerStatusPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ExecutionTypeRequestPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.LocalResourcePBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.NMTokenPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.NodeAttributePBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.NodeIdPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.NodeLabelPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.NodeReportPBImpl;
@@ -211,6 +214,7 @@ import org.apache.hadoop.yarn.proto.YarnProtos.ContainerRetryContextProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStatusProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ExecutionTypeRequestProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.LocalResourceProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.NodeAttributeProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.NodeIdProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.NodeLabelProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.NodeReportProto;
@@ -219,6 +223,7 @@ import org.apache.hadoop.yarn.proto.YarnProtos.PreemptionContractProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.PreemptionMessageProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.PreemptionResourceRequestProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.PriorityProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ProfileCapabilityProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.QueueInfoProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.QueueUserACLInfoProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ResourceBlacklistRequestProto;
@@ -233,6 +238,8 @@ import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.AddTo
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.AddToClusterNodeLabelsResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.CheckForDecommissioningNodesRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.CheckForDecommissioningNodesResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.NodeToAttributesProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.NodesToAttributesMappingRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshAdminAclsRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshAdminAclsResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshNodesRequestProto;
@@ -258,6 +265,7 @@ import org.apache.hadoop.yarn.proto.YarnServiceProtos.AllocateRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.AllocateResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.FinishApplicationMasterRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.FinishApplicationMasterResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetAllResourceProfilesResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetApplicationAttemptReportRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetApplicationAttemptReportResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetApplicationAttemptsRequestProto;
@@ -288,6 +296,8 @@ import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetQueueInfoRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetQueueInfoResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetQueueUserAclsInfoRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetQueueUserAclsInfoResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetResourceProfileRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetResourceProfileResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.IncreaseContainersResourceRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.IncreaseContainersResourceResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.KillApplicationRequestProto;
@@ -312,14 +322,14 @@ import org.apache.hadoop.yarn.proto.YarnServiceProtos.StopContainersRequestProto
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.StopContainersResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.SubmitApplicationRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.SubmitApplicationResponseProto;
-import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetAllResourceProfilesResponseProto;
-import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetResourceProfileRequestProto;
-import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetResourceProfileResponseProto;
-import org.apache.hadoop.yarn.proto.YarnProtos.ProfileCapabilityProto;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodeToAttributes;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodesToAttributesMappingRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.AddToClusterNodeLabelsRequestPBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.AddToClusterNodeLabelsResponsePBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.CheckForDecommissioningNodesRequestPBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.CheckForDecommissioningNodesResponsePBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.NodeToAttributesPBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.NodesToAttributesMappingRequestPBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshAdminAclsRequestPBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshAdminAclsResponsePBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshNodesRequestPBImpl;
@@ -347,7 +357,6 @@ import org.junit.Ignore;
 import org.junit.Test;
 
 import com.google.common.collect.ImmutableSet;
-import java.util.Arrays;
 
 /**
  * Test class for YARN API protocol records.
@@ -428,6 +437,11 @@ public class TestPBImplRecords extends BasePBImplRecordsTest {
     generateByNewInstance(QueueConfigurations.class);
     generateByNewInstance(CollectorInfo.class);
     generateByNewInstance(ResourceTypeInfo.class);
+    //for Node attribute support
+    generateByNewInstance(NodeAttribute.class);
+    generateByNewInstance(NodeToAttributes.class);
+    generateByNewInstance(NodesToAttributesMappingRequest.class);
+
   }
 
   @Test
@@ -1217,4 +1231,22 @@ public class TestPBImplRecords extends BasePBImplRecordsTest {
     validatePBImplRecord(GetAllResourceTypeInfoResponsePBImpl.class,
         YarnServiceProtos.GetAllResourceTypeInfoResponseProto.class);
   }
+
+  @Test
+  public void testNodeAttributePBImpl() throws Exception {
+    validatePBImplRecord(NodeAttributePBImpl.class,
+	NodeAttributeProto.class);
+  }
+
+  @Test
+  public void testNodeToAttributesPBImpl() throws Exception {
+    validatePBImplRecord(NodeToAttributesPBImpl.class,
+	NodeToAttributesProto.class);
+  }
+
+  @Test
+  public void testNodesToAttributesMappingRequestPBImpl() throws Exception {
+    validatePBImplRecord(NodesToAttributesMappingRequestPBImpl.class,
+	NodesToAttributesMappingRequestProto.class);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/84ebbfbf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java
index 15e1cea..a5bf48c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/MockResourceManagerFacade.java
@@ -163,6 +163,8 @@ import org.apache.hadoop.yarn.util.Records;
 import org.junit.Assert;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodesToAttributesMappingRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodesToAttributesMappingResponse;
 
 import com.google.common.base.Strings;
 
@@ -180,15 +182,15 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
 
   private HashSet<ApplicationId> applicationMap = new HashSet<>();
   private HashSet<ApplicationId> keepContainerOnUams = new HashSet<>();
-  private HashMap<ApplicationAttemptId, List<ContainerId>>
-      applicationContainerIdMap = new HashMap<>();
+  private HashMap<ApplicationAttemptId, List<ContainerId>> applicationContainerIdMap =
+      new HashMap<ApplicationAttemptId, List<ContainerId>>();
   private AtomicInteger containerIndex = new AtomicInteger(0);
   private Configuration conf;
   private int subClusterId;
   final private AtomicInteger applicationCounter = new AtomicInteger(0);
 
   // True if the Mock RM is running, false otherwise.
-  // This property allows us to write tests for specific scenario as YARN RM
+  // This property allows us to write tests for specific scenario as Yarn RM
   // down e.g. network issue, failover.
   private boolean isRunning;
 
@@ -273,7 +275,7 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
       } else {
         // Keep track of the containers that are returned to this application
         applicationContainerIdMap.put(attemptId, new ArrayList<ContainerId>());
-      }
+      }     
     }
 
     // Make sure we wait for certain test cases last in the method
@@ -382,7 +384,7 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
                     + attemptId,
                 applicationContainerIdMap.containsKey(attemptId));
             List<ContainerId> ids = applicationContainerIdMap.get(attemptId);
-            ids.add(containerId);
+            ids.add(containerId);            
           }
         }
       }
@@ -493,7 +495,7 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
     if (request.getApplicationSubmissionContext().getUnmanagedAM()
         || request.getApplicationSubmissionContext()
             .getKeepContainersAcrossApplicationAttempts()) {
-      keepContainerOnUams.add(appId);
+    	keepContainerOnUams.add(appId);
     }
     return SubmitApplicationResponse.newInstance();
   }
@@ -511,7 +513,6 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
         throw new ApplicationNotFoundException(
             "Trying to kill an absent application: " + appId);
       }
-      keepContainerOnUams.remove(appId);
     }
     LOG.info("Force killing application: " + appId);
     return KillApplicationResponse.newInstance(true);
@@ -899,4 +900,10 @@ public class MockResourceManagerFacade implements ApplicationClientProtocol,
       GetAllResourceTypeInfoRequest request) throws YarnException, IOException {
     return null;
   }
+
+  @Override
+  public NodesToAttributesMappingResponse mapAttributesToNodes(NodesToAttributesMappingRequest request)
+    throws YarnException, IOException {
+    return null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/84ebbfbf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
index accf901..7444283 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
@@ -66,6 +66,8 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.AddToClusterNodeLabelsR
 import org.apache.hadoop.yarn.server.api.protocolrecords.AddToClusterNodeLabelsResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.CheckForDecommissioningNodesRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.CheckForDecommissioningNodesResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodesToAttributesMappingRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodesToAttributesMappingResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshAdminAclsRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshAdminAclsResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshClusterMaxPriorityRequest;
@@ -960,4 +962,12 @@ public class AdminService extends CompositeService implements
 
     rm.getRMContext().getScheduler().setClusterMaxPriority(conf);
   }
+
+  @Override
+  public NodesToAttributesMappingResponse mapAttributesToNodes(
+      NodesToAttributesMappingRequest request)
+      throws YarnException, IOException {
+    return recordFactory
+        .newRecordInstance(NodesToAttributesMappingResponse.class);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/84ebbfbf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/DefaultRMAdminRequestInterceptor.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/DefaultRMAdminRequestInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/DefaultRMAdminRequestInterceptor.java
index 7e6a1ff..b4140c1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/DefaultRMAdminRequestInterceptor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/DefaultRMAdminRequestInterceptor.java
@@ -32,6 +32,8 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.AddToClusterNodeLabelsR
 import org.apache.hadoop.yarn.server.api.protocolrecords.AddToClusterNodeLabelsResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.CheckForDecommissioningNodesRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.CheckForDecommissioningNodesResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodesToAttributesMappingRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodesToAttributesMappingResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshAdminAclsRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshAdminAclsResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshClusterMaxPriorityRequest;
@@ -212,4 +214,11 @@ public class DefaultRMAdminRequestInterceptor
   public String[] getGroupsForUser(String userName) throws IOException {
     return rmAdminProxy.getGroupsForUser(userName);
   }
+
+  @Override
+  public NodesToAttributesMappingResponse mapAttributesToNodes(
+      NodesToAttributesMappingRequest request)
+      throws YarnException, IOException {
+    return rmAdminProxy.mapAttributesToNodes(request);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/84ebbfbf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/RouterRMAdminService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/RouterRMAdminService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/RouterRMAdminService.java
index b8b7ad8..8d83b23 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/RouterRMAdminService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/RouterRMAdminService.java
@@ -43,6 +43,8 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.AddToClusterNodeLabelsR
 import org.apache.hadoop.yarn.server.api.protocolrecords.AddToClusterNodeLabelsResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.CheckForDecommissioningNodesRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.CheckForDecommissioningNodesResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodesToAttributesMappingRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodesToAttributesMappingResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshAdminAclsRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshAdminAclsResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshClusterMaxPriorityRequest;
@@ -420,4 +422,12 @@ public class RouterRMAdminService extends AbstractService
     RequestInterceptorChainWrapper pipeline = getInterceptorChain();
     return pipeline.getRootInterceptor().refreshClusterMaxPriority(request);
   }
+
+  @Override
+  public NodesToAttributesMappingResponse mapAttributesToNodes(
+      NodesToAttributesMappingRequest request)
+      throws YarnException, IOException {
+    RequestInterceptorChainWrapper pipeline = getInterceptorChain();
+    return pipeline.getRootInterceptor().mapAttributesToNodes(request);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/84ebbfbf/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/rmadmin/PassThroughRMAdminRequestInterceptor.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/rmadmin/PassThroughRMAdminRequestInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/rmadmin/PassThroughRMAdminRequestInterceptor.java
index 38dcc3d..4d6a3ea 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/rmadmin/PassThroughRMAdminRequestInterceptor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/rmadmin/PassThroughRMAdminRequestInterceptor.java
@@ -26,6 +26,8 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.AddToClusterNodeLabelsR
 import org.apache.hadoop.yarn.server.api.protocolrecords.AddToClusterNodeLabelsResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.CheckForDecommissioningNodesRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.CheckForDecommissioningNodesResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodesToAttributesMappingRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NodesToAttributesMappingResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshAdminAclsRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshAdminAclsResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshClusterMaxPriorityRequest;
@@ -145,4 +147,11 @@ public class PassThroughRMAdminRequestInterceptor
     return getNextInterceptor().getGroupsForUser(user);
   }
 
+  @Override
+  public NodesToAttributesMappingResponse mapAttributesToNodes(
+      NodesToAttributesMappingRequest request)
+      throws YarnException, IOException {
+    return getNextInterceptor().mapAttributesToNodes(request);
+  }
+
 }


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


[07/50] [abbrv] hadoop git commit: HDFS-12893. [READ] Support replication of Provided blocks with non-default topologies.

Posted by na...@apache.org.
HDFS-12893. [READ] Support replication of Provided blocks with non-default topologies.


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

Branch: refs/heads/yarn-3409
Commit: c89b29bd421152f0e7e16936f18d9e852895c37a
Parents: 0f6aa95
Author: Virajith Jalaparti <vi...@apache.org>
Authored: Fri Dec 8 14:52:48 2017 -0800
Committer: Chris Douglas <cd...@apache.org>
Committed: Fri Dec 15 17:51:41 2017 -0800

----------------------------------------------------------------------
 .../server/blockmanagement/BlockManager.java    | 30 +++++++++++-
 .../blockmanagement/DatanodeStorageInfo.java    | 11 +++--
 .../blockmanagement/ProvidedStorageMap.java     | 18 ++++++-
 .../TestNameNodeProvidedImplementation.java     | 49 ++++++++++++++++++--
 4 files changed, 97 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c89b29bd/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 916cbaa..c1cd4db 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
@@ -2151,6 +2151,22 @@ public class BlockManager implements BlockStatsMXBean {
   }
 
   /**
+   * Get the associated {@link DatanodeDescriptor} for the storage.
+   * If the storage is of type PROVIDED, one of the nodes that reported
+   * PROVIDED storage are returned. If not, this is equivalent to
+   * {@code storage.getDatanodeDescriptor()}.
+   * @param storage
+   * @return the associated {@link DatanodeDescriptor}.
+   */
+  private DatanodeDescriptor getDatanodeDescriptorFromStorage(
+      DatanodeStorageInfo storage) {
+    if (storage.getStorageType() == StorageType.PROVIDED) {
+      return providedStorageMap.chooseProvidedDatanode();
+    }
+    return storage.getDatanodeDescriptor();
+  }
+
+  /**
    * Parse the data-nodes the block belongs to and choose a certain number
    * from them to be the recovery sources.
    *
@@ -2198,10 +2214,14 @@ public class BlockManager implements BlockStatsMXBean {
     BitSet bitSet = isStriped ?
         new BitSet(((BlockInfoStriped) block).getTotalBlockNum()) : null;
     for (DatanodeStorageInfo storage : blocksMap.getStorages(block)) {
-      final DatanodeDescriptor node = storage.getDatanodeDescriptor();
+      final DatanodeDescriptor node = getDatanodeDescriptorFromStorage(storage);
       final StoredReplicaState state = checkReplicaOnStorage(numReplicas, block,
           storage, corruptReplicas.getNodes(block), false);
       if (state == StoredReplicaState.LIVE) {
+        if (storage.getStorageType() == StorageType.PROVIDED) {
+          storage = new DatanodeStorageInfo(node, storage.getStorageID(),
+              storage.getStorageType(), storage.getState());
+        }
         nodesContainingLiveReplicas.add(storage);
       }
       containingNodes.add(node);
@@ -4338,7 +4358,13 @@ public class BlockManager implements BlockStatsMXBean {
     Collection<DatanodeDescriptor> corruptNodes = corruptReplicas
         .getNodes(storedBlock);
     for (DatanodeStorageInfo storage : blocksMap.getStorages(storedBlock)) {
-      final DatanodeDescriptor cur = storage.getDatanodeDescriptor();
+      if (storage.getStorageType() == StorageType.PROVIDED
+          && storage.getState() == State.NORMAL) {
+        // assume the policy is satisfied for blocks on PROVIDED storage
+        // as long as the storage is in normal state.
+        return true;
+      }
+      final DatanodeDescriptor cur = getDatanodeDescriptorFromStorage(storage);
       // Nodes under maintenance should be counted as valid replicas from
       // rack policy point of view.
       if (!cur.isDecommissionInProgress() && !cur.isDecommissioned()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c89b29bd/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java
index 76bf915..3a56ef1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java
@@ -120,10 +120,15 @@ public class DatanodeStorageInfo {
   private boolean blockContentsStale = true;
 
   DatanodeStorageInfo(DatanodeDescriptor dn, DatanodeStorage s) {
+    this(dn, s.getStorageID(), s.getStorageType(), s.getState());
+  }
+
+  DatanodeStorageInfo(DatanodeDescriptor dn, String storageID,
+      StorageType storageType, State state) {
     this.dn = dn;
-    this.storageID = s.getStorageID();
-    this.storageType = s.getStorageType();
-    this.state = s.getState();
+    this.storageID = storageID;
+    this.storageType = storageType;
+    this.state = state;
   }
 
   public int getBlockReportCount() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c89b29bd/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ProvidedStorageMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ProvidedStorageMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ProvidedStorageMap.java
index 208ed3e..08d1434 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ProvidedStorageMap.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ProvidedStorageMap.java
@@ -192,7 +192,7 @@ public class ProvidedStorageMap {
   }
 
   public void updateStorage(DatanodeDescriptor node, DatanodeStorage storage) {
-    if (providedEnabled && storageId.equals(storage.getStorageID())) {
+    if (isProvidedStorage(storage.getStorageID())) {
       if (StorageType.PROVIDED.equals(storage.getStorageType())) {
         node.injectStorage(providedStorageInfo);
         return;
@@ -204,6 +204,22 @@ public class ProvidedStorageMap {
     node.updateStorage(storage);
   }
 
+  private boolean isProvidedStorage(String dnStorageId) {
+    return providedEnabled && storageId.equals(dnStorageId);
+  }
+
+  /**
+   * Choose a datanode that reported a volume of {@link StorageType} PROVIDED.
+   *
+   * @return the {@link DatanodeDescriptor} corresponding to a datanode that
+   *         reported a volume with {@link StorageType} PROVIDED. If multiple
+   *         datanodes report a PROVIDED volume, one is chosen uniformly at
+   *         random.
+   */
+  public DatanodeDescriptor chooseProvidedDatanode() {
+    return providedDescriptor.chooseRandom();
+  }
+
   /**
    * Builder used for creating {@link LocatedBlocks} when a block is provided.
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c89b29bd/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java
index 394e8d8..2917a34 100644
--- a/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java
+++ b/hadoop-tools/hadoop-fs2img/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeProvidedImplementation.java
@@ -65,6 +65,7 @@ import org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.InMemoryLevelDBAl
 import org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.TextFileRegionAliasMap;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_REPLICATOR_CLASSNAME_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY;
 
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
@@ -201,8 +202,15 @@ public class TestNameNodeProvidedImplementation {
   void startCluster(Path nspath, int numDatanodes,
       StorageType[] storageTypes,
       StorageType[][] storageTypesPerDatanode,
-      boolean doFormat)
-      throws IOException {
+      boolean doFormat) throws IOException {
+    startCluster(nspath, numDatanodes, storageTypes, storageTypesPerDatanode,
+        doFormat, null);
+  }
+
+  void startCluster(Path nspath, int numDatanodes,
+      StorageType[] storageTypes,
+      StorageType[][] storageTypesPerDatanode,
+      boolean doFormat, String[] racks) throws IOException {
     conf.set(DFS_NAMENODE_NAME_DIR_KEY, nspath.toString());
 
     if (storageTypesPerDatanode != null) {
@@ -211,6 +219,7 @@ public class TestNameNodeProvidedImplementation {
           .manageNameDfsDirs(doFormat)
           .numDataNodes(numDatanodes)
           .storageTypes(storageTypesPerDatanode)
+          .racks(racks)
           .build();
     } else if (storageTypes != null) {
       cluster = new MiniDFSCluster.Builder(conf)
@@ -219,12 +228,14 @@ public class TestNameNodeProvidedImplementation {
           .numDataNodes(numDatanodes)
           .storagesPerDatanode(storageTypes.length)
           .storageTypes(storageTypes)
+          .racks(racks)
           .build();
     } else {
       cluster = new MiniDFSCluster.Builder(conf)
           .format(doFormat)
           .manageNameDfsDirs(doFormat)
           .numDataNodes(numDatanodes)
+          .racks(racks)
           .build();
     }
     cluster.waitActive();
@@ -515,11 +526,12 @@ public class TestNameNodeProvidedImplementation {
             StorageType.PROVIDED, StorageType.DISK},
         null,
         false);
+    setAndUnsetReplication("/" + filePrefix + (numFiles - 1) + fileSuffix);
+  }
 
-    String filename = "/" + filePrefix + (numFiles - 1) + fileSuffix;
+  private void setAndUnsetReplication(String filename) throws Exception {
     Path file = new Path(filename);
     FileSystem fs = cluster.getFileSystem();
-
     // set the replication to 4, and test that the file has
     // the required replication.
     short newReplication = 4;
@@ -833,7 +845,7 @@ public class TestNameNodeProvidedImplementation {
         new StorageType[] {StorageType.PROVIDED, StorageType.DISK},
         null, false);
 
-    int fileIndex = numFiles -1;
+    int fileIndex = numFiles - 1;
 
     final BlockManager blockManager = cluster.getNamesystem().getBlockManager();
     final DatanodeManager dnm = blockManager.getDatanodeManager();
@@ -890,4 +902,31 @@ public class TestNameNodeProvidedImplementation {
     // reports all 3 replicas
     verifyFileLocation(fileIndex, 3);
   }
+
+  @Test
+  public void testProvidedWithHierarchicalTopology() throws Exception {
+    conf.setClass(ImageWriter.Options.UGI_CLASS, FsUGIResolver.class,
+        UGIResolver.class);
+    String packageName = "org.apache.hadoop.hdfs.server.blockmanagement";
+    String[] policies = new String[] {
+        "BlockPlacementPolicyDefault",
+        "BlockPlacementPolicyRackFaultTolerant",
+        "BlockPlacementPolicyWithNodeGroup",
+        "BlockPlacementPolicyWithUpgradeDomain"};
+    createImage(new FSTreeWalk(NAMEPATH, conf), NNDIRPATH,
+        FixedBlockResolver.class);
+    String[] racks =
+        {"/pod0/rack0", "/pod0/rack0", "/pod0/rack1", "/pod0/rack1",
+            "/pod1/rack0", "/pod1/rack0", "/pod1/rack1", "/pod1/rack1" };
+    for (String policy: policies) {
+      LOG.info("Using policy: " + packageName + "." + policy);
+      conf.set(DFS_BLOCK_REPLICATOR_CLASSNAME_KEY, packageName + "." + policy);
+      startCluster(NNDIRPATH, racks.length,
+          new StorageType[]{StorageType.PROVIDED, StorageType.DISK},
+          null, false, racks);
+      verifyFileSystemContents();
+      setAndUnsetReplication("/" + filePrefix + (numFiles - 1) + fileSuffix);
+      cluster.shutdown();
+    }
+  }
 }


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


[46/50] [abbrv] hadoop git commit: HDFS-12938. TestErasureCodigCLI testAll failing consistently. (Contributed by Ajay Kumar)

Posted by na...@apache.org.
HDFS-12938. TestErasureCodigCLI testAll failing consistently. (Contributed by Ajay Kumar)


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

Branch: refs/heads/yarn-3409
Commit: b318bed01affa150d70661f263efff9a5c9422f6
Parents: c8ff0cc
Author: Lei Xu <le...@apache.org>
Authored: Thu Dec 21 10:28:24 2017 -0800
Committer: Lei Xu <le...@apache.org>
Committed: Thu Dec 21 10:28:24 2017 -0800

----------------------------------------------------------------------
 .../hadoop-hdfs/src/test/resources/testErasureCodingConf.xml     | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b318bed0/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml
index bd451eb..fc0c060 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml
@@ -311,7 +311,7 @@
       <comparators>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>Warning: setting erasure coding policy on an non-empty directory will not automatically convert existing data to RS-6-3-1024</expected-output>
+          <expected-output>Warning: setting erasure coding policy on a non-empty directory will not automatically convert existing files to RS-6-3-1024</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -353,7 +353,7 @@
       <comparators>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>Warning: unsetting erasure coding policy on an non-empty directory will not automatically convert existing data to replicated data</expected-output>
+          <expected-output>Warning: unsetting erasure coding policy on a non-empty directory will not automatically convert existing files to replicated data</expected-output>
         </comparator>
       </comparators>
     </test>


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


[23/50] [abbrv] hadoop git commit: Add 2.8.3 release jdiff files.

Posted by na...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/a7f8caf5/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Server_Common_2.8.3.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Server_Common_2.8.3.xml b/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Server_Common_2.8.3.xml
new file mode 100644
index 0000000..f3191e4
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Server_Common_2.8.3.xml
@@ -0,0 +1,829 @@
+<?xml version="1.0" encoding="iso-8859-1" standalone="no"?>
+<!-- Generated by the JDiff Javadoc doclet -->
+<!-- (http://www.jdiff.org) -->
+<!-- on Tue Dec 05 05:32:45 UTC 2017 -->
+
+<api
+  xmlns:xsi='http://www.w3.org/2001/XMLSchema-instance'
+  xsi:noNamespaceSchemaLocation='api.xsd'
+  name="Apache Hadoop YARN Server Common 2.8.3"
+  jdversion="1.0.9">
+
+<!--  Command line arguments =  -doclet org.apache.hadoop.classification.tools.IncludePublicAnnotationsJDiffDoclet -docletpath /build/source/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/target/hadoop-annotations.jar:/build/source/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/target/jdiff.jar -verbose -classpath /build/source/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/target/classes:/build/source/hadoop-common-project/hadoop-common/target/hadoop-common-2.8.3.jar:/maven/commons-cli/commons-cli/1.2/commons-cli-1.2.jar:/maven/org/apache/commons/commons-math3/3.1.1/commons-math3-3.1.1.jar:/maven/xmlenc/xmlenc/0.52/xmlenc-0.52.jar:/maven/org/apache/httpcomponents/httpclient/4.5.2/httpclient-4.5.2.jar:/maven/org/apache/httpcomponents/httpcore/4.4.4/httpcore-4.4.4.jar:/maven/commons-codec/commons-codec/1.4/commons-codec-1.4.jar:/maven/commons-io/commons-io/2.4/commons-io-2.4.jar:/maven/commons-n
 et/commons-net/3.1/commons-net-3.1.jar:/maven/commons-collections/commons-collections/3.2.2/commons-collections-3.2.2.jar:/maven/javax/servlet/servlet-api/2.5/servlet-api-2.5.jar:/maven/org/mortbay/jetty/jetty/6.1.26/jetty-6.1.26.jar:/maven/org/mortbay/jetty/jetty-util/6.1.26/jetty-util-6.1.26.jar:/maven/org/mortbay/jetty/jetty-sslengine/6.1.26/jetty-sslengine-6.1.26.jar:/maven/javax/servlet/jsp/jsp-api/2.1/jsp-api-2.1.jar:/maven/com/sun/jersey/jersey-core/1.9/jersey-core-1.9.jar:/maven/com/sun/jersey/jersey-json/1.9/jersey-json-1.9.jar:/maven/org/codehaus/jettison/jettison/1.1/jettison-1.1.jar:/maven/com/sun/xml/bind/jaxb-impl/2.2.3-1/jaxb-impl-2.2.3-1.jar:/maven/com/sun/jersey/jersey-server/1.9/jersey-server-1.9.jar:/maven/asm/asm/3.2/asm-3.2.jar:/maven/log4j/log4j/1.2.17/log4j-1.2.17.jar:/maven/net/java/dev/jets3t/jets3t/0.9.0/jets3t-0.9.0.jar:/maven/com/jamesmurty/utils/java-xmlbuilder/0.4/java-xmlbuilder-0.4.jar:/maven/commons-lang/commons-lang/2.6/commons-lang-2.6.jar:/maven/c
 ommons-configuration/commons-configuration/1.6/commons-configuration-1.6.jar:/maven/commons-digester/commons-digester/1.8/commons-digester-1.8.jar:/maven/commons-beanutils/commons-beanutils/1.7.0/commons-beanutils-1.7.0.jar:/maven/commons-beanutils/commons-beanutils-core/1.8.0/commons-beanutils-core-1.8.0.jar:/maven/org/slf4j/slf4j-api/1.7.10/slf4j-api-1.7.10.jar:/maven/org/slf4j/slf4j-log4j12/1.7.10/slf4j-log4j12-1.7.10.jar:/maven/org/codehaus/jackson/jackson-core-asl/1.9.13/jackson-core-asl-1.9.13.jar:/maven/org/codehaus/jackson/jackson-mapper-asl/1.9.13/jackson-mapper-asl-1.9.13.jar:/maven/org/apache/avro/avro/1.7.4/avro-1.7.4.jar:/maven/com/thoughtworks/paranamer/paranamer/2.3/paranamer-2.3.jar:/maven/org/xerial/snappy/snappy-java/1.0.4.1/snappy-java-1.0.4.1.jar:/maven/com/google/code/gson/gson/2.2.4/gson-2.2.4.jar:/build/source/hadoop-common-project/hadoop-auth/target/hadoop-auth-2.8.3.jar:/maven/com/nimbusds/nimbus-jose-jwt/3.9/nimbus-jose-jwt-3.9.jar:/maven/net/jcip/jcip-anno
 tations/1.0/jcip-annotations-1.0.jar:/maven/net/minidev/json-smart/1.1.1/json-smart-1.1.1.jar:/maven/org/apache/directory/server/apacheds-kerberos-codec/2.0.0-M15/apacheds-kerberos-codec-2.0.0-M15.jar:/maven/org/apache/directory/server/apacheds-i18n/2.0.0-M15/apacheds-i18n-2.0.0-M15.jar:/maven/org/apache/directory/api/api-asn1-api/1.0.0-M20/api-asn1-api-1.0.0-M20.jar:/maven/org/apache/directory/api/api-util/1.0.0-M20/api-util-1.0.0-M20.jar:/maven/org/apache/curator/curator-framework/2.7.1/curator-framework-2.7.1.jar:/maven/com/jcraft/jsch/0.1.54/jsch-0.1.54.jar:/maven/org/apache/curator/curator-client/2.7.1/curator-client-2.7.1.jar:/maven/org/apache/curator/curator-recipes/2.7.1/curator-recipes-2.7.1.jar:/maven/com/google/code/findbugs/jsr305/3.0.0/jsr305-3.0.0.jar:/maven/org/apache/htrace/htrace-core4/4.0.1-incubating/htrace-core4-4.0.1-incubating.jar:/maven/org/apache/commons/commons-compress/1.4.1/commons-compress-1.4.1.jar:/maven/org/tukaani/xz/1.0/xz-1.0.jar:/build/source/hadoo
 p-yarn-project/hadoop-yarn/hadoop-yarn-api/target/hadoop-yarn-api-2.8.3.jar:/build/source/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/target/hadoop-yarn-common-2.8.3.jar:/maven/javax/xml/bind/jaxb-api/2.2.2/jaxb-api-2.2.2.jar:/maven/javax/xml/stream/stax-api/1.0-2/stax-api-1.0-2.jar:/maven/javax/activation/activation/1.1/activation-1.1.jar:/maven/com/sun/jersey/jersey-client/1.9/jersey-client-1.9.jar:/maven/org/codehaus/jackson/jackson-jaxrs/1.9.13/jackson-jaxrs-1.9.13.jar:/maven/org/codehaus/jackson/jackson-xc/1.9.13/jackson-xc-1.9.13.jar:/maven/com/google/inject/extensions/guice-servlet/3.0/guice-servlet-3.0.jar:/maven/com/google/inject/guice/3.0/guice-3.0.jar:/maven/javax/inject/javax.inject/1/javax.inject-1.jar:/maven/aopalliance/aopalliance/1.0/aopalliance-1.0.jar:/maven/com/sun/jersey/contribs/jersey-guice/1.9/jersey-guice-1.9.jar:/maven/com/google/guava/guava/11.0.2/guava-11.0.2.jar:/maven/commons-logging/commons-logging/1.1.3/commons-logging-1.1.3.jar:/build/source/ha
 doop-common-project/hadoop-annotations/target/hadoop-annotations-2.8.3.jar:/usr/lib/jvm/java-7-openjdk-amd64/lib/tools.jar:/maven/com/google/protobuf/protobuf-java/2.5.0/protobuf-java-2.5.0.jar:/maven/org/apache/zookeeper/zookeeper/3.4.6/zookeeper-3.4.6.jar:/maven/io/netty/netty/3.6.2.Final/netty-3.6.2.Final.jar:/maven/org/fusesource/leveldbjni/leveldbjni-all/1.8/leveldbjni-all-1.8.jar -sourcepath /build/source/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java -apidir /build/source/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/target/site/jdiff/xml -apiname Apache Hadoop YARN Server Common 2.8.3 -->
+<package name="org.apache.hadoop.yarn.server">
+</package>
+<package name="org.apache.hadoop.yarn.server.api">
+</package>
+<package name="org.apache.hadoop.yarn.server.api.impl.pb.client">
+</package>
+<package name="org.apache.hadoop.yarn.server.api.impl.pb.service">
+</package>
+<package name="org.apache.hadoop.yarn.server.api.records">
+  <!-- start class org.apache.hadoop.yarn.server.api.records.NodeHealthStatus -->
+  <class name="NodeHealthStatus" extends="java.lang.Object"
+    abstract="true"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="NodeHealthStatus"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="getIsNodeHealthy" return="boolean"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Is the node healthy?
+ @return <code>true</code> if the node is healthy, else <code>false</code>]]>
+      </doc>
+    </method>
+    <method name="getHealthReport" return="java.lang.String"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the <em>diagnostic health report</em> of the node.
+ @return <em>diagnostic health report</em> of the node]]>
+      </doc>
+    </method>
+    <method name="getLastHealthReportTime" return="long"
+      abstract="true" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Get the <em>last timestamp</em> at which the health report was received.
+ @return <em>last timestamp</em> at which the health report was received]]>
+      </doc>
+    </method>
+    <doc>
+    <![CDATA[{@code NodeHealthStatus} is a summary of the health status of the node.
+ <p>
+ It includes information such as:
+ <ul>
+   <li>
+     An indicator of whether the node is healthy, as determined by the
+     health-check script.
+   </li>
+   <li>The previous time at which the health status was reported.</li>
+   <li>A diagnostic report on the health status.</li>
+ </ul>
+ 
+ @see NodeReport
+ @see ApplicationClientProtocol#getClusterNodes(org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesRequest)]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.api.records.NodeHealthStatus -->
+</package>
+<package name="org.apache.hadoop.yarn.server.api.records.impl.pb">
+</package>
+<package name="org.apache.hadoop.yarn.server.metrics">
+</package>
+<package name="org.apache.hadoop.yarn.server.records">
+</package>
+<package name="org.apache.hadoop.yarn.server.records.impl.pb">
+</package>
+<package name="org.apache.hadoop.yarn.server.security.http">
+</package>
+<package name="org.apache.hadoop.yarn.server.sharedcache">
+</package>
+<package name="org.apache.hadoop.yarn.server.utils">
+  <!-- start class org.apache.hadoop.yarn.server.utils.LeveldbIterator -->
+  <class name="LeveldbIterator" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <implements name="java.util.Iterator"/>
+    <implements name="java.io.Closeable"/>
+    <constructor name="LeveldbIterator" type="org.iq80.leveldb.DB"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Create an iterator for the specified database]]>
+      </doc>
+    </constructor>
+    <constructor name="LeveldbIterator" type="org.iq80.leveldb.DB, org.iq80.leveldb.ReadOptions"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Create an iterator for the specified database]]>
+      </doc>
+    </constructor>
+    <constructor name="LeveldbIterator" type="org.iq80.leveldb.DBIterator"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <doc>
+      <![CDATA[Create an iterator using the specified underlying DBIterator]]>
+      </doc>
+    </constructor>
+    <method name="seek"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="key" type="byte[]"/>
+      <exception name="DBException" type="org.iq80.leveldb.DBException"/>
+      <doc>
+      <![CDATA[Repositions the iterator so the key of the next BlockElement
+ returned greater than or equal to the specified targetKey.]]>
+      </doc>
+    </method>
+    <method name="seekToFirst"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="DBException" type="org.iq80.leveldb.DBException"/>
+      <doc>
+      <![CDATA[Repositions the iterator so is is at the beginning of the Database.]]>
+      </doc>
+    </method>
+    <method name="seekToLast"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="DBException" type="org.iq80.leveldb.DBException"/>
+      <doc>
+      <![CDATA[Repositions the iterator so it is at the end of of the Database.]]>
+      </doc>
+    </method>
+    <method name="hasNext" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="DBException" type="org.iq80.leveldb.DBException"/>
+      <doc>
+      <![CDATA[Returns <tt>true</tt> if the iteration has more elements.]]>
+      </doc>
+    </method>
+    <method name="next" return="java.util.Map.Entry"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="DBException" type="org.iq80.leveldb.DBException"/>
+      <doc>
+      <![CDATA[Returns the next element in the iteration.]]>
+      </doc>
+    </method>
+    <method name="peekNext" return="java.util.Map.Entry"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="DBException" type="org.iq80.leveldb.DBException"/>
+      <doc>
+      <![CDATA[Returns the next element in the iteration, without advancing the
+ iteration.]]>
+      </doc>
+    </method>
+    <method name="hasPrev" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="DBException" type="org.iq80.leveldb.DBException"/>
+      <doc>
+      <![CDATA[@return true if there is a previous entry in the iteration.]]>
+      </doc>
+    </method>
+    <method name="prev" return="java.util.Map.Entry"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="DBException" type="org.iq80.leveldb.DBException"/>
+      <doc>
+      <![CDATA[@return the previous element in the iteration and rewinds the iteration.]]>
+      </doc>
+    </method>
+    <method name="peekPrev" return="java.util.Map.Entry"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="DBException" type="org.iq80.leveldb.DBException"/>
+      <doc>
+      <![CDATA[@return the previous element in the iteration, without rewinding the
+ iteration.]]>
+      </doc>
+    </method>
+    <method name="remove"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="DBException" type="org.iq80.leveldb.DBException"/>
+      <doc>
+      <![CDATA[Removes from the database the last element returned by the iterator.]]>
+      </doc>
+    </method>
+    <method name="close"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <exception name="IOException" type="java.io.IOException"/>
+      <doc>
+      <![CDATA[Closes the iterator.]]>
+      </doc>
+    </method>
+    <doc>
+    <![CDATA[A wrapper for a DBIterator to translate the raw RuntimeExceptions that
+ can be thrown into DBExceptions.]]>
+    </doc>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.utils.LeveldbIterator -->
+</package>
+<package name="org.apache.hadoop.yarn.server.webapp">
+</package>
+<package name="org.apache.hadoop.yarn.server.webapp.dao">
+  <!-- start class org.apache.hadoop.yarn.server.webapp.dao.AppAttemptInfo -->
+  <class name="AppAttemptInfo" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="AppAttemptInfo"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="AppAttemptInfo" type="org.apache.hadoop.yarn.api.records.ApplicationAttemptReport"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="getAppAttemptId" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getHost" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getRpcPort" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getTrackingUrl" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getOriginalTrackingUrl" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getDiagnosticsInfo" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getAppAttemptState" return="org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getAmContainerId" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getStartedTime" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getFinishedTime" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <field name="appAttemptId" type="java.lang.String"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="host" type="java.lang.String"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="rpcPort" type="int"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="trackingUrl" type="java.lang.String"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="originalTrackingUrl" type="java.lang.String"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="diagnosticsInfo" type="java.lang.String"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="appAttemptState" type="org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="amContainerId" type="java.lang.String"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="startedTime" type="long"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="finishedTime" type="long"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.webapp.dao.AppAttemptInfo -->
+  <!-- start class org.apache.hadoop.yarn.server.webapp.dao.AppAttemptsInfo -->
+  <class name="AppAttemptsInfo" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="AppAttemptsInfo"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="add"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="info" type="org.apache.hadoop.yarn.server.webapp.dao.AppAttemptInfo"/>
+    </method>
+    <method name="getAttempts" return="java.util.ArrayList"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <field name="attempt" type="java.util.ArrayList"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.webapp.dao.AppAttemptsInfo -->
+  <!-- start class org.apache.hadoop.yarn.server.webapp.dao.AppInfo -->
+  <class name="AppInfo" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="AppInfo"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="AppInfo" type="org.apache.hadoop.yarn.api.records.ApplicationReport"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="getAppId" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getCurrentAppAttemptId" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getUser" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getName" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getQueue" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getType" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getHost" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getRpcPort" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getAppState" return="org.apache.hadoop.yarn.api.records.YarnApplicationState"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getRunningContainers" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getAllocatedCpuVcores" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getAllocatedMemoryMB" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getProgress" return="float"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getDiagnosticsInfo" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getOriginalTrackingUrl" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getTrackingUrl" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getFinalAppStatus" return="org.apache.hadoop.yarn.api.records.FinalApplicationStatus"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getSubmittedTime" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getStartedTime" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getFinishedTime" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getElapsedTime" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getApplicationTags" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="isUnmanagedApp" return="boolean"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getPriority" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getAppNodeLabelExpression" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getAmNodeLabelExpression" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <field name="appId" type="java.lang.String"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="currentAppAttemptId" type="java.lang.String"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="user" type="java.lang.String"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="name" type="java.lang.String"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="queue" type="java.lang.String"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="type" type="java.lang.String"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="host" type="java.lang.String"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="rpcPort" type="int"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="appState" type="org.apache.hadoop.yarn.api.records.YarnApplicationState"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="runningContainers" type="int"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="progress" type="float"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="diagnosticsInfo" type="java.lang.String"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="originalTrackingUrl" type="java.lang.String"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="trackingUrl" type="java.lang.String"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="finalAppStatus" type="org.apache.hadoop.yarn.api.records.FinalApplicationStatus"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="submittedTime" type="long"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="startedTime" type="long"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="finishedTime" type="long"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="elapsedTime" type="long"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="applicationTags" type="java.lang.String"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="priority" type="int"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="unmanagedApplication" type="boolean"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.webapp.dao.AppInfo -->
+  <!-- start class org.apache.hadoop.yarn.server.webapp.dao.AppsInfo -->
+  <class name="AppsInfo" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="AppsInfo"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="add"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="appinfo" type="org.apache.hadoop.yarn.server.webapp.dao.AppInfo"/>
+    </method>
+    <method name="getApps" return="java.util.ArrayList"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <field name="app" type="java.util.ArrayList"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.webapp.dao.AppsInfo -->
+  <!-- start class org.apache.hadoop.yarn.server.webapp.dao.ContainerInfo -->
+  <class name="ContainerInfo" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="ContainerInfo"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <constructor name="ContainerInfo" type="org.apache.hadoop.yarn.api.records.ContainerReport"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="getContainerId" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getAllocatedMB" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getAllocatedVCores" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getAssignedNodeId" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getPriority" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getStartedTime" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getFinishedTime" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getElapsedTime" return="long"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getDiagnosticsInfo" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getLogUrl" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getContainerExitStatus" return="int"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getContainerState" return="org.apache.hadoop.yarn.api.records.ContainerState"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <method name="getNodeHttpAddress" return="java.lang.String"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <field name="containerId" type="java.lang.String"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="allocatedMB" type="long"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="allocatedVCores" type="long"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="assignedNodeId" type="java.lang.String"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="priority" type="int"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="startedTime" type="long"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="finishedTime" type="long"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="elapsedTime" type="long"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="diagnosticsInfo" type="java.lang.String"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="logUrl" type="java.lang.String"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="containerExitStatus" type="int"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="containerState" type="org.apache.hadoop.yarn.api.records.ContainerState"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+    <field name="nodeHttpAddress" type="java.lang.String"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.webapp.dao.ContainerInfo -->
+  <!-- start class org.apache.hadoop.yarn.server.webapp.dao.ContainersInfo -->
+  <class name="ContainersInfo" extends="java.lang.Object"
+    abstract="false"
+    static="false" final="false" visibility="public"
+    deprecated="not deprecated">
+    <constructor name="ContainersInfo"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </constructor>
+    <method name="add"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+      <param name="containerInfo" type="org.apache.hadoop.yarn.server.webapp.dao.ContainerInfo"/>
+    </method>
+    <method name="getContainers" return="java.util.ArrayList"
+      abstract="false" native="false" synchronized="false"
+      static="false" final="false" visibility="public"
+      deprecated="not deprecated">
+    </method>
+    <field name="container" type="java.util.ArrayList"
+      transient="false" volatile="false"
+      static="false" final="false" visibility="protected"
+      deprecated="not deprecated">
+    </field>
+  </class>
+  <!-- end class org.apache.hadoop.yarn.server.webapp.dao.ContainersInfo -->
+</package>
+
+</api>


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